HBASE-17320 Add inclusive/exclusive support for startRow and endRow of scan
This commit is contained in:
parent
a3e0e0df0d
commit
05b1d918b0
|
@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;
|
||||
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.getLocateType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
@ -64,10 +64,10 @@ class AsyncClientScanner {
|
|||
public AsyncClientScanner(Scan scan, RawScanResultConsumer consumer, TableName tableName,
|
||||
AsyncConnectionImpl conn, long scanTimeoutNs, long rpcTimeoutNs) {
|
||||
if (scan.getStartRow() == null) {
|
||||
scan.setStartRow(EMPTY_START_ROW);
|
||||
scan.withStartRow(EMPTY_START_ROW, scan.includeStartRow());
|
||||
}
|
||||
if (scan.getStopRow() == null) {
|
||||
scan.setStopRow(EMPTY_END_ROW);
|
||||
scan.withStopRow(EMPTY_END_ROW, scan.includeStopRow());
|
||||
}
|
||||
this.scan = scan;
|
||||
this.consumer = consumer;
|
||||
|
@ -117,23 +117,22 @@ class AsyncClientScanner {
|
|||
conn.callerFactory.scanSingleRegion().id(resp.scannerId).location(resp.loc).stub(resp.stub)
|
||||
.setScan(scan).consumer(consumer).resultCache(resultCache)
|
||||
.rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
|
||||
.scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).start()
|
||||
.whenComplete((locateType, error) -> {
|
||||
.scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).start().whenComplete((hasMore, error) -> {
|
||||
if (error != null) {
|
||||
consumer.onError(error);
|
||||
return;
|
||||
}
|
||||
if (locateType == null) {
|
||||
consumer.onComplete();
|
||||
if (hasMore) {
|
||||
openScanner();
|
||||
} else {
|
||||
openScanner(locateType);
|
||||
consumer.onComplete();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void openScanner(RegionLocateType locateType) {
|
||||
private void openScanner() {
|
||||
conn.callerFactory.<OpenScannerResponse> single().table(tableName).row(scan.getStartRow())
|
||||
.locateType(locateType).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
|
||||
.locateType(getLocateType(scan)).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
|
||||
.operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).action(this::callOpenScanner).call()
|
||||
.whenComplete((resp, error) -> {
|
||||
if (error != null) {
|
||||
|
@ -145,7 +144,6 @@ class AsyncClientScanner {
|
|||
}
|
||||
|
||||
public void start() {
|
||||
openScanner(scan.isReversed() && isEmptyStartRow(scan.getStartRow()) ? RegionLocateType.BEFORE
|
||||
: RegionLocateType.CURRENT);
|
||||
openScanner();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -246,7 +246,7 @@ class AsyncRpcRetryingCallerFactory {
|
|||
/**
|
||||
* Short cut for {@code build().start()}.
|
||||
*/
|
||||
public CompletableFuture<RegionLocateType> start() {
|
||||
public CompletableFuture<Boolean> start() {
|
||||
return build().start();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,11 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowBefore;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
|
||||
|
@ -34,7 +32,6 @@ import java.util.List;
|
|||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -53,7 +50,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientServ
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.Interface;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
/**
|
||||
|
@ -91,11 +87,9 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||
|
||||
private final int startLogErrorsCnt;
|
||||
|
||||
private final Supplier<byte[]> createNextStartRowWhenError;
|
||||
|
||||
private final Runnable completeWhenNoMoreResultsInRegion;
|
||||
|
||||
private final CompletableFuture<RegionLocateType> future;
|
||||
private final CompletableFuture<Boolean> future;
|
||||
|
||||
private final HBaseRpcController controller;
|
||||
|
||||
|
@ -128,10 +122,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||
this.rpcTimeoutNs = rpcTimeoutNs;
|
||||
this.startLogErrorsCnt = startLogErrorsCnt;
|
||||
if (scan.isReversed()) {
|
||||
createNextStartRowWhenError = this::createReversedNextStartRowWhenError;
|
||||
completeWhenNoMoreResultsInRegion = this::completeReversedWhenNoMoreResultsInRegion;
|
||||
} else {
|
||||
createNextStartRowWhenError = this::createNextStartRowWhenError;
|
||||
completeWhenNoMoreResultsInRegion = this::completeWhenNoMoreResultsInRegion;
|
||||
}
|
||||
this.future = new CompletableFuture<>();
|
||||
|
@ -164,23 +156,13 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||
future.completeExceptionally(new RetriesExhaustedException(tries - 1, exceptions));
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NONNULL_PARAM_VIOLATION",
|
||||
justification = "https://github.com/findbugsproject/findbugs/issues/79")
|
||||
private void completeNoMoreResults() {
|
||||
future.complete(null);
|
||||
future.complete(false);
|
||||
}
|
||||
|
||||
private void completeWithNextStartRow(byte[] nextStartRow) {
|
||||
scan.setStartRow(nextStartRow);
|
||||
future.complete(scan.isReversed() ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
|
||||
}
|
||||
|
||||
private byte[] createNextStartRowWhenError() {
|
||||
return createClosestRowAfter(nextStartRowWhenError);
|
||||
}
|
||||
|
||||
private byte[] createReversedNextStartRowWhenError() {
|
||||
return createClosestRowBefore(nextStartRowWhenError);
|
||||
private void completeWithNextStartRow(byte[] row, boolean inclusive) {
|
||||
scan.withStartRow(row, inclusive);
|
||||
future.complete(true);
|
||||
}
|
||||
|
||||
private void completeWhenError(boolean closeScanner) {
|
||||
|
@ -189,12 +171,9 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||
closeScanner();
|
||||
}
|
||||
if (nextStartRowWhenError != null) {
|
||||
scan.setStartRow(
|
||||
includeNextStartRowWhenError ? nextStartRowWhenError : createNextStartRowWhenError.get());
|
||||
scan.withStartRow(nextStartRowWhenError, includeNextStartRowWhenError);
|
||||
}
|
||||
future.complete(
|
||||
scan.isReversed() && Bytes.equals(scan.getStartRow(), loc.getRegionInfo().getEndKey())
|
||||
? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
|
||||
future.complete(true);
|
||||
}
|
||||
|
||||
private void onError(Throwable error) {
|
||||
|
@ -251,29 +230,19 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||
}
|
||||
|
||||
private void completeWhenNoMoreResultsInRegion() {
|
||||
if (isEmptyStopRow(scan.getStopRow())) {
|
||||
if (isEmptyStopRow(loc.getRegionInfo().getEndKey())) {
|
||||
completeNoMoreResults();
|
||||
}
|
||||
if (noMoreResultsForScan(scan, loc.getRegionInfo())) {
|
||||
completeNoMoreResults();
|
||||
} else {
|
||||
if (Bytes.compareTo(loc.getRegionInfo().getEndKey(), scan.getStopRow()) >= 0) {
|
||||
completeNoMoreResults();
|
||||
}
|
||||
completeWithNextStartRow(loc.getRegionInfo().getEndKey(), true);
|
||||
}
|
||||
completeWithNextStartRow(loc.getRegionInfo().getEndKey());
|
||||
}
|
||||
|
||||
private void completeReversedWhenNoMoreResultsInRegion() {
|
||||
if (isEmptyStopRow(scan.getStopRow())) {
|
||||
if (isEmptyStartRow(loc.getRegionInfo().getStartKey())) {
|
||||
completeNoMoreResults();
|
||||
}
|
||||
if (noMoreResultsForReverseScan(scan, loc.getRegionInfo())) {
|
||||
completeNoMoreResults();
|
||||
} else {
|
||||
if (Bytes.compareTo(loc.getRegionInfo().getStartKey(), scan.getStopRow()) <= 0) {
|
||||
completeNoMoreResults();
|
||||
}
|
||||
completeWithNextStartRow(loc.getRegionInfo().getStartKey(), false);
|
||||
}
|
||||
completeWithNextStartRow(loc.getRegionInfo().getStartKey());
|
||||
}
|
||||
|
||||
private void onComplete(ScanResponse resp) {
|
||||
|
@ -343,9 +312,9 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return return locate direction for next open scanner call, or null if we should stop.
|
||||
* @return {@code true} if we should continue, otherwise {@code false}.
|
||||
*/
|
||||
public CompletableFuture<RegionLocateType> start() {
|
||||
public CompletableFuture<Boolean> start() {
|
||||
next();
|
||||
return future;
|
||||
}
|
||||
|
|
|
@ -17,8 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.getLocateType;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* Retry caller for smaller scan.
|
||||
|
@ -57,10 +57,6 @@ class AsyncSmallScanRpcRetryingCaller {
|
|||
|
||||
private final long rpcTimeoutNs;
|
||||
|
||||
private final Function<byte[], byte[]> createClosestNextRow;
|
||||
|
||||
private final Runnable firstScan;
|
||||
|
||||
private final Function<HRegionInfo, Boolean> nextScan;
|
||||
|
||||
private final List<Result> resultList;
|
||||
|
@ -76,12 +72,8 @@ class AsyncSmallScanRpcRetryingCaller {
|
|||
this.scanTimeoutNs = scanTimeoutNs;
|
||||
this.rpcTimeoutNs = rpcTimeoutNs;
|
||||
if (scan.isReversed()) {
|
||||
this.createClosestNextRow = ConnectionUtils::createClosestRowBefore;
|
||||
this.firstScan = this::reversedFirstScan;
|
||||
this.nextScan = this::reversedNextScan;
|
||||
} else {
|
||||
this.createClosestNextRow = ConnectionUtils::createClosestRowAfter;
|
||||
this.firstScan = this::firstScan;
|
||||
this.nextScan = this::nextScan;
|
||||
}
|
||||
this.resultList = new ArrayList<>();
|
||||
|
@ -141,10 +133,9 @@ class AsyncSmallScanRpcRetryingCaller {
|
|||
}
|
||||
if (resp.hasMoreResultsInRegion) {
|
||||
if (resp.results.length > 0) {
|
||||
scan.setStartRow(
|
||||
createClosestNextRow.apply(resp.results[resp.results.length - 1].getRow()));
|
||||
scan.withStartRow(resp.results[resp.results.length - 1].getRow(), false);
|
||||
}
|
||||
scan(RegionLocateType.CURRENT);
|
||||
scan();
|
||||
return;
|
||||
}
|
||||
if (!nextScan.apply(resp.currentRegion)) {
|
||||
|
@ -152,11 +143,11 @@ class AsyncSmallScanRpcRetryingCaller {
|
|||
}
|
||||
}
|
||||
|
||||
private void scan(RegionLocateType locateType) {
|
||||
private void scan() {
|
||||
conn.callerFactory.<SmallScanResponse> single().table(tableName).row(scan.getStartRow())
|
||||
.rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
|
||||
.operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).locateType(locateType)
|
||||
.action(this::scan).call().whenComplete((resp, error) -> {
|
||||
.locateType(getLocateType(scan)).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
|
||||
.operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).action(this::scan).call()
|
||||
.whenComplete((resp, error) -> {
|
||||
if (error != null) {
|
||||
future.completeExceptionally(error);
|
||||
} else {
|
||||
|
@ -166,45 +157,27 @@ class AsyncSmallScanRpcRetryingCaller {
|
|||
}
|
||||
|
||||
public CompletableFuture<List<Result>> call() {
|
||||
firstScan.run();
|
||||
scan();
|
||||
return future;
|
||||
}
|
||||
|
||||
private void firstScan() {
|
||||
scan(RegionLocateType.CURRENT);
|
||||
}
|
||||
|
||||
private void reversedFirstScan() {
|
||||
scan(isEmptyStartRow(scan.getStartRow()) ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
|
||||
}
|
||||
|
||||
private boolean nextScan(HRegionInfo region) {
|
||||
if (isEmptyStopRow(scan.getStopRow())) {
|
||||
if (isEmptyStopRow(region.getEndKey())) {
|
||||
return false;
|
||||
}
|
||||
private boolean nextScan(HRegionInfo info) {
|
||||
if (noMoreResultsForScan(scan, info)) {
|
||||
return false;
|
||||
} else {
|
||||
if (Bytes.compareTo(region.getEndKey(), scan.getStopRow()) >= 0) {
|
||||
return false;
|
||||
}
|
||||
scan.withStartRow(info.getEndKey());
|
||||
scan();
|
||||
return true;
|
||||
}
|
||||
scan.setStartRow(region.getEndKey());
|
||||
scan(RegionLocateType.CURRENT);
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean reversedNextScan(HRegionInfo region) {
|
||||
if (isEmptyStopRow(scan.getStopRow())) {
|
||||
if (isEmptyStartRow(region.getStartKey())) {
|
||||
return false;
|
||||
}
|
||||
private boolean reversedNextScan(HRegionInfo info) {
|
||||
if (noMoreResultsForReverseScan(scan, info)) {
|
||||
return false;
|
||||
} else {
|
||||
if (Bytes.compareTo(region.getStartKey(), scan.getStopRow()) <= 0) {
|
||||
return false;
|
||||
}
|
||||
scan.withStartRow(info.getStartKey(), false);
|
||||
scan();
|
||||
return true;
|
||||
}
|
||||
scan.setStartRow(region.getStartKey());
|
||||
scan(RegionLocateType.BEFORE);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.io.IOException;
|
|||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Queue;
|
||||
import java.util.function.Function;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -65,14 +64,10 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
|
|||
// used to filter out cells that already returned when we restart a scan
|
||||
private Cell lastCell;
|
||||
|
||||
private Function<byte[], byte[]> createClosestRow;
|
||||
|
||||
public AsyncTableResultScanner(RawAsyncTable table, Scan scan, long maxCacheSize) {
|
||||
this.rawTable = table;
|
||||
this.scan = scan;
|
||||
this.maxCacheSize = maxCacheSize;
|
||||
this.createClosestRow = scan.isReversed() ? ConnectionUtils::createClosestRowBefore
|
||||
: ConnectionUtils::createClosestRowAfter;
|
||||
table.scan(scan, this);
|
||||
}
|
||||
|
||||
|
@ -84,16 +79,17 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
|
|||
private void stopPrefetch(Result lastResult) {
|
||||
prefetchStopped = true;
|
||||
if (lastResult.isPartial() || scan.getBatch() > 0) {
|
||||
scan.setStartRow(lastResult.getRow());
|
||||
scan.withStartRow(lastResult.getRow());
|
||||
lastCell = lastResult.rawCells()[lastResult.rawCells().length - 1];
|
||||
} else {
|
||||
scan.setStartRow(createClosestRow.apply(lastResult.getRow()));
|
||||
scan.withStartRow(lastResult.getRow(), false);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(System.identityHashCode(this) + " stop prefetching when scanning "
|
||||
+ rawTable.getName() + " as the cache size " + cacheSize
|
||||
+ " is greater than the maxCacheSize + " + maxCacheSize + ", the next start row is "
|
||||
+ Bytes.toStringBinary(scan.getStartRow()) + ", lastCell is " + lastCell);
|
||||
LOG.debug(
|
||||
String.format("0x%x", System.identityHashCode(this)) + " stop prefetching when scanning "
|
||||
+ rawTable.getName() + " as the cache size " + cacheSize
|
||||
+ " is greater than the maxCacheSize " + maxCacheSize + ", the next start row is "
|
||||
+ Bytes.toStringBinary(scan.getStartRow()) + ", lastCell is " + lastCell);
|
||||
}
|
||||
// Ignore an onComplete call as the scan is stopped by us.
|
||||
// Here we can not use a simple boolean flag. A scan operation can cross multiple regions and
|
||||
|
@ -166,7 +162,7 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
|
|||
|
||||
private void resumePrefetch() {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(System.identityHashCode(this) + " resume prefetching");
|
||||
LOG.debug(String.format("0x%x", System.identityHashCode(this)) + " resume prefetching");
|
||||
}
|
||||
prefetchStopped = false;
|
||||
rawTable.scan(scan, this);
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -248,7 +249,10 @@ public final class ConnectionUtils {
|
|||
|
||||
/**
|
||||
* Create the closest row before the specified row
|
||||
* @deprecated in fact, we do not know the closest row before the given row, the result is only a
|
||||
* row very close to the current row. Avoid using this method in the future.
|
||||
*/
|
||||
@Deprecated
|
||||
static byte[] createClosestRowBefore(byte[] row) {
|
||||
if (row.length == 0) {
|
||||
return MAX_BYTE_ARRAY;
|
||||
|
@ -347,4 +351,42 @@ public final class ConnectionUtils {
|
|||
static CompletableFuture<Void> voidBatchAll(AsyncTableBase table, List<? extends Row> actions) {
|
||||
return table.<Object> batchAll(actions).thenApply(r -> null);
|
||||
}
|
||||
|
||||
static RegionLocateType getLocateType(Scan scan) {
|
||||
if (scan.isReversed()) {
|
||||
if (isEmptyStartRow(scan.getStartRow())) {
|
||||
return RegionLocateType.BEFORE;
|
||||
} else {
|
||||
return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.BEFORE;
|
||||
}
|
||||
} else {
|
||||
return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.AFTER;
|
||||
}
|
||||
}
|
||||
|
||||
static boolean noMoreResultsForScan(Scan scan, HRegionInfo info) {
|
||||
if (isEmptyStopRow(info.getEndKey())) {
|
||||
return true;
|
||||
}
|
||||
if (isEmptyStopRow(scan.getStopRow())) {
|
||||
return false;
|
||||
}
|
||||
int c = Bytes.compareTo(info.getEndKey(), scan.getStopRow());
|
||||
// 1. if our stop row is less than the endKey of the region
|
||||
// 2. if our stop row is equal to the endKey of the region and we do not include the stop row
|
||||
// for scan.
|
||||
return c > 0 || (c == 0 && !scan.includeStopRow());
|
||||
}
|
||||
|
||||
static boolean noMoreResultsForReverseScan(Scan scan, HRegionInfo info) {
|
||||
if (isEmptyStartRow(info.getStartKey())) {
|
||||
return true;
|
||||
}
|
||||
if (isEmptyStopRow(scan.getStopRow())) {
|
||||
return false;
|
||||
}
|
||||
// no need to test the inclusive of the stop row as the start key of a region is included in
|
||||
// the region.
|
||||
return Bytes.compareTo(info.getStartKey(), scan.getStopRow()) <= 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -87,7 +87,9 @@ public class Scan extends Query {
|
|||
private static final String RAW_ATTR = "_raw_";
|
||||
|
||||
private byte [] startRow = HConstants.EMPTY_START_ROW;
|
||||
private boolean includeStartRow = true;
|
||||
private byte [] stopRow = HConstants.EMPTY_END_ROW;
|
||||
private boolean includeStopRow = false;
|
||||
private int maxVersions = 1;
|
||||
private int batch = -1;
|
||||
|
||||
|
@ -106,7 +108,6 @@ public class Scan extends Query {
|
|||
|
||||
private int storeLimit = -1;
|
||||
private int storeOffset = 0;
|
||||
private boolean getScan;
|
||||
|
||||
/**
|
||||
* @deprecated since 1.0.0. Use {@link #setScanMetricsEnabled(boolean)}
|
||||
|
@ -135,8 +136,8 @@ public class Scan extends Query {
|
|||
private long maxResultSize = -1;
|
||||
private boolean cacheBlocks = true;
|
||||
private boolean reversed = false;
|
||||
private Map<byte [], NavigableSet<byte []>> familyMap =
|
||||
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
|
||||
private Map<byte[], NavigableSet<byte[]>> familyMap =
|
||||
new TreeMap<byte[], NavigableSet<byte[]>>(Bytes.BYTES_COMPARATOR);
|
||||
private Boolean asyncPrefetch = null;
|
||||
|
||||
/**
|
||||
|
@ -175,7 +176,11 @@ public class Scan extends Query {
|
|||
*/
|
||||
public Scan() {}
|
||||
|
||||
public Scan(byte [] startRow, Filter filter) {
|
||||
/**
|
||||
* @deprecated use {@code new Scan().withStartRow(startRow).setFilter(filter)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public Scan(byte[] startRow, Filter filter) {
|
||||
this(startRow);
|
||||
this.filter = filter;
|
||||
}
|
||||
|
@ -183,24 +188,26 @@ public class Scan extends Query {
|
|||
/**
|
||||
* Create a Scan operation starting at the specified row.
|
||||
* <p>
|
||||
* If the specified row does not exist, the Scanner will start from the
|
||||
* next closest row after the specified row.
|
||||
* If the specified row does not exist, the Scanner will start from the next closest row after the
|
||||
* specified row.
|
||||
* @param startRow row to start scanner at or after
|
||||
* @deprecated use {@code new Scan().withStartRow(startRow)} instead.
|
||||
*/
|
||||
public Scan(byte [] startRow) {
|
||||
this.startRow = startRow;
|
||||
@Deprecated
|
||||
public Scan(byte[] startRow) {
|
||||
setStartRow(startRow);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Scan operation for the range of rows specified.
|
||||
* @param startRow row to start scanner at or after (inclusive)
|
||||
* @param stopRow row to stop scanner before (exclusive)
|
||||
* @deprecated use {@code new Scan().withStartRow(startRow).withStopRow(stopRow)} instead.
|
||||
*/
|
||||
public Scan(byte [] startRow, byte [] stopRow) {
|
||||
this.startRow = startRow;
|
||||
this.stopRow = stopRow;
|
||||
//if the startRow and stopRow both are empty, it is not a Get
|
||||
this.getScan = isStartRowAndEqualsStopRow();
|
||||
@Deprecated
|
||||
public Scan(byte[] startRow, byte[] stopRow) {
|
||||
setStartRow(startRow);
|
||||
setStopRow(stopRow);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -211,7 +218,9 @@ public class Scan extends Query {
|
|||
*/
|
||||
public Scan(Scan scan) throws IOException {
|
||||
startRow = scan.getStartRow();
|
||||
includeStartRow = scan.includeStartRow();
|
||||
stopRow = scan.getStopRow();
|
||||
includeStopRow = scan.includeStopRow();
|
||||
maxVersions = scan.getMaxVersions();
|
||||
batch = scan.getBatch();
|
||||
storeLimit = scan.getMaxResultsPerColumnFamily();
|
||||
|
@ -219,7 +228,6 @@ public class Scan extends Query {
|
|||
caching = scan.getCaching();
|
||||
maxResultSize = scan.getMaxResultSize();
|
||||
cacheBlocks = scan.getCacheBlocks();
|
||||
getScan = scan.isGetScan();
|
||||
filter = scan.getFilter(); // clone?
|
||||
loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();
|
||||
consistency = scan.getConsistency();
|
||||
|
@ -228,8 +236,7 @@ public class Scan extends Query {
|
|||
asyncPrefetch = scan.isAsyncPrefetch();
|
||||
small = scan.isSmall();
|
||||
allowPartialResults = scan.getAllowPartialResults();
|
||||
TimeRange ctr = scan.getTimeRange();
|
||||
tr = new TimeRange(ctr.getMin(), ctr.getMax());
|
||||
tr = scan.getTimeRange(); // TimeRange is immutable
|
||||
Map<byte[], NavigableSet<byte[]>> fams = scan.getFamilyMap();
|
||||
for (Map.Entry<byte[],NavigableSet<byte[]>> entry : fams.entrySet()) {
|
||||
byte [] fam = entry.getKey();
|
||||
|
@ -258,7 +265,9 @@ public class Scan extends Query {
|
|||
*/
|
||||
public Scan(Get get) {
|
||||
this.startRow = get.getRow();
|
||||
this.includeStartRow = true;
|
||||
this.stopRow = get.getRow();
|
||||
this.includeStopRow = true;
|
||||
this.filter = get.getFilter();
|
||||
this.cacheBlocks = get.getCacheBlocks();
|
||||
this.maxVersions = get.getMaxVersions();
|
||||
|
@ -266,7 +275,6 @@ public class Scan extends Query {
|
|||
this.storeOffset = get.getRowOffsetPerColumnFamily();
|
||||
this.tr = get.getTimeRange();
|
||||
this.familyMap = get.getFamilyMap();
|
||||
this.getScan = true;
|
||||
this.asyncPrefetch = false;
|
||||
this.consistency = get.getConsistency();
|
||||
this.setIsolationLevel(get.getIsolationLevel());
|
||||
|
@ -282,13 +290,13 @@ public class Scan extends Query {
|
|||
}
|
||||
|
||||
public boolean isGetScan() {
|
||||
return this.getScan || isStartRowAndEqualsStopRow();
|
||||
return includeStartRow && includeStopRow && areStartRowAndStopRowEqual(startRow, stopRow);
|
||||
}
|
||||
|
||||
private boolean isStartRowAndEqualsStopRow() {
|
||||
return this.startRow != null && this.startRow.length > 0 &&
|
||||
Bytes.equals(this.startRow, this.stopRow);
|
||||
private static boolean areStartRowAndStopRowEqual(byte[] startRow, byte[] stopRow) {
|
||||
return startRow != null && startRow.length > 0 && Bytes.equals(startRow, stopRow);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all columns from the specified family.
|
||||
* <p>
|
||||
|
@ -383,45 +391,120 @@ public class Scan extends Query {
|
|||
/**
|
||||
* Set the start row of the scan.
|
||||
* <p>
|
||||
* If the specified row does not exist, the Scanner will start from the
|
||||
* next closest row after the specified row.
|
||||
* If the specified row does not exist, the Scanner will start from the next closest row after the
|
||||
* specified row.
|
||||
* @param startRow row to start scanner at or after
|
||||
* @return this
|
||||
* @throws IllegalArgumentException if startRow does not meet criteria
|
||||
* for a row key (when length exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
* @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
|
||||
* exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
* @deprecated use {@link #withStartRow(byte[])} instead. This method may change the inclusive of
|
||||
* the stop row to keep compatible with the old behavior.
|
||||
*/
|
||||
public Scan setStartRow(byte [] startRow) {
|
||||
@Deprecated
|
||||
public Scan setStartRow(byte[] startRow) {
|
||||
withStartRow(startRow);
|
||||
if (areStartRowAndStopRowEqual(startRow, stopRow)) {
|
||||
// for keeping the old behavior that a scan with the same start and stop row is a get scan.
|
||||
this.includeStopRow = true;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the start row of the scan.
|
||||
* <p>
|
||||
* If the specified row does not exist, the Scanner will start from the next closest row after the
|
||||
* specified row.
|
||||
* @param startRow row to start scanner at or after
|
||||
* @return this
|
||||
* @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
|
||||
* exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
*/
|
||||
public Scan withStartRow(byte[] startRow) {
|
||||
return withStartRow(startRow, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the start row of the scan.
|
||||
* <p>
|
||||
* If the specified row does not exist, or the {@code inclusive} is {@code false}, the Scanner
|
||||
* will start from the next closest row after the specified row.
|
||||
* @param startRow row to start scanner at or after
|
||||
* @param inclusive whether we should include the start row when scan
|
||||
* @return this
|
||||
* @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
|
||||
* exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
*/
|
||||
public Scan withStartRow(byte[] startRow, boolean inclusive) {
|
||||
if (Bytes.len(startRow) > HConstants.MAX_ROW_LENGTH) {
|
||||
throw new IllegalArgumentException(
|
||||
"startRow's length must be less than or equal to " +
|
||||
HConstants.MAX_ROW_LENGTH + " to meet the criteria" +
|
||||
" for a row key.");
|
||||
throw new IllegalArgumentException("startRow's length must be less than or equal to "
|
||||
+ HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");
|
||||
}
|
||||
this.startRow = startRow;
|
||||
this.includeStartRow = inclusive;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the stop row of the scan.
|
||||
* @param stopRow row to end at (exclusive)
|
||||
* <p>
|
||||
* The scan will include rows that are lexicographically less than
|
||||
* the provided stopRow.
|
||||
* <p><b>Note:</b> When doing a filter for a rowKey <u>Prefix</u>
|
||||
* use {@link #setRowPrefixFilter(byte[])}.
|
||||
* The 'trailing 0' will not yield the desired result.</p>
|
||||
* The scan will include rows that are lexicographically less than the provided stopRow.
|
||||
* <p>
|
||||
* <b>Note:</b> When doing a filter for a rowKey <u>Prefix</u> use
|
||||
* {@link #setRowPrefixFilter(byte[])}. The 'trailing 0' will not yield the desired result.
|
||||
* </p>
|
||||
* @param stopRow row to end at (exclusive)
|
||||
* @return this
|
||||
* @throws IllegalArgumentException if stopRow does not meet criteria
|
||||
* for a row key (when length exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
* @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
|
||||
* exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
* @deprecated use {@link #withStartRow(byte[])} instead. This method may change the inclusive of
|
||||
* the stop row to keep compatible with the old behavior.
|
||||
*/
|
||||
public Scan setStopRow(byte [] stopRow) {
|
||||
@Deprecated
|
||||
public Scan setStopRow(byte[] stopRow) {
|
||||
withStopRow(stopRow);
|
||||
if (areStartRowAndStopRowEqual(startRow, stopRow)) {
|
||||
// for keeping the old behavior that a scan with the same start and stop row is a get scan.
|
||||
this.includeStopRow = true;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the stop row of the scan.
|
||||
* <p>
|
||||
* The scan will include rows that are lexicographically less than the provided stopRow.
|
||||
* <p>
|
||||
* <b>Note:</b> When doing a filter for a rowKey <u>Prefix</u> use
|
||||
* {@link #setRowPrefixFilter(byte[])}. The 'trailing 0' will not yield the desired result.
|
||||
* </p>
|
||||
* @param stopRow row to end at (exclusive)
|
||||
* @return this
|
||||
* @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
|
||||
* exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
*/
|
||||
public Scan withStopRow(byte[] stopRow) {
|
||||
return withStopRow(stopRow, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the stop row of the scan.
|
||||
* <p>
|
||||
* The scan will include rows that are lexicographically less than (or equal to if
|
||||
* {@code inclusive} is {@code true}) the provided stopRow.
|
||||
* @param stopRow row to end at
|
||||
* @param inclusive whether we should include the stop row when scan
|
||||
* @return this
|
||||
* @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
|
||||
* exceeds {@link HConstants#MAX_ROW_LENGTH})
|
||||
*/
|
||||
public Scan withStopRow(byte[] stopRow, boolean inclusive) {
|
||||
if (Bytes.len(stopRow) > HConstants.MAX_ROW_LENGTH) {
|
||||
throw new IllegalArgumentException(
|
||||
"stopRow's length must be less than or equal to " +
|
||||
HConstants.MAX_ROW_LENGTH + " to meet the criteria" +
|
||||
" for a row key.");
|
||||
throw new IllegalArgumentException("stopRow's length must be less than or equal to "
|
||||
+ HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");
|
||||
}
|
||||
this.stopRow = stopRow;
|
||||
this.includeStopRow = inclusive;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -635,13 +718,27 @@ public class Scan extends Query {
|
|||
return this.startRow;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return if we should include start row when scan
|
||||
*/
|
||||
public boolean includeStartRow() {
|
||||
return includeStartRow;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the stoprow
|
||||
*/
|
||||
public byte [] getStopRow() {
|
||||
public byte[] getStopRow() {
|
||||
return this.stopRow;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return if we should include stop row when scan
|
||||
*/
|
||||
public boolean includeStopRow() {
|
||||
return includeStopRow;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the max number of versions to fetch
|
||||
*/
|
||||
|
|
|
@ -911,6 +911,12 @@ public final class ProtobufUtil {
|
|||
if (mvccReadPoint > 0) {
|
||||
scanBuilder.setMvccReadPoint(mvccReadPoint);
|
||||
}
|
||||
if (!scan.includeStartRow()) {
|
||||
scanBuilder.setIncludeStartRow(false);
|
||||
}
|
||||
if (scan.includeStopRow()) {
|
||||
scanBuilder.setIncludeStopRow(true);
|
||||
}
|
||||
return scanBuilder.build();
|
||||
}
|
||||
|
||||
|
@ -923,15 +929,24 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static Scan toScan(
|
||||
final ClientProtos.Scan proto) throws IOException {
|
||||
byte [] startRow = HConstants.EMPTY_START_ROW;
|
||||
byte [] stopRow = HConstants.EMPTY_END_ROW;
|
||||
byte[] startRow = HConstants.EMPTY_START_ROW;
|
||||
byte[] stopRow = HConstants.EMPTY_END_ROW;
|
||||
boolean includeStartRow = true;
|
||||
boolean includeStopRow = false;
|
||||
if (proto.hasStartRow()) {
|
||||
startRow = proto.getStartRow().toByteArray();
|
||||
}
|
||||
if (proto.hasStopRow()) {
|
||||
stopRow = proto.getStopRow().toByteArray();
|
||||
}
|
||||
Scan scan = new Scan(startRow, stopRow);
|
||||
if (proto.hasIncludeStartRow()) {
|
||||
includeStartRow = proto.getIncludeStartRow();
|
||||
}
|
||||
if (proto.hasIncludeStopRow()) {
|
||||
includeStopRow = proto.getIncludeStopRow();
|
||||
}
|
||||
Scan scan =
|
||||
new Scan().withStartRow(startRow, includeStartRow).withStopRow(stopRow, includeStopRow);
|
||||
if (proto.hasCacheBlocks()) {
|
||||
scan.setCacheBlocks(proto.getCacheBlocks());
|
||||
}
|
||||
|
|
|
@ -1024,6 +1024,12 @@ public final class ProtobufUtil {
|
|||
if (mvccReadPoint > 0) {
|
||||
scanBuilder.setMvccReadPoint(mvccReadPoint);
|
||||
}
|
||||
if (!scan.includeStartRow()) {
|
||||
scanBuilder.setIncludeStartRow(false);
|
||||
}
|
||||
if (scan.includeStopRow()) {
|
||||
scanBuilder.setIncludeStopRow(true);
|
||||
}
|
||||
return scanBuilder.build();
|
||||
}
|
||||
|
||||
|
@ -1036,15 +1042,24 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static Scan toScan(
|
||||
final ClientProtos.Scan proto) throws IOException {
|
||||
byte [] startRow = HConstants.EMPTY_START_ROW;
|
||||
byte [] stopRow = HConstants.EMPTY_END_ROW;
|
||||
byte[] startRow = HConstants.EMPTY_START_ROW;
|
||||
byte[] stopRow = HConstants.EMPTY_END_ROW;
|
||||
boolean includeStartRow = true;
|
||||
boolean includeStopRow = false;
|
||||
if (proto.hasStartRow()) {
|
||||
startRow = proto.getStartRow().toByteArray();
|
||||
}
|
||||
if (proto.hasStopRow()) {
|
||||
stopRow = proto.getStopRow().toByteArray();
|
||||
}
|
||||
Scan scan = new Scan(startRow, stopRow);
|
||||
if (proto.hasIncludeStartRow()) {
|
||||
includeStartRow = proto.getIncludeStartRow();
|
||||
}
|
||||
if (proto.hasIncludeStopRow()) {
|
||||
includeStopRow = proto.getIncludeStopRow();
|
||||
}
|
||||
Scan scan =
|
||||
new Scan().withStartRow(startRow, includeStartRow).withStopRow(stopRow, includeStopRow);
|
||||
if (proto.hasCacheBlocks()) {
|
||||
scan.setCacheBlocks(proto.getCacheBlocks());
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.io;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
|
|
@ -14563,6 +14563,24 @@ public final class ClientProtos {
|
|||
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
|
||||
*/
|
||||
long getMvccReadPoint();
|
||||
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
boolean hasIncludeStartRow();
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
boolean getIncludeStartRow();
|
||||
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
boolean hasIncludeStopRow();
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
boolean getIncludeStopRow();
|
||||
}
|
||||
/**
|
||||
* <pre>
|
||||
|
@ -14604,6 +14622,8 @@ public final class ClientProtos {
|
|||
allowPartialResults_ = false;
|
||||
cfTimeRange_ = java.util.Collections.emptyList();
|
||||
mvccReadPoint_ = 0L;
|
||||
includeStartRow_ = true;
|
||||
includeStopRow_ = false;
|
||||
}
|
||||
|
||||
@java.lang.Override
|
||||
|
@ -14768,6 +14788,16 @@ public final class ClientProtos {
|
|||
mvccReadPoint_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
case 168: {
|
||||
bitField0_ |= 0x00020000;
|
||||
includeStartRow_ = input.readBool();
|
||||
break;
|
||||
}
|
||||
case 176: {
|
||||
bitField0_ |= 0x00040000;
|
||||
includeStopRow_ = input.readBool();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
|
||||
|
@ -15183,6 +15213,36 @@ public final class ClientProtos {
|
|||
return mvccReadPoint_;
|
||||
}
|
||||
|
||||
public static final int INCLUDE_START_ROW_FIELD_NUMBER = 21;
|
||||
private boolean includeStartRow_;
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean hasIncludeStartRow() {
|
||||
return ((bitField0_ & 0x00020000) == 0x00020000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean getIncludeStartRow() {
|
||||
return includeStartRow_;
|
||||
}
|
||||
|
||||
public static final int INCLUDE_STOP_ROW_FIELD_NUMBER = 22;
|
||||
private boolean includeStopRow_;
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean hasIncludeStopRow() {
|
||||
return ((bitField0_ & 0x00040000) == 0x00040000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean getIncludeStopRow() {
|
||||
return includeStopRow_;
|
||||
}
|
||||
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
byte isInitialized = memoizedIsInitialized;
|
||||
|
@ -15279,6 +15339,12 @@ public final class ClientProtos {
|
|||
if (((bitField0_ & 0x00010000) == 0x00010000)) {
|
||||
output.writeUInt64(20, mvccReadPoint_);
|
||||
}
|
||||
if (((bitField0_ & 0x00020000) == 0x00020000)) {
|
||||
output.writeBool(21, includeStartRow_);
|
||||
}
|
||||
if (((bitField0_ & 0x00040000) == 0x00040000)) {
|
||||
output.writeBool(22, includeStopRow_);
|
||||
}
|
||||
unknownFields.writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -15367,6 +15433,14 @@ public final class ClientProtos {
|
|||
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(20, mvccReadPoint_);
|
||||
}
|
||||
if (((bitField0_ & 0x00020000) == 0x00020000)) {
|
||||
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
|
||||
.computeBoolSize(21, includeStartRow_);
|
||||
}
|
||||
if (((bitField0_ & 0x00040000) == 0x00040000)) {
|
||||
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
|
||||
.computeBoolSize(22, includeStopRow_);
|
||||
}
|
||||
size += unknownFields.getSerializedSize();
|
||||
memoizedSize = size;
|
||||
return size;
|
||||
|
@ -15474,6 +15548,16 @@ public final class ClientProtos {
|
|||
result = result && (getMvccReadPoint()
|
||||
== other.getMvccReadPoint());
|
||||
}
|
||||
result = result && (hasIncludeStartRow() == other.hasIncludeStartRow());
|
||||
if (hasIncludeStartRow()) {
|
||||
result = result && (getIncludeStartRow()
|
||||
== other.getIncludeStartRow());
|
||||
}
|
||||
result = result && (hasIncludeStopRow() == other.hasIncludeStopRow());
|
||||
if (hasIncludeStopRow()) {
|
||||
result = result && (getIncludeStopRow()
|
||||
== other.getIncludeStopRow());
|
||||
}
|
||||
result = result && unknownFields.equals(other.unknownFields);
|
||||
return result;
|
||||
}
|
||||
|
@ -15572,6 +15656,16 @@ public final class ClientProtos {
|
|||
hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
|
||||
getMvccReadPoint());
|
||||
}
|
||||
if (hasIncludeStartRow()) {
|
||||
hash = (37 * hash) + INCLUDE_START_ROW_FIELD_NUMBER;
|
||||
hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
|
||||
getIncludeStartRow());
|
||||
}
|
||||
if (hasIncludeStopRow()) {
|
||||
hash = (37 * hash) + INCLUDE_STOP_ROW_FIELD_NUMBER;
|
||||
hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
|
||||
getIncludeStopRow());
|
||||
}
|
||||
hash = (29 * hash) + unknownFields.hashCode();
|
||||
memoizedHashCode = hash;
|
||||
return hash;
|
||||
|
@ -15765,6 +15859,10 @@ public final class ClientProtos {
|
|||
}
|
||||
mvccReadPoint_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00080000);
|
||||
includeStartRow_ = true;
|
||||
bitField0_ = (bitField0_ & ~0x00100000);
|
||||
includeStopRow_ = false;
|
||||
bitField0_ = (bitField0_ & ~0x00200000);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -15892,6 +15990,14 @@ public final class ClientProtos {
|
|||
to_bitField0_ |= 0x00010000;
|
||||
}
|
||||
result.mvccReadPoint_ = mvccReadPoint_;
|
||||
if (((from_bitField0_ & 0x00100000) == 0x00100000)) {
|
||||
to_bitField0_ |= 0x00020000;
|
||||
}
|
||||
result.includeStartRow_ = includeStartRow_;
|
||||
if (((from_bitField0_ & 0x00200000) == 0x00200000)) {
|
||||
to_bitField0_ |= 0x00040000;
|
||||
}
|
||||
result.includeStopRow_ = includeStopRow_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -16063,6 +16169,12 @@ public final class ClientProtos {
|
|||
if (other.hasMvccReadPoint()) {
|
||||
setMvccReadPoint(other.getMvccReadPoint());
|
||||
}
|
||||
if (other.hasIncludeStartRow()) {
|
||||
setIncludeStartRow(other.getIncludeStartRow());
|
||||
}
|
||||
if (other.hasIncludeStopRow()) {
|
||||
setIncludeStopRow(other.getIncludeStopRow());
|
||||
}
|
||||
this.mergeUnknownFields(other.unknownFields);
|
||||
onChanged();
|
||||
return this;
|
||||
|
@ -17572,6 +17684,70 @@ public final class ClientProtos {
|
|||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
private boolean includeStartRow_ = true;
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean hasIncludeStartRow() {
|
||||
return ((bitField0_ & 0x00100000) == 0x00100000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean getIncludeStartRow() {
|
||||
return includeStartRow_;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public Builder setIncludeStartRow(boolean value) {
|
||||
bitField0_ |= 0x00100000;
|
||||
includeStartRow_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public Builder clearIncludeStartRow() {
|
||||
bitField0_ = (bitField0_ & ~0x00100000);
|
||||
includeStartRow_ = true;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
private boolean includeStopRow_ ;
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean hasIncludeStopRow() {
|
||||
return ((bitField0_ & 0x00200000) == 0x00200000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean getIncludeStopRow() {
|
||||
return includeStopRow_;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public Builder setIncludeStopRow(boolean value) {
|
||||
bitField0_ |= 0x00200000;
|
||||
includeStopRow_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public Builder clearIncludeStopRow() {
|
||||
bitField0_ = (bitField0_ & ~0x00200000);
|
||||
includeStopRow_ = false;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public final Builder setUnknownFields(
|
||||
final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
|
||||
return super.setUnknownFields(unknownFields);
|
||||
|
@ -40658,7 +40834,7 @@ public final class ClientProtos {
|
|||
"tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" +
|
||||
"_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" +
|
||||
"\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " +
|
||||
"\001(\010\"\331\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
|
||||
"\001(\010\"\233\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
|
||||
"Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" +
|
||||
"eBytesPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_ro" +
|
||||
"w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" +
|
||||
|
@ -40673,97 +40849,98 @@ public final class ClientProtos {
|
|||
"aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " +
|
||||
"\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" +
|
||||
"lumnFamilyTimeRange\022\032\n\017mvcc_read_point\030\024",
|
||||
" \001(\004:\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132" +
|
||||
"\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\013" +
|
||||
"2\016.hbase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016" +
|
||||
"number_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 " +
|
||||
"\001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_han" +
|
||||
"dles_partials\030\007 \001(\010\022!\n\031client_handles_he" +
|
||||
"artbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001" +
|
||||
"(\010\022\024\n\005renew\030\n \001(\010:\005false\"\266\002\n\014ScanRespons" +
|
||||
"e\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_i" +
|
||||
"d\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001",
|
||||
"(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n" +
|
||||
"\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_result\030" +
|
||||
"\007 \003(\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n" +
|
||||
"\021heartbeat_message\030\t \001(\010\022+\n\014scan_metrics" +
|
||||
"\030\n \001(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_re" +
|
||||
"ad_point\030\013 \001(\004:\0010\"\240\002\n\024BulkLoadHFileReque" +
|
||||
"st\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" +
|
||||
"ifier\022>\n\013family_path\030\002 \003(\0132).hbase.pb.Bu" +
|
||||
"lkLoadHFileRequest.FamilyPath\022\026\n\016assign_" +
|
||||
"seq_num\030\003 \001(\010\022+\n\010fs_token\030\004 \001(\0132\031.hbase.",
|
||||
"pb.DelegationToken\022\022\n\nbulk_token\030\005 \001(\t\022\030" +
|
||||
"\n\tcopy_file\030\006 \001(\010:\005false\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\"V\n\017Delegat" +
|
||||
"ionToken\022\022\n\nidentifier\030\001 \001(\014\022\020\n\010password" +
|
||||
"\030\002 \001(\014\022\014\n\004kind\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n" +
|
||||
"\026PrepareBulkLoadRequest\022\'\n\ntable_name\030\001 " +
|
||||
"\002(\0132\023.hbase.pb.TableName\022)\n\006region\030\002 \001(\013" +
|
||||
"2\031.hbase.pb.RegionSpecifier\"-\n\027PrepareBu" +
|
||||
"lkLoadResponse\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026Cl",
|
||||
"eanupBulkLoadRequest\022\022\n\nbulk_token\030\001 \002(\t" +
|
||||
"\022)\n\006region\030\002 \001(\0132\031.hbase.pb.RegionSpecif" +
|
||||
"ier\"\031\n\027CleanupBulkLoadResponse\"a\n\026Coproc" +
|
||||
"essorServiceCall\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\007requ" +
|
||||
"est\030\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n" +
|
||||
"\005value\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n" +
|
||||
"\031CoprocessorServiceRequest\022)\n\006region\030\001 \002" +
|
||||
"(\0132\031.hbase.pb.RegionSpecifier\022.\n\004call\030\002 " +
|
||||
"\002(\0132 .hbase.pb.CoprocessorServiceCall\"o\n",
|
||||
"\032CoprocessorServiceResponse\022)\n\006region\030\001 " +
|
||||
"\002(\0132\031.hbase.pb.RegionSpecifier\022&\n\005value\030" +
|
||||
"\002 \002(\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Actio" +
|
||||
"n\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hba" +
|
||||
"se.pb.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase" +
|
||||
".pb.Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb" +
|
||||
".CoprocessorServiceCall\"k\n\014RegionAction\022" +
|
||||
")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
|
||||
"er\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hba" +
|
||||
"se.pb.Action\"c\n\017RegionLoadStats\022\027\n\014memst",
|
||||
"oreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:" +
|
||||
"\0010\022\035\n\022compactionPressure\030\003 \001(\005:\0010\"j\n\024Mul" +
|
||||
"tiRegionLoadStats\022)\n\006region\030\001 \003(\0132\031.hbas" +
|
||||
"e.pb.RegionSpecifier\022\'\n\004stat\030\002 \003(\0132\031.hba" +
|
||||
"se.pb.RegionLoadStats\"\336\001\n\021ResultOrExcept" +
|
||||
"ion\022\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hba" +
|
||||
"se.pb.Result\022*\n\texception\030\003 \001(\0132\027.hbase." +
|
||||
"pb.NameBytesPair\022:\n\016service_result\030\004 \001(\013" +
|
||||
"2\".hbase.pb.CoprocessorServiceResult\0220\n\t" +
|
||||
"loadStats\030\005 \001(\0132\031.hbase.pb.RegionLoadSta",
|
||||
"tsB\002\030\001\"x\n\022RegionActionResult\0226\n\021resultOr" +
|
||||
"Exception\030\001 \003(\0132\033.hbase.pb.ResultOrExcep" +
|
||||
"tion\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameB" +
|
||||
"ytesPair\"x\n\014MultiRequest\022,\n\014regionAction" +
|
||||
"\030\001 \003(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceG" +
|
||||
"roup\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb" +
|
||||
".Condition\"\226\001\n\rMultiResponse\0228\n\022regionAc" +
|
||||
"tionResult\030\001 \003(\0132\034.hbase.pb.RegionAction" +
|
||||
"Result\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regionStati" +
|
||||
"stics\030\003 \001(\0132\036.hbase.pb.MultiRegionLoadSt",
|
||||
"ats*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELI" +
|
||||
"NE\020\0012\263\005\n\rClientService\0222\n\003Get\022\024.hbase.pb" +
|
||||
".GetRequest\032\025.hbase.pb.GetResponse\022;\n\006Mu" +
|
||||
"tate\022\027.hbase.pb.MutateRequest\032\030.hbase.pb" +
|
||||
".MutateResponse\0225\n\004Scan\022\025.hbase.pb.ScanR" +
|
||||
"equest\032\026.hbase.pb.ScanResponse\022P\n\rBulkLo" +
|
||||
"adHFile\022\036.hbase.pb.BulkLoadHFileRequest\032" +
|
||||
"\037.hbase.pb.BulkLoadHFileResponse\022V\n\017Prep" +
|
||||
"areBulkLoad\022 .hbase.pb.PrepareBulkLoadRe" +
|
||||
"quest\032!.hbase.pb.PrepareBulkLoadResponse",
|
||||
"\022V\n\017CleanupBulkLoad\022 .hbase.pb.CleanupBu" +
|
||||
"lkLoadRequest\032!.hbase.pb.CleanupBulkLoad" +
|
||||
"Response\022X\n\013ExecService\022#.hbase.pb.Copro" +
|
||||
"cessorServiceRequest\032$.hbase.pb.Coproces" +
|
||||
"sorServiceResponse\022d\n\027ExecRegionServerSe" +
|
||||
"rvice\022#.hbase.pb.CoprocessorServiceReque" +
|
||||
"st\032$.hbase.pb.CoprocessorServiceResponse" +
|
||||
"\0228\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbas" +
|
||||
"e.pb.MultiResponseBI\n1org.apache.hadoop." +
|
||||
"hbase.shaded.protobuf.generatedB\014ClientP",
|
||||
"rotosH\001\210\001\001\240\001\001"
|
||||
" \001(\004:\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true" +
|
||||
"\022\037\n\020include_stop_row\030\026 \001(\010:\005false\"\246\002\n\013Sc" +
|
||||
"anRequest\022)\n\006region\030\001 \001(\0132\031.hbase.pb.Reg" +
|
||||
"ionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hbase.pb.Sc" +
|
||||
"an\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\rnext_ca" +
|
||||
"ll_seq\030\006 \001(\004\022\037\n\027client_handles_partials\030" +
|
||||
"\007 \001(\010\022!\n\031client_handles_heartbeats\030\010 \001(\010" +
|
||||
"\022\032\n\022track_scan_metrics\030\t \001(\010\022\024\n\005renew\030\n " +
|
||||
"\001(\010:\005false\"\266\002\n\014ScanResponse\022\030\n\020cells_per",
|
||||
"_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mor" +
|
||||
"e_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030" +
|
||||
"\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001(\010\022\037" +
|
||||
"\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026more_" +
|
||||
"results_in_region\030\010 \001(\010\022\031\n\021heartbeat_mes" +
|
||||
"sage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbase" +
|
||||
".pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(\004" +
|
||||
":\0010\"\240\002\n\024BulkLoadHFileRequest\022)\n\006region\030\001" +
|
||||
" \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013famil" +
|
||||
"y_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReq",
|
||||
"uest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\022" +
|
||||
"+\n\010fs_token\030\004 \001(\0132\031.hbase.pb.DelegationT" +
|
||||
"oken\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tcopy_file\030\006 " +
|
||||
"\001(\010:\005false\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014" +
|
||||
"\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022" +
|
||||
"\016\n\006loaded\030\001 \002(\010\"V\n\017DelegationToken\022\022\n\nid" +
|
||||
"entifier\030\001 \001(\014\022\020\n\010password\030\002 \001(\014\022\014\n\004kind" +
|
||||
"\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026PrepareBulkLo" +
|
||||
"adRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb" +
|
||||
".TableName\022)\n\006region\030\002 \001(\0132\031.hbase.pb.Re",
|
||||
"gionSpecifier\"-\n\027PrepareBulkLoadResponse" +
|
||||
"\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026CleanupBulkLoadR" +
|
||||
"equest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006region\030\002 \001" +
|
||||
"(\0132\031.hbase.pb.RegionSpecifier\"\031\n\027Cleanup" +
|
||||
"BulkLoadResponse\"a\n\026CoprocessorServiceCa" +
|
||||
"ll\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\"B\n\030C" +
|
||||
"oprocessorServiceResult\022&\n\005value\030\001 \001(\0132\027" +
|
||||
".hbase.pb.NameBytesPair\"v\n\031CoprocessorSe" +
|
||||
"rviceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.",
|
||||
"RegionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb" +
|
||||
".CoprocessorServiceCall\"o\n\032CoprocessorSe" +
|
||||
"rviceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb" +
|
||||
".RegionSpecifier\022&\n\005value\030\002 \002(\0132\027.hbase." +
|
||||
"pb.NameBytesPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001" +
|
||||
"(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.pb.Mutation" +
|
||||
"Proto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014ser" +
|
||||
"vice_call\030\004 \001(\0132 .hbase.pb.CoprocessorSe" +
|
||||
"rviceCall\"k\n\014RegionAction\022)\n\006region\030\001 \002(" +
|
||||
"\0132\031.hbase.pb.RegionSpecifier\022\016\n\006atomic\030\002",
|
||||
" \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Action\"c" +
|
||||
"\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:" +
|
||||
"\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compacti" +
|
||||
"onPressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadSt" +
|
||||
"ats\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpe" +
|
||||
"cifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLo" +
|
||||
"adStats\"\336\001\n\021ResultOrException\022\r\n\005index\030\001" +
|
||||
" \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*" +
|
||||
"\n\texception\030\003 \001(\0132\027.hbase.pb.NameBytesPa" +
|
||||
"ir\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Co",
|
||||
"processorServiceResult\0220\n\tloadStats\030\005 \001(" +
|
||||
"\0132\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Regi" +
|
||||
"onActionResult\0226\n\021resultOrException\030\001 \003(" +
|
||||
"\0132\033.hbase.pb.ResultOrException\022*\n\texcept" +
|
||||
"ion\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mu" +
|
||||
"ltiRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase" +
|
||||
".pb.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\t" +
|
||||
"condition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n" +
|
||||
"\rMultiResponse\0228\n\022regionActionResult\030\001 \003" +
|
||||
"(\0132\034.hbase.pb.RegionActionResult\022\021\n\tproc",
|
||||
"essed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036." +
|
||||
"hbase.pb.MultiRegionLoadStats*\'\n\013Consist" +
|
||||
"ency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\263\005\n\rClien" +
|
||||
"tService\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025." +
|
||||
"hbase.pb.GetResponse\022;\n\006Mutate\022\027.hbase.p" +
|
||||
"b.MutateRequest\032\030.hbase.pb.MutateRespons" +
|
||||
"e\0225\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase" +
|
||||
".pb.ScanResponse\022P\n\rBulkLoadHFile\022\036.hbas" +
|
||||
"e.pb.BulkLoadHFileRequest\032\037.hbase.pb.Bul" +
|
||||
"kLoadHFileResponse\022V\n\017PrepareBulkLoad\022 .",
|
||||
"hbase.pb.PrepareBulkLoadRequest\032!.hbase." +
|
||||
"pb.PrepareBulkLoadResponse\022V\n\017CleanupBul" +
|
||||
"kLoad\022 .hbase.pb.CleanupBulkLoadRequest\032" +
|
||||
"!.hbase.pb.CleanupBulkLoadResponse\022X\n\013Ex" +
|
||||
"ecService\022#.hbase.pb.CoprocessorServiceR" +
|
||||
"equest\032$.hbase.pb.CoprocessorServiceResp" +
|
||||
"onse\022d\n\027ExecRegionServerService\022#.hbase." +
|
||||
"pb.CoprocessorServiceRequest\032$.hbase.pb." +
|
||||
"CoprocessorServiceResponse\0228\n\005Multi\022\026.hb" +
|
||||
"ase.pb.MultiRequest\032\027.hbase.pb.MultiResp",
|
||||
"onseBI\n1org.apache.hadoop.hbase.shaded.p" +
|
||||
"rotobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
|
||||
};
|
||||
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() {
|
||||
|
@ -40865,7 +41042,7 @@ public final class ClientProtos {
|
|||
internal_static_hbase_pb_Scan_fieldAccessorTable = new
|
||||
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
|
||||
internal_static_hbase_pb_Scan_descriptor,
|
||||
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", });
|
||||
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", "IncludeStartRow", "IncludeStopRow", });
|
||||
internal_static_hbase_pb_ScanRequest_descriptor =
|
||||
getDescriptor().getMessageTypes().get(12);
|
||||
internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
|
||||
|
|
|
@ -256,6 +256,8 @@ message Scan {
|
|||
optional bool allow_partial_results = 18;
|
||||
repeated ColumnFamilyTimeRange cf_time_range = 19;
|
||||
optional uint64 mvcc_read_point = 20 [default = 0];
|
||||
optional bool include_start_row = 21 [default = true];
|
||||
optional bool include_stop_row = 22 [default = false];
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -14229,6 +14229,26 @@ public final class ClientProtos {
|
|||
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
|
||||
*/
|
||||
long getMvccReadPoint();
|
||||
|
||||
// optional bool include_start_row = 21 [default = true];
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
boolean hasIncludeStartRow();
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
boolean getIncludeStartRow();
|
||||
|
||||
// optional bool include_stop_row = 22 [default = false];
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
boolean hasIncludeStopRow();
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
boolean getIncludeStopRow();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code hbase.pb.Scan}
|
||||
|
@ -14423,6 +14443,16 @@ public final class ClientProtos {
|
|||
mvccReadPoint_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
case 168: {
|
||||
bitField0_ |= 0x00020000;
|
||||
includeStartRow_ = input.readBool();
|
||||
break;
|
||||
}
|
||||
case 176: {
|
||||
bitField0_ |= 0x00040000;
|
||||
includeStopRow_ = input.readBool();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||
|
@ -14872,6 +14902,38 @@ public final class ClientProtos {
|
|||
return mvccReadPoint_;
|
||||
}
|
||||
|
||||
// optional bool include_start_row = 21 [default = true];
|
||||
public static final int INCLUDE_START_ROW_FIELD_NUMBER = 21;
|
||||
private boolean includeStartRow_;
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean hasIncludeStartRow() {
|
||||
return ((bitField0_ & 0x00020000) == 0x00020000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean getIncludeStartRow() {
|
||||
return includeStartRow_;
|
||||
}
|
||||
|
||||
// optional bool include_stop_row = 22 [default = false];
|
||||
public static final int INCLUDE_STOP_ROW_FIELD_NUMBER = 22;
|
||||
private boolean includeStopRow_;
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean hasIncludeStopRow() {
|
||||
return ((bitField0_ & 0x00040000) == 0x00040000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean getIncludeStopRow() {
|
||||
return includeStopRow_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
column_ = java.util.Collections.emptyList();
|
||||
attribute_ = java.util.Collections.emptyList();
|
||||
|
@ -14893,6 +14955,8 @@ public final class ClientProtos {
|
|||
allowPartialResults_ = false;
|
||||
cfTimeRange_ = java.util.Collections.emptyList();
|
||||
mvccReadPoint_ = 0L;
|
||||
includeStartRow_ = true;
|
||||
includeStopRow_ = false;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -14990,6 +15054,12 @@ public final class ClientProtos {
|
|||
if (((bitField0_ & 0x00010000) == 0x00010000)) {
|
||||
output.writeUInt64(20, mvccReadPoint_);
|
||||
}
|
||||
if (((bitField0_ & 0x00020000) == 0x00020000)) {
|
||||
output.writeBool(21, includeStartRow_);
|
||||
}
|
||||
if (((bitField0_ & 0x00040000) == 0x00040000)) {
|
||||
output.writeBool(22, includeStopRow_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -15079,6 +15149,14 @@ public final class ClientProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(20, mvccReadPoint_);
|
||||
}
|
||||
if (((bitField0_ & 0x00020000) == 0x00020000)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBoolSize(21, includeStartRow_);
|
||||
}
|
||||
if (((bitField0_ & 0x00040000) == 0x00040000)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBoolSize(22, includeStopRow_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -15193,6 +15271,16 @@ public final class ClientProtos {
|
|||
result = result && (getMvccReadPoint()
|
||||
== other.getMvccReadPoint());
|
||||
}
|
||||
result = result && (hasIncludeStartRow() == other.hasIncludeStartRow());
|
||||
if (hasIncludeStartRow()) {
|
||||
result = result && (getIncludeStartRow()
|
||||
== other.getIncludeStartRow());
|
||||
}
|
||||
result = result && (hasIncludeStopRow() == other.hasIncludeStopRow());
|
||||
if (hasIncludeStopRow()) {
|
||||
result = result && (getIncludeStopRow()
|
||||
== other.getIncludeStopRow());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -15286,6 +15374,14 @@ public final class ClientProtos {
|
|||
hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getMvccReadPoint());
|
||||
}
|
||||
if (hasIncludeStartRow()) {
|
||||
hash = (37 * hash) + INCLUDE_START_ROW_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashBoolean(getIncludeStartRow());
|
||||
}
|
||||
if (hasIncludeStopRow()) {
|
||||
hash = (37 * hash) + INCLUDE_STOP_ROW_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashBoolean(getIncludeStopRow());
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
memoizedHashCode = hash;
|
||||
return hash;
|
||||
|
@ -15471,6 +15567,10 @@ public final class ClientProtos {
|
|||
}
|
||||
mvccReadPoint_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00080000);
|
||||
includeStartRow_ = true;
|
||||
bitField0_ = (bitField0_ & ~0x00100000);
|
||||
includeStopRow_ = false;
|
||||
bitField0_ = (bitField0_ & ~0x00200000);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -15602,6 +15702,14 @@ public final class ClientProtos {
|
|||
to_bitField0_ |= 0x00010000;
|
||||
}
|
||||
result.mvccReadPoint_ = mvccReadPoint_;
|
||||
if (((from_bitField0_ & 0x00100000) == 0x00100000)) {
|
||||
to_bitField0_ |= 0x00020000;
|
||||
}
|
||||
result.includeStartRow_ = includeStartRow_;
|
||||
if (((from_bitField0_ & 0x00200000) == 0x00200000)) {
|
||||
to_bitField0_ |= 0x00040000;
|
||||
}
|
||||
result.includeStopRow_ = includeStopRow_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -15747,6 +15855,12 @@ public final class ClientProtos {
|
|||
if (other.hasMvccReadPoint()) {
|
||||
setMvccReadPoint(other.getMvccReadPoint());
|
||||
}
|
||||
if (other.hasIncludeStartRow()) {
|
||||
setIncludeStartRow(other.getIncludeStartRow());
|
||||
}
|
||||
if (other.hasIncludeStopRow()) {
|
||||
setIncludeStopRow(other.getIncludeStopRow());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -17272,6 +17386,72 @@ public final class ClientProtos {
|
|||
return this;
|
||||
}
|
||||
|
||||
// optional bool include_start_row = 21 [default = true];
|
||||
private boolean includeStartRow_ = true;
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean hasIncludeStartRow() {
|
||||
return ((bitField0_ & 0x00100000) == 0x00100000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public boolean getIncludeStartRow() {
|
||||
return includeStartRow_;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public Builder setIncludeStartRow(boolean value) {
|
||||
bitField0_ |= 0x00100000;
|
||||
includeStartRow_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_start_row = 21 [default = true];</code>
|
||||
*/
|
||||
public Builder clearIncludeStartRow() {
|
||||
bitField0_ = (bitField0_ & ~0x00100000);
|
||||
includeStartRow_ = true;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// optional bool include_stop_row = 22 [default = false];
|
||||
private boolean includeStopRow_ ;
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean hasIncludeStopRow() {
|
||||
return ((bitField0_ & 0x00200000) == 0x00200000);
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public boolean getIncludeStopRow() {
|
||||
return includeStopRow_;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public Builder setIncludeStopRow(boolean value) {
|
||||
bitField0_ |= 0x00200000;
|
||||
includeStopRow_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional bool include_stop_row = 22 [default = false];</code>
|
||||
*/
|
||||
public Builder clearIncludeStopRow() {
|
||||
bitField0_ = (bitField0_ & ~0x00200000);
|
||||
includeStopRow_ = false;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:hbase.pb.Scan)
|
||||
}
|
||||
|
||||
|
@ -39732,7 +39912,7 @@ public final class ClientProtos {
|
|||
"tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" +
|
||||
"_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" +
|
||||
"\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " +
|
||||
"\001(\010\"\331\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
|
||||
"\001(\010\"\233\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
|
||||
"Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" +
|
||||
"eBytesPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_ro" +
|
||||
"w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" +
|
||||
|
@ -39747,97 +39927,98 @@ public final class ClientProtos {
|
|||
"aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " +
|
||||
"\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" +
|
||||
"lumnFamilyTimeRange\022\032\n\017mvcc_read_point\030\024",
|
||||
" \001(\004:\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132" +
|
||||
"\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\013" +
|
||||
"2\016.hbase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016" +
|
||||
"number_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 " +
|
||||
"\001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_han" +
|
||||
"dles_partials\030\007 \001(\010\022!\n\031client_handles_he" +
|
||||
"artbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001" +
|
||||
"(\010\022\024\n\005renew\030\n \001(\010:\005false\"\266\002\n\014ScanRespons" +
|
||||
"e\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_i" +
|
||||
"d\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001",
|
||||
"(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n" +
|
||||
"\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_result\030" +
|
||||
"\007 \003(\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n" +
|
||||
"\021heartbeat_message\030\t \001(\010\022+\n\014scan_metrics" +
|
||||
"\030\n \001(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_re" +
|
||||
"ad_point\030\013 \001(\004:\0010\"\240\002\n\024BulkLoadHFileReque" +
|
||||
"st\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" +
|
||||
"ifier\022>\n\013family_path\030\002 \003(\0132).hbase.pb.Bu" +
|
||||
"lkLoadHFileRequest.FamilyPath\022\026\n\016assign_" +
|
||||
"seq_num\030\003 \001(\010\022+\n\010fs_token\030\004 \001(\0132\031.hbase.",
|
||||
"pb.DelegationToken\022\022\n\nbulk_token\030\005 \001(\t\022\030" +
|
||||
"\n\tcopy_file\030\006 \001(\010:\005false\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\"V\n\017Delegat" +
|
||||
"ionToken\022\022\n\nidentifier\030\001 \001(\014\022\020\n\010password" +
|
||||
"\030\002 \001(\014\022\014\n\004kind\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n" +
|
||||
"\026PrepareBulkLoadRequest\022\'\n\ntable_name\030\001 " +
|
||||
"\002(\0132\023.hbase.pb.TableName\022)\n\006region\030\002 \001(\013" +
|
||||
"2\031.hbase.pb.RegionSpecifier\"-\n\027PrepareBu" +
|
||||
"lkLoadResponse\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026Cl",
|
||||
"eanupBulkLoadRequest\022\022\n\nbulk_token\030\001 \002(\t" +
|
||||
"\022)\n\006region\030\002 \001(\0132\031.hbase.pb.RegionSpecif" +
|
||||
"ier\"\031\n\027CleanupBulkLoadResponse\"a\n\026Coproc" +
|
||||
"essorServiceCall\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\007requ" +
|
||||
"est\030\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n" +
|
||||
"\005value\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n" +
|
||||
"\031CoprocessorServiceRequest\022)\n\006region\030\001 \002" +
|
||||
"(\0132\031.hbase.pb.RegionSpecifier\022.\n\004call\030\002 " +
|
||||
"\002(\0132 .hbase.pb.CoprocessorServiceCall\"o\n",
|
||||
"\032CoprocessorServiceResponse\022)\n\006region\030\001 " +
|
||||
"\002(\0132\031.hbase.pb.RegionSpecifier\022&\n\005value\030" +
|
||||
"\002 \002(\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Actio" +
|
||||
"n\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hba" +
|
||||
"se.pb.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase" +
|
||||
".pb.Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb" +
|
||||
".CoprocessorServiceCall\"k\n\014RegionAction\022" +
|
||||
")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
|
||||
"er\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hba" +
|
||||
"se.pb.Action\"c\n\017RegionLoadStats\022\027\n\014memst",
|
||||
"oreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:" +
|
||||
"\0010\022\035\n\022compactionPressure\030\003 \001(\005:\0010\"j\n\024Mul" +
|
||||
"tiRegionLoadStats\022)\n\006region\030\001 \003(\0132\031.hbas" +
|
||||
"e.pb.RegionSpecifier\022\'\n\004stat\030\002 \003(\0132\031.hba" +
|
||||
"se.pb.RegionLoadStats\"\336\001\n\021ResultOrExcept" +
|
||||
"ion\022\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hba" +
|
||||
"se.pb.Result\022*\n\texception\030\003 \001(\0132\027.hbase." +
|
||||
"pb.NameBytesPair\022:\n\016service_result\030\004 \001(\013" +
|
||||
"2\".hbase.pb.CoprocessorServiceResult\0220\n\t" +
|
||||
"loadStats\030\005 \001(\0132\031.hbase.pb.RegionLoadSta",
|
||||
"tsB\002\030\001\"x\n\022RegionActionResult\0226\n\021resultOr" +
|
||||
"Exception\030\001 \003(\0132\033.hbase.pb.ResultOrExcep" +
|
||||
"tion\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameB" +
|
||||
"ytesPair\"x\n\014MultiRequest\022,\n\014regionAction" +
|
||||
"\030\001 \003(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceG" +
|
||||
"roup\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb" +
|
||||
".Condition\"\226\001\n\rMultiResponse\0228\n\022regionAc" +
|
||||
"tionResult\030\001 \003(\0132\034.hbase.pb.RegionAction" +
|
||||
"Result\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regionStati" +
|
||||
"stics\030\003 \001(\0132\036.hbase.pb.MultiRegionLoadSt",
|
||||
"ats*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELI" +
|
||||
"NE\020\0012\263\005\n\rClientService\0222\n\003Get\022\024.hbase.pb" +
|
||||
".GetRequest\032\025.hbase.pb.GetResponse\022;\n\006Mu" +
|
||||
"tate\022\027.hbase.pb.MutateRequest\032\030.hbase.pb" +
|
||||
".MutateResponse\0225\n\004Scan\022\025.hbase.pb.ScanR" +
|
||||
"equest\032\026.hbase.pb.ScanResponse\022P\n\rBulkLo" +
|
||||
"adHFile\022\036.hbase.pb.BulkLoadHFileRequest\032" +
|
||||
"\037.hbase.pb.BulkLoadHFileResponse\022V\n\017Prep" +
|
||||
"areBulkLoad\022 .hbase.pb.PrepareBulkLoadRe" +
|
||||
"quest\032!.hbase.pb.PrepareBulkLoadResponse",
|
||||
"\022V\n\017CleanupBulkLoad\022 .hbase.pb.CleanupBu" +
|
||||
"lkLoadRequest\032!.hbase.pb.CleanupBulkLoad" +
|
||||
"Response\022X\n\013ExecService\022#.hbase.pb.Copro" +
|
||||
"cessorServiceRequest\032$.hbase.pb.Coproces" +
|
||||
"sorServiceResponse\022d\n\027ExecRegionServerSe" +
|
||||
"rvice\022#.hbase.pb.CoprocessorServiceReque" +
|
||||
"st\032$.hbase.pb.CoprocessorServiceResponse" +
|
||||
"\0228\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbas" +
|
||||
"e.pb.MultiResponseBB\n*org.apache.hadoop." +
|
||||
"hbase.protobuf.generatedB\014ClientProtosH\001",
|
||||
"\210\001\001\240\001\001"
|
||||
" \001(\004:\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true" +
|
||||
"\022\037\n\020include_stop_row\030\026 \001(\010:\005false\"\246\002\n\013Sc" +
|
||||
"anRequest\022)\n\006region\030\001 \001(\0132\031.hbase.pb.Reg" +
|
||||
"ionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hbase.pb.Sc" +
|
||||
"an\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\rnext_ca" +
|
||||
"ll_seq\030\006 \001(\004\022\037\n\027client_handles_partials\030" +
|
||||
"\007 \001(\010\022!\n\031client_handles_heartbeats\030\010 \001(\010" +
|
||||
"\022\032\n\022track_scan_metrics\030\t \001(\010\022\024\n\005renew\030\n " +
|
||||
"\001(\010:\005false\"\266\002\n\014ScanResponse\022\030\n\020cells_per",
|
||||
"_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mor" +
|
||||
"e_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030" +
|
||||
"\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001(\010\022\037" +
|
||||
"\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026more_" +
|
||||
"results_in_region\030\010 \001(\010\022\031\n\021heartbeat_mes" +
|
||||
"sage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbase" +
|
||||
".pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(\004" +
|
||||
":\0010\"\240\002\n\024BulkLoadHFileRequest\022)\n\006region\030\001" +
|
||||
" \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013famil" +
|
||||
"y_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReq",
|
||||
"uest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\022" +
|
||||
"+\n\010fs_token\030\004 \001(\0132\031.hbase.pb.DelegationT" +
|
||||
"oken\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tcopy_file\030\006 " +
|
||||
"\001(\010:\005false\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014" +
|
||||
"\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022" +
|
||||
"\016\n\006loaded\030\001 \002(\010\"V\n\017DelegationToken\022\022\n\nid" +
|
||||
"entifier\030\001 \001(\014\022\020\n\010password\030\002 \001(\014\022\014\n\004kind" +
|
||||
"\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026PrepareBulkLo" +
|
||||
"adRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb" +
|
||||
".TableName\022)\n\006region\030\002 \001(\0132\031.hbase.pb.Re",
|
||||
"gionSpecifier\"-\n\027PrepareBulkLoadResponse" +
|
||||
"\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026CleanupBulkLoadR" +
|
||||
"equest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006region\030\002 \001" +
|
||||
"(\0132\031.hbase.pb.RegionSpecifier\"\031\n\027Cleanup" +
|
||||
"BulkLoadResponse\"a\n\026CoprocessorServiceCa" +
|
||||
"ll\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\"B\n\030C" +
|
||||
"oprocessorServiceResult\022&\n\005value\030\001 \001(\0132\027" +
|
||||
".hbase.pb.NameBytesPair\"v\n\031CoprocessorSe" +
|
||||
"rviceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.",
|
||||
"RegionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb" +
|
||||
".CoprocessorServiceCall\"o\n\032CoprocessorSe" +
|
||||
"rviceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb" +
|
||||
".RegionSpecifier\022&\n\005value\030\002 \002(\0132\027.hbase." +
|
||||
"pb.NameBytesPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001" +
|
||||
"(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.pb.Mutation" +
|
||||
"Proto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014ser" +
|
||||
"vice_call\030\004 \001(\0132 .hbase.pb.CoprocessorSe" +
|
||||
"rviceCall\"k\n\014RegionAction\022)\n\006region\030\001 \002(" +
|
||||
"\0132\031.hbase.pb.RegionSpecifier\022\016\n\006atomic\030\002",
|
||||
" \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Action\"c" +
|
||||
"\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:" +
|
||||
"\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compacti" +
|
||||
"onPressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadSt" +
|
||||
"ats\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpe" +
|
||||
"cifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLo" +
|
||||
"adStats\"\336\001\n\021ResultOrException\022\r\n\005index\030\001" +
|
||||
" \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*" +
|
||||
"\n\texception\030\003 \001(\0132\027.hbase.pb.NameBytesPa" +
|
||||
"ir\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Co",
|
||||
"processorServiceResult\0220\n\tloadStats\030\005 \001(" +
|
||||
"\0132\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Regi" +
|
||||
"onActionResult\0226\n\021resultOrException\030\001 \003(" +
|
||||
"\0132\033.hbase.pb.ResultOrException\022*\n\texcept" +
|
||||
"ion\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mu" +
|
||||
"ltiRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase" +
|
||||
".pb.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\t" +
|
||||
"condition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n" +
|
||||
"\rMultiResponse\0228\n\022regionActionResult\030\001 \003" +
|
||||
"(\0132\034.hbase.pb.RegionActionResult\022\021\n\tproc",
|
||||
"essed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036." +
|
||||
"hbase.pb.MultiRegionLoadStats*\'\n\013Consist" +
|
||||
"ency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\263\005\n\rClien" +
|
||||
"tService\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025." +
|
||||
"hbase.pb.GetResponse\022;\n\006Mutate\022\027.hbase.p" +
|
||||
"b.MutateRequest\032\030.hbase.pb.MutateRespons" +
|
||||
"e\0225\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase" +
|
||||
".pb.ScanResponse\022P\n\rBulkLoadHFile\022\036.hbas" +
|
||||
"e.pb.BulkLoadHFileRequest\032\037.hbase.pb.Bul" +
|
||||
"kLoadHFileResponse\022V\n\017PrepareBulkLoad\022 .",
|
||||
"hbase.pb.PrepareBulkLoadRequest\032!.hbase." +
|
||||
"pb.PrepareBulkLoadResponse\022V\n\017CleanupBul" +
|
||||
"kLoad\022 .hbase.pb.CleanupBulkLoadRequest\032" +
|
||||
"!.hbase.pb.CleanupBulkLoadResponse\022X\n\013Ex" +
|
||||
"ecService\022#.hbase.pb.CoprocessorServiceR" +
|
||||
"equest\032$.hbase.pb.CoprocessorServiceResp" +
|
||||
"onse\022d\n\027ExecRegionServerService\022#.hbase." +
|
||||
"pb.CoprocessorServiceRequest\032$.hbase.pb." +
|
||||
"CoprocessorServiceResponse\0228\n\005Multi\022\026.hb" +
|
||||
"ase.pb.MultiRequest\032\027.hbase.pb.MultiResp",
|
||||
"onseBB\n*org.apache.hadoop.hbase.protobuf" +
|
||||
".generatedB\014ClientProtosH\001\210\001\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -39927,7 +40108,7 @@ public final class ClientProtos {
|
|||
internal_static_hbase_pb_Scan_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_hbase_pb_Scan_descriptor,
|
||||
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", });
|
||||
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", "IncludeStartRow", "IncludeStopRow", });
|
||||
internal_static_hbase_pb_ScanRequest_descriptor =
|
||||
getDescriptor().getMessageTypes().get(12);
|
||||
internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
|
||||
|
|
|
@ -256,6 +256,8 @@ message Scan {
|
|||
optional bool allow_partial_results = 18;
|
||||
repeated ColumnFamilyTimeRange cf_time_range = 19;
|
||||
optional uint64 mvcc_read_point = 20 [default = 0];
|
||||
optional bool include_start_row = 21 [default = true];
|
||||
optional bool include_stop_row = 22 [default = false];
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -179,8 +179,8 @@ class DefaultStoreFileManager implements StoreFileManager {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final Collection<StoreFile> getFilesForScanOrGet(boolean isGet,
|
||||
byte[] startRow, byte[] stopRow) {
|
||||
public final Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
|
||||
byte[] stopRow, boolean includeStopRow) {
|
||||
// We cannot provide any useful input and already have the files sorted by seqNum.
|
||||
return getStorefiles();
|
||||
}
|
||||
|
|
|
@ -5760,8 +5760,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
protected Cell joinedContinuationRow = null;
|
||||
private boolean filterClosed = false;
|
||||
|
||||
protected final int isScan;
|
||||
protected final byte[] stopRow;
|
||||
protected final boolean includeStopRow;
|
||||
protected final HRegion region;
|
||||
protected final CellComparator comparator;
|
||||
|
||||
|
@ -5797,15 +5797,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
*/
|
||||
defaultScannerContext = ScannerContext.newBuilder()
|
||||
.setBatchLimit(scan.getBatch()).build();
|
||||
|
||||
if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) && !scan.isGetScan()) {
|
||||
this.stopRow = null;
|
||||
} else {
|
||||
this.stopRow = scan.getStopRow();
|
||||
}
|
||||
// If we are doing a get, we want to be [startRow,endRow]. Normally
|
||||
// it is [startRow,endRow) and if startRow=endRow we get nothing.
|
||||
this.isScan = scan.isGetScan() ? 1 : 0;
|
||||
this.stopRow = scan.getStopRow();
|
||||
this.includeStopRow = scan.includeStopRow();
|
||||
|
||||
// synchronize on scannerReadPoints so that nobody calculates
|
||||
// getSmallestReadPoint, before scannerReadPoints is updated.
|
||||
|
@ -6118,7 +6111,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// Let's see what we have in the storeHeap.
|
||||
Cell current = this.storeHeap.peek();
|
||||
|
||||
boolean stopRow = isStopRow(current);
|
||||
boolean shouldStop = shouldStop(current);
|
||||
// When has filter row is true it means that the all the cells for a particular row must be
|
||||
// read before a filtering decision can be made. This means that filters where hasFilterRow
|
||||
// run the risk of enLongAddering out of memory errors in the case that they are applied to a
|
||||
|
@ -6142,7 +6135,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// 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 (shouldStop) {
|
||||
if (hasFilterRow) {
|
||||
filter.filterRowCells(results);
|
||||
}
|
||||
|
@ -6182,7 +6175,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
Cell nextKv = this.storeHeap.peek();
|
||||
stopRow = nextKv == null || isStopRow(nextKv);
|
||||
shouldStop = shouldStop(nextKv);
|
||||
// save that the row was empty before filters applied to it.
|
||||
final boolean isEmptyRow = results.isEmpty();
|
||||
|
||||
|
@ -6219,7 +6212,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
// 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;
|
||||
if (!shouldStop) continue;
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
|
@ -6260,10 +6253,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
if (!moreRows) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
if (!stopRow) continue;
|
||||
if (!shouldStop) continue;
|
||||
}
|
||||
|
||||
if (stopRow) {
|
||||
if (shouldStop) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
} else {
|
||||
return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
|
||||
|
@ -6343,10 +6336,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
.postScannerFilterRow(this, curRowCell);
|
||||
}
|
||||
|
||||
protected boolean isStopRow(Cell currentRowCell) {
|
||||
return currentRowCell == null
|
||||
|| (stopRow != null && comparator.compareRows(currentRowCell, stopRow, 0, stopRow
|
||||
.length) >= isScan);
|
||||
protected boolean shouldStop(Cell currentRowCell) {
|
||||
if (currentRowCell == null) {
|
||||
return true;
|
||||
}
|
||||
if (stopRow == null || Bytes.equals(stopRow, HConstants.EMPTY_END_ROW)) {
|
||||
return false;
|
||||
}
|
||||
int c = comparator.compareRows(currentRowCell, stopRow, 0, stopRow.length);
|
||||
return c > 0 || (c == 0 && !includeStopRow);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,6 +18,13 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -90,13 +97,6 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
|
|||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
/**
|
||||
* A Store holds a column family in a Region. Its a memstore and a set of zero
|
||||
* or more StoreFiles, which stretch backwards over time.
|
||||
|
@ -1138,20 +1138,19 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get all scanners with no filtering based on TTL (that happens further down
|
||||
* the line).
|
||||
* Get all scanners with no filtering based on TTL (that happens further down the line).
|
||||
* @return all scanners for this store
|
||||
*/
|
||||
@Override
|
||||
public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet,
|
||||
boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
|
||||
byte[] stopRow, long readPt) throws IOException {
|
||||
public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean usePread,
|
||||
boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow,
|
||||
byte[] stopRow, boolean includeStopRow, long readPt) throws IOException {
|
||||
Collection<StoreFile> storeFilesToScan;
|
||||
List<KeyValueScanner> memStoreScanners;
|
||||
this.lock.readLock().lock();
|
||||
try {
|
||||
storeFilesToScan =
|
||||
this.storeEngine.getStoreFileManager().getFilesForScanOrGet(isGet, startRow, stopRow);
|
||||
storeFilesToScan = this.storeEngine.getStoreFileManager().getFilesForScan(startRow,
|
||||
includeStartRow, stopRow, includeStopRow);
|
||||
memStoreScanners = this.memstore.getScanners(readPt);
|
||||
} finally {
|
||||
this.lock.readLock().unlock();
|
||||
|
@ -1163,9 +1162,8 @@ public class HStore implements Store {
|
|||
// but now we get them in ascending order, which I think is
|
||||
// actually more correct, since memstore get put at the end.
|
||||
List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(storeFilesToScan,
|
||||
cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
|
||||
List<KeyValueScanner> scanners =
|
||||
new ArrayList<KeyValueScanner>(sfScanners.size()+1);
|
||||
cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
|
||||
List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(sfScanners.size() + 1);
|
||||
scanners.addAll(sfScanners);
|
||||
// Then the memstore scanners
|
||||
scanners.addAll(memStoreScanners);
|
||||
|
@ -1174,8 +1172,9 @@ public class HStore implements Store {
|
|||
|
||||
@Override
|
||||
public List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
|
||||
boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
|
||||
byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException {
|
||||
boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
|
||||
boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
|
||||
boolean includeMemstoreScanner) throws IOException {
|
||||
List<KeyValueScanner> memStoreScanners = null;
|
||||
if (includeMemstoreScanner) {
|
||||
this.lock.readLock().lock();
|
||||
|
@ -2234,8 +2233,8 @@ public class HStore implements Store {
|
|||
if (LOG.isInfoEnabled()) {
|
||||
LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
|
||||
" added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
|
||||
", sequenceid=" + + storeFile.getReader().getSequenceID() +
|
||||
", filesize=" + StringUtils.humanReadableInt(storeFile.getReader().length()));
|
||||
", sequenceid=" + +storeFile.getReader().getSequenceID() + ", filesize="
|
||||
+ TraditionalBinaryPrefix.long2String(storeFile.getReader().length(), "", 1));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,9 +23,11 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.regionserver.HRegion.RegionScannerImpl;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* ReversibleRegionScannerImpl extends from RegionScannerImpl, and is used to
|
||||
|
@ -56,10 +58,15 @@ class ReversedRegionScannerImpl extends RegionScannerImpl {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected boolean isStopRow(Cell currentRowCell) {
|
||||
return currentRowCell == null
|
||||
|| (super.stopRow != null && comparator.compareRows(currentRowCell, stopRow, 0,
|
||||
stopRow.length) <= super.isScan);
|
||||
protected boolean shouldStop(Cell currentRowCell) {
|
||||
if (currentRowCell == null) {
|
||||
return true;
|
||||
}
|
||||
if (stopRow == null || Bytes.equals(stopRow, HConstants.EMPTY_START_ROW)) {
|
||||
return false;
|
||||
}
|
||||
int c = comparator.compareRows(currentRowCell, stopRow, 0, stopRow.length);
|
||||
return c < 0 || (c == 0 && !includeStopRow);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -81,35 +81,45 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get all scanners with no filtering based on TTL (that happens further down
|
||||
* the line).
|
||||
* @param cacheBlocks
|
||||
* @param isGet
|
||||
* @param usePread
|
||||
* @param isCompaction
|
||||
* @param matcher
|
||||
* @param startRow
|
||||
* @param stopRow
|
||||
* @param readPt
|
||||
* Get all scanners with no filtering based on TTL (that happens further down the line).
|
||||
* @param cacheBlocks cache the blocks or not
|
||||
* @param usePread true to use pread, false if not
|
||||
* @param isCompaction true if the scanner is created for compaction
|
||||
* @param matcher the scan query matcher
|
||||
* @param startRow the start row
|
||||
* @param stopRow the stop row
|
||||
* @param readPt the read point of the current scan
|
||||
* @return all scanners for this store
|
||||
*/
|
||||
List<KeyValueScanner> getScanners(
|
||||
boolean cacheBlocks,
|
||||
boolean isGet,
|
||||
boolean usePread,
|
||||
boolean isCompaction,
|
||||
ScanQueryMatcher matcher,
|
||||
byte[] startRow,
|
||||
byte[] stopRow,
|
||||
long readPt
|
||||
) throws IOException;
|
||||
default List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet, boolean usePread,
|
||||
boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt)
|
||||
throws IOException {
|
||||
return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false,
|
||||
readPt);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all scanners with no filtering based on TTL (that happens further down the line).
|
||||
* @param cacheBlocks cache the blocks or not
|
||||
* @param usePread true to use pread, false if not
|
||||
* @param isCompaction true if the scanner is created for compaction
|
||||
* @param matcher the scan query matcher
|
||||
* @param startRow the start row
|
||||
* @param includeStartRow true to include start row, false if not
|
||||
* @param stopRow the stop row
|
||||
* @param includeStopRow true to include stop row, false if not
|
||||
* @param readPt the read point of the current scan
|
||||
* @return all scanners for this store
|
||||
*/
|
||||
List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean usePread, boolean isCompaction,
|
||||
ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow,
|
||||
boolean includeStopRow, long readPt) throws IOException;
|
||||
|
||||
/**
|
||||
* Create scanners on the given files and if needed on the memstore with no filtering based on TTL
|
||||
* (that happens further down the line).
|
||||
* @param files the list of files on which the scanners has to be created
|
||||
* @param cacheBlocks cache the blocks or not
|
||||
* @param isGet true if it is get, false if not
|
||||
* @param usePread true to use pread, false if not
|
||||
* @param isCompaction true if the scanner is created for compaction
|
||||
* @param matcher the scan query matcher
|
||||
|
@ -119,9 +129,34 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
* @param includeMemstoreScanner true if memstore has to be included
|
||||
* @return scanners on the given files and on the memstore if specified
|
||||
*/
|
||||
List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean isGet,
|
||||
boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
|
||||
byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException;
|
||||
default List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
|
||||
boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
|
||||
byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner)
|
||||
throws IOException {
|
||||
return getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow,
|
||||
false, readPt, includeMemstoreScanner);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create scanners on the given files and if needed on the memstore with no filtering based on TTL
|
||||
* (that happens further down the line).
|
||||
* @param files the list of files on which the scanners has to be created
|
||||
* @param cacheBlocks ache the blocks or not
|
||||
* @param usePread true to use pread, false if not
|
||||
* @param isCompaction true if the scanner is created for compaction
|
||||
* @param matcher the scan query matcher
|
||||
* @param startRow the start row
|
||||
* @param includeStartRow true to include start row, false if not
|
||||
* @param stopRow the stop row
|
||||
* @param includeStopRow true to include stop row, false if not
|
||||
* @param readPt the read point of the current scan
|
||||
* @param includeMemstoreScanner true if memstore has to be included
|
||||
* @return scanners on the given files and on the memstore if specified
|
||||
*/
|
||||
List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean usePread,
|
||||
boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow,
|
||||
byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner)
|
||||
throws IOException;
|
||||
|
||||
ScanInfo getScanInfo();
|
||||
|
||||
|
|
|
@ -105,14 +105,12 @@ public interface StoreFileManager {
|
|||
|
||||
/**
|
||||
* Gets the store files to scan for a Scan or Get request.
|
||||
* @param isGet Whether it's a get.
|
||||
* @param startRow Start row of the request.
|
||||
* @param stopRow Stop row of the request.
|
||||
* @return The list of files that are to be read for this request.
|
||||
*/
|
||||
Collection<StoreFile> getFilesForScanOrGet(
|
||||
boolean isGet, byte[] startRow, byte[] stopRow
|
||||
);
|
||||
Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow,
|
||||
boolean includeStopRow);
|
||||
|
||||
/**
|
||||
* Gets initial, full list of candidate store files to check for row-key-before.
|
||||
|
|
|
@ -289,8 +289,8 @@ public class StripeStoreFileManager
|
|||
}
|
||||
|
||||
@Override
|
||||
public Collection<StoreFile> getFilesForScanOrGet(
|
||||
boolean isGet, byte[] startRow, byte[] stopRow) {
|
||||
public Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
|
||||
byte[] stopRow, boolean includeStopRow) {
|
||||
if (state.stripeFiles.isEmpty()) {
|
||||
return state.level0Files; // There's just L0.
|
||||
}
|
||||
|
|
|
@ -17,10 +17,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.querymatcher;
|
||||
|
||||
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
|
@ -45,7 +46,7 @@ public abstract class CompactionScanQueryMatcher extends ScanQueryMatcher {
|
|||
|
||||
protected CompactionScanQueryMatcher(ScanInfo scanInfo, DeleteTracker deletes,
|
||||
long readPointToUse, long oldestUnexpiredTS, long now) {
|
||||
super(HConstants.EMPTY_START_ROW, scanInfo,
|
||||
super(createStartKeyFromRow(EMPTY_START_ROW, scanInfo), scanInfo,
|
||||
new ScanWildcardColumnTracker(scanInfo.getMinVersions(), scanInfo.getMaxVersions(),
|
||||
oldestUnexpiredTS),
|
||||
oldestUnexpiredTS, now);
|
||||
|
|
|
@ -116,7 +116,8 @@ public class LegacyScanQueryMatcher extends ScanQueryMatcher {
|
|||
private LegacyScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns,
|
||||
boolean hasNullColumn, DeleteTracker deletes, ScanType scanType, long readPointToUse,
|
||||
long earliestPutTs, long oldestUnexpiredTS, long now) {
|
||||
super(scan.getStartRow(), scanInfo, columns, oldestUnexpiredTS, now);
|
||||
super(createStartKeyFromRow(scan.getStartRow(), scanInfo), scanInfo, columns, oldestUnexpiredTS,
|
||||
now);
|
||||
TimeRange timeRange = scan.getColumnFamilyTimeRange().get(scanInfo.getFamily());
|
||||
if (timeRange == null) {
|
||||
this.tr = scan.getTimeRange();
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
|||
* Query matcher for normal user scan.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class NormalUserScanQueryMatcher extends UserScanQueryMatcher {
|
||||
public abstract class NormalUserScanQueryMatcher extends UserScanQueryMatcher {
|
||||
|
||||
/** Keeps track of deletes */
|
||||
private final DeleteTracker deletes;
|
||||
|
@ -91,17 +91,45 @@ public class NormalUserScanQueryMatcher extends UserScanQueryMatcher {
|
|||
RegionCoprocessorHost regionCoprocessorHost) throws IOException {
|
||||
DeleteTracker deletes = instantiateDeleteTracker(regionCoprocessorHost);
|
||||
if (scan.isReversed()) {
|
||||
return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes,
|
||||
oldestUnexpiredTS, now) {
|
||||
if (scan.includeStopRow()) {
|
||||
return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes,
|
||||
oldestUnexpiredTS, now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow > 0;
|
||||
}
|
||||
};
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow >= 0;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes,
|
||||
oldestUnexpiredTS, now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow > 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
} else {
|
||||
return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes,
|
||||
oldestUnexpiredTS, now);
|
||||
if (scan.includeStopRow()) {
|
||||
return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes,
|
||||
oldestUnexpiredTS, now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow <= 0;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes,
|
||||
oldestUnexpiredTS, now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow < 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
|||
* Query matcher for raw scan.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RawScanQueryMatcher extends UserScanQueryMatcher {
|
||||
public abstract class RawScanQueryMatcher extends UserScanQueryMatcher {
|
||||
|
||||
protected RawScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns,
|
||||
boolean hasNullColumn, long oldestUnexpiredTS, long now) {
|
||||
|
@ -63,17 +63,45 @@ public class RawScanQueryMatcher extends UserScanQueryMatcher {
|
|||
public static RawScanQueryMatcher create(Scan scan, ScanInfo scanInfo, ColumnTracker columns,
|
||||
boolean hasNullColumn, long oldestUnexpiredTS, long now) {
|
||||
if (scan.isReversed()) {
|
||||
return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS,
|
||||
now) {
|
||||
if (scan.includeStopRow()) {
|
||||
return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS,
|
||||
now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow > 0;
|
||||
}
|
||||
};
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow >= 0;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS,
|
||||
now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow > 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
} else {
|
||||
return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS,
|
||||
now);
|
||||
if (scan.includeStopRow()) {
|
||||
return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS,
|
||||
now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow <= 0;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS,
|
||||
now) {
|
||||
|
||||
@Override
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow < 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -127,10 +127,10 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
|
||||
protected boolean stickyNextRow;
|
||||
|
||||
protected ScanQueryMatcher(byte[] startRow, ScanInfo scanInfo, ColumnTracker columns,
|
||||
protected ScanQueryMatcher(Cell startKey, ScanInfo scanInfo, ColumnTracker columns,
|
||||
long oldestUnexpiredTS, long now) {
|
||||
this.rowComparator = scanInfo.getComparator();
|
||||
this.startKey = CellUtil.createFirstDeleteFamilyCellOnRow(startRow, scanInfo.getFamily());
|
||||
this.startKey = startKey;
|
||||
this.oldestUnexpiredTS = oldestUnexpiredTS;
|
||||
this.now = now;
|
||||
this.columns = columns;
|
||||
|
@ -345,6 +345,10 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
}
|
||||
}
|
||||
|
||||
protected static Cell createStartKeyFromRow(byte[] startRow, ScanInfo scanInfo) {
|
||||
return CellUtil.createFirstDeleteFamilyCellOnRow(startRow, scanInfo.getFamily());
|
||||
}
|
||||
|
||||
protected static DeleteTracker instantiateDeleteTracker(RegionCoprocessorHost host)
|
||||
throws IOException {
|
||||
DeleteTracker tracker = new ScanDeleteTracker();
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.NavigableSet;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
|
@ -50,9 +51,17 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
|
|||
|
||||
protected final TimeRange tr;
|
||||
|
||||
private static Cell createStartKey(Scan scan, ScanInfo scanInfo) {
|
||||
if (scan.includeStartRow()) {
|
||||
return createStartKeyFromRow(scan.getStartRow(), scanInfo);
|
||||
} else {
|
||||
return CellUtil.createLastOnRow(scan.getStartRow());
|
||||
}
|
||||
}
|
||||
|
||||
protected UserScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns,
|
||||
boolean hasNullColumn, long oldestUnexpiredTS, long now) {
|
||||
super(scan.getStartRow(), scanInfo, columns, oldestUnexpiredTS, now);
|
||||
super(createStartKey(scan, scanInfo), scanInfo, columns, oldestUnexpiredTS, now);
|
||||
this.hasNullColumn = hasNullColumn;
|
||||
this.filter = scan.getFilter();
|
||||
this.stopRow = scan.getStopRow();
|
||||
|
@ -163,9 +172,7 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
|
|||
|
||||
protected abstract boolean isGet();
|
||||
|
||||
protected boolean moreRowsMayExistsAfter(int cmpToStopRow) {
|
||||
return cmpToStopRow < 0;
|
||||
}
|
||||
protected abstract boolean moreRowsMayExistsAfter(int cmpToStopRow);
|
||||
|
||||
@Override
|
||||
public boolean moreRowsMayExistAfter(Cell cell) {
|
||||
|
|
|
@ -120,7 +120,7 @@ public abstract class AbstractTestAsyncTableScan {
|
|||
public void testScanNoStopKey() throws Exception {
|
||||
int start = 345;
|
||||
List<Result> results =
|
||||
doScan(createScan().setStartRow(Bytes.toBytes(String.format("%03d", start))));
|
||||
doScan(createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))));
|
||||
assertEquals(COUNT - start, results.size());
|
||||
IntStream.range(0, COUNT - start).forEach(i -> assertResultEquals(results.get(i), start + i));
|
||||
}
|
||||
|
@ -129,44 +129,66 @@ public abstract class AbstractTestAsyncTableScan {
|
|||
public void testReverseScanNoStopKey() throws Exception {
|
||||
int start = 765;
|
||||
List<Result> results = doScan(
|
||||
createScan().setStartRow(Bytes.toBytes(String.format("%03d", start))).setReversed(true));
|
||||
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))).setReversed(true));
|
||||
assertEquals(start + 1, results.size());
|
||||
IntStream.range(0, start + 1).forEach(i -> assertResultEquals(results.get(i), start - i));
|
||||
}
|
||||
|
||||
private void testScan(int start, int stop) throws Exception {
|
||||
List<Result> results =
|
||||
doScan(createScan().setStartRow(Bytes.toBytes(String.format("%03d", start)))
|
||||
.setStopRow(Bytes.toBytes(String.format("%03d", stop))));
|
||||
assertEquals(stop - start, results.size());
|
||||
IntStream.range(0, stop - start).forEach(i -> assertResultEquals(results.get(i), start + i));
|
||||
private void testScan(int start, boolean startInclusive, int stop, boolean stopInclusive)
|
||||
throws Exception {
|
||||
List<Result> results = doScan(
|
||||
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
|
||||
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive));
|
||||
int actualStart = startInclusive ? start : start + 1;
|
||||
int actualStop = stopInclusive ? stop + 1 : stop;
|
||||
assertEquals(actualStop - actualStart, results.size());
|
||||
IntStream.range(0, actualStop - actualStart)
|
||||
.forEach(i -> assertResultEquals(results.get(i), actualStart + i));
|
||||
}
|
||||
|
||||
private void testReversedScan(int start, int stop) throws Exception {
|
||||
List<Result> results =
|
||||
doScan(createScan().setStartRow(Bytes.toBytes(String.format("%03d", start)))
|
||||
.setStopRow(Bytes.toBytes(String.format("%03d", stop))).setReversed(true));
|
||||
assertEquals(start - stop, results.size());
|
||||
IntStream.range(0, start - stop).forEach(i -> assertResultEquals(results.get(i), start - i));
|
||||
private void testReversedScan(int start, boolean startInclusive, int stop, boolean stopInclusive)
|
||||
throws Exception {
|
||||
List<Result> results = doScan(createScan()
|
||||
.withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
|
||||
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive).setReversed(true));
|
||||
int actualStart = startInclusive ? start : start - 1;
|
||||
int actualStop = stopInclusive ? stop - 1 : stop;
|
||||
assertEquals(actualStart - actualStop, results.size());
|
||||
IntStream.range(0, actualStart - actualStop)
|
||||
.forEach(i -> assertResultEquals(results.get(i), actualStart - i));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScanWithStartKeyAndStopKey() throws Exception {
|
||||
testScan(345, 567);
|
||||
testScan(1, true, 998, false); // from first region to last region
|
||||
testScan(123, true, 345, true);
|
||||
testScan(234, true, 456, false);
|
||||
testScan(345, false, 567, true);
|
||||
testScan(456, false, 678, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReversedScanWithStartKeyAndStopKey() throws Exception {
|
||||
testReversedScan(765, 543);
|
||||
testReversedScan(998, true, 1, false); // from first region to first region
|
||||
testReversedScan(543, true, 321, true);
|
||||
testReversedScan(654, true, 432, false);
|
||||
testReversedScan(765, false, 543, true);
|
||||
testReversedScan(876, false, 654, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScanAtRegionBoundary() throws Exception {
|
||||
testScan(222, 333);
|
||||
testScan(222, true, 333, true);
|
||||
testScan(333, true, 444, false);
|
||||
testScan(444, false, 555, true);
|
||||
testScan(555, false, 666, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReversedScanAtRegionBoundary() throws Exception {
|
||||
testScan(222, 333);
|
||||
testReversedScan(333, true, 222, true);
|
||||
testReversedScan(444, true, 333, false);
|
||||
testReversedScan(555, false, 444, true);
|
||||
testReversedScan(666, false, 555, false);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
|
@ -92,13 +93,17 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
|
|||
}
|
||||
}
|
||||
|
||||
@Parameter
|
||||
@Parameter(0)
|
||||
public String scanType;
|
||||
|
||||
@Parameter(1)
|
||||
public Supplier<Scan> scanCreater;
|
||||
|
||||
@Parameters
|
||||
@Parameters(name = "{index}: type={0}")
|
||||
public static List<Object[]> params() {
|
||||
return Arrays.asList(new Supplier<?>[] { TestRawAsyncTableScan::createNormalScan },
|
||||
new Supplier<?>[] { TestRawAsyncTableScan::createBatchScan });
|
||||
Supplier<Scan> normal = TestRawAsyncTableScan::createNormalScan;
|
||||
Supplier<Scan> batch = TestRawAsyncTableScan::createBatchScan;
|
||||
return Arrays.asList(new Object[] { "normal", normal }, new Object[] { "batch", batch });
|
||||
}
|
||||
|
||||
private static Scan createNormalScan() {
|
||||
|
@ -117,7 +122,10 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
|
|||
@Override
|
||||
protected List<Result> doScan(Scan scan) throws Exception {
|
||||
SimpleRawScanResultConsumer scanConsumer = new SimpleRawScanResultConsumer();
|
||||
ASYNC_CONN.getRawTable(TABLE_NAME).scan(scan, scanConsumer);
|
||||
RawAsyncTable table = ASYNC_CONN.getRawTable(TABLE_NAME);
|
||||
table.setScanTimeout(1, TimeUnit.HOURS);
|
||||
table.setReadRpcTimeout(1, TimeUnit.HOURS);
|
||||
table.scan(scan, scanConsumer);
|
||||
List<Result> results = new ArrayList<>();
|
||||
for (Result result; (result = scanConsumer.take()) != null;) {
|
||||
results.add(result);
|
||||
|
|
|
@ -89,15 +89,15 @@ public class TestStripeStoreFileManager {
|
|||
MockStoreFile sf = createFile();
|
||||
manager.insertNewFiles(al(sf));
|
||||
assertEquals(1, manager.getStorefileCount());
|
||||
Collection<StoreFile> filesForGet = manager.getFilesForScanOrGet(true, KEY_A, KEY_A);
|
||||
Collection<StoreFile> filesForGet = manager.getFilesForScan(KEY_A, true, KEY_A, true);
|
||||
assertEquals(1, filesForGet.size());
|
||||
assertTrue(filesForGet.contains(sf));
|
||||
|
||||
// Add some stripes and make sure we get this file for every stripe.
|
||||
manager.addCompactionResults(al(), al(createFile(OPEN_KEY, KEY_B),
|
||||
createFile(KEY_B, OPEN_KEY)));
|
||||
assertTrue(manager.getFilesForScanOrGet(true, KEY_A, KEY_A).contains(sf));
|
||||
assertTrue(manager.getFilesForScanOrGet(true, KEY_C, KEY_C).contains(sf));
|
||||
assertTrue(manager.getFilesForScan(KEY_A, true, KEY_A, true).contains(sf));
|
||||
assertTrue(manager.getFilesForScan(KEY_C, true, KEY_C, true).contains(sf));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -290,10 +290,9 @@ public class TestStripeStoreFileManager {
|
|||
verifyGetAndScanScenario(manager, keyAfter(KEY_B), keyAfter(KEY_C), sf0, sfC, sfD);
|
||||
}
|
||||
|
||||
private void verifyGetAndScanScenario(StripeStoreFileManager manager,
|
||||
byte[] start, byte[] end, StoreFile... results) throws Exception {
|
||||
verifyGetOrScanScenario(manager, true, start, end, results);
|
||||
verifyGetOrScanScenario(manager, false, start, end, results);
|
||||
private void verifyGetAndScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end,
|
||||
StoreFile... results) throws Exception {
|
||||
verifyGetOrScanScenario(manager, start, end, results);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -548,16 +547,16 @@ public class TestStripeStoreFileManager {
|
|||
verifyAllFiles(manager, allFiles); // must have the same files.
|
||||
}
|
||||
|
||||
private void verifyGetOrScanScenario(StripeStoreFileManager manager, boolean isGet,
|
||||
byte[] start, byte[] end, StoreFile... results) throws Exception {
|
||||
verifyGetOrScanScenario(manager, isGet, start, end, Arrays.asList(results));
|
||||
private void verifyGetOrScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end,
|
||||
StoreFile... results) throws Exception {
|
||||
verifyGetOrScanScenario(manager, start, end, Arrays.asList(results));
|
||||
}
|
||||
|
||||
private void verifyGetOrScanScenario(StripeStoreFileManager manager, boolean isGet,
|
||||
byte[] start, byte[] end, Collection<StoreFile> results) throws Exception {
|
||||
private void verifyGetOrScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end,
|
||||
Collection<StoreFile> results) throws Exception {
|
||||
start = start != null ? start : HConstants.EMPTY_START_ROW;
|
||||
end = end != null ? end : HConstants.EMPTY_END_ROW;
|
||||
Collection<StoreFile> sfs = manager.getFilesForScanOrGet(isGet, start, end);
|
||||
Collection<StoreFile> sfs = manager.getFilesForScan(start, true, end, false);
|
||||
assertEquals(results.size(), sfs.size());
|
||||
for (StoreFile result : results) {
|
||||
assertTrue(sfs.contains(result));
|
||||
|
@ -566,7 +565,7 @@ public class TestStripeStoreFileManager {
|
|||
|
||||
private void verifyAllFiles(
|
||||
StripeStoreFileManager manager, Collection<StoreFile> results) throws Exception {
|
||||
verifyGetOrScanScenario(manager, false, null, null, results);
|
||||
verifyGetOrScanScenario(manager, null, null, results);
|
||||
}
|
||||
|
||||
// TODO: replace with Mockito?
|
||||
|
|
Loading…
Reference in New Issue