HBASE-17045 Unify the implementation of small scan and regular scan

This commit is contained in:
zhangduo 2017-01-24 21:07:25 +08:00
parent 616f4801b0
commit 85d701892e
24 changed files with 1295 additions and 668 deletions

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.Interface; 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.ScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
/** /**
* The asynchronous client scanner implementation. * The asynchronous client scanner implementation.
@ -95,12 +96,16 @@ class AsyncClientScanner {
public final ClientService.Interface stub; public final ClientService.Interface stub;
public final long scannerId; public final HBaseRpcController controller;
public OpenScannerResponse(HRegionLocation loc, Interface stub, long scannerId) { public final ScanResponse resp;
public OpenScannerResponse(HRegionLocation loc, Interface stub, HBaseRpcController controller,
ScanResponse resp) {
this.loc = loc; this.loc = loc;
this.stub = stub; this.stub = stub;
this.scannerId = scannerId; this.controller = controller;
this.resp = resp;
} }
} }
@ -108,14 +113,14 @@ class AsyncClientScanner {
HRegionLocation loc, ClientService.Interface stub) { HRegionLocation loc, ClientService.Interface stub) {
CompletableFuture<OpenScannerResponse> future = new CompletableFuture<>(); CompletableFuture<OpenScannerResponse> future = new CompletableFuture<>();
try { try {
ScanRequest request = ScanRequest request = RequestConverter.buildScanRequest(loc.getRegionInfo().getRegionName(),
RequestConverter.buildScanRequest(loc.getRegionInfo().getRegionName(), scan, 0, false); scan, scan.getCaching(), false);
stub.scan(controller, request, resp -> { stub.scan(controller, request, resp -> {
if (controller.failed()) { if (controller.failed()) {
future.completeExceptionally(controller.getFailed()); future.completeExceptionally(controller.getFailed());
return; return;
} }
future.complete(new OpenScannerResponse(loc, stub, resp.getScannerId())); future.complete(new OpenScannerResponse(loc, stub, controller, resp));
}); });
} catch (IOException e) { } catch (IOException e) {
future.completeExceptionally(e); future.completeExceptionally(e);
@ -124,11 +129,11 @@ class AsyncClientScanner {
} }
private void startScan(OpenScannerResponse resp) { private void startScan(OpenScannerResponse resp) {
conn.callerFactory.scanSingleRegion().id(resp.scannerId).location(resp.loc).stub(resp.stub) conn.callerFactory.scanSingleRegion().id(resp.resp.getScannerId()).location(resp.loc)
.setScan(scan).consumer(consumer).resultCache(resultCache) .stub(resp.stub).setScan(scan).consumer(consumer).resultCache(resultCache)
.rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
.scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) .scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
.maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).start() .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).start(resp.controller, resp.resp)
.whenComplete((hasMore, error) -> { .whenComplete((hasMore, error) -> {
if (error != null) { if (error != null) {
consumer.onError(error); consumer.onError(error);

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
/** /**
@ -376,7 +377,8 @@ class AsyncNonMetaRegionLocator {
metaKey = createRegionName(tableName, req.row, NINES, false); metaKey = createRegionName(tableName, req.row, NINES, false);
} }
conn.getRawTable(META_TABLE_NAME) conn.getRawTable(META_TABLE_NAME)
.smallScan(new Scan(metaKey).setReversed(true).setSmall(true).addFamily(CATALOG_FAMILY), 1) .scanAll(new Scan().withStartRow(metaKey).setReversed(true).setReadType(ReadType.PREAD)
.addFamily(CATALOG_FAMILY).setLimit(1))
.whenComplete((results, error) -> onScanComplete(tableName, req, results, error)); .whenComplete((results, error) -> onScanComplete(tableName, req, results, error));
} }

View File

@ -30,7 +30,9 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
/** /**
* Factory to create an AsyncRpcRetryCaller. * Factory to create an AsyncRpcRetryCaller.
@ -138,81 +140,6 @@ class AsyncRpcRetryingCallerFactory {
return new SingleRequestCallerBuilder<>(); return new SingleRequestCallerBuilder<>();
} }
public class SmallScanCallerBuilder extends BuilderBase {
private TableName tableName;
private Scan scan;
private int limit;
private long scanTimeoutNs = -1L;
private long rpcTimeoutNs = -1L;
public SmallScanCallerBuilder table(TableName tableName) {
this.tableName = tableName;
return this;
}
public SmallScanCallerBuilder setScan(Scan scan) {
this.scan = scan;
return this;
}
public SmallScanCallerBuilder limit(int limit) {
this.limit = limit;
return this;
}
public SmallScanCallerBuilder scanTimeout(long scanTimeout, TimeUnit unit) {
this.scanTimeoutNs = unit.toNanos(scanTimeout);
return this;
}
public SmallScanCallerBuilder rpcTimeout(long rpcTimeout, TimeUnit unit) {
this.rpcTimeoutNs = unit.toNanos(rpcTimeout);
return this;
}
public SmallScanCallerBuilder pause(long pause, TimeUnit unit) {
this.pauseNs = unit.toNanos(pause);
return this;
}
public SmallScanCallerBuilder maxAttempts(int maxAttempts) {
this.maxAttempts = maxAttempts;
return this;
}
public SmallScanCallerBuilder startLogErrorsCnt(int startLogErrorsCnt) {
this.startLogErrorsCnt = startLogErrorsCnt;
return this;
}
public AsyncSmallScanRpcRetryingCaller build() {
TableName tableName = checkNotNull(this.tableName, "tableName is null");
Scan scan = checkNotNull(this.scan, "scan is null");
checkArgument(limit > 0, "invalid limit %d", limit);
return new AsyncSmallScanRpcRetryingCaller(conn, tableName, scan, limit, pauseNs, maxAttempts,
scanTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
}
/**
* Shortcut for {@code build().call()}
*/
public CompletableFuture<List<Result>> call() {
return build().call();
}
}
/**
* Create retry caller for small scan.
*/
public SmallScanCallerBuilder smallScan() {
return new SmallScanCallerBuilder();
}
public class ScanSingleRegionCallerBuilder extends BuilderBase { public class ScanSingleRegionCallerBuilder extends BuilderBase {
private long scannerId = -1L; private long scannerId = -1L;
@ -297,10 +224,11 @@ class AsyncRpcRetryingCallerFactory {
} }
/** /**
* Short cut for {@code build().start()}. * Short cut for {@code build().start(HBaseRpcController, ScanResponse)}.
*/ */
public CompletableFuture<Boolean> start() { public CompletableFuture<Boolean> start(HBaseRpcController controller,
return build().start(); ScanResponse respWhenOpen) {
return build().start(controller, respWhenOpen);
} }
} }

View File

