caller) {
- scan.setStartRow(localStartKey);
- SmallScannerCallable s = new SmallScannerCallable(
- connection, table, scan, scanMetrics, controllerFactory, cacheNum, 0);
- ScannerCallableWithReplicas scannerCallableWithReplicas =
- new ScannerCallableWithReplicas(table, connection,
- s, pool, primaryOperationTimeout, scan, retries,
- scannerTimeout, cacheNum, conf, caller);
- return scannerCallableWithReplicas;
- }
-
- }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index 5f5badf586c..ab6cb8d518a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -1257,8 +1257,7 @@ class ConnectionManager {
Scan s = new Scan();
s.setReversed(true);
s.setStartRow(metaKey);
- s.setSmall(true);
- s.setCaching(1);
+ s.setOneRowLimit();
if (this.useMetaReplicas) {
s.setConsistency(Consistency.TIMELINE);
}
@@ -1286,15 +1285,11 @@ class ConnectionManager {
long pauseBase = this.pause;
try {
Result regionInfoRow = null;
- ReversedClientScanner rcs = null;
- try {
- rcs = new ClientSmallReversedScanner(conf, s, TableName.META_TABLE_NAME, this,
- rpcCallerFactory, rpcControllerFactory, getMetaLookupPool(), 0);
+ s.resetMvccReadPoint();
+ try (ReversedClientScanner rcs =
+ new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory,
+ rpcControllerFactory, getMetaLookupPool(), 0)) {
regionInfoRow = rcs.next();
- } finally {
- if (rcs != null) {
- rcs.close();
- }
}
if (regionInfoRow == null) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index c1e6d23d3e1..e7b41142e39 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -17,14 +17,16 @@
*/
package org.apache.hadoop.hbase.client;
+import com.google.common.annotations.VisibleForTesting;
+
import java.io.IOException;
+import java.util.Arrays;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -32,8 +34,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
-
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.util.Bytes;
/**
* Utility used by client connections.
@@ -195,4 +196,36 @@ public class ConnectionUtils {
return false;
}
}
+
+ // A byte array in which all elements are the max byte, and it is used to
+ // construct closest front row
+ static final byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
+
+ /**
+ * Create the closest row after the specified row
+ */
+ static byte[] createClosestRowAfter(byte[] row) {
+ return Arrays.copyOf(row, row.length + 1);
+ }
+
+ /**
+ * Create the closest row before the specified row
+ * @deprecated in fact, we do not know the closest row before the given row, the result is only a
+ * row very close to the current row. Avoid using this method in the future.
+ */
+ @Deprecated
+ static byte[] createClosestRowBefore(byte[] row) {
+ if (row.length == 0) {
+ return MAX_BYTE_ARRAY;
+ }
+ if (row[row.length - 1] == 0) {
+ return Arrays.copyOf(row, row.length - 1);
+ } else {
+ byte[] nextRow = new byte[row.length + MAX_BYTE_ARRAY.length];
+ System.arraycopy(row, 0, nextRow, 0, row.length - 1);
+ nextRow[row.length - 1] = (byte) ((row[row.length - 1] & 0xFF) - 1);
+ System.arraycopy(MAX_BYTE_ARRAY, 0, nextRow, row.length, MAX_BYTE_ARRAY.length);
+ return nextRow;
+ }
+ }
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 83e7217afcb..1e3a900c2f0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -795,21 +795,9 @@ public class HTable implements HTableInterface, RegionLocator {
}
if (scan.isReversed()) {
- if (scan.isSmall()) {
- return new ClientSmallReversedScanner(getConfiguration(), scan, getName(),
- this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
- pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
- } else {
- return new ReversedClientScanner(getConfiguration(), scan, getName(),
- this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
- pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
- }
- }
-
- if (scan.isSmall()) {
- return new ClientSmallScanner(getConfiguration(), scan, getName(),
- this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
- pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
+ return new ReversedClientScanner(getConfiguration(), scan, getName(),
+ this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
+ pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
} else {
return new ClientScanner(getConfiguration(), scan, getName(), this.connection,
this.rpcCallerFactory, this.rpcControllerFactory,
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
index ca998ae26b5..edb66c5a1d7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
@@ -18,15 +18,18 @@
*/
package org.apache.hadoop.hbase.client;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowBefore;
+
import java.io.IOException;
import java.util.concurrent.ExecutorService;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ExceptionUtil;
@@ -58,8 +61,7 @@ public class ReversedClientScanner extends ClientScanner {
}
@Override
- protected boolean nextScanner(int nbRows, final boolean done)
- throws IOException {
+ protected Result[] nextScanner(int nbRows) throws IOException {
// Close the previous scanner if it's open
closeScanner();
@@ -69,16 +71,17 @@ public class ReversedClientScanner extends ClientScanner {
// if we're at start of table, close and return false to stop iterating
if (this.currentRegion != null) {
byte[] startKey = this.currentRegion.getStartKey();
- if (startKey == null
- || Bytes.equals(startKey, HConstants.EMPTY_BYTE_ARRAY)
- || checkScanStopRow(startKey) || done) {
+ if (startKey == null || Bytes.equals(startKey, HConstants.EMPTY_BYTE_ARRAY)
+ || checkScanStopRow(startKey)) {
close();
if (LOG.isDebugEnabled()) {
LOG.debug("Finished " + this.currentRegion);
}
- return false;
+ return null;
}
localStartKey = startKey;
+ // clear mvcc read point if we are going to switch regions
+ scan.resetMvccReadPoint();
if (LOG.isDebugEnabled()) {
LOG.debug("Finished " + this.currentRegion);
}
@@ -109,17 +112,21 @@ public class ReversedClientScanner extends ClientScanner {
// beginning of the region
// callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas,
// we do a callWithRetries
- this.caller.callWithoutRetries(callable, scannerTimeout);
+ Result[] rrs = this.caller.callWithoutRetries(callable, scannerTimeout);
this.currentRegion = callable.getHRegionInfo();
if (this.scanMetrics != null) {
this.scanMetrics.countOfRegions.incrementAndGet();
}
+ if (rrs != null && rrs.length == 0 && callable.moreResultsForScan() == MoreResults.NO) {
+ // no results for the scan, return null to terminate the scan.
+ return null;
+ }
+ return rrs;
} catch (IOException e) {
ExceptionUtil.rethrowIfInterrupt(e);
close();
throw e;
}
- return true;
}
protected ScannerCallableWithReplicas getScannerCallable(byte[] localStartKey,
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
index e169f7a018e..840af97f90c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
@@ -44,7 +44,7 @@ public class ReversedScannerCallable extends ScannerCallable {
/**
* The start row for locating regions. In reversed scanner, may locate the
* regions for a range of keys when doing
- * {@link ReversedClientScanner#nextScanner(int, boolean)}
+ * {@link ReversedClientScanner#nextScanner(int)}
*/
protected final byte[] locateStartRow;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
index 128e7e1c1a7..84f1ca92e2d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
@@ -174,6 +174,17 @@ public class Scan extends Query {
*/
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.
*/
@@ -253,6 +264,7 @@ public class Scan extends Query {
setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
}
this.mvccReadPoint = scan.getMvccReadPoint();
+ this.limit = scan.getLimit();
}
/**
@@ -1013,6 +1025,62 @@ public class Scan extends Query {
return ProtobufUtil.toScanMetrics(bytes);
}
+ /**
+ * @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.
+ *
+ * This condition will be tested at last, after all other conditions such as stopRow, filter, etc.
+ *
+ * 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;
+ }
+
+ /**
+ * Call this when you only want to get one row. It will set {@code limit} to {@code 1}, and also
+ * set {@code readType} to {@link ReadType#PREAD}.
+ * @return this
+ */
+ public Scan setOneRowLimit() {
+ return setLimit(1).setReadType(ReadType.PREAD);
+ }
+
+ @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.
+ *
+ * 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.
*/
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index 2dee7ceaa54..55be6da5fab 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -18,6 +18,9 @@
package org.apache.hadoop.hbase.client;
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.TextFormat;
+
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.UnknownHostException;
@@ -26,10 +29,8 @@ import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
@@ -37,10 +38,10 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@@ -51,12 +52,8 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
-import com.google.protobuf.ServiceException;
-import com.google.protobuf.TextFormat;
-
/**
* Scanner operations such as create, next, etc.
* Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as
@@ -82,9 +79,15 @@ public class ScannerCallable extends RegionServerCallable {
private int logCutOffLatency = 1000;
private static String myAddress;
protected final int id;
- protected boolean serverHasMoreResultsContext;
- protected boolean serverHasMoreResults;
+ enum MoreResults {
+ YES, NO, UNKNOWN
+ }
+
+ private MoreResults moreResultsInRegion;
+ private MoreResults moreResultsForScan;
+
+ private boolean openScanner;
/**
* Saves whether or not the most recent response from the server was a heartbeat message.
* Heartbeat messages are identified by the flag {@link ScanResponse#getHeartbeatMessage()}
@@ -136,6 +139,7 @@ public class ScannerCallable extends RegionServerCallable {
logScannerActivity = conf.getBoolean(LOG_SCANNER_ACTIVITY, false);
logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000);
this.controllerFactory = rpcControllerFactory;
+ this.controller = rpcControllerFactory.newController();
}
PayloadCarryingRpcController getController() {
@@ -189,135 +193,124 @@ public class ScannerCallable extends RegionServerCallable {
}
}
+ private ScanResponse next() throws IOException {
+ // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server
+ setHeartbeatMessage(false);
+ incRPCcallsMetrics();
+ ScanRequest request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
+ this.scanMetrics != null, renew, scan.getLimit());
+ try {
+ ScanResponse response = getStub().scan(controller, request);
+ nextCallSeq++;
+ return response;
+ } catch (Exception e) {
+ IOException ioe = ProtobufUtil.handleRemoteException(e);
+ if (logScannerActivity) {
+ LOG.info("Got exception making request " + TextFormat.shortDebugString(request) + " to " +
+ getLocation(),
+ e);
+ }
+ if (logScannerActivity) {
+ if (ioe instanceof UnknownScannerException) {
+ try {
+ HRegionLocation location =
+ getConnection().relocateRegion(getTableName(), scan.getStartRow());
+ LOG.info("Scanner=" + scannerId + " expired, current region location is "
+ + location.toString());
+ } catch (Throwable t) {
+ LOG.info("Failed to relocate region", t);
+ }
+ } else if (ioe instanceof ScannerResetException) {
+ LOG.info("Scanner=" + scannerId + " has received an exception, and the server "
+ + "asked us to reset the scanner state.",
+ ioe);
+ }
+ }
+ // The below convertion of exceptions into DoNotRetryExceptions is a little strange.
+ // Why not just have these exceptions implment DNRIOE you ask? Well, usually we want
+ // ServerCallable#withRetries to just retry when it gets these exceptions. In here in
+ // a scan when doing a next in particular, we want to break out and get the scanner to
+ // reset itself up again. Throwing a DNRIOE is how we signal this to happen (its ugly,
+ // yeah and hard to follow and in need of a refactor).
+ if (ioe instanceof NotServingRegionException) {
+ // Throw a DNRE so that we break out of cycle of calling NSRE
+ // when what we need is to open scanner against new location.
+ // Attach NSRE to signal client that it needs to re-setup scanner.
+ if (this.scanMetrics != null) {
+ this.scanMetrics.countOfNSRE.incrementAndGet();
+ }
+ throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe);
+ } else if (ioe instanceof RegionServerStoppedException) {
+ // Throw a DNRE so that we break out of cycle of the retries and instead go and
+ // open scanner against new location.
+ throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe);
+ } else {
+ // The outer layers will retry
+ throw ioe;
+ }
+ }
+ }
+
+ private void setAlreadyClosed() {
+ this.scannerId = -1L;
+ this.closed = true;
+ }
@Override
- public Result [] call(int callTimeout) throws IOException {
+ public Result[] call(int callTimeout) throws IOException {
if (Thread.interrupted()) {
throw new InterruptedIOException();
}
-
- if (controller == null) {
- controller = controllerFactory.newController();
- controller.setPriority(getTableName());
- controller.setCallTimeout(callTimeout);
- }
-
if (closed) {
- if (scannerId != -1) {
- close();
+ close();
+ return null;
+ }
+ controller.reset();
+ controller.setPriority(getTableName());
+ controller.setCallTimeout(callTimeout);
+ ScanResponse response;
+ if (this.scannerId == -1L) {
+ this.openScanner = true;
+ response = openScanner();
+ } else {
+ this.openScanner = false;
+ response = next();
+ }
+ long timestamp = System.currentTimeMillis();
+ setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
+ Result[] rrs = ResponseConverter.getResults(controller.cellScanner(), response);
+ if (logScannerActivity) {
+ long now = System.currentTimeMillis();
+ if (now - timestamp > logCutOffLatency) {
+ int rows = rrs == null ? 0 : rrs.length;
+ LOG.info("Took " + (now - timestamp) + "ms to fetch " + rows + " rows from scanner="
+ + scannerId);
+ }
+ }
+ updateServerSideMetrics(response);
+ // moreResults is only used for the case where a filter exhausts all elements
+ if (response.hasMoreResults()) {
+ if (response.getMoreResults()) {
+ setMoreResultsForScan(MoreResults.YES);
+ } else {
+ setMoreResultsForScan(MoreResults.NO);
+ setAlreadyClosed();
}
} else {
- if (scannerId == -1L) {
- this.scannerId = openScanner();
- } else {
- Result [] rrs = null;
- ScanRequest request = null;
- // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server
- setHeartbeatMessage(false);
- try {
- incRPCcallsMetrics();
- request =
- RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
- this.scanMetrics != null, renew);
- ScanResponse response = null;
- try {
- response = getStub().scan(controller, request);
- // Client and RS maintain a nextCallSeq number during the scan. Every next() call
- // from client to server will increment this number in both sides. Client passes this
- // number along with the request and at RS side both the incoming nextCallSeq and its
- // nextCallSeq will be matched. In case of a timeout this increment at the client side
- // should not happen. If at the server side fetching of next batch of data was over,
- // there will be mismatch in the nextCallSeq number. Server will throw
- // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
- // as the last successfully retrieved row.
- // See HBASE-5974
- nextCallSeq++;
- long timestamp = System.currentTimeMillis();
- setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
- // Results are returned via controller
- CellScanner cellScanner = controller.cellScanner();
- rrs = ResponseConverter.getResults(cellScanner, response);
- if (logScannerActivity) {
- long now = System.currentTimeMillis();
- if (now - timestamp > logCutOffLatency) {
- int rows = rrs == null ? 0 : rrs.length;
- LOG.info("Took " + (now-timestamp) + "ms to fetch "
- + rows + " rows from scanner=" + scannerId);
- }
- }
- updateServerSideMetrics(response);
- // moreResults is only used for the case where a filter exhausts all elements
- if (response.hasMoreResults() && !response.getMoreResults()) {
- scannerId = -1L;
- closed = true;
- // Implied that no results were returned back, either.
- return null;
- }
- // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
- // to size or quantity of results in the response.
- if (response.hasMoreResultsInRegion()) {
- // Set what the RS said
- setHasMoreResultsContext(true);
- setServerHasMoreResults(response.getMoreResultsInRegion());
- } else {
- // Server didn't respond whether it has more results or not.
- setHasMoreResultsContext(false);
- }
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
- }
- updateResultsMetrics(rrs);
- } catch (IOException e) {
- if (logScannerActivity) {
- LOG.info("Got exception making request " + TextFormat.shortDebugString(request)
- + " to " + getLocation(), e);
- }
- IOException ioe = e;
- if (e instanceof RemoteException) {
- ioe = RemoteExceptionHandler.decodeRemoteException((RemoteException)e);
- }
- if (logScannerActivity) {
- if (ioe instanceof UnknownScannerException) {
- try {
- HRegionLocation location =
- getConnection().relocateRegion(getTableName(), scan.getStartRow());
- LOG.info("Scanner=" + scannerId
- + " expired, current region location is " + location.toString());
- } catch (Throwable t) {
- LOG.info("Failed to relocate region", t);
- }
- } else if (ioe instanceof ScannerResetException) {
- LOG.info("Scanner=" + scannerId + " has received an exception, and the server "
- + "asked us to reset the scanner state.", ioe);
- }
- }
- // The below convertion of exceptions into DoNotRetryExceptions is a little strange.
- // Why not just have these exceptions implment DNRIOE you ask? Well, usually we want
- // ServerCallable#withRetries to just retry when it gets these exceptions. In here in
- // a scan when doing a next in particular, we want to break out and get the scanner to
- // reset itself up again. Throwing a DNRIOE is how we signal this to happen (its ugly,
- // yeah and hard to follow and in need of a refactor).
- if (ioe instanceof NotServingRegionException) {
- // Throw a DNRE so that we break out of cycle of calling NSRE
- // when what we need is to open scanner against new location.
- // Attach NSRE to signal client that it needs to re-setup scanner.
- if (this.scanMetrics != null) {
- this.scanMetrics.countOfNSRE.incrementAndGet();
- }
- throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe);
- } else if (ioe instanceof RegionServerStoppedException) {
- // Throw a DNRE so that we break out of cycle of the retries and instead go and
- // open scanner against new location.
- throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe);
- } else {
- // The outer layers will retry
- throw ioe;
- }
- }
- return rrs;
- }
+ setMoreResultsForScan(MoreResults.UNKNOWN);
}
- return null;
+ if (response.hasMoreResultsInRegion()) {
+ if (response.getMoreResultsInRegion()) {
+ setMoreResultsInRegion(MoreResults.YES);
+ } else {
+ setMoreResultsInRegion(MoreResults.NO);
+ setAlreadyClosed();
+ }
+ } else {
+ setMoreResultsInRegion(MoreResults.UNKNOWN);
+ }
+ updateResultsMetrics(rrs);
+ return rrs;
}
/**
@@ -326,11 +319,11 @@ public class ScannerCallable extends RegionServerCallable {
* scan request exceeds a certain time threshold. Heartbeats allow the server to avoid
* timeouts during long running scan operations.
*/
- protected boolean isHeartbeatMessage() {
+ boolean isHeartbeatMessage() {
return heartbeatMessage;
}
- protected void setHeartbeatMessage(boolean heartbeatMessage) {
+ private void setHeartbeatMessage(boolean heartbeatMessage) {
this.heartbeatMessage = heartbeatMessage;
}
@@ -397,12 +390,10 @@ public class ScannerCallable extends RegionServerCallable {
this.scannerId = -1L;
}
- protected long openScanner() throws IOException {
+ private ScanResponse openScanner() throws IOException {
incRPCcallsMetrics();
- ScanRequest request =
- RequestConverter.buildScanRequest(
- getLocation().getRegionInfo().getRegionName(),
- this.scan, 0, false);
+ ScanRequest request = RequestConverter.buildScanRequest(
+ getLocation().getRegionInfo().getRegionName(), this.scan, this.caching, false);
try {
ScanResponse response = getStub().scan(controller, request);
long id = response.getScannerId();
@@ -413,9 +404,10 @@ public class ScannerCallable extends RegionServerCallable {
if (response.hasMvccReadPoint()) {
this.scan.setMvccReadPoint(response.getMvccReadPoint());
}
- return id;
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
+ this.scannerId = id;
+ return response;
+ } catch (Exception e) {
+ throw ProtobufUtil.handleRemoteException(e);
}
}
@@ -480,27 +472,31 @@ public class ScannerCallable extends RegionServerCallable {
/**
* Should the client attempt to fetch more results from this region
- * @return True if the client should attempt to fetch more results, false otherwise.
*/
- protected boolean getServerHasMoreResults() {
- assert serverHasMoreResultsContext;
- return this.serverHasMoreResults;
+ MoreResults moreResultsInRegion() {
+ return moreResultsInRegion;
}
- protected void setServerHasMoreResults(boolean serverHasMoreResults) {
- this.serverHasMoreResults = serverHasMoreResults;
+ void setMoreResultsInRegion(MoreResults moreResults) {
+ this.moreResultsInRegion = moreResults;
}
/**
- * Did the server respond with information about whether more results might exist.
- * Not guaranteed to respond with older server versions
- * @return True if the server responded with information about more results.
+ * Should the client attempt to fetch more results for the whole scan.
*/
- protected boolean hasMoreResultsContext() {
- return serverHasMoreResultsContext;
+ MoreResults moreResultsForScan() {
+ return moreResultsForScan;
}
- protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
- this.serverHasMoreResultsContext = serverHasMoreResultsContext;
+ void setMoreResultsForScan(MoreResults moreResults) {
+ this.moreResultsForScan = moreResults;
+ }
+
+ /**
+ * Whether the previous call is openScanner. This is used to keep compatible with the old
+ * implementation that we always returns empty result for openScanner.
+ */
+ boolean isOpenScanner() {
+ return openScanner;
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index a030e67efc6..46c8f9c0c27 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -18,7 +18,10 @@
package org.apache.hadoop.hbase.client;
-import static org.apache.hadoop.hbase.client.ClientScanner.createClosestRowBefore;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowBefore;
+
+import com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.io.InterruptedIOException;
@@ -30,22 +33,18 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
import org.apache.hadoop.hbase.util.Pair;
-import com.google.common.annotations.VisibleForTesting;
-
/**
* This class has the logic for handling scanners for regions with and without replicas.
* 1. A scan is attempted on the default (primary) region
@@ -115,20 +114,16 @@ class ScannerCallableWithReplicas implements RetryingCallable {
return currentScannerCallable.getHRegionInfo();
}
- public boolean getServerHasMoreResults() {
- return currentScannerCallable.getServerHasMoreResults();
+ public MoreResults moreResultsInRegion() {
+ return currentScannerCallable.moreResultsInRegion();
}
- public void setServerHasMoreResults(boolean serverHasMoreResults) {
- currentScannerCallable.setServerHasMoreResults(serverHasMoreResults);
+ public MoreResults moreResultsForScan() {
+ return currentScannerCallable.moreResultsForScan();
}
- public boolean hasMoreResultsContext() {
- return currentScannerCallable.hasMoreResultsContext();
- }
-
- public void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
- currentScannerCallable.setHasMoreResultsContext(serverHasMoreResultsContext);
+ public boolean isOpenScanner() {
+ return currentScannerCallable.isOpenScanner();
}
@Override
@@ -342,7 +337,7 @@ class ScannerCallableWithReplicas implements RetryingCallable {
if (callable.getScan().isReversed()) {
callable.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow()));
} else {
- callable.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1]));
+ callable.getScan().setStartRow(createClosestRowAfter(this.lastResult.getRow()));
}
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 715d1066b20..b3ade65342c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -20,6 +20,20 @@ package org.apache.hadoop.hbase.protobuf;
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+import com.google.protobuf.Parser;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.TextFormat;
+
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
@@ -47,6 +61,7 @@ import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
@@ -128,12 +143,12 @@ import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
-import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.quotas.QuotaScope;
import org.apache.hadoop.hbase.quotas.QuotaType;
@@ -158,20 +173,6 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.token.Token;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ListMultimap;
-import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.Message;
-import com.google.protobuf.Parser;
-import com.google.protobuf.RpcChannel;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-import com.google.protobuf.TextFormat;
-
/**
* Protobufs utility.
*/
@@ -322,17 +323,32 @@ public final class ProtobufUtil {
* a new IOException that wraps the unexpected ServiceException.
*/
public static IOException getRemoteException(ServiceException se) {
- Throwable e = se.getCause();
- if (e == null) {
- return new IOException(se);
+ return makeIOExceptionOfException(se);
+ }
+
+ /**
+ * Like {@link #getRemoteException(ServiceException)} but more generic, able to handle more than
+ * just {@link ServiceException}. Prefer this method to
+ * {@link #getRemoteException(ServiceException)} because trying to
+ * contain direct protobuf references.
+ * @param e
+ */
+ public static IOException handleRemoteException(Exception e) {
+ return makeIOExceptionOfException(e);
+ }
+
+ private static IOException makeIOExceptionOfException(Exception e) {
+ Throwable t = e;
+ if (e instanceof ServiceException) {
+ t = e.getCause();
}
- if (ExceptionUtil.isInterrupt(e)) {
- return ExceptionUtil.asInterrupt(e);
+ if (ExceptionUtil.isInterrupt(t)) {
+ return ExceptionUtil.asInterrupt(t);
}
- if (e instanceof RemoteException) {
- e = ((RemoteException) e).unwrapRemoteException();
+ if (t instanceof RemoteException) {
+ t = ((RemoteException) t).unwrapRemoteException();
}
- return e instanceof IOException ? (IOException) e : new IOException(se);
+ return t instanceof IOException ? (IOException) t : new HBaseIOException(t);
}
/**
@@ -908,6 +924,32 @@ public final class ProtobufUtil {
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
*
@@ -1005,6 +1047,9 @@ public final class ProtobufUtil {
if (mvccReadPoint > 0) {
scanBuilder.setMvccReadPoint(mvccReadPoint);
}
+ if (scan.getReadType() != Scan.ReadType.DEFAULT) {
+ scanBuilder.setReadType(toReadType(scan.getReadType()));
+ }
return scanBuilder.build();
}
@@ -1095,6 +1140,11 @@ public final class ProtobufUtil {
if (proto.hasMvccReadPoint()) {
PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint());
}
+ if (scan.isSmall()) {
+ scan.setReadType(Scan.ReadType.PREAD);
+ } else if (proto.hasReadType()) {
+ scan.setReadType(toReadType(proto.getReadType()));
+ }
return scan;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 3b9d366754f..cac4e97e4d5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -498,6 +498,9 @@ public final class RequestConverter {
builder.setClientHandlesPartials(true);
builder.setClientHandlesHeartbeats(true);
builder.setTrackScanMetrics(scan.isScanMetricsEnabled());
+ if (scan.getLimit() > 0) {
+ builder.setLimitOfRows(scan.getLimit());
+ }
return builder.build();
}
@@ -532,7 +535,7 @@ public final class RequestConverter {
*/
public static ScanRequest buildScanRequest(final long scannerId, final int numberOfRows,
final boolean closeScanner, final long nextCallSeq, final boolean trackMetrics,
- final boolean renew) {
+ final boolean renew, int limitOfRows) {
ScanRequest.Builder builder = ScanRequest.newBuilder();
builder.setNumberOfRows(numberOfRows);
builder.setCloseScanner(closeScanner);
@@ -542,6 +545,9 @@ public final class RequestConverter {
builder.setClientHandlesHeartbeats(true);
builder.setTrackScanMetrics(trackMetrics);
builder.setRenew(renew);
+ if (limitOfRows > 0) {
+ builder.setLimitOfRows(limitOfRows);
+ }
return builder.build();
}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
index 828d435500e..88c32ac15fd 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
@@ -17,9 +17,11 @@
*/
package org.apache.hadoop.hbase.client;
+import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
@@ -39,6 +41,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.After;
@@ -87,6 +90,7 @@ public class TestClientScanner {
private boolean rpcFinished = false;
private boolean rpcFinishedFired = false;
+ private boolean initialized = false;
public MockClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
@@ -97,9 +101,13 @@ public class TestClientScanner {
}
@Override
- protected boolean nextScanner(int nbRows, final boolean done) throws IOException {
+ protected Result[] nextScanner(int nbRows) throws IOException {
+ if (!initialized) {
+ initialized = true;
+ return super.nextScanner(nbRows);
+ }
if (!rpcFinished) {
- return super.nextScanner(nbRows, done);
+ return super.nextScanner(nbRows);
}
// Enforce that we don't short-circuit more than once
@@ -108,7 +116,7 @@ public class TestClientScanner {
" short-circuit was triggered.");
}
rpcFinishedFired = true;
- return false;
+ return null;
}
@Override
@@ -150,14 +158,13 @@ public class TestClientScanner {
ScannerCallableWithReplicas.class);
switch (count) {
case 0: // initialize
- case 2: // detect no more results
- case 3: // close
+ count++;
+ callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.UNKNOWN);
+ return results;
+ case 1: // detect no more results
+ case 2: // close
count++;
return null;
- case 1:
- count++;
- callable.setHasMoreResultsContext(false);
- return results;
default:
throw new RuntimeException("Expected only 2 invocations");
}
@@ -213,15 +220,13 @@ public class TestClientScanner {
ScannerCallableWithReplicas.class);
switch (count) {
case 0: // initialize
- case 2: // close
+ count++;
+ // if we set no here the implementation will trigger a close
+ callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES);
+ return results;
+ case 1: // close
count++;
return null;
- case 1:
- count++;
- callable.setHasMoreResultsContext(true);
- // if we set false here the implementation will trigger a close
- callable.setServerHasMoreResults(true);
- return results;
default:
throw new RuntimeException("Expected only 2 invocations");
}
@@ -237,16 +242,11 @@ public class TestClientScanner {
try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- // Due to initializeScannerInConstruction()
- Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class),
- Mockito.anyInt());
-
InOrder inOrder = Mockito.inOrder(caller);
scanner.loadCache();
- inOrder.verify(caller, Mockito.times(2)).callWithoutRetries(
+ inOrder.verify(caller, Mockito.times(1)).callWithoutRetries(
Mockito.any(RetryingCallable.class), Mockito.anyInt());
assertEquals(1, scanner.cache.size());
@@ -281,15 +281,13 @@ public class TestClientScanner {
ScannerCallableWithReplicas.class);
switch (count) {
case 0: // initialize
- case 2: // close
+ count++;
+ // if we set no here the implementation will trigger a close
+ callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES);
+ return results;
+ case 1: // close
count++;
return null;
- case 1:
- count++;
- callable.setHasMoreResultsContext(true);
- // if we set false here the implementation will trigger a close
- callable.setServerHasMoreResults(true);
- return results;
default:
throw new RuntimeException("Expected only 2 invocations");
}
@@ -305,18 +303,11 @@ public class TestClientScanner {
try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- // Due to initializeScannerInConstruction()
- Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class),
- Mockito.anyInt());
-
InOrder inOrder = Mockito.inOrder(caller);
scanner.loadCache();
- // Ensures that possiblyNextScanner isn't called at the end which would trigger
- // another call to callWithoutRetries
- inOrder.verify(caller, Mockito.times(2)).callWithoutRetries(
+ inOrder.verify(caller, Mockito.times(1)).callWithoutRetries(
Mockito.any(RetryingCallable.class), Mockito.anyInt());
assertEquals(3, scanner.cache.size());
@@ -363,14 +354,12 @@ public class TestClientScanner {
ScannerCallableWithReplicas.class);
switch (count) {
case 0: // initialize
- case 2: // close
+ count++;
+ callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO);
+ return results;
+ case 1: // close
count++;
return null;
- case 1:
- count++;
- callable.setHasMoreResultsContext(true);
- callable.setServerHasMoreResults(false);
- return results;
default:
throw new RuntimeException("Expected only 2 invocations");
}
@@ -385,18 +374,13 @@ public class TestClientScanner {
try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- // Due to initializeScannerInConstruction()
- Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class),
- Mockito.anyInt());
-
scanner.setRpcFinished(true);
InOrder inOrder = Mockito.inOrder(caller);
scanner.loadCache();
- inOrder.verify(caller, Mockito.times(2)).callWithoutRetries(
+ inOrder.verify(caller, Mockito.times(1)).callWithoutRetries(
Mockito.any(RetryingCallable.class), Mockito.anyInt());
assertEquals(1, scanner.cache.size());
@@ -435,22 +419,19 @@ public class TestClientScanner {
ScannerCallableWithReplicas.class);
switch (count) {
case 0: // initialize
- case 3: // close
count++;
- return null;
+ callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES);
+ return results1;
case 1:
- count++;
- callable.setHasMoreResultsContext(true);
- callable.setServerHasMoreResults(true);
- return results1;
- case 2:
count++;
// The server reports back false WRT more results
- callable.setHasMoreResultsContext(true);
- callable.setServerHasMoreResults(false);
+ callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO);
return results2;
+ case 2: // close
+ count++;
+ return null;
default:
- throw new RuntimeException("Expected only 2 invocations");
+ throw new RuntimeException("Expected only 3 invocations");
}
}
});
@@ -461,17 +442,12 @@ public class TestClientScanner {
try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- // Due to initializeScannerInConstruction()
- Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class),
- Mockito.anyInt());
-
InOrder inOrder = Mockito.inOrder(caller);
scanner.setRpcFinished(true);
scanner.loadCache();
- inOrder.verify(caller, Mockito.times(3)).callWithoutRetries(
+ inOrder.verify(caller, Mockito.times(2)).callWithoutRetries(
Mockito.any(RetryingCallable.class), Mockito.anyInt());
assertEquals(2, scanner.cache.size());
@@ -516,8 +492,8 @@ public class TestClientScanner {
iter.next();
}
fail("Should have failed with RetriesExhaustedException");
- } catch (RetriesExhaustedException expected) {
-
+ } catch (RuntimeException expected) {
+ assertThat(expected.getCause(), instanceOf(RetriesExhaustedException.class));
}
}
@@ -552,7 +528,5 @@ public class TestClientScanner {
}
};
}
-
}
-
}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java
deleted file mode 100644
index 57b52e6c801..00000000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java
+++ /dev/null
@@ -1,349 +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.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClientSmallReversedScanner.SmallReversedScannerCallableFactory;
-import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test the ClientSmallReversedScanner.
- */
-@Category(SmallTests.class)
-public class TestClientSmallReversedScanner {
-
- Scan scan;
- ExecutorService pool;
- Configuration conf;
-
- ClusterConnection clusterConn;
- RpcRetryingCallerFactory rpcFactory;
- RpcControllerFactory controllerFactory;
- RpcRetryingCaller caller;
-
- @Before
- @SuppressWarnings({"deprecation", "unchecked"})
- public void setup() throws IOException {
- clusterConn = Mockito.mock(ClusterConnection.class);
- rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);
- controllerFactory = Mockito.mock(RpcControllerFactory.class);
- pool = Executors.newSingleThreadExecutor();
- scan = new Scan();
- conf = new Configuration();
- Mockito.when(clusterConn.getConfiguration()).thenReturn(conf);
- // Mock out the RpcCaller
- caller = Mockito.mock(RpcRetryingCaller.class);
- // Return the mock from the factory
- Mockito.when(rpcFactory. newCaller()).thenReturn(caller);
- }
-
- @After
- public void teardown() {
- if (null != pool) {
- pool.shutdownNow();
- }
- }
-
- /**
- * Create a simple Answer which returns true the first time, and false every time after.
- */
- private Answer createTrueThenFalseAnswer() {
- return new Answer() {
- boolean first = true;
-
- @Override
- public Boolean answer(InvocationOnMock invocation) {
- if (first) {
- first = false;
- return true;
- }
- return false;
- }
- };
- }
-
- private SmallReversedScannerCallableFactory getFactory(
- final ScannerCallableWithReplicas callableWithReplicas) {
- return new SmallReversedScannerCallableFactory() {
- @Override
- public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table,
- Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum,
- RpcControllerFactory controllerFactory, ExecutorService pool,
- int primaryOperationTimeout, int retries, int scannerTimeout, Configuration conf,
- RpcRetryingCaller caller, boolean isFirstRegionToLocate) {
- return callableWithReplicas;
- }
- };
- }
-
- @Test
- public void testContextPresent() throws Exception {
- final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum);
-
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- // Mock out the RpcCaller
- @SuppressWarnings("unchecked")
- RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class);
- // Return the mock from the factory
- Mockito.when(rpcFactory. newCaller()).thenReturn(caller);
-
- // Intentionally leave a "default" caching size in the Scan. No matter the value, we
- // should continue based on the server context
-
- SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
- TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,
- Integer.MAX_VALUE)) {
-
- csrs.setScannerCallableFactory(factory);
-
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout()))
- .thenAnswer(new Answer() {
- int count = 0;
-
- @Override
- public Result[] answer(InvocationOnMock invocation) {
- Result[] results;
- if (0 == count) {
- results = new Result[] {Result.create(new Cell[] {kv3}),
- Result.create(new Cell[] {kv2})};
- } else if (1 == count) {
- results = new Result[] {Result.create(new Cell[] {kv1})};
- } else {
- results = new Result[0];
- }
- count++;
- return results;
- }
- });
-
- // Pass back the context always
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true);
- // Only have more results the first time
- Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenAnswer(
- createTrueThenFalseAnswer());
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- csrs.loadCache();
-
- List results = csrs.cache;
- Iterator iter = results.iterator();
- assertEquals(3, results.size());
- for (int i = 3; i >= 1 && iter.hasNext(); i--) {
- Result result = iter.next();
- byte[] row = result.getRow();
- assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
- assertEquals(1, result.getMap().size());
- }
- assertTrue(csrs.closed);
- }
- }
-
- @Test
- public void testNoContextFewerRecords() throws Exception {
- final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum);
-
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- // While the server returns 2 records per batch, we expect more records.
- scan.setCaching(2);
-
- SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
- TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,
- Integer.MAX_VALUE)) {
-
- csrs.setScannerCallableFactory(factory);
-
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout()))
- .thenAnswer(new Answer() {
- int count = 0;
-
- @Override
- public Result[] answer(InvocationOnMock invocation) {
- Result[] results;
- if (0 == count) {
- results = new Result[] {Result.create(new Cell[] {kv3}),
- Result.create(new Cell[] {kv2})};
- } else if (1 == count) {
- // Return fewer records than expected (2)
- results = new Result[] {Result.create(new Cell[] {kv1})};
- } else {
- throw new RuntimeException("Should not fetch a third batch from the server");
- }
- count++;
- return results;
- }
- });
-
- // Server doesn't return the context
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false);
- // getServerHasMoreResults shouldn't be called when hasMoreResultsContext returns false
- Mockito.when(callableWithReplicas.getServerHasMoreResults())
- .thenThrow(new RuntimeException("Should not be called"));
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- csrs.loadCache();
-
- List results = csrs.cache;
- Iterator iter = results.iterator();
- assertEquals(2, results.size());
- for (int i = 3; i >= 2 && iter.hasNext(); i--) {
- Result result = iter.next();
- byte[] row = result.getRow();
- assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
- assertEquals(1, result.getMap().size());
- }
-
- // "consume" the Results
- results.clear();
-
- csrs.loadCache();
-
- assertEquals(1, results.size());
- Result result = results.get(0);
- assertEquals("row1", new String(result.getRow(), StandardCharsets.UTF_8));
- assertEquals(1, result.getMap().size());
-
- assertTrue(csrs.closed);
- }
- }
-
- @Test
- public void testNoContextNoRecords() throws Exception {
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- // While the server return 2 records per RPC, we expect there to be more records.
- scan.setCaching(2);
-
- SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
- TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,
- Integer.MAX_VALUE)) {
-
- csrs.setScannerCallableFactory(factory);
-
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout()))
- .thenReturn(new Result[0]);
-
- // Server doesn't return the context
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false);
- // Only have more results the first time
- Mockito.when(callableWithReplicas.getServerHasMoreResults())
- .thenThrow(new RuntimeException("Should not be called"));
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- csrs.loadCache();
-
- assertEquals(0, csrs.cache.size());
- assertTrue(csrs.closed);
- }
- }
-
- @Test
- public void testContextNoRecords() throws Exception {
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
- TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,
- Integer.MAX_VALUE)) {
-
- csrs.setScannerCallableFactory(factory);
-
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout()))
- .thenReturn(new Result[0]);
-
- // Server doesn't return the context
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true);
- // Only have more results the first time
- Mockito.when(callableWithReplicas.getServerHasMoreResults())
- .thenReturn(false);
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- csrs.loadCache();
-
- assertEquals(0, csrs.cache.size());
- assertTrue(csrs.closed);
- }
- }
-}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java
deleted file mode 100644
index 90bf4bbd3ab..00000000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java
+++ /dev/null
@@ -1,339 +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.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
-import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test the ClientSmallScanner.
- */
-@Category(SmallTests.class)
-public class TestClientSmallScanner {
-
- Scan scan;
- ExecutorService pool;
- Configuration conf;
-
- ClusterConnection clusterConn;
- RpcRetryingCallerFactory rpcFactory;
- RpcControllerFactory controllerFactory;
- RpcRetryingCaller caller;
-
- @Before
- @SuppressWarnings({"deprecation", "unchecked"})
- public void setup() throws IOException {
- clusterConn = Mockito.mock(ClusterConnection.class);
- rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);
- controllerFactory = Mockito.mock(RpcControllerFactory.class);
- pool = Executors.newSingleThreadExecutor();
- scan = new Scan();
- conf = new Configuration();
- Mockito.when(clusterConn.getConfiguration()).thenReturn(conf);
- // Mock out the RpcCaller
- caller = Mockito.mock(RpcRetryingCaller.class);
- // Return the mock from the factory
- Mockito.when(rpcFactory. newCaller()).thenReturn(caller);
- }
-
- @After
- public void teardown() {
- if (null != pool) {
- pool.shutdownNow();
- }
- }
-
- /**
- * Create a simple Answer which returns true the first time, and false every time after.
- */
- private Answer createTrueThenFalseAnswer() {
- return new Answer() {
- boolean first = true;
-
- @Override
- public Boolean answer(InvocationOnMock invocation) {
- if (first) {
- first = false;
- return true;
- }
- return false;
- }
- };
- }
-
- private SmallScannerCallableFactory getFactory(
- final ScannerCallableWithReplicas callableWithReplicas) {
- return new SmallScannerCallableFactory() {
- @Override
- public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table,
- Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum,
- RpcControllerFactory controllerFactory, ExecutorService pool,
- int primaryOperationTimeout, int retries, int scannerTimeout, Configuration conf,
- RpcRetryingCaller caller) {
- return callableWithReplicas;
- }
- };
- }
-
- @Test
- public void testContextPresent() throws Exception {
- final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum);
-
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- // Mock out the RpcCaller
- @SuppressWarnings("unchecked")
- RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class);
- // Return the mock from the factory
- Mockito.when(rpcFactory. newCaller()).thenReturn(caller);
-
- SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- // Intentionally leave a "default" caching size in the Scan. No matter the value, we
- // should continue based on the server context
-
- try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
- clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- css.setScannerCallableFactory(factory);
-
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout()))
- .thenAnswer(new Answer() {
- int count = 0;
-
- @Override
- public Result[] answer(InvocationOnMock invocation) {
- Result[] results;
- if (0 == count) {
- results = new Result[] {Result.create(new Cell[] {kv1}),
- Result.create(new Cell[] {kv2})};
- } else if (1 == count) {
- results = new Result[] {Result.create(new Cell[] {kv3})};
- } else {
- results = new Result[0];
- }
- count++;
- return results;
- }
- });
-
- // Pass back the context always
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true);
- // Only have more results the first time
- Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenAnswer(
- createTrueThenFalseAnswer());
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- css.loadCache();
-
- List results = css.cache;
- assertEquals(3, results.size());
- for (int i = 1; i <= 3; i++) {
- Result result = results.get(i - 1);
- byte[] row = result.getRow();
- assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
- assertEquals(1, result.getMap().size());
- }
-
- assertTrue(css.closed);
- }
- }
-
- @Test
- public void testNoContextFewerRecords() throws Exception {
- final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
- Type.Maximum);
-
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- // While the server returns 2 records per batch, we expect more records.
- scan.setCaching(2);
- SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
- clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- css.setScannerCallableFactory(factory);
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout()))
- .thenAnswer(new Answer() {
- int count = 0;
-
- @Override
- public Result[] answer(InvocationOnMock invocation) {
- Result[] results;
- if (0 == count) {
- results = new Result[] {Result.create(new Cell[] {kv1}),
- Result.create(new Cell[] {kv2})};
- } else if (1 == count) {
- // Return fewer records than expected (2)
- results = new Result[] {Result.create(new Cell[] {kv3})};
- } else {
- throw new RuntimeException("Should not fetch a third batch from the server");
- }
- count++;
- return results;
- }
- });
-
- // Server doesn't return the context
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false);
- // Only have more results the first time
- Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenThrow(
- new RuntimeException("Should not be called"));
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- css.loadCache();
-
- List results = css.cache;
- assertEquals(2, results.size());
- for (int i = 1; i <= 2; i++) {
- Result result = results.get(i - 1);
- byte[] row = result.getRow();
- assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
- assertEquals(1, result.getMap().size());
- }
-
- // "consume" the results we verified
- results.clear();
-
- css.loadCache();
-
- assertEquals(1, results.size());
- Result result = results.get(0);
- assertEquals("row3", new String(result.getRow(), StandardCharsets.UTF_8));
- assertEquals(1, result.getMap().size());
- assertTrue(css.closed);
- }
- }
-
- @Test
- public void testNoContextNoRecords() throws Exception {
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- // While the server return 2 records per RPC, we expect there to be more records.
- scan.setCaching(2);
-
- SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
- clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- css.setScannerCallableFactory(factory);
-
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout()))
- .thenReturn(new Result[0]);
-
- // Server doesn't return the context
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false);
- // Only have more results the first time
- Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenThrow(
- new RuntimeException("Should not be called"));
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- css.loadCache();
-
- assertEquals(0, css.cache.size());
- assertTrue(css.closed);
- }
- }
-
- @Test
- public void testContextNoRecords() throws Exception {
- ScannerCallableWithReplicas callableWithReplicas = Mockito
- .mock(ScannerCallableWithReplicas.class);
-
- SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
-
- try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
- clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
-
- css.setScannerCallableFactory(factory);
-
- // Return some data the first time, less the second, and none after that
- Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout()))
- .thenReturn(new Result[0]);
-
- // Server doesn't return the context
- Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true);
- // Only have more results the first time
- Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenReturn(false);
-
- // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
- HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
- Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
- // Trigger the "no more data" branch for #nextScanner(...)
- Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
-
- css.loadCache();
-
- assertEquals(0, css.cache.size());
- assertTrue(css.closed);
- }
- }
-}
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
index d1f28ba7c12..cefdd386490 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
@@ -14239,6 +14239,36 @@ public final class ClientProtos {
* optional uint64 mvcc_read_point = 20 [default = 0];
*/
long getMvccReadPoint();
+
+ // optional bool include_start_row = 21 [default = true];
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ boolean hasIncludeStartRow();
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ boolean getIncludeStartRow();
+
+ // optional bool include_stop_row = 22 [default = false];
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ boolean hasIncludeStopRow();
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ boolean getIncludeStopRow();
+
+ // optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ boolean hasReadType();
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType();
}
/**
* Protobuf type {@code hbase.pb.Scan}
@@ -14433,6 +14463,27 @@ public final class ClientProtos {
mvccReadPoint_ = input.readUInt64();
break;
}
+ case 168: {
+ bitField0_ |= 0x00020000;
+ includeStartRow_ = input.readBool();
+ break;
+ }
+ case 176: {
+ bitField0_ |= 0x00040000;
+ includeStopRow_ = input.readBool();
+ 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) {
@@ -14481,6 +14532,97 @@ public final class ClientProtos {
return PARSER;
}
+ /**
+ * Protobuf enum {@code hbase.pb.Scan.ReadType}
+ */
+ public enum ReadType
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * DEFAULT = 0;
+ */
+ DEFAULT(0, 0),
+ /**
+ * STREAM = 1;
+ */
+ STREAM(1, 1),
+ /**
+ * PREAD = 2;
+ */
+ PREAD(2, 2),
+ ;
+
+ /**
+ * DEFAULT = 0;
+ */
+ public static final int DEFAULT_VALUE = 0;
+ /**
+ * STREAM = 1;
+ */
+ public static final int STREAM_VALUE = 1;
+ /**
+ * PREAD = 2;
+ */
+ 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
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap() {
+ 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_;
// repeated .hbase.pb.Column column = 1;
public static final int COLUMN_FIELD_NUMBER = 1;
@@ -14882,6 +15024,54 @@ public final class ClientProtos {
return mvccReadPoint_;
}
+ // optional bool include_start_row = 21 [default = true];
+ public static final int INCLUDE_START_ROW_FIELD_NUMBER = 21;
+ private boolean includeStartRow_;
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ public boolean hasIncludeStartRow() {
+ return ((bitField0_ & 0x00020000) == 0x00020000);
+ }
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ public boolean getIncludeStartRow() {
+ return includeStartRow_;
+ }
+
+ // optional bool include_stop_row = 22 [default = false];
+ public static final int INCLUDE_STOP_ROW_FIELD_NUMBER = 22;
+ private boolean includeStopRow_;
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ public boolean hasIncludeStopRow() {
+ return ((bitField0_ & 0x00040000) == 0x00040000);
+ }
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ public boolean getIncludeStopRow() {
+ 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_;
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ public boolean hasReadType() {
+ return ((bitField0_ & 0x00080000) == 0x00080000);
+ }
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType() {
+ return readType_;
+ }
+
private void initFields() {
column_ = java.util.Collections.emptyList();
attribute_ = java.util.Collections.emptyList();
@@ -14903,6 +15093,9 @@ public final class ClientProtos {
allowPartialResults_ = false;
cfTimeRange_ = java.util.Collections.emptyList();
mvccReadPoint_ = 0L;
+ includeStartRow_ = true;
+ includeStopRow_ = false;
+ readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -15000,6 +15193,15 @@ public final class ClientProtos {
if (((bitField0_ & 0x00010000) == 0x00010000)) {
output.writeUInt64(20, mvccReadPoint_);
}
+ if (((bitField0_ & 0x00020000) == 0x00020000)) {
+ output.writeBool(21, includeStartRow_);
+ }
+ if (((bitField0_ & 0x00040000) == 0x00040000)) {
+ output.writeBool(22, includeStopRow_);
+ }
+ if (((bitField0_ & 0x00080000) == 0x00080000)) {
+ output.writeEnum(23, readType_.getNumber());
+ }
getUnknownFields().writeTo(output);
}
@@ -15089,6 +15291,18 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(20, mvccReadPoint_);
}
+ if (((bitField0_ & 0x00020000) == 0x00020000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(21, includeStartRow_);
+ }
+ if (((bitField0_ & 0x00040000) == 0x00040000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(22, includeStopRow_);
+ }
+ if (((bitField0_ & 0x00080000) == 0x00080000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(23, readType_.getNumber());
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -15203,6 +15417,21 @@ public final class ClientProtos {
result = result && (getMvccReadPoint()
== other.getMvccReadPoint());
}
+ result = result && (hasIncludeStartRow() == other.hasIncludeStartRow());
+ if (hasIncludeStartRow()) {
+ result = result && (getIncludeStartRow()
+ == other.getIncludeStartRow());
+ }
+ result = result && (hasIncludeStopRow() == other.hasIncludeStopRow());
+ if (hasIncludeStopRow()) {
+ result = result && (getIncludeStopRow()
+ == other.getIncludeStopRow());
+ }
+ result = result && (hasReadType() == other.hasReadType());
+ if (hasReadType()) {
+ result = result &&
+ (getReadType() == other.getReadType());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -15296,6 +15525,18 @@ public final class ClientProtos {
hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getMvccReadPoint());
}
+ if (hasIncludeStartRow()) {
+ hash = (37 * hash) + INCLUDE_START_ROW_FIELD_NUMBER;
+ hash = (53 * hash) + hashBoolean(getIncludeStartRow());
+ }
+ if (hasIncludeStopRow()) {
+ hash = (37 * hash) + INCLUDE_STOP_ROW_FIELD_NUMBER;
+ hash = (53 * hash) + hashBoolean(getIncludeStopRow());
+ }
+ if (hasReadType()) {
+ hash = (37 * hash) + READTYPE_FIELD_NUMBER;
+ hash = (53 * hash) + hashEnum(getReadType());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -15481,6 +15722,12 @@ public final class ClientProtos {
}
mvccReadPoint_ = 0L;
bitField0_ = (bitField0_ & ~0x00080000);
+ includeStartRow_ = true;
+ bitField0_ = (bitField0_ & ~0x00100000);
+ includeStopRow_ = false;
+ bitField0_ = (bitField0_ & ~0x00200000);
+ readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
+ bitField0_ = (bitField0_ & ~0x00400000);
return this;
}
@@ -15612,6 +15859,18 @@ public final class ClientProtos {
to_bitField0_ |= 0x00010000;
}
result.mvccReadPoint_ = mvccReadPoint_;
+ if (((from_bitField0_ & 0x00100000) == 0x00100000)) {
+ to_bitField0_ |= 0x00020000;
+ }
+ result.includeStartRow_ = includeStartRow_;
+ if (((from_bitField0_ & 0x00200000) == 0x00200000)) {
+ to_bitField0_ |= 0x00040000;
+ }
+ result.includeStopRow_ = includeStopRow_;
+ if (((from_bitField0_ & 0x00400000) == 0x00400000)) {
+ to_bitField0_ |= 0x00080000;
+ }
+ result.readType_ = readType_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -15757,6 +16016,15 @@ public final class ClientProtos {
if (other.hasMvccReadPoint()) {
setMvccReadPoint(other.getMvccReadPoint());
}
+ if (other.hasIncludeStartRow()) {
+ setIncludeStartRow(other.getIncludeStartRow());
+ }
+ if (other.hasIncludeStopRow()) {
+ setIncludeStopRow(other.getIncludeStopRow());
+ }
+ if (other.hasReadType()) {
+ setReadType(other.getReadType());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -17282,6 +17550,108 @@ public final class ClientProtos {
return this;
}
+ // optional bool include_start_row = 21 [default = true];
+ private boolean includeStartRow_ = true;
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ public boolean hasIncludeStartRow() {
+ return ((bitField0_ & 0x00100000) == 0x00100000);
+ }
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ public boolean getIncludeStartRow() {
+ return includeStartRow_;
+ }
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ public Builder setIncludeStartRow(boolean value) {
+ bitField0_ |= 0x00100000;
+ includeStartRow_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * optional bool include_start_row = 21 [default = true];
+ */
+ public Builder clearIncludeStartRow() {
+ bitField0_ = (bitField0_ & ~0x00100000);
+ includeStartRow_ = true;
+ onChanged();
+ return this;
+ }
+
+ // optional bool include_stop_row = 22 [default = false];
+ private boolean includeStopRow_ ;
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ public boolean hasIncludeStopRow() {
+ return ((bitField0_ & 0x00200000) == 0x00200000);
+ }
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ public boolean getIncludeStopRow() {
+ return includeStopRow_;
+ }
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ public Builder setIncludeStopRow(boolean value) {
+ bitField0_ |= 0x00200000;
+ includeStopRow_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * optional bool include_stop_row = 22 [default = false];
+ */
+ public Builder clearIncludeStopRow() {
+ bitField0_ = (bitField0_ & ~0x00200000);
+ includeStopRow_ = false;
+ onChanged();
+ 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;
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ public boolean hasReadType() {
+ return ((bitField0_ & 0x00400000) == 0x00400000);
+ }
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType() {
+ return readType_;
+ }
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ 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;
+ }
+ /**
+ * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];
+ */
+ 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)
}
@@ -17403,6 +17773,24 @@ public final class ClientProtos {
* optional bool renew = 10 [default = false];
*/
boolean getRenew();
+
+ // optional uint32 limit_of_rows = 11 [default = 0];
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ boolean hasLimitOfRows();
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ int getLimitOfRows();
}
/**
* Protobuf type {@code hbase.pb.ScanRequest}
@@ -17534,6 +17922,11 @@ public final class ClientProtos {
renew_ = input.readBool();
break;
}
+ case 88: {
+ bitField0_ |= 0x00000400;
+ limitOfRows_ = input.readUInt32();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -17746,6 +18139,30 @@ public final class ClientProtos {
return renew_;
}
+ // optional uint32 limit_of_rows = 11 [default = 0];
+ public static final int LIMIT_OF_ROWS_FIELD_NUMBER = 11;
+ private int limitOfRows_;
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ public boolean hasLimitOfRows() {
+ return ((bitField0_ & 0x00000400) == 0x00000400);
+ }
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ public int getLimitOfRows() {
+ return limitOfRows_;
+ }
+
private void initFields() {
region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
@@ -17757,6 +18174,7 @@ public final class ClientProtos {
clientHandlesHeartbeats_ = false;
trackScanMetrics_ = false;
renew_ = false;
+ limitOfRows_ = 0;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -17812,6 +18230,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000200) == 0x00000200)) {
output.writeBool(10, renew_);
}
+ if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ output.writeUInt32(11, limitOfRows_);
+ }
getUnknownFields().writeTo(output);
}
@@ -17861,6 +18282,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(10, renew_);
}
+ if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt32Size(11, limitOfRows_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -17934,6 +18359,11 @@ public final class ClientProtos {
result = result && (getRenew()
== other.getRenew());
}
+ result = result && (hasLimitOfRows() == other.hasLimitOfRows());
+ if (hasLimitOfRows()) {
+ result = result && (getLimitOfRows()
+ == other.getLimitOfRows());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -17987,6 +18417,10 @@ public final class ClientProtos {
hash = (37 * hash) + RENEW_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getRenew());
}
+ if (hasLimitOfRows()) {
+ hash = (37 * hash) + LIMIT_OF_ROWS_FIELD_NUMBER;
+ hash = (53 * hash) + getLimitOfRows();
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -18139,6 +18573,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000100);
renew_ = false;
bitField0_ = (bitField0_ & ~0x00000200);
+ limitOfRows_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000400);
return this;
}
@@ -18215,6 +18651,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000200;
}
result.renew_ = renew_;
+ if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
+ to_bitField0_ |= 0x00000400;
+ }
+ result.limitOfRows_ = limitOfRows_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -18261,6 +18701,9 @@ public final class ClientProtos {
if (other.hasRenew()) {
setRenew(other.getRenew());
}
+ if (other.hasLimitOfRows()) {
+ setLimitOfRows(other.getLimitOfRows());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -18798,6 +19241,55 @@ public final class ClientProtos {
return this;
}
+ // optional uint32 limit_of_rows = 11 [default = 0];
+ private int limitOfRows_ ;
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ public boolean hasLimitOfRows() {
+ return ((bitField0_ & 0x00000400) == 0x00000400);
+ }
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ public int getLimitOfRows() {
+ return limitOfRows_;
+ }
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ public Builder setLimitOfRows(int value) {
+ bitField0_ |= 0x00000400;
+ limitOfRows_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * optional uint32 limit_of_rows = 11 [default = 0];
+ *
+ *
+ * if we have returned limit_of_rows rows to client, then close the scanner.
+ *
+ */
+ public Builder clearLimitOfRows() {
+ bitField0_ = (bitField0_ & ~0x00000400);
+ limitOfRows_ = 0;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest)
}
@@ -35899,8 +36391,8 @@ public final class ClientProtos {
"\0132\027.hbase.pb.MutationProto\022&\n\tcondition\030" +
"\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce_grou" +
"p\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result\030\001 \001(" +
- "\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 \001(\010\"\331" +
- "\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb.Colum" +
+ "\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 \001(\010\"\377" +
+ "\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb.Colum" +
"n\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.NameByte" +
"sPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_row\030\004 \001" +
"(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filter\022\'\n\n" +
@@ -35915,81 +36407,86 @@ public final class ClientProtos {
"g\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 \001(\010\0226" +
"\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.ColumnF" +
"amilyTimeRange\022\032\n\017mvcc_read_point\030\024 \001(\004:",
- "\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132\031.hba" +
- "se.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hb" +
- "ase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016numbe" +
- "r_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025" +
- "\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_handles_" +
- "partials\030\007 \001(\010\022!\n\031client_handles_heartbe" +
- "ats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001(\010\022\024\n" +
- "\005renew\030\n \001(\010:\005false\"\266\002\n\014ScanResponse\022\030\n\020" +
- "cells_per_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001" +
- "(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n",
- "\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stal" +
- "e\030\006 \001(\010\022\037\n\027partial_flag_per_result\030\007 \003(\010" +
- "\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n\021hear" +
- "tbeat_message\030\t \001(\010\022+\n\014scan_metrics\030\n \001(" +
- "\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_read_po" +
- "int\030\013 \001(\004:\0010\"\305\001\n\024BulkLoadHFileRequest\022)\n" +
- "\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifier" +
- "\022>\n\013family_path\030\002 \003(\0132).hbase.pb.BulkLoa" +
- "dHFileRequest.FamilyPath\022\026\n\016assign_seq_n" +
- "um\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014",
- "\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n" +
- "\006loaded\030\001 \002(\010\"a\n\026CoprocessorServiceCall\022" +
- "\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013me" +
- "thod_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030Copr" +
- "ocessorServiceResult\022&\n\005value\030\001 \001(\0132\027.hb" +
- "ase.pb.NameBytesPair\"v\n\031CoprocessorServi" +
- "ceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Reg" +
- "ionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb.Co" +
- "processorServiceCall\"o\n\032CoprocessorServi" +
- "ceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re",
- "gionSpecifier\022&\n\005value\030\002 \002(\0132\027.hbase.pb." +
- "NameBytesPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001(\r\022" +
- ")\n\010mutation\030\002 \001(\0132\027.hbase.pb.MutationPro" +
- "to\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014servic" +
- "e_call\030\004 \001(\0132 .hbase.pb.CoprocessorServi" +
- "ceCall\"k\n\014RegionAction\022)\n\006region\030\001 \002(\0132\031" +
- ".hbase.pb.RegionSpecifier\022\016\n\006atomic\030\002 \001(" +
- "\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Action\"c\n\017R" +
- "egionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022" +
- "\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compactionP",
- "ressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadStats" +
- "\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpecif" +
- "ier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLoadS" +
- "tats\"\336\001\n\021ResultOrException\022\r\n\005index\030\001 \001(" +
- "\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*\n\te" +
- "xception\030\003 \001(\0132\027.hbase.pb.NameBytesPair\022" +
- ":\n\016service_result\030\004 \001(\0132\".hbase.pb.Copro" +
- "cessorServiceResult\0220\n\tloadStats\030\005 \001(\0132\031" +
- ".hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022RegionA" +
- "ctionResult\0226\n\021resultOrException\030\001 \003(\0132\033",
- ".hbase.pb.ResultOrException\022*\n\texception" +
- "\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Multi" +
- "Request\022,\n\014regionAction\030\001 \003(\0132\026.hbase.pb" +
- ".RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcon" +
- "dition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n\rMu" +
- "ltiResponse\0228\n\022regionActionResult\030\001 \003(\0132" +
- "\034.hbase.pb.RegionActionResult\022\021\n\tprocess" +
- "ed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036.hba" +
- "se.pb.MultiRegionLoadStats*\'\n\013Consistenc" +
- "y\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientSe",
- "rvice\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025.hba" +
- "se.pb.GetResponse\022;\n\006Mutate\022\027.hbase.pb.M" +
- "utateRequest\032\030.hbase.pb.MutateResponse\0225" +
- "\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase.pb" +
- ".ScanResponse\022P\n\rBulkLoadHFile\022\036.hbase.p" +
- "b.BulkLoadHFileRequest\032\037.hbase.pb.BulkLo" +
- "adHFileResponse\022X\n\013ExecService\022#.hbase.p" +
- "b.CoprocessorServiceRequest\032$.hbase.pb.C" +
- "oprocessorServiceResponse\022d\n\027ExecRegionS" +
- "erverService\022#.hbase.pb.CoprocessorServi",
- "ceRequest\032$.hbase.pb.CoprocessorServiceR" +
- "esponse\0228\n\005Multi\022\026.hbase.pb.MultiRequest" +
- "\032\027.hbase.pb.MultiResponseBB\n*org.apache." +
- "hadoop.hbase.protobuf.generatedB\014ClientP" +
- "rotosH\001\210\001\001\240\001\001"
+ "\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true\022\037\n\020i" +
+ "nclude_stop_row\030\026 \001(\010:\005false\0222\n\010readType" +
+ "\030\027 \001(\0162\027.hbase.pb.Scan.ReadType:\007DEFAULT" +
+ "\".\n\010ReadType\022\013\n\007DEFAULT\020\000\022\n\n\006STREAM\020\001\022\t\n" +
+ "\005PREAD\020\002\"\300\002\n\013ScanRequest\022)\n\006region\030\001 \001(\013" +
+ "2\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(" +
+ "\0132\016.hbase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n" +
+ "\016number_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005" +
+ " \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_ha" +
+ "ndles_partials\030\007 \001(\010\022!\n\031client_handles_h",
+ "eartbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t " +
+ "\001(\010\022\024\n\005renew\030\n \001(\010:\005false\022\030\n\rlimit_of_ro" +
+ "ws\030\013 \001(\r:\0010\"\266\002\n\014ScanResponse\022\030\n\020cells_pe" +
+ "r_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mo" +
+ "re_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results" +
+ "\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001(\010\022" +
+ "\037\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026more" +
+ "_results_in_region\030\010 \001(\010\022\031\n\021heartbeat_me" +
+ "ssage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbas" +
+ "e.pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(",
+ "\004:\0010\"\305\001\n\024BulkLoadHFileRequest\022)\n\006region\030" +
+ "\001 \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013fami" +
+ "ly_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileRe" +
+ "quest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010" +
+ "\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002" +
+ " \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n\006loaded\030" +
+ "\001 \002(\010\"a\n\026CoprocessorServiceCall\022\013\n\003row\030\001" +
+ " \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013method_nam" +
+ "e\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorS" +
+ "erviceResult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.N",
+ "ameBytesPair\"v\n\031CoprocessorServiceReques" +
+ "t\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpeci" +
+ "fier\022.\n\004call\030\002 \002(\0132 .hbase.pb.Coprocesso" +
+ "rServiceCall\"o\n\032CoprocessorServiceRespon" +
+ "se\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" +
+ "ifier\022&\n\005value\030\002 \002(\0132\027.hbase.pb.NameByte" +
+ "sPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001(\r\022)\n\010mutat" +
+ "ion\030\002 \001(\0132\027.hbase.pb.MutationProto\022\032\n\003ge" +
+ "t\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014service_call\030\004" +
+ " \001(\0132 .hbase.pb.CoprocessorServiceCall\"k",
+ "\n\014RegionAction\022)\n\006region\030\001 \002(\0132\031.hbase.p" +
+ "b.RegionSpecifier\022\016\n\006atomic\030\002 \001(\010\022 \n\006act" +
+ "ion\030\003 \003(\0132\020.hbase.pb.Action\"c\n\017RegionLoa" +
+ "dStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rheapO" +
+ "ccupancy\030\002 \001(\005:\0010\022\035\n\022compactionPressure\030" +
+ "\003 \001(\005:\0010\"j\n\024MultiRegionLoadStats\022)\n\006regi" +
+ "on\030\001 \003(\0132\031.hbase.pb.RegionSpecifier\022\'\n\004s" +
+ "tat\030\002 \003(\0132\031.hbase.pb.RegionLoadStats\"\336\001\n" +
+ "\021ResultOrException\022\r\n\005index\030\001 \001(\r\022 \n\006res" +
+ "ult\030\002 \001(\0132\020.hbase.pb.Result\022*\n\texception",
+ "\030\003 \001(\0132\027.hbase.pb.NameBytesPair\022:\n\016servi" +
+ "ce_result\030\004 \001(\0132\".hbase.pb.CoprocessorSe" +
+ "rviceResult\0220\n\tloadStats\030\005 \001(\0132\031.hbase.p" +
+ "b.RegionLoadStatsB\002\030\001\"x\n\022RegionActionRes" +
+ "ult\0226\n\021resultOrException\030\001 \003(\0132\033.hbase.p" +
+ "b.ResultOrException\022*\n\texception\030\002 \001(\0132\027" +
+ ".hbase.pb.NameBytesPair\"x\n\014MultiRequest\022" +
+ ",\n\014regionAction\030\001 \003(\0132\026.hbase.pb.RegionA" +
+ "ction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcondition\030\003" +
+ " \001(\0132\023.hbase.pb.Condition\"\226\001\n\rMultiRespo",
+ "nse\0228\n\022regionActionResult\030\001 \003(\0132\034.hbase." +
+ "pb.RegionActionResult\022\021\n\tprocessed\030\002 \001(\010" +
+ "\0228\n\020regionStatistics\030\003 \001(\0132\036.hbase.pb.Mu" +
+ "ltiRegionLoadStats*\'\n\013Consistency\022\n\n\006STR" +
+ "ONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientService\0222\n" +
+ "\003Get\022\024.hbase.pb.GetRequest\032\025.hbase.pb.Ge" +
+ "tResponse\022;\n\006Mutate\022\027.hbase.pb.MutateReq" +
+ "uest\032\030.hbase.pb.MutateResponse\0225\n\004Scan\022\025" +
+ ".hbase.pb.ScanRequest\032\026.hbase.pb.ScanRes" +
+ "ponse\022P\n\rBulkLoadHFile\022\036.hbase.pb.BulkLo",
+ "adHFileRequest\032\037.hbase.pb.BulkLoadHFileR" +
+ "esponse\022X\n\013ExecService\022#.hbase.pb.Coproc" +
+ "essorServiceRequest\032$.hbase.pb.Coprocess" +
+ "orServiceResponse\022d\n\027ExecRegionServerSer" +
+ "vice\022#.hbase.pb.CoprocessorServiceReques" +
+ "t\032$.hbase.pb.CoprocessorServiceResponse\022" +
+ "8\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbase" +
+ ".pb.MultiResponseBB\n*org.apache.hadoop.h" +
+ "base.protobuf.generatedB\014ClientProtosH\001\210" +
+ "\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -36079,13 +36576,13 @@ public final class ClientProtos {
internal_static_hbase_pb_Scan_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_Scan_descriptor,
- new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", });
+ new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", "IncludeStartRow", "IncludeStopRow", "ReadType", });
internal_static_hbase_pb_ScanRequest_descriptor =
getDescriptor().getMessageTypes().get(12);
internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
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 =
getDescriptor().getMessageTypes().get(13);
internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new
diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto
index f330ea4829c..4fb64bd85c6 100644
--- a/hbase-protocol/src/main/protobuf/Client.proto
+++ b/hbase-protocol/src/main/protobuf/Client.proto
@@ -260,6 +260,15 @@ message Scan {
optional bool allow_partial_results = 18;
repeated ColumnFamilyTimeRange cf_time_range = 19;
optional uint64 mvcc_read_point = 20 [default = 0];
+ optional bool include_start_row = 21 [default = true];
+ optional bool include_stop_row = 22 [default = false];
+
+ enum ReadType {
+ DEFAULT = 0;
+ STREAM = 1;
+ PREAD = 2;
+ }
+ optional ReadType readType = 23 [default = DEFAULT];
}
/**
@@ -284,6 +293,8 @@ message ScanRequest {
optional bool client_handles_heartbeats = 8;
optional bool track_scan_metrics = 9;
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];
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
index b19b4825464..27518ca9087 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
@@ -63,27 +63,27 @@ public class SyncTable extends Configured implements Tool {
static final String SOURCE_ZK_CLUSTER_CONF_KEY = "sync.table.source.zk.cluster";
static final String TARGET_ZK_CLUSTER_CONF_KEY = "sync.table.target.zk.cluster";
static final String DRY_RUN_CONF_KEY="sync.table.dry.run";
-
+
Path sourceHashDir;
String sourceTableName;
String targetTableName;
-
+
String sourceZkCluster;
String targetZkCluster;
boolean dryRun;
-
+
Counters counters;
-
+
public SyncTable(Configuration conf) {
super(conf);
}
-
+
public Job createSubmittableJob(String[] args) throws IOException {
FileSystem fs = sourceHashDir.getFileSystem(getConf());
if (!fs.exists(sourceHashDir)) {
throw new IOException("Source hash dir not found: " + sourceHashDir);
}
-
+
HashTable.TableHash tableHash = HashTable.TableHash.read(getConf(), sourceHashDir);
LOG.info("Read source hash manifest: " + tableHash);
LOG.info("Read " + tableHash.partitions.size() + " partition keys");
@@ -97,7 +97,7 @@ public class SyncTable extends Configured implements Tool {
+ " says numHashFiles=" + tableHash.numHashFiles + " but the number of partition keys"
+ " found in the partitions file is " + tableHash.partitions.size());
}
-
+
Path dataDir = new Path(sourceHashDir, HashTable.HASH_DATA_DIR);
int dataSubdirCount = 0;
for (FileStatus file : fs.listStatus(dataDir)) {
@@ -105,14 +105,14 @@ public class SyncTable extends Configured implements Tool {
dataSubdirCount++;
}
}
-
+
if (dataSubdirCount != tableHash.numHashFiles) {
throw new RuntimeException("Hash data appears corrupt. The number of of hash files created"
+ " should be 1 more than the number of partition keys. However, the number of data dirs"
+ " found is " + dataSubdirCount + " but the number of partition keys"
+ " found in the partitions file is " + tableHash.partitions.size());
}
-
+
Job job = Job.getInstance(getConf(),getConf().get("mapreduce.job.name",
"syncTable_" + sourceTableName + "-" + targetTableName));
Configuration jobConf = job.getConfiguration();
@@ -127,12 +127,12 @@ public class SyncTable extends Configured implements Tool {
jobConf.set(TARGET_ZK_CLUSTER_CONF_KEY, targetZkCluster);
}
jobConf.setBoolean(DRY_RUN_CONF_KEY, dryRun);
-
+
TableMapReduceUtil.initTableMapperJob(targetTableName, tableHash.initScan(),
SyncMapper.class, null, null, job);
-
+
job.setNumReduceTasks(0);
-
+
if (dryRun) {
job.setOutputFormatClass(NullOutputFormat.class);
} else {
@@ -140,37 +140,37 @@ public class SyncTable extends Configured implements Tool {
// because it sets up the TableOutputFormat.
TableMapReduceUtil.initTableReducerJob(targetTableName, null, job, null,
targetZkCluster, null, null);
-
+
// would be nice to add an option for bulk load instead
}
-
+
return job;
}
-
+
public static class SyncMapper extends TableMapper {
Path sourceHashDir;
-
+
Connection sourceConnection;
Connection targetConnection;
Table sourceTable;
Table targetTable;
boolean dryRun;
-
+
HashTable.TableHash sourceTableHash;
HashTable.TableHash.Reader sourceHashReader;
ImmutableBytesWritable currentSourceHash;
ImmutableBytesWritable nextSourceKey;
HashTable.ResultHasher targetHasher;
-
+
Throwable mapperException;
-
+
public static enum Counter {BATCHES, HASHES_MATCHED, HASHES_NOT_MATCHED, SOURCEMISSINGROWS,
SOURCEMISSINGCELLS, TARGETMISSINGROWS, TARGETMISSINGCELLS, ROWSWITHDIFFS, DIFFERENTCELLVALUES,
MATCHINGROWS, MATCHINGCELLS, EMPTY_BATCHES, RANGESMATCHED, RANGESNOTMATCHED};
-
+
@Override
protected void setup(Context context) throws IOException {
-
+
Configuration conf = context.getConfiguration();
sourceHashDir = new Path(conf.get(SOURCE_HASH_DIR_CONF_KEY));
sourceConnection = openConnection(conf, SOURCE_ZK_CLUSTER_CONF_KEY, null);
@@ -179,23 +179,23 @@ public class SyncTable extends Configured implements Tool {
sourceTable = openTable(sourceConnection, conf, SOURCE_TABLE_CONF_KEY);
targetTable = openTable(targetConnection, conf, TARGET_TABLE_CONF_KEY);
dryRun = conf.getBoolean(SOURCE_TABLE_CONF_KEY, false);
-
+
sourceTableHash = HashTable.TableHash.read(conf, sourceHashDir);
LOG.info("Read source hash manifest: " + sourceTableHash);
LOG.info("Read " + sourceTableHash.partitions.size() + " partition keys");
-
+
TableSplit split = (TableSplit) context.getInputSplit();
ImmutableBytesWritable splitStartKey = new ImmutableBytesWritable(split.getStartRow());
-
+
sourceHashReader = sourceTableHash.newReader(conf, splitStartKey);
findNextKeyHashPair();
-
+
// create a hasher, but don't start it right away
// instead, find the first hash batch at or after the start row
// and skip any rows that come before. they will be caught by the previous task
targetHasher = new HashTable.ResultHasher();
}
-
+
private static Connection openConnection(Configuration conf, String zkClusterConfKey,
String configPrefix)
throws IOException {
@@ -204,12 +204,12 @@ public class SyncTable extends Configured implements Tool {
zkCluster, configPrefix);
return ConnectionFactory.createConnection(clusterConf);
}
-
+
private static Table openTable(Connection connection, Configuration conf,
String tableNameConfKey) throws IOException {
return connection.getTable(TableName.valueOf(conf.get(tableNameConfKey)));
}
-
+
/**
* Attempt to read the next source key/hash pair.
* If there are no more, set nextSourceKey to null
@@ -223,7 +223,7 @@ public class SyncTable extends Configured implements Tool {
nextSourceKey = null;
}
}
-
+
@Override
protected void map(ImmutableBytesWritable key, Result value, Context context)
throws IOException, InterruptedException {
@@ -232,7 +232,7 @@ public class SyncTable extends Configured implements Tool {
while (nextSourceKey != null && key.compareTo(nextSourceKey) >= 0) {
moveToNextBatch(context);
}
-
+
// next, add the scanned row (as long as we've reached the first batch)
if (targetHasher.isBatchStarted()) {
targetHasher.hashResult(value);
@@ -247,7 +247,7 @@ public class SyncTable extends Configured implements Tool {
/**
* If there is an open hash batch, complete it and sync if there are diffs.
- * Start a new batch, and seek to read the
+ * Start a new batch, and seek to read the
*/
private void moveToNextBatch(Context context) throws IOException, InterruptedException {
if (targetHasher.isBatchStarted()) {
@@ -255,7 +255,7 @@ public class SyncTable extends Configured implements Tool {
}
targetHasher.startBatch(nextSourceKey);
currentSourceHash = sourceHashReader.getCurrentHash();
-
+
findNextKeyHashPair();
}
@@ -276,28 +276,28 @@ public class SyncTable extends Configured implements Tool {
context.getCounter(Counter.HASHES_MATCHED).increment(1);
} else {
context.getCounter(Counter.HASHES_NOT_MATCHED).increment(1);
-
+
ImmutableBytesWritable stopRow = nextSourceKey == null
? new ImmutableBytesWritable(sourceTableHash.stopRow)
: nextSourceKey;
-
+
if (LOG.isDebugEnabled()) {
LOG.debug("Hash mismatch. Key range: " + toHex(targetHasher.getBatchStartKey())
+ " to " + toHex(stopRow)
+ " sourceHash: " + toHex(currentSourceHash)
+ " targetHash: " + toHex(targetHash));
}
-
+
syncRange(context, targetHasher.getBatchStartKey(), stopRow);
}
}
private static String toHex(ImmutableBytesWritable bytes) {
return Bytes.toHex(bytes.get(), bytes.getOffset(), bytes.getLength());
}
-
+
private static final CellScanner EMPTY_CELL_SCANNER
= new CellScanner(Iterators.emptyIterator());
-
+
/**
* Rescan the given range directly from the source and target tables.
* Count and log differences, and if this is not a dry run, output Puts and Deletes
@@ -305,17 +305,16 @@ public class SyncTable extends Configured implements Tool {
*/
private void syncRange(Context context, ImmutableBytesWritable startRow,
ImmutableBytesWritable stopRow) throws IOException, InterruptedException {
-
Scan scan = sourceTableHash.initScan();
scan.setStartRow(startRow.copyBytes());
scan.setStopRow(stopRow.copyBytes());
-
+
ResultScanner sourceScanner = sourceTable.getScanner(scan);
CellScanner sourceCells = new CellScanner(sourceScanner.iterator());
- ResultScanner targetScanner = targetTable.getScanner(scan);
+ ResultScanner targetScanner = targetTable.getScanner(new Scan(scan));
CellScanner targetCells = new CellScanner(targetScanner.iterator());
-
+
boolean rangeMatched = true;
byte[] nextSourceRow = sourceCells.nextRow();
byte[] nextTargetRow = targetCells.nextRow();
@@ -327,7 +326,7 @@ public class SyncTable extends Configured implements Tool {
LOG.info("Target missing row: " + Bytes.toHex(nextSourceRow));
}
context.getCounter(Counter.TARGETMISSINGROWS).increment(1);
-
+
rowMatched = syncRowCells(context, nextSourceRow, sourceCells, EMPTY_CELL_SCANNER);
nextSourceRow = sourceCells.nextRow(); // advance only source to next row
} else if (rowComparison > 0) {
@@ -335,41 +334,41 @@ public class SyncTable extends Configured implements Tool {
LOG.info("Source missing row: " + Bytes.toHex(nextTargetRow));
}
context.getCounter(Counter.SOURCEMISSINGROWS).increment(1);
-
+
rowMatched = syncRowCells(context, nextTargetRow, EMPTY_CELL_SCANNER, targetCells);
nextTargetRow = targetCells.nextRow(); // advance only target to next row
} else {
// current row is the same on both sides, compare cell by cell
rowMatched = syncRowCells(context, nextSourceRow, sourceCells, targetCells);
- nextSourceRow = sourceCells.nextRow();
+ nextSourceRow = sourceCells.nextRow();
nextTargetRow = targetCells.nextRow();
}
-
+
if (!rowMatched) {
rangeMatched = false;
}
}
-
+
sourceScanner.close();
targetScanner.close();
-
+
context.getCounter(rangeMatched ? Counter.RANGESMATCHED : Counter.RANGESNOTMATCHED)
.increment(1);
}
-
+
private static class CellScanner {
private final Iterator results;
-
+
private byte[] currentRow;
private Result currentRowResult;
private int nextCellInRow;
-
+
private Result nextRowResult;
-
+
public CellScanner(Iterator results) {
this.results = results;
}
-
+
/**
* Advance to the next row and return its row key.
* Returns null iff there are no more rows.
@@ -390,7 +389,7 @@ public class SyncTable extends Configured implements Tool {
nextRowResult = null;
}
}
-
+
if (nextRowResult == null) {
// end of data, no more rows
currentRowResult = null;
@@ -398,7 +397,7 @@ public class SyncTable extends Configured implements Tool {
return null;
}
}
-
+
// advance to cached result for next row
currentRowResult = nextRowResult;
nextCellInRow = 0;
@@ -406,7 +405,7 @@ public class SyncTable extends Configured implements Tool {
nextRowResult = null;
return currentRow;
}
-
+
/**
* Returns the next Cell in the current row or null iff none remain.
*/
@@ -415,7 +414,7 @@ public class SyncTable extends Configured implements Tool {
// nothing left in current row
return null;
}
-
+
Cell nextCell = currentRowResult.rawCells()[nextCellInRow];
nextCellInRow++;
if (nextCellInRow == currentRowResult.size()) {
@@ -441,7 +440,7 @@ public class SyncTable extends Configured implements Tool {
return nextCell;
}
}
-
+
/**
* Compare the cells for the given row from the source and target tables.
* Count and log any differences.
@@ -465,14 +464,14 @@ public class SyncTable extends Configured implements Tool {
}
context.getCounter(Counter.TARGETMISSINGCELLS).increment(1);
matchingRow = false;
-
+
if (!dryRun) {
if (put == null) {
put = new Put(rowKey);
}
put.add(sourceCell);
}
-
+
sourceCell = sourceCells.nextCellInRow();
} else if (cellKeyComparison > 0) {
if (LOG.isDebugEnabled()) {
@@ -480,7 +479,7 @@ public class SyncTable extends Configured implements Tool {
}
context.getCounter(Counter.SOURCEMISSINGCELLS).increment(1);
matchingRow = false;
-
+
if (!dryRun) {
if (delete == null) {
delete = new Delete(rowKey);
@@ -489,7 +488,7 @@ public class SyncTable extends Configured implements Tool {
delete.addColumn(CellUtil.cloneFamily(targetCell),
CellUtil.cloneQualifier(targetCell), targetCell.getTimestamp());
}
-
+
targetCell = targetCells.nextCellInRow();
} else {
// the cell keys are equal, now check values
@@ -507,7 +506,7 @@ public class SyncTable extends Configured implements Tool {
}
context.getCounter(Counter.DIFFERENTCELLVALUES).increment(1);
matchingRow = false;
-
+
if (!dryRun) {
// overwrite target cell
if (put == null) {
@@ -519,7 +518,7 @@ public class SyncTable extends Configured implements Tool {
sourceCell = sourceCells.nextCellInRow();
targetCell = targetCells.nextCellInRow();
}
-
+
if (!dryRun && sourceTableHash.scanBatch > 0) {
if (put != null && put.size() >= sourceTableHash.scanBatch) {
context.write(new ImmutableBytesWritable(rowKey), put);
@@ -531,7 +530,7 @@ public class SyncTable extends Configured implements Tool {
}
}
}
-
+
if (!dryRun) {
if (put != null) {
context.write(new ImmutableBytesWritable(rowKey), put);
@@ -540,7 +539,7 @@ public class SyncTable extends Configured implements Tool {
context.write(new ImmutableBytesWritable(rowKey), delete);
}
}
-
+
if (matchingCells > 0) {
context.getCounter(Counter.MATCHINGCELLS).increment(matchingCells);
}
@@ -580,21 +579,21 @@ public class SyncTable extends Configured implements Tool {
if (c2 == null) {
return -1; // target missing cell
}
-
+
int result = CellComparator.compareFamilies(c1, c2);
if (result != 0) {
return result;
}
-
+
result = CellComparator.compareQualifiers(c1, c2);
if (result != 0) {
return result;
}
-
+
// note timestamp comparison is inverted - more recent cells first
return CellComparator.compareTimestamps(c1, c2);
}
-
+
@Override
protected void cleanup(Context context)
throws IOException, InterruptedException {
@@ -605,7 +604,7 @@ public class SyncTable extends Configured implements Tool {
mapperException = t;
}
}
-
+
try {
sourceTable.close();
targetTable.close();
@@ -618,7 +617,7 @@ public class SyncTable extends Configured implements Tool {
LOG.error("Suppressing exception from closing tables", t);
}
}
-
+
// propagate first exception
if (mapperException != null) {
Throwables.propagateIfInstanceOf(mapperException, IOException.class);
@@ -638,7 +637,7 @@ public class SyncTable extends Configured implements Tool {
&& (nextSourceKey.compareTo(splitEndRow) < 0 || reachedEndOfTable)) {
moveToNextBatch(context);
}
-
+
if (targetHasher.isBatchStarted()) {
// need to complete the final open hash batch
@@ -653,7 +652,7 @@ public class SyncTable extends Configured implements Tool {
} else {
scan.setStopRow(nextSourceKey.copyBytes());
}
-
+
ResultScanner targetScanner = null;
try {
targetScanner = targetTable.getScanner(scan);
@@ -671,7 +670,7 @@ public class SyncTable extends Configured implements Tool {
}
}
}
-
+
private static final int NUM_ARGS = 3;
private static void printUsage(final String errorMsg) {
if (errorMsg != null && errorMsg.length() > 0) {
@@ -681,7 +680,7 @@ public class SyncTable extends Configured implements Tool {
System.err.println("Usage: SyncTable [options] ");
System.err.println();
System.err.println("Options:");
-
+
System.err.println(" sourcezkcluster ZK cluster key of the source table");
System.err.println(" (defaults to cluster in classpath's config)");
System.err.println(" targetzkcluster ZK cluster key of the target table");
@@ -703,7 +702,7 @@ public class SyncTable extends Configured implements Tool {
+ " --sourcezkcluster=zk1.example.com,zk2.example.com,zk3.example.com:2181:/hbase"
+ " hdfs://nn:9000/hashes/tableA tableA tableA");
}
-
+
private boolean doCommandLine(final String[] args) {
if (args.length < NUM_ARGS) {
printUsage(null);
@@ -713,37 +712,37 @@ public class SyncTable extends Configured implements Tool {
sourceHashDir = new Path(args[args.length - 3]);
sourceTableName = args[args.length - 2];
targetTableName = args[args.length - 1];
-
+
for (int i = 0; i < args.length - NUM_ARGS; i++) {
String cmd = args[i];
if (cmd.equals("-h") || cmd.startsWith("--h")) {
printUsage(null);
return false;
}
-
+
final String sourceZkClusterKey = "--sourcezkcluster=";
if (cmd.startsWith(sourceZkClusterKey)) {
sourceZkCluster = cmd.substring(sourceZkClusterKey.length());
continue;
}
-
+
final String targetZkClusterKey = "--targetzkcluster=";
if (cmd.startsWith(targetZkClusterKey)) {
targetZkCluster = cmd.substring(targetZkClusterKey.length());
continue;
}
-
+
final String dryRunKey = "--dryrun=";
if (cmd.startsWith(dryRunKey)) {
dryRun = Boolean.parseBoolean(cmd.substring(dryRunKey.length()));
continue;
}
-
+
printUsage("Invalid argument '" + cmd + "'");
return false;
}
-
+
} catch (Exception e) {
e.printStackTrace();
printUsage("Can't start because " + e.getMessage());
@@ -751,7 +750,7 @@ public class SyncTable extends Configured implements Tool {
}
return true;
}
-
+
/**
* Main entry point.
*/
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index c5a850b9c4e..7339d87041c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -274,11 +274,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
private final String scannerName;
private final RegionScanner s;
private final Region r;
+ private final boolean allowPartial;
- public RegionScannerHolder(String scannerName, RegionScanner s, Region r) {
+ public RegionScannerHolder(String scannerName, RegionScanner s, Region r, boolean allowPartial) {
this.scannerName = scannerName;
this.s = s;
this.r = r;
+ this.allowPartial = allowPartial;
}
public long getNextCallSeq() {
@@ -1110,11 +1112,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return lastBlock;
}
- private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r)
- throws LeaseStillHeldException {
+ private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r,
+ boolean allowPartial) throws LeaseStillHeldException {
regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod,
new ScannerListener(scannerName));
- RegionScannerHolder rsh = new RegionScannerHolder(scannerName, s, r);
+ RegionScannerHolder rsh = new RegionScannerHolder(scannerName, s, r, allowPartial);
RegionScannerHolder existing = scanners.putIfAbsent(scannerName, rsh);
assert existing == null : "scannerId must be unique within regionserver's whole lifecycle!";
return rsh;
@@ -2460,8 +2462,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return rsh;
}
- private Pair newRegionScanner(ScanRequest request,
- ScanResponse.Builder builder) throws IOException {
+ private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder)
+ throws IOException {
Region region = getRegion(request.getRegion());
ClientProtos.Scan protoScan = request.getScan();
boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();
@@ -2491,7 +2493,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
builder.setMvccReadPoint(scanner.getMvccReadPoint());
builder.setTtl(scannerLeaseTimeoutPeriod);
String scannerName = String.valueOf(scannerId);
- return Pair.newPair(addScanner(scannerName, scanner, region), scan.isSmall());
+ return addScanner(scannerName, scanner, region,
+ !scan.isSmall() && !(request.hasLimitOfRows() && request.getLimitOfRows() > 0));
}
private void checkScanNextCallSeq(ScanRequest request, RegionScannerHolder rsh)
@@ -2548,9 +2551,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// return whether we have more results in region.
private boolean scan(PayloadCarryingRpcController controller, ScanRequest request,
- RegionScannerHolder rsh, boolean isSmallScan, long maxQuotaResultSize, int rows,
- List results, ScanResponse.Builder builder, MutableObject lastBlock,
- RpcCallContext context) throws IOException {
+ RegionScannerHolder rsh, long maxQuotaResultSize, int rows, List results,
+ ScanResponse.Builder builder, MutableObject lastBlock, RpcCallContext context)
+ throws IOException {
Region region = rsh.r;
RegionScanner scanner = rsh.s;
long maxResultSize;
@@ -2581,7 +2584,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// formed.
boolean serverGuaranteesOrderOfPartials = results.isEmpty();
boolean allowPartialResults =
- clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan;
+ clientHandlesPartials && serverGuaranteesOrderOfPartials && rsh.allowPartial;
boolean moreRows = false;
// Heartbeat messages occur when the processing of the ScanRequest is exceeds a
@@ -2738,15 +2741,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
rpcScanRequestCount.increment();
RegionScannerHolder rsh;
ScanResponse.Builder builder = ScanResponse.newBuilder();
- boolean isSmallScan;
try {
if (request.hasScannerId()) {
rsh = getRegionScanner(request);
- isSmallScan = false;
} else {
- Pair pair = newRegionScanner(request, builder);
- rsh = pair.getFirst();
- isSmallScan = pair.getSecond().booleanValue();
+ rsh = newRegionScanner(request, builder);
}
} catch (IOException e) {
if (e == SCANNER_ALREADY_CLOSED) {
@@ -2805,6 +2804,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
RegionScanner scanner = rsh.s;
boolean moreResults = 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();
boolean scannerClosed = false;
try {
@@ -2825,7 +2833,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
if (!done) {
moreResultsInRegion = scan((PayloadCarryingRpcController) controller, request, rsh,
- isSmallScan, maxQuotaResultSize, rows, results, builder, lastBlock, context);
+ maxQuotaResultSize, rows, results, builder, lastBlock, context);
}
}
@@ -2837,6 +2845,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// 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.
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, (PayloadCarryingRpcController) controller,
RegionReplicaUtil.isDefaultReplica(region.getRegionInfo()));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 4fa987508e3..419b7c507c8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.tool.Canary.RegionTask.TaskType;
import org.apache.hadoop.hbase.util.Bytes;
@@ -365,7 +366,7 @@ public final class Canary implements Tool {
scan.setFilter(new FirstKeyOnlyFilter());
scan.addFamily(column.getName());
scan.setMaxResultSize(1L);
- scan.setSmall(true);
+ scan.setOneRowLimit();
}
if (LOG.isDebugEnabled()) {
@@ -500,7 +501,7 @@ public final class Canary implements Tool {
scan.setFilter(new FirstKeyOnlyFilter());
scan.setCaching(1);
scan.setMaxResultSize(1L);
- scan.setSmall(true);
+ scan.setOneRowLimit();
stopWatch.start();
ResultScanner s = table.getScanner(scan);
s.next();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index f3becfe7d5c..6b785984d65 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -2427,14 +2427,17 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* Return the number of rows in the given table.
*/
public int countRows(final Table table) throws IOException {
- Scan scan = new Scan();
- ResultScanner results = table.getScanner(scan);
- int count = 0;
- for (@SuppressWarnings("unused") Result res : results) {
- count++;
+ return countRows(table, new Scan());
+ }
+
+ public int countRows(final Table table, final Scan scan) throws IOException {
+ try (ResultScanner results = table.getScanner(scan)) {
+ int count = 0;
+ while (results.next() != null) {
+ count++;
+ }
+ return count;
}
- results.close();
- return count;
}
public int countRows(final Table table, final byte[]... families) throws IOException {
@@ -2442,13 +2445,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
for (byte[] family: families) {
scan.addFamily(family);
}
- ResultScanner results = table.getScanner(scan);
- int count = 0;
- for (@SuppressWarnings("unused") Result res : results) {
- count++;
- }
- results.close();
- return count;
+ return countRows(table, scan);
}
/**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
index 3bf91a4a1a9..df38b8e29ac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
@@ -166,7 +166,7 @@ public class TestMetaTableAccessorNoCluster {
public ScanResponse answer(InvocationOnMock invocation) throws Throwable {
((PayloadCarryingRpcController) invocation.getArguments()[0]).setCellScanner(CellUtil
.createCellScanner(cellScannables));
- return builder.build();
+ return builder.setScannerId(1234567890L).build();
}
}).thenReturn(ScanResponse.newBuilder().setMoreResults(false).build());
// Associate a spied-upon HConnection with UTIL.getConfiguration. Need
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index ed8e848cc03..181d55a3cb4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -607,7 +607,6 @@ public class TestPartialResultsFromClientSide {
scan.setAllowPartialResults(true);
scan.setSmall(true);
scan.setMaxResultSize(1);
-
ResultScanner scanner = TABLE.getScanner(scan);
Result r = null;
@@ -731,11 +730,13 @@ public class TestPartialResultsFromClientSide {
byte[] value = Bytes.createMaxByteArray(100);
Table tmpTable = createTestTable(testName, rows, families, qualifiers, value);
-
// Open scanner before deletes
ResultScanner scanner =
tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
-
+ // now the openScanner will also fetch data and will be executed lazily, i.e, only openScanner
+ // when you call next, so here we need to make a next call to open scanner. The maxResultSize
+ // limit can make sure that we will not fetch all the data at once, so the test sill works.
+ int scannerCount = scanner.next().rawCells().length;
Delete delete1 = new Delete(rows[0]);
delete1.addColumn(families[0], qualifiers[0], 0);
tmpTable.delete(delete1);
@@ -745,7 +746,7 @@ public class TestPartialResultsFromClientSide {
tmpTable.delete(delete2);
// Should see all cells because scanner was opened prior to deletes
- int scannerCount = countCellsFromScanner(scanner);
+ scannerCount += countCellsFromScanner(scanner);
int expectedCount = numRows * numFamilies * numQualifiers;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
scannerCount == expectedCount);
@@ -758,6 +759,7 @@ public class TestPartialResultsFromClientSide {
scannerCount == expectedCount);
scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
+ scannerCount = scanner.next().rawCells().length;
// Put in 2 new rows. The timestamps differ from the deleted rows
Put put1 = new Put(rows[0]);
put1.add(new KeyValue(rows[0], families[0], qualifiers[0], 1, value));
@@ -768,7 +770,7 @@ public class TestPartialResultsFromClientSide {
tmpTable.put(put2);
// Scanner opened prior to puts. Cell count shouldn't have changed
- scannerCount = countCellsFromScanner(scanner);
+ scannerCount += countCellsFromScanner(scanner);
expectedCount = numRows * numFamilies * numQualifiers - 2;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
scannerCount == expectedCount);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java
index 4a3b15253df..803a81da3a2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java
@@ -85,9 +85,11 @@ public class TestClientScannerRPCTimeout {
public void testScannerNextRPCTimesout() throws Exception {
final TableName TABLE_NAME = TableName.valueOf("testScannerNextRPCTimesout");
Table ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY);
+ byte[] r0 = Bytes.toBytes("row-0");
byte[] r1 = Bytes.toBytes("row-1");
byte[] r2 = Bytes.toBytes("row-2");
byte[] r3 = Bytes.toBytes("row-3");
+ putToTable(ht, r0);
putToTable(ht, r1);
putToTable(ht, r2);
putToTable(ht, r3);
@@ -97,6 +99,9 @@ public class TestClientScannerRPCTimeout {
scan.setCaching(1);
ResultScanner scanner = ht.getScanner(scan);
Result result = scanner.next();
+ // fetched when openScanner
+ assertTrue("Expected row: row-0", Bytes.equals(r0, result.getRow()));
+ result = scanner.next();
assertTrue("Expected row: row-1", Bytes.equals(r1, result.getRow()));
LOG.info("Got expected first row");
long t1 = System.currentTimeMillis();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 69fdf6a0bd9..713688c8170 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -6108,7 +6108,7 @@ public class TestFromClientSide {
public void testReversedScanUnderMultiRegions() throws Exception {
// Test Initialization.
TableName TABLE = TableName.valueOf("testReversedScanUnderMultiRegions");
- byte[] maxByteArray = ReversedClientScanner.MAX_BYTE_ARRAY;
+ byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY;
byte[][] splitRows = new byte[][] { Bytes.toBytes("005"),
Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)),
Bytes.toBytes("006"),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
index 3333f54be0f..288872e7977 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
@@ -106,8 +106,8 @@ public class TestLeaseRenewal {
Scan s = new Scan();
s.setCaching(1);
ResultScanner rs = table.getScanner(s);
- // make sure that calling renewLease does not impact the scan results
- assertTrue(((AbstractClientScanner)rs).renewLease());
+ // we haven't open the scanner yet so nothing happens
+ assertFalse(((AbstractClientScanner) rs).renewLease());
assertTrue(Arrays.equals(rs.next().getRow(), ANOTHERROW));
// renew the lease a few times, long enough to be sure
// the lease would have expired otherwise
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index e2ca677db84..261915e8733 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -169,7 +169,7 @@ public class TestRegionServerMetrics {
// By default, master doesn't host meta now.
// Adding some meta related requests
- requests += 3;
+ requests += 1;
readRequests ++;
metricsRegionServer.getRegionServerWrapper().forceRecompute();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
index 1d7ec464820..6febd45b20d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
@@ -21,8 +21,6 @@ import java.io.IOException;
import java.util.List;
import java.util.concurrent.CountDownLatch;
-import junit.framework.Assert;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
+import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -199,6 +198,7 @@ public class TestScannerWithBulkload {
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
createTable(admin, tableName);
Scan scan = createScan();
+ scan.setCaching(1);
final HTable table = init(admin, l, scan, tableName);
// use bulkload
final Path hfilePath = writeToHFile(l, "/temp/testBulkLoadWithParallelScan/",
@@ -207,6 +207,7 @@ public class TestScannerWithBulkload {
conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
ResultScanner scanner = table.getScanner(scan);
+ Result result = scanner.next();
// Create a scanner and then do bulk load
final CountDownLatch latch = new CountDownLatch(1);
new Thread() {
@@ -226,7 +227,6 @@ public class TestScannerWithBulkload {
latch.await();
// By the time we do next() the bulk loaded files are also added to the kv
// scanner
- Result result = scanner.next();
scanAfterBulkLoad(scanner, result, "version1");
scanner.close();
table.close();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 79d65cda4b5..1c1ab398c93 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -947,7 +947,6 @@ public class TestAccessController extends SecureTestUtil {
for (Result r = scanner.next(); r != null; r = scanner.next()) {
// do nothing
}
- } catch (IOException e) {
} finally {
scanner.close();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
index d400fa644fc..ec8bc955f1d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
@@ -381,6 +381,7 @@ public class TestAccessController2 extends SecureTestUtil {
Scan s1 = new Scan();
s1.addFamily(TEST_FAMILY_2);
try (ResultScanner scanner1 = table.getScanner(s1);) {
+ scanner1.next();
}
}
return null;
@@ -411,6 +412,7 @@ public class TestAccessController2 extends SecureTestUtil {
Scan s1 = new Scan();
s1.addFamily(TEST_FAMILY_2);
try (ResultScanner scanner1 = table.getScanner(s1);) {
+ scanner1.next();
}
}
return null;
@@ -425,6 +427,7 @@ public class TestAccessController2 extends SecureTestUtil {
Scan s1 = new Scan();
s1.addColumn(TEST_FAMILY, Q2);
try (ResultScanner scanner1 = table.getScanner(s1);) {
+ scanner1.next();
}
}
return null;