@ -17,11 +17,10 @@
*/ */
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime; import static org.apache.hadoop.hbase.client.ConnectionUtils.*;
import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan; 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.noMoreResultsForScan;
import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController; 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; import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
import io.netty.util.HashedWheelTimer; import io.netty.util.HashedWheelTimer;
@ -135,6 +134,10 @@ class AsyncScanSingleRegionRpcRetryingCaller {
return TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - nextCallStartNs); return TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - nextCallStartNs);
} }
private long remainingTimeNs() {
return scanTimeoutNs - (System.nanoTime() - nextCallStartNs);
}
private void closeScanner() { private void closeScanner() {
resetController(controller, rpcTimeoutNs); resetController(controller, rpcTimeoutNs);
ScanRequest req = RequestConverter.buildScanRequest(this.scannerId, 0, true, false); ScanRequest req = RequestConverter.buildScanRequest(this.scannerId, 0, true, false);
@ -199,7 +202,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
} }
long delayNs; long delayNs;
if (scanTimeoutNs > 0) { if (scanTimeoutNs > 0) {
long maxDelayNs = scanTimeoutNs - (System.nanoTime() - nextCallStartNs); long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
if (maxDelayNs <= 0) { if (maxDelayNs <= 0) {
completeExceptionally(!scannerClosed); completeExceptionally(!scannerClosed);
return; return;
@ -245,7 +248,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
} }
} }
private void onComplete(ScanResponse resp) { private void onComplete(HBaseRpcController controller, ScanResponse resp) {
if (controller.failed()) { if (controller.failed()) {
onError(controller.getFailed()); onError(controller.getFailed());
return; return;
@ -288,6 +291,13 @@ class AsyncScanSingleRegionRpcRetryingCaller {
completeNoMoreResults(); completeNoMoreResults();
return; return;
} }
if (scan.getLimit() > 0) {
// The RS should have set the moreResults field in ScanResponse to false when we have reached
// the limit.
int limit = scan.getLimit() - results.length;
assert limit > 0;
scan.setLimit(limit);
}
// as in 2.0 this value will always be set // as in 2.0 this value will always be set
if (!resp.getMoreResultsInRegion()) { if (!resp.getMoreResultsInRegion()) {
completeWhenNoMoreResultsInRegion.run(); completeWhenNoMoreResultsInRegion.run();
@ -297,10 +307,26 @@ class AsyncScanSingleRegionRpcRetryingCaller {
} }
private void call() { private void call() {
resetController(controller, rpcTimeoutNs); // As we have a call sequence for scan, it is useless to have a different rpc timeout which is
// less than the scan timeout. If the server does not respond in time(usually this will not
// happen as we have heartbeat now), we will get an OutOfOrderScannerNextException when
// resending the next request and the only way to fix this is to close the scanner and open a
// new one.
long callTimeoutNs;
if (scanTimeoutNs > 0) {
long remainingNs = scanTimeoutNs - (System.nanoTime() - nextCallStartNs);
if (remainingNs <= 0) {
completeExceptionally(true);
return;
}
callTimeoutNs = remainingNs;
} else {
callTimeoutNs = 0L;
}
resetController(controller, callTimeoutNs);
ScanRequest req = RequestConverter.buildScanRequest(scannerId, scan.getCaching(), false, ScanRequest req = RequestConverter.buildScanRequest(scannerId, scan.getCaching(), false,
nextCallSeq, false, false); nextCallSeq, false, false, scan.getLimit());
stub.scan(controller, req, this::onComplete); stub.scan(controller, req, resp -> onComplete(controller, resp));
} }
private void next() { private void next() {
@ -312,10 +338,15 @@ class AsyncScanSingleRegionRpcRetryingCaller {
} }
/** /**
* Now we will also fetch some cells along with the scanner id when opening a scanner, so we also
* need to process the ScanResponse for the open scanner request. The HBaseRpcController for the
* open scanner request is also needed because we may have some data in the CellScanner which is
* contained in the controller.
* @return {@code true} if we should continue, otherwise {@code false}. * @return {@code true} if we should continue, otherwise {@code false}.
*/ */
public CompletableFuture<Boolean> start() { public CompletableFuture<Boolean> start(HBaseRpcController controller,
next(); ScanResponse respWhenOpen) {
onComplete(controller, respWhenOpen);
return future; return future;
} }
} }

View File

@ -1,194 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
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;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
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;
/**
* Retry caller for smaller scan.
*/
@InterfaceAudience.Private
class AsyncSmallScanRpcRetryingCaller {
private final AsyncConnectionImpl conn;
private final TableName tableName;
private final Scan scan;
private final int limit;
private final long scanTimeoutNs;
private final long rpcTimeoutNs;
private final long pauseNs;
private final int maxAttempts;
private final int startLogErrosCnt;
private final Function<HRegionInfo, Boolean> nextScan;
private final List<Result> resultList;
private final CompletableFuture<List<Result>> future;
public AsyncSmallScanRpcRetryingCaller(AsyncConnectionImpl conn, TableName tableName, Scan scan,
int limit, long pauseNs, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs,
int startLogErrosCnt) {
this.conn = conn;
this.tableName = tableName;
this.scan = scan;
this.limit = limit;
this.scanTimeoutNs = scanTimeoutNs;
this.rpcTimeoutNs = rpcTimeoutNs;
this.pauseNs = pauseNs;
this.maxAttempts = maxAttempts;
this.startLogErrosCnt = startLogErrosCnt;
if (scan.isReversed()) {
this.nextScan = this::reversedNextScan;
} else {
this.nextScan = this::nextScan;
}
this.resultList = new ArrayList<>();
this.future = new CompletableFuture<>();
}
private static final class SmallScanResponse {
public final Result[] results;
public final HRegionInfo currentRegion;
public final boolean hasMoreResultsInRegion;
public SmallScanResponse(Result[] results, HRegionInfo currentRegion,
boolean hasMoreResultsInRegion) {
this.results = results;
this.currentRegion = currentRegion;
this.hasMoreResultsInRegion = hasMoreResultsInRegion;
}
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
justification = "Findbugs seems to be confused by lambda expression.")
private CompletableFuture<SmallScanResponse> scan(HBaseRpcController controller,
HRegionLocation loc, ClientService.Interface stub) {
CompletableFuture<SmallScanResponse> future = new CompletableFuture<>();
ScanRequest req;
try {
req = RequestConverter.buildScanRequest(loc.getRegionInfo().getRegionName(), scan,
limit - resultList.size(), true);
} catch (IOException e) {
future.completeExceptionally(e);
return future;
}
stub.scan(controller, req, resp -> {
if (controller.failed()) {
future.completeExceptionally(controller.getFailed());
} else {
try {
Result[] results = ResponseConverter.getResults(controller.cellScanner(), resp);
future.complete(
new SmallScanResponse(results, loc.getRegionInfo(), resp.getMoreResultsInRegion()));
} catch (IOException e) {
future.completeExceptionally(e);
}
}
});
return future;
}
private void onComplete(SmallScanResponse resp) {
resultList.addAll(Arrays.asList(resp.results));
if (resultList.size() == limit) {
future.complete(resultList);
return;
}
if (resp.hasMoreResultsInRegion) {
if (resp.results.length > 0) {
scan.withStartRow(resp.results[resp.results.length - 1].getRow(), false);
}
scan();
return;
}
if (!nextScan.apply(resp.currentRegion)) {
future.complete(resultList);
}
}
private void scan() {
conn.callerFactory.<SmallScanResponse> single().table(tableName).row(scan.getStartRow())
.locateType(getLocateType(scan)).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
.operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
.maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrosCnt).action(this::scan).call()
.whenComplete((resp, error) -> {
if (error != null) {
future.completeExceptionally(error);
} else {
onComplete(resp);
}
});
}
public CompletableFuture<List<Result>> call() {
scan();
return future;
}
private boolean nextScan(HRegionInfo info) {
if (noMoreResultsForScan(scan, info)) {
return false;
} else {
scan.withStartRow(info.getEndKey());
scan();
return true;
}
}
private boolean reversedNextScan(HRegionInfo info) {
if (noMoreResultsForReverseScan(scan, info)) {
return false;
} else {
scan.withStartRow(info.getStartKey(), false);
scan();
return true;
}
}
}

View File

@ -300,26 +300,34 @@ public interface AsyncTableBase {
CompareOp compareOp, byte[] value, RowMutations mutation); CompareOp compareOp, byte[] value, RowMutations mutation);
/** /**
* Just call {@link #smallScan(Scan, int)} with {@link Integer#MAX_VALUE}. * Return all the results that match the given scan object.
* @see #smallScan(Scan, int)
*/
default CompletableFuture<List<Result>> smallScan(Scan scan) {
return smallScan(scan, Integer.MAX_VALUE);
}
/**
* Return all the results that match the given scan object. The number of the returned results
* will not be greater than {@code limit}.
* <p> * <p>
* Notice that the scan must be small, and should not use batch or allowPartialResults. The * Notice that usually you should use this method with a {@link Scan} object that has limit set.
* {@code caching} property of the scan object is also ignored as we will use {@code limit} * For example, if you want to get the closest row after a given row, you could do this:
* instead. * <p>
* @param scan A configured {@link Scan} object. *
* @param limit the limit of results count * <pre>
* <code>
* table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -> {
* if (results.isEmpty()) {
* System.out.println("No row after " + Bytes.toStringBinary(row));
* } else {
* System.out.println("The closest row after " + Bytes.toStringBinary(row) + " is "
* + Bytes.toStringBinary(results.stream().findFirst().get().getRow()));
* }
* });
* </code>
* </pre>
* <p>
* If your result set is very large, you should use other scan method to get a scanner or use
* callback to process the results. They will do chunking to prevent OOM. The scanAll method will
* fetch all the results and store them in a List and then return the list to you.
* @param scan A configured {@link Scan} object. SO if you use this method to fetch a really large
* result set, it is likely to cause OOM.
* @return The results of this small scan operation. The return value will be wrapped by a * @return The results of this small scan operation. The return value will be wrapped by a
* {@link CompletableFuture}. * {@link CompletableFuture}.
*/ */
CompletableFuture<List<Result>> smallScan(Scan scan, int limit); CompletableFuture<List<Result>> scanAll(Scan scan);
/** /**
* Test for the existence of columns in the table, as specified by the Gets. * Test for the existence of columns in the table, as specified by the Gets.

View File

@ -144,8 +144,8 @@ class AsyncTableImpl implements AsyncTable {
} }
@Override @Override
public CompletableFuture<List<Result>> smallScan(Scan scan, int limit) { public CompletableFuture<List<Result>> scanAll(Scan scan) {
return wrap(rawTable.smallScan(scan, limit)); return wrap(rawTable.scanAll(scan));
} }
private long resultSize2CacheSize(long maxResultSize) { private long resultSize2CacheSize(long maxResultSize) {
@ -197,4 +197,5 @@ class AsyncTableImpl implements AsyncTable {
public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) { public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {
return rawTable.<T> batch(actions).stream().map(this::wrap).collect(toList()); return rawTable.<T> batch(actions).stream().map(this::wrap).collect(toList());
} }
} }

View File

@ -21,13 +21,13 @@ import static java.util.stream.Collectors.toList;
import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies; import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.function.Function; import java.util.function.Function;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -58,8 +58,6 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
@InterfaceAudience.Private @InterfaceAudience.Private
class RawAsyncTableImpl implements RawAsyncTable { class RawAsyncTableImpl implements RawAsyncTable {
private static final Log LOG = LogFactory.getLog(RawAsyncTableImpl.class);
private final AsyncConnectionImpl conn; private final AsyncConnectionImpl conn;
private final TableName tableName; private final TableName tableName;
@ -332,12 +330,6 @@ class RawAsyncTableImpl implements RawAsyncTable {
.call(); .call();
} }
private <T> CompletableFuture<T> failedFuture(Throwable error) {
CompletableFuture<T> future = new CompletableFuture<>();
future.completeExceptionally(error);
return future;
}
private Scan setDefaultScanConfig(Scan scan) { private Scan setDefaultScanConfig(Scan scan) {
// always create a new scan object as we may reset the start row later. // always create a new scan object as we may reset the start row later.
Scan newScan = ReflectionUtils.newInstance(scan.getClass(), scan); Scan newScan = ReflectionUtils.newInstance(scan.getClass(), scan);
@ -351,27 +343,35 @@ class RawAsyncTableImpl implements RawAsyncTable {
} }
@Override @Override
public CompletableFuture<List<Result>> smallScan(Scan scan, int limit) { public CompletableFuture<List<Result>> scanAll(Scan scan) {
if (!scan.isSmall()) { CompletableFuture<List<Result>> future = new CompletableFuture<>();
return failedFuture(new IllegalArgumentException("Only small scan is allowed")); List<Result> scanResults = new ArrayList<>();
} scan(scan, new RawScanResultConsumer() {
if (scan.getBatch() > 0 || scan.getAllowPartialResults()) {
return failedFuture( @Override
new IllegalArgumentException("Batch and allowPartial is not allowed for small scan")); public boolean onNext(Result[] results) {
} scanResults.addAll(Arrays.asList(results));
return conn.callerFactory.smallScan().table(tableName).setScan(setDefaultScanConfig(scan)) return true;
.limit(limit).scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS) }
.rpcTimeout(readRpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
.maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).call(); @Override
public void onError(Throwable error) {
future.completeExceptionally(error);
}
@Override
public void onComplete() {
future.complete(scanResults);
}
});
return future;
} }
public void scan(Scan scan, RawScanResultConsumer consumer) { public void scan(Scan scan, RawScanResultConsumer consumer) {
if (scan.isSmall()) { if (scan.isSmall() || scan.getLimit() > 0) {
if (scan.getBatch() > 0 || scan.getAllowPartialResults()) { if (scan.getBatch() > 0 || scan.getAllowPartialResults()) {
consumer.onError( consumer.onError(new IllegalArgumentException(
new IllegalArgumentException("Batch and allowPartial is not allowed for small scan")); "Batch and allowPartial is not allowed for small scan or limited scan"));
} else {
LOG.warn("This is small scan " + scan + ", consider using smallScan directly?");
} }
} }
scan = setDefaultScanConfig(scan); scan = setDefaultScanConfig(scan);
@ -388,6 +388,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
public List<CompletableFuture<Void>> put(List<Put> puts) { public List<CompletableFuture<Void>> put(List<Put> puts) {
return voidMutate(puts); return voidMutate(puts);
} }
@Override @Override
public List<CompletableFuture<Void>> delete(List<Delete> deletes) { public List<CompletableFuture<Void>> delete(List<Delete> deletes) {
return voidMutate(deletes); return voidMutate(deletes);
@ -434,4 +435,5 @@ class RawAsyncTableImpl implements RawAsyncTable {
public long getScanTimeout(TimeUnit unit) { public long getScanTimeout(TimeUnit unit) {
return unit.convert(scanTimeoutNs, TimeUnit.NANOSECONDS); return unit.convert(scanTimeoutNs, TimeUnit.NANOSECONDS);
} }
} }

View File

@ -47,7 +47,9 @@ public interface RawScanResultConsumer {
* This method give you a chance to terminate a slow scan operation. * This method give you a chance to terminate a slow scan operation.
* @return {@code false} if you want to terminate the scan process. Otherwise {@code true} * @return {@code false} if you want to terminate the scan process. Otherwise {@code true}
*/ */
boolean onHeartbeat(); default boolean onHeartbeat() {
return true;
}
/** /**
* Indicate that we hit an unrecoverable error and the scan operation is terminated. * Indicate that we hit an unrecoverable error and the scan operation is terminated.

View File

@ -46,38 +46,45 @@ import org.apache.hadoop.hbase.util.Bytes;
/** /**
* Used to perform Scan operations. * Used to perform Scan operations.
* <p> * <p>
* All operations are identical to {@link Get} with the exception of * All operations are identical to {@link Get} with the exception of instantiation. Rather than
* instantiation. Rather than specifying a single row, an optional startRow * specifying a single row, an optional startRow and stopRow may be defined. If rows are not
* and stopRow may be defined. If rows are not specified, the Scanner will * specified, the Scanner will iterate over all rows.
* iterate over all rows.
* <p> * <p>
* To get all columns from all rows of a Table, create an instance with no constraints; use the * To get all columns from all rows of a Table, create an instance with no constraints; use the
* {@link #Scan()} constructor. To constrain the scan to specific column families, * {@link #Scan()} constructor. To constrain the scan to specific column families, call
* call {@link #addFamily(byte[]) addFamily} for each family to retrieve on your Scan instance. * {@link #addFamily(byte[]) addFamily} for each family to retrieve on your Scan instance.
* <p> * <p>
* To get specific columns, call {@link #addColumn(byte[], byte[]) addColumn} * To get specific columns, call {@link #addColumn(byte[], byte[]) addColumn} for each column to
* for each column to retrieve. * retrieve.
* <p> * <p>
* To only retrieve columns within a specific range of version timestamps, * To only retrieve columns within a specific range of version timestamps, call
* call {@link #setTimeRange(long, long) setTimeRange}. * {@link #setTimeRange(long, long) setTimeRange}.
* <p> * <p>
* To only retrieve columns with a specific timestamp, call * To only retrieve columns with a specific timestamp, call {@link #setTimeStamp(long) setTimestamp}
* {@link #setTimeStamp(long) setTimestamp}. * .
* <p> * <p>
* To limit the number of versions of each column to be returned, call * To limit the number of versions of each column to be returned, call {@link #setMaxVersions(int)
* {@link #setMaxVersions(int) setMaxVersions}. * setMaxVersions}.
* <p> * <p>
* To limit the maximum number of values returned for each call to next(), * To limit the maximum number of values returned for each call to next(), call
* call {@link #setBatch(int) setBatch}. * {@link #setBatch(int) setBatch}.
* <p> * <p>
* To add a filter, call {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}. * To add a filter, call {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}.
* <p> * <p>
* Expert: To explicitly disable server-side block caching for this scan, * For small scan, it is deprecated in 2.0.0. Now we have a {@link #setLimit(int)} method in Scan
* execute {@link #setCacheBlocks(boolean)}. * object which is used to tell RS how many rows we want. If the rows return reaches the limit, the
* <p><em>Note:</em> Usage alters Scan instances. Internally, attributes are updated as the Scan * RS will close the RegionScanner automatically. And we will also fetch data when openScanner in
* runs and if enabled, metrics accumulate in the Scan instance. Be aware this is the case when * the new implementation, this means we can also finish a scan operation in one rpc call. And we
* you go to clone a Scan instance or if you go to reuse a created Scan instance; safer is create * have also introduced a {@link #setReadType(ReadType)} method. You can use this method to tell RS
* a Scan instance per usage. * to use pread explicitly.
* <p>
* Expert: To explicitly disable server-side block caching for this scan, execute
* {@link #setCacheBlocks(boolean)}.
* <p>
* <em>Note:</em> Usage alters Scan instances. Internally, attributes are updated as the Scan runs
* and if enabled, metrics accumulate in the Scan instance. Be aware this is the case when you go to
* clone a Scan instance or if you go to reuse a created Scan instance; safer is create a Scan
* instance per usage.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
@ -86,9 +93,9 @@ public class Scan extends Query {
private static final String RAW_ATTR = "_raw_"; private static final String RAW_ATTR = "_raw_";
private byte [] startRow = HConstants.EMPTY_START_ROW; private byte[] startRow = HConstants.EMPTY_START_ROW;
private boolean includeStartRow = true; private boolean includeStartRow = true;
private byte [] stopRow = HConstants.EMPTY_END_ROW; private byte[] stopRow = HConstants.EMPTY_END_ROW;
private boolean includeStopRow = false; private boolean includeStopRow = false;
private int maxVersions = 1; private int maxVersions = 1;
private int batch = -1; private int batch = -1;
@ -171,6 +178,16 @@ public class Scan extends Query {
*/ */
private long mvccReadPoint = -1L; private long mvccReadPoint = -1L;
/**
* The number of rows we want for this scan. We will terminate the scan if the number of return
* rows reaches this value.
*/
private int limit = -1;
/**
* Control whether to use pread at server side.
*/
private ReadType readType = ReadType.DEFAULT;
/** /**
* Create a Scan operation across all rows. * Create a Scan operation across all rows.
*/ */
@ -257,6 +274,7 @@ public class Scan extends Query {
setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax()); setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
} }
this.mvccReadPoint = scan.getMvccReadPoint(); this.mvccReadPoint = scan.getMvccReadPoint();
this.limit = scan.getLimit();
} }
/** /**
@ -969,37 +987,36 @@ public class Scan extends Query {
return attr == null ? false : Bytes.toBoolean(attr); return attr == null ? false : Bytes.toBoolean(attr);
} }
/** /**
* Set whether this scan is a small scan * Set whether this scan is a small scan
* <p> * <p>
* Small scan should use pread and big scan can use seek + read * Small scan should use pread and big scan can use seek + read seek + read is fast but can cause
* * two problem (1) resource contention (2) cause too much network io [89-fb] Using pread for
* seek + read is fast but can cause two problem (1) resource contention (2) * non-compaction read request https://issues.apache.org/jira/browse/HBASE-7266 On the other hand,
* cause too much network io * if setting it true, we would do openScanner,next,closeScanner in one RPC call. It means the
* * better performance for small scan. [HBASE-9488]. Generally, if the scan range is within one
* [89-fb] Using pread for non-compaction read request * data block(64KB), it could be considered as a small scan.
* https://issues.apache.org/jira/browse/HBASE-7266
*
* On the other hand, if setting it true, we would do
* openScanner,next,closeScanner in one RPC call. It means the better
* performance for small scan. [HBASE-9488].
*
* Generally, if the scan range is within one data block(64KB), it could be
* considered as a small scan.
*
* @param small * @param small
* @deprecated since 2.0.0. Use {@link #setLimit(int)} and {@link #setReadType(ReadType)} instead.
* And for the one rpc optimization, now we will also fetch data when openScanner, and
* if the number of rows reaches the limit then we will close the scanner
* automatically which means we will fall back to one rpc.
* @see #setLimit(int)
* @see #setReadType(ReadType)
*/ */
@Deprecated
public Scan setSmall(boolean small) { public Scan setSmall(boolean small) {
this.small = small; this.small = small;
this.readType = ReadType.PREAD;
return this; return this;
} }
/** /**
* Get whether this scan is a small scan * Get whether this scan is a small scan
* @return true if small scan * @return true if small scan
* @deprecated since 2.0.0. See the comment of {@link #setSmall(boolean)}
*/ */
@Deprecated
public boolean isSmall() { public boolean isSmall() {
return small; return small;
} }
@ -1080,6 +1097,53 @@ public class Scan extends Query {
return this; return this;
} }
/**
* @return the limit of rows for this scan
*/
public int getLimit() {
return limit;
}
/**
* Set the limit of rows for this scan. We will terminate the scan if the number of returned rows
* reaches this value.
* <p>
* This condition will be tested at last, after all other conditions such as stopRow, filter, etc.
* <p>
* Can not be used together with batch and allowPartial.
* @param limit the limit of rows for this scan
* @return this
*/
public Scan setLimit(int limit) {
this.limit = limit;
return this;
}
@InterfaceAudience.Public
@InterfaceStability.Unstable
public enum ReadType {
DEFAULT, STREAM, PREAD
}
/**
* @return the read type for this scan
*/
public ReadType getReadType() {
return readType;
}
/**
* Set the read type for this scan.
* <p>
* Notice that we may choose to use pread even if you specific {@link ReadType#STREAM} here. For
* example, we will always use pread if this is a get scan.
* @return this
*/
public Scan setReadType(ReadType readType) {
this.readType = readType;
return this;
}
/** /**
* Get the mvcc read point used to open a scanner. * Get the mvcc read point used to open a scanner.
*/ */

View File

@ -194,7 +194,7 @@ public class ScannerCallable extends ClientServiceCallable<Result[]> {
try { try {
incRPCcallsMetrics(); incRPCcallsMetrics();
request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
this.scanMetrics != null, renew); this.scanMetrics != null, renew, -1);
ScanResponse response = null; ScanResponse response = null;
response = getStub().scan(getRpcController(), request); response = getStub().scan(getRpcController(), request);
// Client and RS maintain a nextCallSeq number during the scan. Every next() call // Client and RS maintain a nextCallSeq number during the scan. Every next() call

View File

@ -814,6 +814,32 @@ public final class ProtobufUtil {
return get; return get;
} }
public static ClientProtos.Scan.ReadType toReadType(Scan.ReadType readType) {
switch (readType) {
case DEFAULT:
return ClientProtos.Scan.ReadType.DEFAULT;
case STREAM:
return ClientProtos.Scan.ReadType.STREAM;
case PREAD:
return ClientProtos.Scan.ReadType.PREAD;
default:
throw new IllegalArgumentException("Unknown ReadType: " + readType);
}
}
public static Scan.ReadType toReadType(ClientProtos.Scan.ReadType readType) {
switch (readType) {
case DEFAULT:
return Scan.ReadType.DEFAULT;
case STREAM:
return Scan.ReadType.STREAM;
case PREAD:
return Scan.ReadType.PREAD;
default:
throw new IllegalArgumentException("Unknown ReadType: " + readType);
}
}
/** /**
* Convert a client Scan to a protocol buffer Scan * Convert a client Scan to a protocol buffer Scan
* *
@ -917,6 +943,9 @@ public final class ProtobufUtil {
if (scan.includeStopRow()) { if (scan.includeStopRow()) {
scanBuilder.setIncludeStopRow(true); scanBuilder.setIncludeStopRow(true);
} }
if (scan.getReadType() != Scan.ReadType.DEFAULT) {
scanBuilder.setReadType(toReadType(scan.getReadType()));
}
return scanBuilder.build(); return scanBuilder.build();
} }
@ -1015,6 +1044,11 @@ public final class ProtobufUtil {
if (proto.hasMvccReadPoint()) { if (proto.hasMvccReadPoint()) {
PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint()); PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint());
} }
if (scan.isSmall()) {
scan.setReadType(Scan.ReadType.PREAD);
} else if (proto.hasReadType()) {
scan.setReadType(toReadType(proto.getReadType()));
}
return scan; return scan;
} }

View File

@ -74,6 +74,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLoadStats; import org.apache.hadoop.hbase.client.RegionLoadStats;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.client.SnapshotType; import org.apache.hadoop.hbase.client.SnapshotType;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
@ -928,6 +929,32 @@ public final class ProtobufUtil {
return get; return get;
} }
public static ClientProtos.Scan.ReadType toReadType(Scan.ReadType readType) {
switch (readType) {
case DEFAULT:
return ClientProtos.Scan.ReadType.DEFAULT;
case STREAM:
return ClientProtos.Scan.ReadType.STREAM;
case PREAD:
return ClientProtos.Scan.ReadType.PREAD;
default:
throw new IllegalArgumentException("Unknown ReadType: " + readType);
}
}
public static Scan.ReadType toReadType(ClientProtos.Scan.ReadType readType) {
switch (readType) {
case DEFAULT:
return Scan.ReadType.DEFAULT;
case STREAM:
return Scan.ReadType.STREAM;
case PREAD:
return Scan.ReadType.PREAD;
default:
throw new IllegalArgumentException("Unknown ReadType: " + readType);
}
}
/** /**
* Convert a client Scan to a protocol buffer Scan * Convert a client Scan to a protocol buffer Scan
* *
@ -1031,6 +1058,9 @@ public final class ProtobufUtil {
if (scan.includeStopRow()) { if (scan.includeStopRow()) {
scanBuilder.setIncludeStopRow(true); scanBuilder.setIncludeStopRow(true);
} }
if (scan.getReadType() != Scan.ReadType.DEFAULT) {
scanBuilder.setReadType(toReadType(scan.getReadType()));
}
return scanBuilder.build(); return scanBuilder.build();
} }
@ -1129,6 +1159,11 @@ public final class ProtobufUtil {
if (proto.hasMvccReadPoint()) { if (proto.hasMvccReadPoint()) {
PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint()); PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint());
} }
if (scan.isSmall()) {
scan.setReadType(Scan.ReadType.PREAD);
} else if (proto.hasReadType()) {
scan.setReadType(toReadType(proto.getReadType()));
}
return scan; return scan;
} }

View File

@ -462,11 +462,10 @@ public final class RequestConverter {
* @return a scan request * @return a scan request
* @throws IOException * @throws IOException
*/ */
public static ScanRequest buildScanRequest(final byte[] regionName, final Scan scan, public static ScanRequest buildScanRequest(byte[] regionName, Scan scan, int numberOfRows,
final int numberOfRows, final boolean closeScanner) throws IOException { boolean closeScanner) throws IOException {
ScanRequest.Builder builder = ScanRequest.newBuilder(); ScanRequest.Builder builder = ScanRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
RegionSpecifierType.REGION_NAME, regionName);
builder.setNumberOfRows(numberOfRows); builder.setNumberOfRows(numberOfRows);
builder.setCloseScanner(closeScanner); builder.setCloseScanner(closeScanner);
builder.setRegion(region); builder.setRegion(region);
@ -474,19 +473,21 @@ public final class RequestConverter {
builder.setClientHandlesPartials(true); builder.setClientHandlesPartials(true);
builder.setClientHandlesHeartbeats(true); builder.setClientHandlesHeartbeats(true);
builder.setTrackScanMetrics(scan.isScanMetricsEnabled()); builder.setTrackScanMetrics(scan.isScanMetricsEnabled());
if (scan.getLimit() > 0) {
builder.setLimitOfRows(scan.getLimit());
}
return builder.build(); return builder.build();
} }
/** /**
* Create a protocol buffer ScanRequest for a scanner id * Create a protocol buffer ScanRequest for a scanner id
*
* @param scannerId * @param scannerId
* @param numberOfRows * @param numberOfRows
* @param closeScanner * @param closeScanner
* @return a scan request * @return a scan request
*/ */
public static ScanRequest buildScanRequest(final long scannerId, final int numberOfRows, public static ScanRequest buildScanRequest(long scannerId, int numberOfRows, boolean closeScanner,
final boolean closeScanner, final boolean trackMetrics) { boolean trackMetrics) {
ScanRequest.Builder builder = ScanRequest.newBuilder(); ScanRequest.Builder builder = ScanRequest.newBuilder();
builder.setNumberOfRows(numberOfRows); builder.setNumberOfRows(numberOfRows);
builder.setCloseScanner(closeScanner); builder.setCloseScanner(closeScanner);
@ -499,16 +500,14 @@ public final class RequestConverter {
/** /**
* Create a protocol buffer ScanRequest for a scanner id * Create a protocol buffer ScanRequest for a scanner id
*
* @param scannerId * @param scannerId
* @param numberOfRows * @param numberOfRows
* @param closeScanner * @param closeScanner
* @param nextCallSeq * @param nextCallSeq
* @return a scan request * @return a scan request
*/ */
public static ScanRequest buildScanRequest(final long scannerId, final int numberOfRows, public static ScanRequest buildScanRequest(long scannerId, int numberOfRows, boolean closeScanner,
final boolean closeScanner, final long nextCallSeq, final boolean trackMetrics, long nextCallSeq, boolean trackMetrics, boolean renew, int limitOfRows) {
final boolean renew) {
ScanRequest.Builder builder = ScanRequest.newBuilder(); ScanRequest.Builder builder = ScanRequest.newBuilder();
builder.setNumberOfRows(numberOfRows); builder.setNumberOfRows(numberOfRows);
builder.setCloseScanner(closeScanner); builder.setCloseScanner(closeScanner);
@ -518,6 +517,9 @@ public final class RequestConverter {
builder.setClientHandlesHeartbeats(true); builder.setClientHandlesHeartbeats(true);
builder.setTrackScanMetrics(trackMetrics); builder.setTrackScanMetrics(trackMetrics);
builder.setRenew(renew); builder.setRenew(renew);
if (limitOfRows > 0) {
builder.setLimitOfRows(limitOfRows);
}
return builder.build(); return builder.build();
} }

View File

@ -14487,13 +14487,13 @@ public final class ClientProtos {
boolean getLoadColumnFamiliesOnDemand(); boolean getLoadColumnFamiliesOnDemand();
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
boolean hasSmall(); @java.lang.Deprecated boolean hasSmall();
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
boolean getSmall(); @java.lang.Deprecated boolean getSmall();
/** /**
* <code>optional bool reversed = 15 [default = false];</code> * <code>optional bool reversed = 15 [default = false];</code>
@ -14581,6 +14581,15 @@ public final class ClientProtos {
* <code>optional bool include_stop_row = 22 [default = false];</code> * <code>optional bool include_stop_row = 22 [default = false];</code>
*/ */
boolean getIncludeStopRow(); boolean getIncludeStopRow();
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
boolean hasReadType();
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType getReadType();
} }
/** /**
* <pre> * <pre>
@ -14624,6 +14633,7 @@ public final class ClientProtos {
mvccReadPoint_ = 0L; mvccReadPoint_ = 0L;
includeStartRow_ = true; includeStartRow_ = true;
includeStopRow_ = false; includeStopRow_ = false;
readType_ = 0;
} }
@java.lang.Override @java.lang.Override
@ -14798,6 +14808,17 @@ public final class ClientProtos {
includeStopRow_ = input.readBool(); includeStopRow_ = input.readBool();
break; break;
} }
case 184: {
int rawValue = input.readEnum();
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType value = org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType.valueOf(rawValue);
if (value == null) {
unknownFields.mergeVarintField(23, rawValue);
} else {
bitField0_ |= 0x00080000;
readType_ = rawValue;
}
break;
}
} }
} }
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@ -14831,6 +14852,105 @@ public final class ClientProtos {
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.Builder.class); org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.Builder.class);
} }
/**
* Protobuf enum {@code hbase.pb.Scan.ReadType}
*/
public enum ReadType
implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
/**
* <code>DEFAULT = 0;</code>
*/
DEFAULT(0),
/**
* <code>STREAM = 1;</code>
*/
STREAM(1),
/**
* <code>PREAD = 2;</code>
*/
PREAD(2),
;
/**
* <code>DEFAULT = 0;</code>
*/
public static final int DEFAULT_VALUE = 0;
/**
* <code>STREAM = 1;</code>
*/
public static final int STREAM_VALUE = 1;
/**
* <code>PREAD = 2;</code>
*/
public static final int PREAD_VALUE = 2;
public final int getNumber() {
return value;
}
/**
* @deprecated Use {@link #forNumber(int)} instead.
*/
@java.lang.Deprecated
public static ReadType valueOf(int value) {
return forNumber(value);
}
public static ReadType forNumber(int value) {
switch (value) {
case 0: return DEFAULT;
case 1: return STREAM;
case 2: return PREAD;
default: return null;
}
}
public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<ReadType>
internalGetValueMap() {
return internalValueMap;
}
private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
ReadType> internalValueMap =
new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<ReadType>() {
public ReadType findValueByNumber(int number) {
return ReadType.forNumber(number);
}
};
public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(ordinal());
}
public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
getDescriptorForType() {
return getDescriptor();
}
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() {
return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.getDescriptor().getEnumTypes().get(0);
}
private static final ReadType[] VALUES = values();
public static ReadType valueOf(
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
if (desc.getType() != getDescriptor()) {
throw new java.lang.IllegalArgumentException(
"EnumValueDescriptor is not for this type.");
}
return VALUES[desc.getIndex()];
}
private final int value;
private ReadType(int value) {
this.value = value;
}
// @@protoc_insertion_point(enum_scope:hbase.pb.Scan.ReadType)
}
private int bitField0_; private int bitField0_;
public static final int COLUMN_FIELD_NUMBER = 1; public static final int COLUMN_FIELD_NUMBER = 1;
private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column> column_; private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column> column_;
@ -15090,15 +15210,15 @@ public final class ClientProtos {
public static final int SMALL_FIELD_NUMBER = 14; public static final int SMALL_FIELD_NUMBER = 14;
private boolean small_; private boolean small_;
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean hasSmall() { @java.lang.Deprecated public boolean hasSmall() {
return ((bitField0_ & 0x00000800) == 0x00000800); return ((bitField0_ & 0x00000800) == 0x00000800);
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean getSmall() { @java.lang.Deprecated public boolean getSmall() {
return small_; return small_;
} }
@ -15243,6 +15363,22 @@ public final class ClientProtos {
return includeStopRow_; return includeStopRow_;
} }
public static final int READTYPE_FIELD_NUMBER = 23;
private int readType_;
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public boolean hasReadType() {
return ((bitField0_ & 0x00080000) == 0x00080000);
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType getReadType() {
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType result = org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType.valueOf(readType_);
return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT : result;
}
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized; byte isInitialized = memoizedIsInitialized;
@ -15345,6 +15481,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00040000) == 0x00040000)) { if (((bitField0_ & 0x00040000) == 0x00040000)) {
output.writeBool(22, includeStopRow_); output.writeBool(22, includeStopRow_);
} }
if (((bitField0_ & 0x00080000) == 0x00080000)) {
output.writeEnum(23, readType_);
}
unknownFields.writeTo(output); unknownFields.writeTo(output);
} }
@ -15441,6 +15580,10 @@ public final class ClientProtos {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeBoolSize(22, includeStopRow_); .computeBoolSize(22, includeStopRow_);
} }
if (((bitField0_ & 0x00080000) == 0x00080000)) {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeEnumSize(23, readType_);
}
size += unknownFields.getSerializedSize(); size += unknownFields.getSerializedSize();
memoizedSize = size; memoizedSize = size;
return size; return size;
@ -15558,6 +15701,10 @@ public final class ClientProtos {
result = result && (getIncludeStopRow() result = result && (getIncludeStopRow()
== other.getIncludeStopRow()); == other.getIncludeStopRow());
} }
result = result && (hasReadType() == other.hasReadType());
if (hasReadType()) {
result = result && readType_ == other.readType_;
}
result = result && unknownFields.equals(other.unknownFields); result = result && unknownFields.equals(other.unknownFields);
return result; return result;
} }
@ -15666,6 +15813,10 @@ public final class ClientProtos {
hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean( hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
getIncludeStopRow()); getIncludeStopRow());
} }
if (hasReadType()) {
hash = (37 * hash) + READTYPE_FIELD_NUMBER;
hash = (53 * hash) + readType_;
}
hash = (29 * hash) + unknownFields.hashCode(); hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -15863,6 +16014,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00100000); bitField0_ = (bitField0_ & ~0x00100000);
includeStopRow_ = false; includeStopRow_ = false;
bitField0_ = (bitField0_ & ~0x00200000); bitField0_ = (bitField0_ & ~0x00200000);
readType_ = 0;
bitField0_ = (bitField0_ & ~0x00400000);
return this; return this;
} }
@ -15998,6 +16151,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00040000; to_bitField0_ |= 0x00040000;
} }
result.includeStopRow_ = includeStopRow_; result.includeStopRow_ = includeStopRow_;
if (((from_bitField0_ & 0x00400000) == 0x00400000)) {
to_bitField0_ |= 0x00080000;
}
result.readType_ = readType_;
result.bitField0_ = to_bitField0_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -16175,6 +16332,9 @@ public final class ClientProtos {
if (other.hasIncludeStopRow()) { if (other.hasIncludeStopRow()) {
setIncludeStopRow(other.getIncludeStopRow()); setIncludeStopRow(other.getIncludeStopRow());
} }
if (other.hasReadType()) {
setReadType(other.getReadType());
}
this.mergeUnknownFields(other.unknownFields); this.mergeUnknownFields(other.unknownFields);
onChanged(); onChanged();
return this; return this;
@ -17251,30 +17411,30 @@ public final class ClientProtos {
private boolean small_ ; private boolean small_ ;
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean hasSmall() { @java.lang.Deprecated public boolean hasSmall() {
return ((bitField0_ & 0x00002000) == 0x00002000); return ((bitField0_ & 0x00002000) == 0x00002000);
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean getSmall() { @java.lang.Deprecated public boolean getSmall() {
return small_; return small_;
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public Builder setSmall(boolean value) { @java.lang.Deprecated public Builder setSmall(boolean value) {
bitField0_ |= 0x00002000; bitField0_ |= 0x00002000;
small_ = value; small_ = value;
onChanged(); onChanged();
return this; return this;
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public Builder clearSmall() { @java.lang.Deprecated public Builder clearSmall() {
bitField0_ = (bitField0_ & ~0x00002000); bitField0_ = (bitField0_ & ~0x00002000);
small_ = false; small_ = false;
onChanged(); onChanged();
@ -17748,6 +17908,42 @@ public final class ClientProtos {
onChanged(); onChanged();
return this; return this;
} }
private int readType_ = 0;
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public boolean hasReadType() {
return ((bitField0_ & 0x00400000) == 0x00400000);
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType getReadType() {
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType result = org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType.valueOf(readType_);
return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT : result;
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public Builder setReadType(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.ReadType value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00400000;
readType_ = value.getNumber();
onChanged();
return this;
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public Builder clearReadType() {
bitField0_ = (bitField0_ & ~0x00400000);
readType_ = 0;
onChanged();
return this;
}
public final Builder setUnknownFields( public final Builder setUnknownFields(
final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields); return super.setUnknownFields(unknownFields);
@ -17898,6 +18094,23 @@ public final class ClientProtos {
* <code>optional bool renew = 10 [default = false];</code> * <code>optional bool renew = 10 [default = false];</code>
*/ */
boolean getRenew(); boolean getRenew();
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
boolean hasLimitOfRows();
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
int getLimitOfRows();
} }
/** /**
* <pre> * <pre>
@ -17930,6 +18143,7 @@ public final class ClientProtos {
clientHandlesHeartbeats_ = false; clientHandlesHeartbeats_ = false;
trackScanMetrics_ = false; trackScanMetrics_ = false;
renew_ = false; renew_ = false;
limitOfRows_ = 0;
} }
@java.lang.Override @java.lang.Override
@ -18026,6 +18240,11 @@ public final class ClientProtos {
renew_ = input.readBool(); renew_ = input.readBool();
break; break;
} }
case 88: {
bitField0_ |= 0x00000400;
limitOfRows_ = input.readUInt32();
break;
}
} }
} }
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@ -18213,6 +18432,29 @@ public final class ClientProtos {
return renew_; return renew_;
} }
public static final int LIMIT_OF_ROWS_FIELD_NUMBER = 11;
private int limitOfRows_;
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
public boolean hasLimitOfRows() {
return ((bitField0_ & 0x00000400) == 0x00000400);
}
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
public int getLimitOfRows() {
return limitOfRows_;
}
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized; byte isInitialized = memoizedIsInitialized;
@ -18267,6 +18509,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000200) == 0x00000200)) { if (((bitField0_ & 0x00000200) == 0x00000200)) {
output.writeBool(10, renew_); output.writeBool(10, renew_);
} }
if (((bitField0_ & 0x00000400) == 0x00000400)) {
output.writeUInt32(11, limitOfRows_);
}
unknownFields.writeTo(output); unknownFields.writeTo(output);
} }
@ -18315,6 +18560,10 @@ public final class ClientProtos {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeBoolSize(10, renew_); .computeBoolSize(10, renew_);
} }
if (((bitField0_ & 0x00000400) == 0x00000400)) {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeUInt32Size(11, limitOfRows_);
}
size += unknownFields.getSerializedSize(); size += unknownFields.getSerializedSize();
memoizedSize = size; memoizedSize = size;
return size; return size;
@ -18382,6 +18631,11 @@ public final class ClientProtos {
result = result && (getRenew() result = result && (getRenew()
== other.getRenew()); == other.getRenew());
} }
result = result && (hasLimitOfRows() == other.hasLimitOfRows());
if (hasLimitOfRows()) {
result = result && (getLimitOfRows()
== other.getLimitOfRows());
}
result = result && unknownFields.equals(other.unknownFields); result = result && unknownFields.equals(other.unknownFields);
return result; return result;
} }
@ -18440,6 +18694,10 @@ public final class ClientProtos {
hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean( hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
getRenew()); getRenew());
} }
if (hasLimitOfRows()) {
hash = (37 * hash) + LIMIT_OF_ROWS_FIELD_NUMBER;
hash = (53 * hash) + getLimitOfRows();
}
hash = (29 * hash) + unknownFields.hashCode(); hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -18599,6 +18857,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000100); bitField0_ = (bitField0_ & ~0x00000100);
renew_ = false; renew_ = false;
bitField0_ = (bitField0_ & ~0x00000200); bitField0_ = (bitField0_ & ~0x00000200);
limitOfRows_ = 0;
bitField0_ = (bitField0_ & ~0x00000400);
return this; return this;
} }
@ -18671,6 +18931,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000200; to_bitField0_ |= 0x00000200;
} }
result.renew_ = renew_; result.renew_ = renew_;
if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
to_bitField0_ |= 0x00000400;
}
result.limitOfRows_ = limitOfRows_;
result.bitField0_ = to_bitField0_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -18743,6 +19007,9 @@ public final class ClientProtos {
if (other.hasRenew()) { if (other.hasRenew()) {
setRenew(other.getRenew()); setRenew(other.getRenew());
} }
if (other.hasLimitOfRows()) {
setLimitOfRows(other.getLimitOfRows());
}
this.mergeUnknownFields(other.unknownFields); this.mergeUnknownFields(other.unknownFields);
onChanged(); onChanged();
return this; return this;
@ -19272,6 +19539,54 @@ public final class ClientProtos {
onChanged(); onChanged();
return this; return this;
} }
private int limitOfRows_ ;
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
public boolean hasLimitOfRows() {
return ((bitField0_ & 0x00000400) == 0x00000400);
}
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
public int getLimitOfRows() {
return limitOfRows_;
}
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
public Builder setLimitOfRows(int value) {
bitField0_ |= 0x00000400;
limitOfRows_ = value;
onChanged();
return this;
}
/**
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*/
public Builder clearLimitOfRows() {
bitField0_ = (bitField0_ & ~0x00000400);
limitOfRows_ = 0;
onChanged();
return this;
}
public final Builder setUnknownFields( public final Builder setUnknownFields(
final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields); return super.setUnknownFields(unknownFields);
@ -40834,7 +41149,7 @@ public final class ClientProtos {
"tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" + "tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" +
"_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" + "_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 " + "\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " +
"\001(\010\"\233\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." + "\001(\010\"\203\006\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
"Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" + "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" + "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" + "w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" +
@ -40843,104 +41158,108 @@ public final class ClientProtos {
"cks\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017m" + "cks\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017m" +
"ax_result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(" + "ax_result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(" +
"\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036load_column_f" + "\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036load_column_f" +
"amilies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\022\027" + "amilies_on_demand\030\r \001(\010\022\021\n\005small\030\016 \001(\010B\002" +
"\n\010reversed\030\017 \001(\010:\005false\0222\n\013consistency\030\020" + "\030\001\022\027\n\010reversed\030\017 \001(\010:\005false\0222\n\013consisten" +
" \001(\0162\025.hbase.pb.Consistency:\006STRONG\022\017\n\007c" + "cy\030\020 \001(\0162\025.hbase.pb.Consistency:\006STRONG\022" +
"aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " + "\017\n\007caching\030\021 \001(\r\022\035\n\025allow_partial_result" +
"\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" + "s\030\022 \001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.p" +
"lumnFamilyTimeRange\022\032\n\017mvcc_read_point\030\024", "b.ColumnFamilyTimeRange\022\032\n\017mvcc_read_poi",
" \001(\004:\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true" + "nt\030\024 \001(\004:\0010\022\037\n\021include_start_row\030\025 \001(\010:\004" +
"\022\037\n\020include_stop_row\030\026 \001(\010:\005false\"\246\002\n\013Sc" + "true\022\037\n\020include_stop_row\030\026 \001(\010:\005false\0222\n" +
"anRequest\022)\n\006region\030\001 \001(\0132\031.hbase.pb.Reg" + "\010readType\030\027 \001(\0162\027.hbase.pb.Scan.ReadType" +
"ionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hbase.pb.Sc" + ":\007DEFAULT\".\n\010ReadType\022\013\n\007DEFAULT\020\000\022\n\n\006ST" +
"an\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016number_of_rows" + "REAM\020\001\022\t\n\005PREAD\020\002\"\300\002\n\013ScanRequest\022)\n\006reg" +
"\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025\n\rnext_ca" + "ion\030\001 \001(\0132\031.hbase.pb.RegionSpecifier\022\034\n\004" +
"ll_seq\030\006 \001(\004\022\037\n\027client_handles_partials\030" + "scan\030\002 \001(\0132\016.hbase.pb.Scan\022\022\n\nscanner_id" +
"\007 \001(\010\022!\n\031client_handles_heartbeats\030\010 \001(\010" + "\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclose_" +
"\022\032\n\022track_scan_metrics\030\t \001(\010\022\024\n\005renew\030\n " + "scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027" +
"\001(\010:\005false\"\266\002\n\014ScanResponse\022\030\n\020cells_per", "client_handles_partials\030\007 \001(\010\022!\n\031client_",
"_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mor" + "handles_heartbeats\030\010 \001(\010\022\032\n\022track_scan_m" +
"e_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030" + "etrics\030\t \001(\010\022\024\n\005renew\030\n \001(\010:\005false\022\030\n\rli" +
"\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001(\010\022\037" + "mit_of_rows\030\013 \001(\r:\0010\"\266\002\n\014ScanResponse\022\030\n" +
"\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026more_" + "\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 " +
"results_in_region\030\010 \001(\010\022\031\n\021heartbeat_mes" + "\001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!" +
"sage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbase" + "\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005sta" +
".pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(\004" + "le\030\006 \001(\010\022\037\n\027partial_flag_per_result\030\007 \003(" +
":\0010\"\240\002\n\024BulkLoadHFileRequest\022)\n\006region\030\001" + "\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n\021hea" +
" \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013famil" + "rtbeat_message\030\t \001(\010\022+\n\014scan_metrics\030\n \001" +
"y_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReq", "(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_read_p",
"uest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\022" + "oint\030\013 \001(\004:\0010\"\240\002\n\024BulkLoadHFileRequest\022)" +
"+\n\010fs_token\030\004 \001(\0132\031.hbase.pb.DelegationT" + "\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifie" +
"oken\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tcopy_file\030\006 " + "r\022>\n\013family_path\030\002 \003(\0132).hbase.pb.BulkLo" +
"\001(\010:\005false\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014" + "adHFileRequest.FamilyPath\022\026\n\016assign_seq_" +
"\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022" + "num\030\003 \001(\010\022+\n\010fs_token\030\004 \001(\0132\031.hbase.pb.D" +
"\016\n\006loaded\030\001 \002(\010\"V\n\017DelegationToken\022\022\n\nid" + "elegationToken\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tco" +
"entifier\030\001 \001(\014\022\020\n\010password\030\002 \001(\014\022\014\n\004kind" + "py_file\030\006 \001(\010:\005false\032*\n\nFamilyPath\022\016\n\006fa" +
"\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026PrepareBulkLo" + "mily\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFil" +
"adRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb" + "eResponse\022\016\n\006loaded\030\001 \002(\010\"V\n\017DelegationT" +
".TableName\022)\n\006region\030\002 \001(\0132\031.hbase.pb.Re", "oken\022\022\n\nidentifier\030\001 \001(\014\022\020\n\010password\030\002 \001",
"gionSpecifier\"-\n\027PrepareBulkLoadResponse" + "(\014\022\014\n\004kind\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026Pre" +
"\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026CleanupBulkLoadR" + "pareBulkLoadRequest\022\'\n\ntable_name\030\001 \002(\0132" +
"equest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006region\030\002 \001" + "\023.hbase.pb.TableName\022)\n\006region\030\002 \001(\0132\031.h" +
"(\0132\031.hbase.pb.RegionSpecifier\"\031\n\027Cleanup" + "base.pb.RegionSpecifier\"-\n\027PrepareBulkLo" +
"BulkLoadResponse\"a\n\026CoprocessorServiceCa" + "adResponse\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026Cleanu" +
"ll\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n" + "pBulkLoadRequest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006" +
"\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030C" + "region\030\002 \001(\0132\031.hbase.pb.RegionSpecifier\"" +
"oprocessorServiceResult\022&\n\005value\030\001 \001(\0132\027" + "\031\n\027CleanupBulkLoadResponse\"a\n\026Coprocesso" +
".hbase.pb.NameBytesPair\"v\n\031CoprocessorSe" + "rServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_nam" +
"rviceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.", "e\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030",
"RegionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb" + "\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n\005val" +
".CoprocessorServiceCall\"o\n\032CoprocessorSe" + "ue\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n\031Cop" +
"rviceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb" + "rocessorServiceRequest\022)\n\006region\030\001 \002(\0132\031" +
".RegionSpecifier\022&\n\005value\030\002 \002(\0132\027.hbase." + ".hbase.pb.RegionSpecifier\022.\n\004call\030\002 \002(\0132" +
"pb.NameBytesPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001" + " .hbase.pb.CoprocessorServiceCall\"o\n\032Cop" +
"(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.pb.Mutation" + "rocessorServiceResponse\022)\n\006region\030\001 \002(\0132" +
"Proto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014ser" + "\031.hbase.pb.RegionSpecifier\022&\n\005value\030\002 \002(" +
"vice_call\030\004 \001(\0132 .hbase.pb.CoprocessorSe" + "\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Action\022\r\n" +
"rviceCall\"k\n\014RegionAction\022)\n\006region\030\001 \002(" + "\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.p" +
"\0132\031.hbase.pb.RegionSpecifier\022\016\n\006atomic\030\002", "b.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.",
" \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Action\"c" + "Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb.Cop" +
"\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:" + "rocessorServiceCall\"k\n\014RegionAction\022)\n\006r" +
"\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compacti" + "egion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\016" +
"onPressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadSt" + "\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.p" +
"ats\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpe" + "b.Action\"c\n\017RegionLoadStats\022\027\n\014memstoreL" +
"cifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLo" + "oad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035" +
"adStats\"\336\001\n\021ResultOrException\022\r\n\005index\030\001" + "\n\022compactionPressure\030\003 \001(\005:\0010\"j\n\024MultiRe" +
" \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*" + "gionLoadStats\022)\n\006region\030\001 \003(\0132\031.hbase.pb" +
"\n\texception\030\003 \001(\0132\027.hbase.pb.NameBytesPa" + ".RegionSpecifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.p" +
"ir\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Co", "b.RegionLoadStats\"\336\001\n\021ResultOrException\022",
"processorServiceResult\0220\n\tloadStats\030\005 \001(" + "\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.p" +
"\0132\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Regi" + "b.Result\022*\n\texception\030\003 \001(\0132\027.hbase.pb.N" +
"onActionResult\0226\n\021resultOrException\030\001 \003(" + "ameBytesPair\022:\n\016service_result\030\004 \001(\0132\".h" +
"\0132\033.hbase.pb.ResultOrException\022*\n\texcept" + "base.pb.CoprocessorServiceResult\0220\n\tload" +
"ion\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mu" + "Stats\030\005 \001(\0132\031.hbase.pb.RegionLoadStatsB\002" +
"ltiRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase" + "\030\001\"x\n\022RegionActionResult\0226\n\021resultOrExce" +
".pb.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\t" + "ption\030\001 \003(\0132\033.hbase.pb.ResultOrException" +
"condition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n" + "\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameBytes" +
"\rMultiResponse\0228\n\022regionActionResult\030\001 \003" + "Pair\"x\n\014MultiRequest\022,\n\014regionAction\030\001 \003" +
"(\0132\034.hbase.pb.RegionActionResult\022\021\n\tproc", "(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceGroup",
"essed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036." + "\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb.Con" +
"hbase.pb.MultiRegionLoadStats*\'\n\013Consist" + "dition\"\226\001\n\rMultiResponse\0228\n\022regionAction" +
"ency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\263\005\n\rClien" + "Result\030\001 \003(\0132\034.hbase.pb.RegionActionResu" +
"tService\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025." + "lt\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regionStatistic" +
"hbase.pb.GetResponse\022;\n\006Mutate\022\027.hbase.p" + "s\030\003 \001(\0132\036.hbase.pb.MultiRegionLoadStats*" +
"b.MutateRequest\032\030.hbase.pb.MutateRespons" + "\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\001" +
"e\0225\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase" + "2\263\005\n\rClientService\0222\n\003Get\022\024.hbase.pb.Get" +
".pb.ScanResponse\022P\n\rBulkLoadHFile\022\036.hbas" + "Request\032\025.hbase.pb.GetResponse\022;\n\006Mutate" +
"e.pb.BulkLoadHFileRequest\032\037.hbase.pb.Bul" + "\022\027.hbase.pb.MutateRequest\032\030.hbase.pb.Mut" +
"kLoadHFileResponse\022V\n\017PrepareBulkLoad\022 .", "ateResponse\0225\n\004Scan\022\025.hbase.pb.ScanReque",
"hbase.pb.PrepareBulkLoadRequest\032!.hbase." + "st\032\026.hbase.pb.ScanResponse\022P\n\rBulkLoadHF" +
"pb.PrepareBulkLoadResponse\022V\n\017CleanupBul" + "ile\022\036.hbase.pb.BulkLoadHFileRequest\032\037.hb" +
"kLoad\022 .hbase.pb.CleanupBulkLoadRequest\032" + "ase.pb.BulkLoadHFileResponse\022V\n\017PrepareB" +
"!.hbase.pb.CleanupBulkLoadResponse\022X\n\013Ex" + "ulkLoad\022 .hbase.pb.PrepareBulkLoadReques" +
"ecService\022#.hbase.pb.CoprocessorServiceR" + "t\032!.hbase.pb.PrepareBulkLoadResponse\022V\n\017" +
"equest\032$.hbase.pb.CoprocessorServiceResp" + "CleanupBulkLoad\022 .hbase.pb.CleanupBulkLo" +
"onse\022d\n\027ExecRegionServerService\022#.hbase." + "adRequest\032!.hbase.pb.CleanupBulkLoadResp" +
"pb.CoprocessorServiceRequest\032$.hbase.pb." + "onse\022X\n\013ExecService\022#.hbase.pb.Coprocess" +
"CoprocessorServiceResponse\0228\n\005Multi\022\026.hb" + "orServiceRequest\032$.hbase.pb.CoprocessorS" +
"ase.pb.MultiRequest\032\027.hbase.pb.MultiResp", "erviceResponse\022d\n\027ExecRegionServerServic",
"onseBI\n1org.apache.hadoop.hbase.shaded.p" + "e\022#.hbase.pb.CoprocessorServiceRequest\032$" +
"rotobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001" ".hbase.pb.CoprocessorServiceResponse\0228\n\005" +
"Multi\022\026.hbase.pb.MultiRequest\032\027.hbase.pb" +
".MultiResponseBI\n1org.apache.hadoop.hbas" +
"e.shaded.protobuf.generatedB\014ClientProto" +
"sH\001\210\001\001\240\001\001"
}; };
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() {
@ -41042,13 +41361,13 @@ public final class ClientProtos {
internal_static_hbase_pb_Scan_fieldAccessorTable = new internal_static_hbase_pb_Scan_fieldAccessorTable = new
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
internal_static_hbase_pb_Scan_descriptor, 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", "IncludeStartRow", "IncludeStopRow", }); 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", "ReadType", });
internal_static_hbase_pb_ScanRequest_descriptor = internal_static_hbase_pb_ScanRequest_descriptor =
getDescriptor().getMessageTypes().get(12); getDescriptor().getMessageTypes().get(12);
internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
internal_static_hbase_pb_ScanRequest_descriptor, internal_static_hbase_pb_ScanRequest_descriptor,
new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", }); new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", "LimitOfRows", });
internal_static_hbase_pb_ScanResponse_descriptor = internal_static_hbase_pb_ScanResponse_descriptor =
getDescriptor().getMessageTypes().get(13); getDescriptor().getMessageTypes().get(13);
internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new

View File

@ -249,7 +249,7 @@ message Scan {
optional uint32 store_limit = 11; optional uint32 store_limit = 11;
optional uint32 store_offset = 12; optional uint32 store_offset = 12;
optional bool load_column_families_on_demand = 13; /* DO NOT add defaults to load_column_families_on_demand. */ optional bool load_column_families_on_demand = 13; /* DO NOT add defaults to load_column_families_on_demand. */
optional bool small = 14; optional bool small = 14 [deprecated = true];
optional bool reversed = 15 [default = false]; optional bool reversed = 15 [default = false];
optional Consistency consistency = 16 [default = STRONG]; optional Consistency consistency = 16 [default = STRONG];
optional uint32 caching = 17; optional uint32 caching = 17;
@ -258,6 +258,13 @@ message Scan {
optional uint64 mvcc_read_point = 20 [default = 0]; optional uint64 mvcc_read_point = 20 [default = 0];
optional bool include_start_row = 21 [default = true]; optional bool include_start_row = 21 [default = true];
optional bool include_stop_row = 22 [default = false]; optional bool include_stop_row = 22 [default = false];
enum ReadType {
DEFAULT = 0;
STREAM = 1;
PREAD = 2;
}
optional ReadType readType = 23 [default = DEFAULT];
} }
/** /**
@ -282,6 +289,8 @@ message ScanRequest {
optional bool client_handles_heartbeats = 8; optional bool client_handles_heartbeats = 8;
optional bool track_scan_metrics = 9; optional bool track_scan_metrics = 9;
optional bool renew = 10 [default = false]; optional bool renew = 10 [default = false];
// if we have returned limit_of_rows rows to client, then close the scanner.
optional uint32 limit_of_rows = 11 [default = 0];
} }
/** /**

View File

@ -14145,15 +14145,15 @@ public final class ClientProtos {
*/ */
boolean getLoadColumnFamiliesOnDemand(); boolean getLoadColumnFamiliesOnDemand();
// optional bool small = 14; // optional bool small = 14 [deprecated = true];
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
boolean hasSmall(); @java.lang.Deprecated boolean hasSmall();
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
boolean getSmall(); @java.lang.Deprecated boolean getSmall();
// optional bool reversed = 15 [default = false]; // optional bool reversed = 15 [default = false];
/** /**
@ -14249,6 +14249,16 @@ public final class ClientProtos {
* <code>optional bool include_stop_row = 22 [default = false];</code> * <code>optional bool include_stop_row = 22 [default = false];</code>
*/ */
boolean getIncludeStopRow(); boolean getIncludeStopRow();
// optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
boolean hasReadType();
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType();
} }
/** /**
* Protobuf type {@code hbase.pb.Scan} * Protobuf type {@code hbase.pb.Scan}
@ -14453,6 +14463,17 @@ public final class ClientProtos {
includeStopRow_ = input.readBool(); includeStopRow_ = input.readBool();
break; break;
} }
case 184: {
int rawValue = input.readEnum();
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType value = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.valueOf(rawValue);
if (value == null) {
unknownFields.mergeVarintField(23, rawValue);
} else {
bitField0_ |= 0x00080000;
readType_ = value;
}
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -14501,6 +14522,97 @@ public final class ClientProtos {
return PARSER; return PARSER;
} }
/**
* Protobuf enum {@code hbase.pb.Scan.ReadType}
*/
public enum ReadType
implements com.google.protobuf.ProtocolMessageEnum {
/**
* <code>DEFAULT = 0;</code>
*/
DEFAULT(0, 0),
/**
* <code>STREAM = 1;</code>
*/
STREAM(1, 1),
/**
* <code>PREAD = 2;</code>
*/
PREAD(2, 2),
;
/**
* <code>DEFAULT = 0;</code>
*/
public static final int DEFAULT_VALUE = 0;
/**
* <code>STREAM = 1;</code>
*/
public static final int STREAM_VALUE = 1;
/**
* <code>PREAD = 2;</code>
*/
public static final int PREAD_VALUE = 2;
public final int getNumber() { return value; }
public static ReadType valueOf(int value) {
switch (value) {
case 0: return DEFAULT;
case 1: return STREAM;
case 2: return PREAD;
default: return null;
}
}
public static com.google.protobuf.Internal.EnumLiteMap<ReadType>
internalGetValueMap() {
return internalValueMap;
}
private static com.google.protobuf.Internal.EnumLiteMap<ReadType>
internalValueMap =
new com.google.protobuf.Internal.EnumLiteMap<ReadType>() {
public ReadType findValueByNumber(int number) {
return ReadType.valueOf(number);
}
};
public final com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(index);
}
public final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptorForType() {
return getDescriptor();
}
public static final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDescriptor().getEnumTypes().get(0);
}
private static final ReadType[] VALUES = values();
public static ReadType valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
if (desc.getType() != getDescriptor()) {
throw new java.lang.IllegalArgumentException(
"EnumValueDescriptor is not for this type.");
}
return VALUES[desc.getIndex()];
}
private final int index;
private final int value;
private ReadType(int index, int value) {
this.index = index;
this.value = value;
}
// @@protoc_insertion_point(enum_scope:hbase.pb.Scan.ReadType)
}
private int bitField0_; private int bitField0_;
// repeated .hbase.pb.Column column = 1; // repeated .hbase.pb.Column column = 1;
public static final int COLUMN_FIELD_NUMBER = 1; public static final int COLUMN_FIELD_NUMBER = 1;
@ -14770,19 +14882,19 @@ public final class ClientProtos {
return loadColumnFamiliesOnDemand_; return loadColumnFamiliesOnDemand_;
} }
// optional bool small = 14; // optional bool small = 14 [deprecated = true];
public static final int SMALL_FIELD_NUMBER = 14; public static final int SMALL_FIELD_NUMBER = 14;
private boolean small_; private boolean small_;
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean hasSmall() { @java.lang.Deprecated public boolean hasSmall() {
return ((bitField0_ & 0x00000800) == 0x00000800); return ((bitField0_ & 0x00000800) == 0x00000800);
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean getSmall() { @java.lang.Deprecated public boolean getSmall() {
return small_; return small_;
} }
@ -14934,6 +15046,22 @@ public final class ClientProtos {
return includeStopRow_; return includeStopRow_;
} }
// optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
public static final int READTYPE_FIELD_NUMBER = 23;
private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType readType_;
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public boolean hasReadType() {
return ((bitField0_ & 0x00080000) == 0x00080000);
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType() {
return readType_;
}
private void initFields() { private void initFields() {
column_ = java.util.Collections.emptyList(); column_ = java.util.Collections.emptyList();
attribute_ = java.util.Collections.emptyList(); attribute_ = java.util.Collections.emptyList();
@ -14957,6 +15085,7 @@ public final class ClientProtos {
mvccReadPoint_ = 0L; mvccReadPoint_ = 0L;
includeStartRow_ = true; includeStartRow_ = true;
includeStopRow_ = false; includeStopRow_ = false;
readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -15060,6 +15189,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00040000) == 0x00040000)) { if (((bitField0_ & 0x00040000) == 0x00040000)) {
output.writeBool(22, includeStopRow_); output.writeBool(22, includeStopRow_);
} }
if (((bitField0_ & 0x00080000) == 0x00080000)) {
output.writeEnum(23, readType_.getNumber());
}
getUnknownFields().writeTo(output); getUnknownFields().writeTo(output);
} }
@ -15157,6 +15289,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeBoolSize(22, includeStopRow_); .computeBoolSize(22, includeStopRow_);
} }
if (((bitField0_ & 0x00080000) == 0x00080000)) {
size += com.google.protobuf.CodedOutputStream
.computeEnumSize(23, readType_.getNumber());
}
size += getUnknownFields().getSerializedSize(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -15281,6 +15417,11 @@ public final class ClientProtos {
result = result && (getIncludeStopRow() result = result && (getIncludeStopRow()
== other.getIncludeStopRow()); == other.getIncludeStopRow());
} }
result = result && (hasReadType() == other.hasReadType());
if (hasReadType()) {
result = result &&
(getReadType() == other.getReadType());
}
result = result && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -15382,6 +15523,10 @@ public final class ClientProtos {
hash = (37 * hash) + INCLUDE_STOP_ROW_FIELD_NUMBER; hash = (37 * hash) + INCLUDE_STOP_ROW_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getIncludeStopRow()); hash = (53 * hash) + hashBoolean(getIncludeStopRow());
} }
if (hasReadType()) {
hash = (37 * hash) + READTYPE_FIELD_NUMBER;
hash = (53 * hash) + hashEnum(getReadType());
}
hash = (29 * hash) + getUnknownFields().hashCode(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -15571,6 +15716,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00100000); bitField0_ = (bitField0_ & ~0x00100000);
includeStopRow_ = false; includeStopRow_ = false;
bitField0_ = (bitField0_ & ~0x00200000); bitField0_ = (bitField0_ & ~0x00200000);
readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
bitField0_ = (bitField0_ & ~0x00400000);
return this; return this;
} }
@ -15710,6 +15857,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00040000; to_bitField0_ |= 0x00040000;
} }
result.includeStopRow_ = includeStopRow_; result.includeStopRow_ = includeStopRow_;
if (((from_bitField0_ & 0x00400000) == 0x00400000)) {
to_bitField0_ |= 0x00080000;
}
result.readType_ = readType_;
result.bitField0_ = to_bitField0_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -15861,6 +16012,9 @@ public final class ClientProtos {
if (other.hasIncludeStopRow()) { if (other.hasIncludeStopRow()) {
setIncludeStopRow(other.getIncludeStopRow()); setIncludeStopRow(other.getIncludeStopRow());
} }
if (other.hasReadType()) {
setReadType(other.getReadType());
}
this.mergeUnknownFields(other.getUnknownFields()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -16945,33 +17099,33 @@ public final class ClientProtos {
return this; return this;
} }
// optional bool small = 14; // optional bool small = 14 [deprecated = true];
private boolean small_ ; private boolean small_ ;
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean hasSmall() { @java.lang.Deprecated public boolean hasSmall() {
return ((bitField0_ & 0x00002000) == 0x00002000); return ((bitField0_ & 0x00002000) == 0x00002000);
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public boolean getSmall() { @java.lang.Deprecated public boolean getSmall() {
return small_; return small_;
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public Builder setSmall(boolean value) { @java.lang.Deprecated public Builder setSmall(boolean value) {
bitField0_ |= 0x00002000; bitField0_ |= 0x00002000;
small_ = value; small_ = value;
onChanged(); onChanged();
return this; return this;
} }
/** /**
* <code>optional bool small = 14;</code> * <code>optional bool small = 14 [deprecated = true];</code>
*/ */
public Builder clearSmall() { @java.lang.Deprecated public Builder clearSmall() {
bitField0_ = (bitField0_ & ~0x00002000); bitField0_ = (bitField0_ & ~0x00002000);
small_ = false; small_ = false;
onChanged(); onChanged();
@ -17452,6 +17606,42 @@ public final class ClientProtos {
return this; return this;
} }
// optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public boolean hasReadType() {
return ((bitField0_ & 0x00400000) == 0x00400000);
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType() {
return readType_;
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public Builder setReadType(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00400000;
readType_ = value;
onChanged();
return this;
}
/**
* <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
*/
public Builder clearReadType() {
bitField0_ = (bitField0_ & ~0x00400000);
readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:hbase.pb.Scan) // @@protoc_insertion_point(builder_scope:hbase.pb.Scan)
} }
@ -17573,6 +17763,24 @@ public final class ClientProtos {
* <code>optional bool renew = 10 [default = false];</code> * <code>optional bool renew = 10 [default = false];</code>
*/ */
boolean getRenew(); boolean getRenew();
// optional uint32 limit_of_rows = 11 [default = 0];
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
boolean hasLimitOfRows();
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
int getLimitOfRows();
} }
/** /**
* Protobuf type {@code hbase.pb.ScanRequest} * Protobuf type {@code hbase.pb.ScanRequest}
@ -17704,6 +17912,11 @@ public final class ClientProtos {
renew_ = input.readBool(); renew_ = input.readBool();
break; break;
} }
case 88: {
bitField0_ |= 0x00000400;
limitOfRows_ = input.readUInt32();
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -17916,6 +18129,30 @@ public final class ClientProtos {
return renew_; return renew_;
} }
// optional uint32 limit_of_rows = 11 [default = 0];
public static final int LIMIT_OF_ROWS_FIELD_NUMBER = 11;
private int limitOfRows_;
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
public boolean hasLimitOfRows() {
return ((bitField0_ & 0x00000400) == 0x00000400);
}
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
public int getLimitOfRows() {
return limitOfRows_;
}
private void initFields() { private void initFields() {
region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
@ -17927,6 +18164,7 @@ public final class ClientProtos {
clientHandlesHeartbeats_ = false; clientHandlesHeartbeats_ = false;
trackScanMetrics_ = false; trackScanMetrics_ = false;
renew_ = false; renew_ = false;
limitOfRows_ = 0;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -17982,6 +18220,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000200) == 0x00000200)) { if (((bitField0_ & 0x00000200) == 0x00000200)) {
output.writeBool(10, renew_); output.writeBool(10, renew_);
} }
if (((bitField0_ & 0x00000400) == 0x00000400)) {
output.writeUInt32(11, limitOfRows_);
}
getUnknownFields().writeTo(output); getUnknownFields().writeTo(output);
} }
@ -18031,6 +18272,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeBoolSize(10, renew_); .computeBoolSize(10, renew_);
} }
if (((bitField0_ & 0x00000400) == 0x00000400)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt32Size(11, limitOfRows_);
}
size += getUnknownFields().getSerializedSize(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -18104,6 +18349,11 @@ public final class ClientProtos {
result = result && (getRenew() result = result && (getRenew()
== other.getRenew()); == other.getRenew());
} }
result = result && (hasLimitOfRows() == other.hasLimitOfRows());
if (hasLimitOfRows()) {
result = result && (getLimitOfRows()
== other.getLimitOfRows());
}
result = result && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -18157,6 +18407,10 @@ public final class ClientProtos {
hash = (37 * hash) + RENEW_FIELD_NUMBER; hash = (37 * hash) + RENEW_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getRenew()); hash = (53 * hash) + hashBoolean(getRenew());
} }
if (hasLimitOfRows()) {
hash = (37 * hash) + LIMIT_OF_ROWS_FIELD_NUMBER;
hash = (53 * hash) + getLimitOfRows();
}
hash = (29 * hash) + getUnknownFields().hashCode(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -18309,6 +18563,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000100); bitField0_ = (bitField0_ & ~0x00000100);
renew_ = false; renew_ = false;
bitField0_ = (bitField0_ & ~0x00000200); bitField0_ = (bitField0_ & ~0x00000200);
limitOfRows_ = 0;
bitField0_ = (bitField0_ & ~0x00000400);
return this; return this;
} }
@ -18385,6 +18641,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000200; to_bitField0_ |= 0x00000200;
} }
result.renew_ = renew_; result.renew_ = renew_;
if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
to_bitField0_ |= 0x00000400;
}
result.limitOfRows_ = limitOfRows_;
result.bitField0_ = to_bitField0_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -18431,6 +18691,9 @@ public final class ClientProtos {
if (other.hasRenew()) { if (other.hasRenew()) {
setRenew(other.getRenew()); setRenew(other.getRenew());
} }
if (other.hasLimitOfRows()) {
setLimitOfRows(other.getLimitOfRows());
}
this.mergeUnknownFields(other.getUnknownFields()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -18968,6 +19231,55 @@ public final class ClientProtos {
return this; return this;
} }
// optional uint32 limit_of_rows = 11 [default = 0];
private int limitOfRows_ ;
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
public boolean hasLimitOfRows() {
return ((bitField0_ & 0x00000400) == 0x00000400);
}
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
public int getLimitOfRows() {
return limitOfRows_;
}
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
public Builder setLimitOfRows(int value) {
bitField0_ |= 0x00000400;
limitOfRows_ = value;
onChanged();
return this;
}
/**
* <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
*
* <pre>
* if we have returned limit_of_rows rows to client, then close the scanner.
* </pre>
*/
public Builder clearLimitOfRows() {
bitField0_ = (bitField0_ & ~0x00000400);
limitOfRows_ = 0;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest) // @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest)
} }
@ -39912,7 +40224,7 @@ public final class ClientProtos {
"tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" + "tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" +
"_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" + "_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 " + "\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " +
"\001(\010\"\233\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." + "\001(\010\"\203\006\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
"Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" + "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" + "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" + "w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" +
@ -39921,104 +40233,108 @@ public final class ClientProtos {
"cks\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017m" + "cks\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017m" +
"ax_result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(" + "ax_result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(" +
"\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036load_column_f" + "\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036load_column_f" +
"amilies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\022\027" + "amilies_on_demand\030\r \001(\010\022\021\n\005small\030\016 \001(\010B\002" +
"\n\010reversed\030\017 \001(\010:\005false\0222\n\013consistency\030\020" + "\030\001\022\027\n\010reversed\030\017 \001(\010:\005false\0222\n\013consisten" +
" \001(\0162\025.hbase.pb.Consistency:\006STRONG\022\017\n\007c" + "cy\030\020 \001(\0162\025.hbase.pb.Consistency:\006STRONG\022" +
"aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " + "\017\n\007caching\030\021 \001(\r\022\035\n\025allow_partial_result" +
"\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" + "s\030\022 \001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.p" +
"lumnFamilyTimeRange\022\032\n\017mvcc_read_point\030\024", "b.ColumnFamilyTimeRange\022\032\n\017mvcc_read_poi",
" \001(\004:\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true" + "nt\030\024 \001(\004:\0010\022\037\n\021include_start_row\030\025 \001(\010:\004" +
"\022\037\n\020include_stop_row\030\026 \001(\010:\005false\"\246\002\n\013Sc" + "true\022\037\n\020include_stop_row\030\026 \001(\010:\005false\0222\n" +
"anRequest\022)\n\006region\030\001 \001(\0132\031.hbase.pb.Reg" + "\010readType\030\027 \001(\0162\027.hbase.pb.Scan.ReadType" +
"ionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hbase.pb.Sc" + ":\007DEFAULT\".\n\010ReadType\022\013\n\007DEFAULT\020\000\022\n\n\006ST" +
"an\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016number_of_rows" + "REAM\020\001\022\t\n\005PREAD\020\002\"\300\002\n\013ScanRequest\022)\n\006reg" +
"\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025\n\rnext_ca" + "ion\030\001 \001(\0132\031.hbase.pb.RegionSpecifier\022\034\n\004" +
"ll_seq\030\006 \001(\004\022\037\n\027client_handles_partials\030" + "scan\030\002 \001(\0132\016.hbase.pb.Scan\022\022\n\nscanner_id" +
"\007 \001(\010\022!\n\031client_handles_heartbeats\030\010 \001(\010" + "\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclose_" +
"\022\032\n\022track_scan_metrics\030\t \001(\010\022\024\n\005renew\030\n " + "scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027" +
"\001(\010:\005false\"\266\002\n\014ScanResponse\022\030\n\020cells_per", "client_handles_partials\030\007 \001(\010\022!\n\031client_",
"_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mor" + "handles_heartbeats\030\010 \001(\010\022\032\n\022track_scan_m" +
"e_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030" + "etrics\030\t \001(\010\022\024\n\005renew\030\n \001(\010:\005false\022\030\n\rli" +
"\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001(\010\022\037" + "mit_of_rows\030\013 \001(\r:\0010\"\266\002\n\014ScanResponse\022\030\n" +
"\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026more_" + "\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 " +
"results_in_region\030\010 \001(\010\022\031\n\021heartbeat_mes" + "\001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!" +
"sage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbase" + "\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005sta" +
".pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(\004" + "le\030\006 \001(\010\022\037\n\027partial_flag_per_result\030\007 \003(" +
":\0010\"\240\002\n\024BulkLoadHFileRequest\022)\n\006region\030\001" + "\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n\021hea" +
" \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013famil" + "rtbeat_message\030\t \001(\010\022+\n\014scan_metrics\030\n \001" +
"y_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReq", "(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_read_p",
"uest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\022" + "oint\030\013 \001(\004:\0010\"\240\002\n\024BulkLoadHFileRequest\022)" +
"+\n\010fs_token\030\004 \001(\0132\031.hbase.pb.DelegationT" + "\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifie" +
"oken\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tcopy_file\030\006 " + "r\022>\n\013family_path\030\002 \003(\0132).hbase.pb.BulkLo" +
"\001(\010:\005false\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014" + "adHFileRequest.FamilyPath\022\026\n\016assign_seq_" +
"\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022" + "num\030\003 \001(\010\022+\n\010fs_token\030\004 \001(\0132\031.hbase.pb.D" +
"\016\n\006loaded\030\001 \002(\010\"V\n\017DelegationToken\022\022\n\nid" + "elegationToken\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tco" +
"entifier\030\001 \001(\014\022\020\n\010password\030\002 \001(\014\022\014\n\004kind" + "py_file\030\006 \001(\010:\005false\032*\n\nFamilyPath\022\016\n\006fa" +
"\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026PrepareBulkLo" + "mily\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFil" +
"adRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb" + "eResponse\022\016\n\006loaded\030\001 \002(\010\"V\n\017DelegationT" +
".TableName\022)\n\006region\030\002 \001(\0132\031.hbase.pb.Re", "oken\022\022\n\nidentifier\030\001 \001(\014\022\020\n\010password\030\002 \001",
"gionSpecifier\"-\n\027PrepareBulkLoadResponse" + "(\014\022\014\n\004kind\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026Pre" +
"\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026CleanupBulkLoadR" + "pareBulkLoadRequest\022\'\n\ntable_name\030\001 \002(\0132" +
"equest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006region\030\002 \001" + "\023.hbase.pb.TableName\022)\n\006region\030\002 \001(\0132\031.h" +
"(\0132\031.hbase.pb.RegionSpecifier\"\031\n\027Cleanup" + "base.pb.RegionSpecifier\"-\n\027PrepareBulkLo" +
"BulkLoadResponse\"a\n\026CoprocessorServiceCa" + "adResponse\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026Cleanu" +
"ll\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n" + "pBulkLoadRequest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006" +
"\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030C" + "region\030\002 \001(\0132\031.hbase.pb.RegionSpecifier\"" +
"oprocessorServiceResult\022&\n\005value\030\001 \001(\0132\027" + "\031\n\027CleanupBulkLoadResponse\"a\n\026Coprocesso" +
".hbase.pb.NameBytesPair\"v\n\031CoprocessorSe" + "rServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_nam" +
"rviceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.", "e\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030",
"RegionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb" + "\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n\005val" +
".CoprocessorServiceCall\"o\n\032CoprocessorSe" + "ue\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n\031Cop" +
"rviceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb" + "rocessorServiceRequest\022)\n\006region\030\001 \002(\0132\031" +
".RegionSpecifier\022&\n\005value\030\002 \002(\0132\027.hbase." + ".hbase.pb.RegionSpecifier\022.\n\004call\030\002 \002(\0132" +
"pb.NameBytesPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001" + " .hbase.pb.CoprocessorServiceCall\"o\n\032Cop" +
"(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.pb.Mutation" + "rocessorServiceResponse\022)\n\006region\030\001 \002(\0132" +
"Proto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014ser" + "\031.hbase.pb.RegionSpecifier\022&\n\005value\030\002 \002(" +
"vice_call\030\004 \001(\0132 .hbase.pb.CoprocessorSe" + "\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Action\022\r\n" +
"rviceCall\"k\n\014RegionAction\022)\n\006region\030\001 \002(" + "\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.p" +
"\0132\031.hbase.pb.RegionSpecifier\022\016\n\006atomic\030\002", "b.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.",
" \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Action\"c" + "Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb.Cop" +
"\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:" + "rocessorServiceCall\"k\n\014RegionAction\022)\n\006r" +
"\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compacti" + "egion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\016" +
"onPressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadSt" + "\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.p" +
"ats\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpe" + "b.Action\"c\n\017RegionLoadStats\022\027\n\014memstoreL" +
"cifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLo" + "oad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035" +
"adStats\"\336\001\n\021ResultOrException\022\r\n\005index\030\001" + "\n\022compactionPressure\030\003 \001(\005:\0010\"j\n\024MultiRe" +
" \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*" + "gionLoadStats\022)\n\006region\030\001 \003(\0132\031.hbase.pb" +
"\n\texception\030\003 \001(\0132\027.hbase.pb.NameBytesPa" + ".RegionSpecifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.p" +
"ir\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Co", "b.RegionLoadStats\"\336\001\n\021ResultOrException\022",
"processorServiceResult\0220\n\tloadStats\030\005 \001(" + "\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.p" +
"\0132\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Regi" + "b.Result\022*\n\texception\030\003 \001(\0132\027.hbase.pb.N" +
"onActionResult\0226\n\021resultOrException\030\001 \003(" + "ameBytesPair\022:\n\016service_result\030\004 \001(\0132\".h" +
"\0132\033.hbase.pb.ResultOrException\022*\n\texcept" + "base.pb.CoprocessorServiceResult\0220\n\tload" +
"ion\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mu" + "Stats\030\005 \001(\0132\031.hbase.pb.RegionLoadStatsB\002" +
"ltiRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase" + "\030\001\"x\n\022RegionActionResult\0226\n\021resultOrExce" +
".pb.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\t" + "ption\030\001 \003(\0132\033.hbase.pb.ResultOrException" +
"condition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n" + "\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameBytes" +
"\rMultiResponse\0228\n\022regionActionResult\030\001 \003" + "Pair\"x\n\014MultiRequest\022,\n\014regionAction\030\001 \003" +
"(\0132\034.hbase.pb.RegionActionResult\022\021\n\tproc", "(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceGroup",
"essed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036." + "\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb.Con" +
"hbase.pb.MultiRegionLoadStats*\'\n\013Consist" + "dition\"\226\001\n\rMultiResponse\0228\n\022regionAction" +
"ency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\263\005\n\rClien" + "Result\030\001 \003(\0132\034.hbase.pb.RegionActionResu" +
"tService\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025." + "lt\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regionStatistic" +
"hbase.pb.GetResponse\022;\n\006Mutate\022\027.hbase.p" + "s\030\003 \001(\0132\036.hbase.pb.MultiRegionLoadStats*" +
"b.MutateRequest\032\030.hbase.pb.MutateRespons" + "\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\001" +
"e\0225\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase" + "2\263\005\n\rClientService\0222\n\003Get\022\024.hbase.pb.Get" +
".pb.ScanResponse\022P\n\rBulkLoadHFile\022\036.hbas" + "Request\032\025.hbase.pb.GetResponse\022;\n\006Mutate" +
"e.pb.BulkLoadHFileRequest\032\037.hbase.pb.Bul" + "\022\027.hbase.pb.MutateRequest\032\030.hbase.pb.Mut" +
"kLoadHFileResponse\022V\n\017PrepareBulkLoad\022 .", "ateResponse\0225\n\004Scan\022\025.hbase.pb.ScanReque",
"hbase.pb.PrepareBulkLoadRequest\032!.hbase." + "st\032\026.hbase.pb.ScanResponse\022P\n\rBulkLoadHF" +
"pb.PrepareBulkLoadResponse\022V\n\017CleanupBul" + "ile\022\036.hbase.pb.BulkLoadHFileRequest\032\037.hb" +
"kLoad\022 .hbase.pb.CleanupBulkLoadRequest\032" + "ase.pb.BulkLoadHFileResponse\022V\n\017PrepareB" +
"!.hbase.pb.CleanupBulkLoadResponse\022X\n\013Ex" + "ulkLoad\022 .hbase.pb.PrepareBulkLoadReques" +
"ecService\022#.hbase.pb.CoprocessorServiceR" + "t\032!.hbase.pb.PrepareBulkLoadResponse\022V\n\017" +
"equest\032$.hbase.pb.CoprocessorServiceResp" + "CleanupBulkLoad\022 .hbase.pb.CleanupBulkLo" +
"onse\022d\n\027ExecRegionServerService\022#.hbase." + "adRequest\032!.hbase.pb.CleanupBulkLoadResp" +
"pb.CoprocessorServiceRequest\032$.hbase.pb." + "onse\022X\n\013ExecService\022#.hbase.pb.Coprocess" +
"CoprocessorServiceResponse\0228\n\005Multi\022\026.hb" + "orServiceRequest\032$.hbase.pb.CoprocessorS" +
"ase.pb.MultiRequest\032\027.hbase.pb.MultiResp", "erviceResponse\022d\n\027ExecRegionServerServic",
"onseBB\n*org.apache.hadoop.hbase.protobuf" + "e\022#.hbase.pb.CoprocessorServiceRequest\032$" +
".generatedB\014ClientProtosH\001\210\001\001\240\001\001" ".hbase.pb.CoprocessorServiceResponse\0228\n\005" +
"Multi\022\026.hbase.pb.MultiRequest\032\027.hbase.pb" +
".MultiResponseBB\n*org.apache.hadoop.hbas" +
"e.protobuf.generatedB\014ClientProtosH\001\210\001\001\240" +
"\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -40108,13 +40424,13 @@ public final class ClientProtos {
internal_static_hbase_pb_Scan_fieldAccessorTable = new internal_static_hbase_pb_Scan_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_Scan_descriptor, 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", "IncludeStartRow", "IncludeStopRow", }); 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", "ReadType", });
internal_static_hbase_pb_ScanRequest_descriptor = internal_static_hbase_pb_ScanRequest_descriptor =
getDescriptor().getMessageTypes().get(12); getDescriptor().getMessageTypes().get(12);
internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_ScanRequest_descriptor, internal_static_hbase_pb_ScanRequest_descriptor,
new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", }); new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", "LimitOfRows", });
internal_static_hbase_pb_ScanResponse_descriptor = internal_static_hbase_pb_ScanResponse_descriptor =
getDescriptor().getMessageTypes().get(13); getDescriptor().getMessageTypes().get(13);
internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new

View File

@ -249,7 +249,7 @@ message Scan {
optional uint32 store_limit = 11; optional uint32 store_limit = 11;
optional uint32 store_offset = 12; optional uint32 store_offset = 12;
optional bool load_column_families_on_demand = 13; /* DO NOT add defaults to load_column_families_on_demand. */ optional bool load_column_families_on_demand = 13; /* DO NOT add defaults to load_column_families_on_demand. */
optional bool small = 14; optional bool small = 14 [deprecated = true];
optional bool reversed = 15 [default = false]; optional bool reversed = 15 [default = false];
optional Consistency consistency = 16 [default = STRONG]; optional Consistency consistency = 16 [default = STRONG];
optional uint32 caching = 17; optional uint32 caching = 17;
@ -258,6 +258,13 @@ message Scan {
optional uint64 mvcc_read_point = 20 [default = 0]; optional uint64 mvcc_read_point = 20 [default = 0];
optional bool include_start_row = 21 [default = true]; optional bool include_start_row = 21 [default = true];
optional bool include_stop_row = 22 [default = false]; optional bool include_stop_row = 22 [default = false];
enum ReadType {
DEFAULT = 0;
STREAM = 1;
PREAD = 2;
}
optional ReadType readType = 23 [default = DEFAULT];
} }
/** /**
@ -282,6 +289,8 @@ message ScanRequest {
optional bool client_handles_heartbeats = 8; optional bool client_handles_heartbeats = 8;
optional bool track_scan_metrics = 9; optional bool track_scan_metrics = 9;
optional bool renew = 10 [default = false]; optional bool renew = 10 [default = false];
// if we have returned limit_of_rows rows to client, then close the scanner.
optional uint32 limit_of_rows = 11 [default = 0];
} }
/** /**

View File

@ -1133,6 +1133,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
} }
} }
@VisibleForTesting
public int getScannersCount() {
return scanners.size();
}
public public
RegionScanner getScanner(long scannerId) { RegionScanner getScanner(long scannerId) {
String scannerIdString = Long.toString(scannerId); String scannerIdString = Long.toString(scannerId);
@ -3014,6 +3019,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
RegionScanner scanner = rsh.s; RegionScanner scanner = rsh.s;
boolean moreResults = true; boolean moreResults = true;
boolean moreResultsInRegion = true; boolean moreResultsInRegion = true;
// this is the limit of rows for this scan, if we the number of rows reach this value, we will
// close the scanner.
int limitOfRows;
if (request.hasLimitOfRows()) {
limitOfRows = request.getLimitOfRows();
rows = Math.min(rows, limitOfRows);
} else {
limitOfRows = -1;
}
MutableObject lastBlock = new MutableObject(); MutableObject lastBlock = new MutableObject();
boolean scannerClosed = false; boolean scannerClosed = false;
try { try {
@ -3046,6 +3060,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// with the old scan implementation where we just ignore the returned results if moreResults // with the old scan implementation where we just ignore the returned results if moreResults
// is false. Can remove the isEmpty check after we get rid of the old implementation. // is false. Can remove the isEmpty check after we get rid of the old implementation.
moreResults = false; moreResults = false;
} else if (limitOfRows > 0 && results.size() >= limitOfRows
&& !results.get(results.size() - 1).isPartial()) {
// if we have reached the limit of rows
moreResults = false;
} }
addResults(builder, results, (HBaseRpcController) controller, addResults(builder, results, (HBaseRpcController) controller,
RegionReplicaUtil.isDefaultReplica(region.getRegionInfo()), RegionReplicaUtil.isDefaultReplica(region.getRegionInfo()),

View File

@ -127,7 +127,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
protected Cell lastTop = null; protected Cell lastTop = null;
// A flag whether use pread for scan // A flag whether use pread for scan
private boolean scanUsePread = false; private final boolean scanUsePread;
// Indicates whether there was flush during the course of the scan // Indicates whether there was flush during the course of the scan
protected volatile boolean flushed = false; protected volatile boolean flushed = false;
// generally we get one file from a flush // generally we get one file from a flush
@ -168,7 +168,21 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
this.useRowColBloom = numCol > 1 || (!get && numCol == 1); this.useRowColBloom = numCol > 1 || (!get && numCol == 1);
this.maxRowSize = scanInfo.getTableMaxRowSize(); this.maxRowSize = scanInfo.getTableMaxRowSize();
this.scanUsePread = scan.isSmall()? true: scanInfo.isUsePread(); if (get) {
this.scanUsePread = true;
} else {
switch (scan.getReadType()) {
case STREAM:
this.scanUsePread = false;
break;
case PREAD:
this.scanUsePread = true;
break;
default:
this.scanUsePread = scanInfo.isUsePread();
break;
}
}
this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck(); this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck();
// Parallel seeking is on if the config allows and more there is more than one store file. // Parallel seeking is on if the config allows and more there is more than one store file.
if (this.store != null && this.store.getStorefilesCount() > 1) { if (this.store != null && this.store.getStorefilesCount() > 1) {
@ -348,10 +362,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @return list of scanners to seek * @return list of scanners to seek
*/ */
protected List<KeyValueScanner> getScannersNoCompaction() throws IOException { protected List<KeyValueScanner> getScannersNoCompaction() throws IOException {
final boolean isCompaction = false; return selectScannersFrom(store.getScanners(cacheBlocks, get, scanUsePread, false, matcher,
boolean usePread = get || scanUsePread; scan.getStartRow(), scan.getStopRow(), this.readPt));
return selectScannersFrom(store.getScanners(cacheBlocks, get, usePread,
isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt));
} }
/** /**
@ -803,18 +815,14 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
} }
protected void resetScannerStack(Cell lastTopKey) throws IOException { protected void resetScannerStack(Cell lastTopKey) throws IOException {
/* When we have the scan object, should we not pass it to getScanners() // When we have the scan object, should we not pass it to getScanners() to get a limited set of
* to get a limited set of scanners? We did so in the constructor and we // scanners? We did so in the constructor and we could have done it now by storing the scan
* could have done it now by storing the scan object from the constructor // object from the constructor
*/
final boolean isCompaction = false;
boolean usePread = get || scanUsePread;
List<KeyValueScanner> scanners = null; List<KeyValueScanner> scanners = null;
try { try {
flushLock.lock(); flushLock.lock();
scanners = selectScannersFrom(store.getScanners(flushedStoreFiles, cacheBlocks, get, usePread, scanners = selectScannersFrom(store.getScanners(flushedStoreFiles, cacheBlocks, get,
isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, true)); scanUsePread, false, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, true));
// Clear the current set of flushed store files so that they don't get added again // Clear the current set of flushed store files so that they don't get added again
flushedStoreFiles.clear(); flushedStoreFiles.clear();
} finally { } finally {

View File

@ -95,6 +95,12 @@ public abstract class AbstractTestAsyncTableScan {
@Test @Test
public void testScanAll() throws Exception { public void testScanAll() throws Exception {
List<Result> results = doScan(createScan()); List<Result> results = doScan(createScan());
// make sure all scanners are closed at RS side
TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
.forEach(rs -> assertEquals(
"The scanner count of " + rs.getServerName() + " is "
+ rs.getRSRpcServices().getScannersCount(),
0, rs.getRSRpcServices().getScannersCount()));
assertEquals(COUNT, results.size()); assertEquals(COUNT, results.size());
IntStream.range(0, COUNT).forEach(i -> { IntStream.range(0, COUNT).forEach(i -> {
Result result = results.get(i); Result result = results.get(i);

View File

@ -26,6 +26,7 @@ import java.util.concurrent.ForkJoinPool;
import java.util.function.Supplier; import java.util.function.Supplier;
import java.util.stream.IntStream; import java.util.stream.IntStream;
import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -38,11 +39,20 @@ import org.junit.runners.Parameterized.Parameters;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
@Category({ MediumTests.class, ClientTests.class }) @Category({ MediumTests.class, ClientTests.class })
public class TestAsyncTableSmallScan extends AbstractTestAsyncTableScan { public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
@Parameter @Parameter(0)
public String tableType;
@Parameter(1)
public Supplier<AsyncTableBase> getTable; public Supplier<AsyncTableBase> getTable;
@Parameter(2)
public String scanType;
@Parameter(3)
public Supplier<Scan> scanCreator;
private static RawAsyncTable getRawTable() { private static RawAsyncTable getRawTable() {
return ASYNC_CONN.getRawTable(TABLE_NAME); return ASYNC_CONN.getRawTable(TABLE_NAME);
} }
@ -51,24 +61,37 @@ public class TestAsyncTableSmallScan extends AbstractTestAsyncTableScan {
return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool()); return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
} }
@Parameters private static Scan createNormalScan() {
return new Scan();
}
// test if we can handle partial result when open scanner.
private static Scan createSmallResultSizeScan() {
return new Scan().setMaxResultSize(1);
}
@Parameters(name = "{index}: table={0}, scan={2}")
public static List<Object[]> params() { public static List<Object[]> params() {
return Arrays.asList(new Supplier<?>[] { TestAsyncTableSmallScan::getRawTable }, Supplier<AsyncTableBase> rawTable = TestAsyncTableScanAll::getRawTable;
new Supplier<?>[] { TestAsyncTableSmallScan::getTable }); Supplier<AsyncTableBase> normalTable = TestAsyncTableScanAll::getTable;
Supplier<Scan> normalScan = TestAsyncTableScanAll::createNormalScan;
Supplier<Scan> smallResultSizeScan = TestAsyncTableScanAll::createSmallResultSizeScan;
return Arrays.asList(new Object[] { "raw", rawTable, "normal", normalScan },
new Object[] { "raw", rawTable, "smallResultSize", smallResultSizeScan },
new Object[] { "normal", normalTable, "normal", normalScan },
new Object[] { "normal", normalTable, "smallResultSize", smallResultSizeScan });
} }
@Test @Test
public void testScanWithLimit() throws InterruptedException, ExecutionException { public void testScanWithLimit() throws InterruptedException, ExecutionException {
AsyncTableBase table = getTable.get();
int start = 111; int start = 111;
int stop = 888; int stop = 888;
int limit = 300; int limit = 300;
List<Result> results = List<Result> results = getTable.get()
table .scanAll(scanCreator.get().withStartRow(Bytes.toBytes(String.format("%03d", start)))
.smallScan(new Scan(Bytes.toBytes(String.format("%03d", start))) .withStopRow(Bytes.toBytes(String.format("%03d", stop))).setLimit(limit)
.setStopRow(Bytes.toBytes(String.format("%03d", stop))).setSmall(true), .setReadType(ReadType.PREAD))
limit) .get();
.get();
assertEquals(limit, results.size()); assertEquals(limit, results.size());
IntStream.range(0, limit).forEach(i -> { IntStream.range(0, limit).forEach(i -> {
Result result = results.get(i); Result result = results.get(i);
@ -80,14 +103,14 @@ public class TestAsyncTableSmallScan extends AbstractTestAsyncTableScan {
@Test @Test
public void testReversedScanWithLimit() throws InterruptedException, ExecutionException { public void testReversedScanWithLimit() throws InterruptedException, ExecutionException {
AsyncTableBase table = getTable.get();
int start = 888; int start = 888;
int stop = 111; int stop = 111;
int limit = 300; int limit = 300;
List<Result> results = table.smallScan( List<Result> results = getTable.get()
new Scan(Bytes.toBytes(String.format("%03d", start))) .scanAll(scanCreator.get().withStartRow(Bytes.toBytes(String.format("%03d", start)))
.setStopRow(Bytes.toBytes(String.format("%03d", stop))).setSmall(true).setReversed(true), .withStopRow(Bytes.toBytes(String.format("%03d", stop))).setLimit(limit)
limit).get(); .setReadType(ReadType.PREAD).setReversed(true))
.get();
assertEquals(limit, results.size()); assertEquals(limit, results.size());
IntStream.range(0, limit).forEach(i -> { IntStream.range(0, limit).forEach(i -> {
Result result = results.get(i); Result result = results.get(i);
@ -99,11 +122,11 @@ public class TestAsyncTableSmallScan extends AbstractTestAsyncTableScan {
@Override @Override
protected Scan createScan() { protected Scan createScan() {
return new Scan().setSmall(true); return scanCreator.get();
} }
@Override @Override
protected List<Result> doScan(Scan scan) throws Exception { protected List<Result> doScan(Scan scan) throws Exception {
return getTable.get().smallScan(scan).get(); return getTable.get().scanAll(scan).get();
} }
} }

View File

@ -56,11 +56,6 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
return true; return true;
} }
@Override
public boolean onHeartbeat() {
return true;
}
@Override @Override
public synchronized void onError(Throwable error) { public synchronized void onError(Throwable error) {
finished = true; finished = true;

View File

@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -369,7 +370,10 @@ public class TestMultiRowRangeFilter {
@Test @Test
public void testMultiRowRangeFilterWithExclusive() throws IOException { public void testMultiRowRangeFilterWithExclusive() throws IOException {
tableName = TableName.valueOf("testMultiRowRangeFilterWithExclusive"); tableName = TableName.valueOf("testMultiRowRangeFilterWithExclusive");
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 6000000);
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE); Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
ht.setReadRpcTimeout(600000);
ht.setOperationTimeout(6000000);
generateRows(numRows, ht, family, qf, value); generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan(); Scan scan = new Scan();