HBASE-17167 Pass mvcc to client when scan

This commit is contained in:
zhangduo 2016-11-29 17:13:49 +08:00
parent 7c43a23c07
commit 890fcbd0e6
17 changed files with 1133 additions and 469 deletions

View File

@ -120,198 +120,192 @@ public abstract class ClientScanner extends AbstractClientScanner {
ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
throws IOException {
if (LOG.isTraceEnabled()) {
LOG.trace("Scan table=" + tableName
+ ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
if (LOG.isTraceEnabled()) {
LOG.trace(
"Scan table=" + tableName + ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
}
this.scan = scan;
this.tableName = tableName;
this.lastNext = System.currentTimeMillis();
this.connection = connection;
this.pool = pool;
this.primaryOperationTimeout = primaryOperationTimeout;
this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
if (scan.getMaxResultSize() > 0) {
this.maxScannerResultSize = scan.getMaxResultSize();
} else {
this.maxScannerResultSize = conf.getLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
}
this.scannerTimeout =
HBaseConfiguration.getInt(conf, HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
// check if application wants to collect scan metrics
initScanMetrics(scan);
// Use the caching from the Scan. If not set, use the default cache setting for this table.
if (this.scan.getCaching() > 0) {
this.caching = this.scan.getCaching();
} else {
this.caching = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING,
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
}
this.caller = rpcFactory.<Result[]> newCaller();
this.rpcControllerFactory = controllerFactory;
this.conf = conf;
initCache();
initializeScannerInConstruction();
}
protected abstract void initCache();
protected void initializeScannerInConstruction() throws IOException {
// initialize the scanner
nextScanner(this.caching, false);
}
protected ClusterConnection getConnection() {
return this.connection;
}
protected TableName getTable() {
return this.tableName;
}
protected int getRetries() {
return this.retries;
}
protected int getScannerTimeout() {
return this.scannerTimeout;
}
protected Configuration getConf() {
return this.conf;
}
protected Scan getScan() {
return scan;
}
protected ExecutorService getPool() {
return pool;
}
protected int getPrimaryOperationTimeout() {
return primaryOperationTimeout;
}
protected int getCaching() {
return caching;
}
protected long getTimestamp() {
return lastNext;
}
@VisibleForTesting
protected long getMaxResultSize() {
return maxScannerResultSize;
}
// returns true if the passed region endKey
protected boolean checkScanStopRow(final byte[] endKey) {
if (this.scan.getStopRow().length > 0) {
// there is a stop row, check to see if we are past it.
byte[] stopRow = scan.getStopRow();
int cmp = Bytes.compareTo(stopRow, 0, stopRow.length, endKey, 0, endKey.length);
if (cmp <= 0) {
// stopRow <= endKey (endKey is equals to or larger than stopRow)
// This is a stop.
return true;
}
this.scan = scan;
this.tableName = tableName;
this.lastNext = System.currentTimeMillis();
this.connection = connection;
this.pool = pool;
this.primaryOperationTimeout = primaryOperationTimeout;
this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
if (scan.getMaxResultSize() > 0) {
this.maxScannerResultSize = scan.getMaxResultSize();
} else {
this.maxScannerResultSize = conf.getLong(
HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
}
this.scannerTimeout = HBaseConfiguration.getInt(conf,
HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
}
return false; // unlikely.
}
// check if application wants to collect scan metrics
initScanMetrics(scan);
private boolean possiblyNextScanner(int nbRows, final boolean done) throws IOException {
// If we have just switched replica, don't go to the next scanner yet. Rather, try
// the scanner operations on the new replica, from the right point in the scan
// Note that when we switched to a different replica we left it at a point
// where we just did the "openScanner" with the appropriate startrow
if (callable != null && callable.switchedToADifferentReplica()) return true;
return nextScanner(nbRows, done);
}
// Use the caching from the Scan. If not set, use the default cache setting for this table.
if (this.scan.getCaching() > 0) {
this.caching = this.scan.getCaching();
} else {
this.caching = conf.getInt(
HConstants.HBASE_CLIENT_SCANNER_CACHING,
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
}
this.caller = rpcFactory.<Result[]> newCaller();
this.rpcControllerFactory = controllerFactory;
this.conf = conf;
initCache();
initializeScannerInConstruction();
/*
* Gets a scanner for the next region. If this.currentRegion != null, then we will move to the
* endrow of this.currentRegion. Else we will get scanner at the scan.getStartRow(). We will go no
* further, just tidy up outstanding scanners, if <code>currentRegion != null</code> and
* <code>done</code> is true.
* @param nbRows
* @param done Server-side says we're done scanning.
*/
protected boolean nextScanner(int nbRows, final boolean done) throws IOException {
// Close the previous scanner if it's open
if (this.callable != null) {
this.callable.setClose();
call(callable, caller, scannerTimeout);
this.callable = null;
}
protected abstract void initCache();
// Where to start the next scanner
byte[] localStartKey;
protected void initializeScannerInConstruction() throws IOException{
// initialize the scanner
nextScanner(this.caching, false);
}
protected ClusterConnection getConnection() {
return this.connection;
}
protected TableName getTable() {
return this.tableName;
}
protected int getRetries() {
return this.retries;
}
protected int getScannerTimeout() {
return this.scannerTimeout;
}
protected Configuration getConf() {
return this.conf;
}
protected Scan getScan() {
return scan;
}
protected ExecutorService getPool() {
return pool;
}
protected int getPrimaryOperationTimeout() {
return primaryOperationTimeout;
}
protected int getCaching() {
return caching;
}
protected long getTimestamp() {
return lastNext;
}
@VisibleForTesting
protected long getMaxResultSize() {
return maxScannerResultSize;
}
// returns true if the passed region endKey
protected boolean checkScanStopRow(final byte [] endKey) {
if (this.scan.getStopRow().length > 0) {
// there is a stop row, check to see if we are past it.
byte [] stopRow = scan.getStopRow();
int cmp = Bytes.compareTo(stopRow, 0, stopRow.length,
endKey, 0, endKey.length);
if (cmp <= 0) {
// stopRow <= endKey (endKey is equals to or larger than stopRow)
// This is a stop.
return true;
}
}
return false; //unlikely.
}
private boolean possiblyNextScanner(int nbRows, final boolean done) throws IOException {
// If we have just switched replica, don't go to the next scanner yet. Rather, try
// the scanner operations on the new replica, from the right point in the scan
// Note that when we switched to a different replica we left it at a point
// where we just did the "openScanner" with the appropriate startrow
if (callable != null && callable.switchedToADifferentReplica()) return true;
return nextScanner(nbRows, done);
}
/*
* Gets a scanner for the next region. If this.currentRegion != null, then
* we will move to the endrow of this.currentRegion. Else we will get
* scanner at the scan.getStartRow(). We will go no further, just tidy
* up outstanding scanners, if <code>currentRegion != null</code> and
* <code>done</code> is true.
* @param nbRows
* @param done Server-side says we're done scanning.
*/
protected boolean nextScanner(int nbRows, final boolean done)
throws IOException {
// Close the previous scanner if it's open
if (this.callable != null) {
this.callable.setClose();
call(callable, caller, scannerTimeout);
this.callable = null;
}
// Where to start the next scanner
byte [] localStartKey;
// if we're at end of table, close and return false to stop iterating
if (this.currentRegion != null) {
byte [] endKey = this.currentRegion.getEndKey();
if (endKey == null ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) ||
checkScanStopRow(endKey) ||
done) {
close();
if (LOG.isTraceEnabled()) {
LOG.trace("Finished " + this.currentRegion);
}
return false;
}
localStartKey = endKey;
// if we're at end of table, close and return false to stop iterating
if (this.currentRegion != null) {
byte[] endKey = this.currentRegion.getEndKey();
if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)
|| checkScanStopRow(endKey) || done) {
close();
if (LOG.isTraceEnabled()) {
LOG.trace("Finished " + this.currentRegion);
}
} else {
localStartKey = this.scan.getStartRow();
return false;
}
if (LOG.isDebugEnabled() && this.currentRegion != null) {
// Only worth logging if NOT first region in scan.
LOG.debug("Advancing internal scanner to startKey at '" +
Bytes.toStringBinary(localStartKey) + "'");
localStartKey = endKey;
// clear mvcc read point if we are going to switch regions
scan.resetMvccReadPoint();
if (LOG.isTraceEnabled()) {
LOG.trace("Finished " + this.currentRegion);
}
try {
callable = getScannerCallable(localStartKey, nbRows);
// Open a scanner on the region server starting at the
// beginning of the region
call(callable, caller, scannerTimeout);
this.currentRegion = callable.getHRegionInfo();
if (this.scanMetrics != null) {
this.scanMetrics.countOfRegions.incrementAndGet();
}
} catch (IOException e) {
close();
throw e;
}
return true;
} else {
localStartKey = this.scan.getStartRow();
}
if (LOG.isDebugEnabled() && this.currentRegion != null) {
// Only worth logging if NOT first region in scan.
LOG.debug(
"Advancing internal scanner to startKey at '" + Bytes.toStringBinary(localStartKey) + "'");
}
try {
callable = getScannerCallable(localStartKey, nbRows);
// Open a scanner on the region server starting at the
// beginning of the region
call(callable, caller, scannerTimeout);
this.currentRegion = callable.getHRegionInfo();
if (this.scanMetrics != null) {
this.scanMetrics.countOfRegions.incrementAndGet();
}
} catch (IOException e) {
close();
throw e;
}
return true;
}
@VisibleForTesting
boolean isAnyRPCcancelled() {
return callable.isAnyRPCcancelled();
}
Result[] call(ScannerCallableWithReplicas callable,
RpcRetryingCaller<Result[]> caller, int scannerTimeout)
throws IOException, RuntimeException {
Result[] call(ScannerCallableWithReplicas callable, RpcRetryingCaller<Result[]> caller,
int scannerTimeout) throws IOException, RuntimeException {
if (Thread.interrupted()) {
throw new InterruptedIOException();
}
@ -320,61 +314,57 @@ public abstract class ClientScanner extends AbstractClientScanner {
return caller.callWithoutRetries(callable, scannerTimeout);
}
@InterfaceAudience.Private
protected ScannerCallableWithReplicas getScannerCallable(byte [] localStartKey,
int nbRows) {
scan.setStartRow(localStartKey);
ScannerCallable s =
new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
this.rpcControllerFactory);
s.setCaching(nbRows);
ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(tableName, getConnection(),
s, pool, primaryOperationTimeout, scan,
retries, scannerTimeout, caching, conf, caller);
return sr;
}
@InterfaceAudience.Private
protected ScannerCallableWithReplicas getScannerCallable(byte[] localStartKey, int nbRows) {
scan.setStartRow(localStartKey);
ScannerCallable s = new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
this.rpcControllerFactory);
s.setCaching(nbRows);
ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(tableName, getConnection(), s,
pool, primaryOperationTimeout, scan, retries, scannerTimeout, caching, conf, caller);
return sr;
}
/**
* Publish the scan metrics. For now, we use scan.setAttribute to pass the metrics back to the
* application or TableInputFormat.Later, we could push it to other systems. We don't use
* metrics framework because it doesn't support multi-instances of the same metrics on the same
* machine; for scan/map reduce scenarios, we will have multiple scans running at the same time.
*
* By default, scan metrics are disabled; if the application wants to collect them, this
* behavior can be turned on by calling calling {@link Scan#setScanMetricsEnabled(boolean)}
*
* <p>This invocation clears the scan metrics. Metrics are aggregated in the Scan instance.
*/
protected void writeScanMetrics() {
if (this.scanMetrics == null || scanMetricsPublished) {
return;
}
MapReduceProtos.ScanMetrics pScanMetrics = ProtobufUtil.toScanMetrics(scanMetrics);
scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA, pScanMetrics.toByteArray());
scanMetricsPublished = true;
/**
* Publish the scan metrics. For now, we use scan.setAttribute to pass the metrics back to the
* application or TableInputFormat.Later, we could push it to other systems. We don't use metrics
* framework because it doesn't support multi-instances of the same metrics on the same machine;
* for scan/map reduce scenarios, we will have multiple scans running at the same time. By
* default, scan metrics are disabled; if the application wants to collect them, this behavior can
* be turned on by calling calling {@link Scan#setScanMetricsEnabled(boolean)}
* <p>
* This invocation clears the scan metrics. Metrics are aggregated in the Scan instance.
*/
protected void writeScanMetrics() {
if (this.scanMetrics == null || scanMetricsPublished) {
return;
}
MapReduceProtos.ScanMetrics pScanMetrics = ProtobufUtil.toScanMetrics(scanMetrics);
scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA, pScanMetrics.toByteArray());
scanMetricsPublished = true;
}
protected void initSyncCache() {
protected void initSyncCache() {
cache = new LinkedList<Result>();
}
protected Result nextWithSyncCache() throws IOException {
// If the scanner is closed and there's nothing left in the cache, next is a no-op.
if (cache.size() == 0 && this.closed) {
return null;
}
if (cache.size() == 0) {
loadCache();
}
if (cache.size() > 0) {
return cache.poll();
}
// if we exhausted this scanner before calling close, write out the scan metrics
writeScanMetrics();
protected Result nextWithSyncCache() throws IOException {
// If the scanner is closed and there's nothing left in the cache, next is a no-op.
if (cache.size() == 0 && this.closed) {
return null;
}
if (cache.size() == 0) {
loadCache();
}
if (cache.size() > 0) {
return cache.poll();
}
// if we exhausted this scanner before calling close, write out the scan metrics
writeScanMetrics();
return null;
}
@VisibleForTesting
public int getCacheSize() {

View File

@ -334,7 +334,7 @@ public class HTable implements Table {
* {@link Table#getScanner(Scan)} has other usage details.
*/
@Override
public ResultScanner getScanner(final Scan scan) throws IOException {
public ResultScanner getScanner(Scan scan) throws IOException {
if (scan.getBatch() > 0 && scan.isSmall()) {
throw new IllegalArgumentException("Small scan should not be used with batching");
}
@ -345,7 +345,10 @@ public class HTable implements Table {
if (scan.getMaxResultSize() <= 0) {
scan.setMaxResultSize(scannerMaxResultSize);
}
if (scan.getMvccReadPoint() > 0) {
// it is not supposed to be set by user, clear
scan.resetMvccReadPoint();
}
Boolean async = scan.isAsyncPrefetch();
if (async == null) {
async = connConfiguration.isClientScannerAsyncPrefetch();

View File

@ -0,0 +1,41 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* A helper class used to access the package private field in o.a.h.h.client package.
* <p>
* This is because we share some data structures between client and server and the data structures
* are marked as {@code InterfaceAudience.Public}, but we do not want to expose some of the fields
* to end user.
* <p>
* TODO: A better solution is to separate the data structures used in client and server.
*/
@InterfaceAudience.Private
public class PackagePrivateFieldAccessor {
public static void setMvccReadPoint(Scan scan, long mvccReadPoint) {
scan.setMvccReadPoint(mvccReadPoint);
}
public static long getMvccReadPoint(Scan scan) {
return scan.getMvccReadPoint();
}
}

View File

@ -154,26 +154,23 @@ public class Scan extends Query {
*/
public static final boolean DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH = false;
/**
* Set it true for small scan to get better performance
*
* Small scan should use pread and big scan can use seek + read
*
* seek + read is fast but can cause two problem (1) resource contention (2)
* cause too much network io
*
* [89-fb] Using pread for non-compaction read request
* https://issues.apache.org/jira/browse/HBASE-7266
*
* On the other hand, if setting it true, we would do
* openScanner,next,closeScanner in one RPC call. It means the better
* performance for small scan. [HBASE-9488].
*
* Generally, if the scan range is within one data block(64KB), it could be
* considered as a small scan.
/**
* Set it true for small scan to get better performance Small scan should use pread and big scan
* can use seek + read seek + read is fast but can cause two problem (1) resource contention (2)
* cause too much network io [89-fb] Using pread for non-compaction read request
* https://issues.apache.org/jira/browse/HBASE-7266 On the other hand, if setting it true, we
* would do openScanner,next,closeScanner in one RPC call. It means the better performance for
* small scan. [HBASE-9488]. Generally, if the scan range is within one data block(64KB), it could
* be considered as a small scan.
*/
private boolean small = false;
/**
* The mvcc read point to use when open a scanner. Remember to clear it after switching regions as
* the mvcc is only valid within region scope.
*/
private long mvccReadPoint = -1L;
/**
* Create a Scan operation across all rows.
*/
@ -253,6 +250,7 @@ public class Scan extends Query {
TimeRange tr = entry.getValue();
setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
}
this.mvccReadPoint = scan.getMvccReadPoint();
}
/**
@ -281,6 +279,7 @@ public class Scan extends Query {
TimeRange tr = entry.getValue();
setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
}
this.mvccReadPoint = -1L;
}
public boolean isGetScan() {
@ -976,4 +975,26 @@ public class Scan extends Query {
this.asyncPrefetch = asyncPrefetch;
return this;
}
/**
* Get the mvcc read point used to open a scanner.
*/
long getMvccReadPoint() {
return mvccReadPoint;
}
/**
* Set the mvcc read point used to open a scanner.
*/
Scan setMvccReadPoint(long mvccReadPoint) {
this.mvccReadPoint = mvccReadPoint;
return this;
}
/**
* Set the mvcc read point to -1 which means do not use it.
*/
Scan resetMvccReadPoint() {
return setMvccReadPoint(-1L);
}
}

View File

@ -375,6 +375,9 @@ public class ScannerCallable extends ClientServiceCallable<Result[]> {
LOG.info("Open scanner=" + id + " for scan=" + scan.toString()
+ " on region " + getLocation().toString());
}
if (response.hasMvccReadPoint()) {
this.scan.setMvccReadPoint(response.getMvccReadPoint());
}
return id;
} catch (Exception e) {
throw ProtobufUtil.handleRemoteException(e);

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@ -907,6 +908,10 @@ public final class ProtobufUtil {
if (scan.getCaching() > 0) {
scanBuilder.setCaching(scan.getCaching());
}
long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
if (mvccReadPoint > 0) {
scanBuilder.setMvccReadPoint(mvccReadPoint);
}
return scanBuilder.build();
}
@ -994,6 +999,9 @@ public final class ProtobufUtil {
if (proto.hasCaching()) {
scan.setCaching(proto.getCaching());
}
if (proto.hasMvccReadPoint()) {
PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint());
}
return scan;
}

View File

@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLoadStats;
import org.apache.hadoop.hbase.client.Result;
@ -1019,6 +1020,10 @@ public final class ProtobufUtil {
if (scan.getCaching() > 0) {
scanBuilder.setCaching(scan.getCaching());
}
long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
if (mvccReadPoint > 0) {
scanBuilder.setMvccReadPoint(mvccReadPoint);
}
return scanBuilder.build();
}
@ -1106,6 +1111,9 @@ public final class ProtobufUtil {
if (proto.hasCaching()) {
scan.setCaching(proto.getCaching());
}
if (proto.hasMvccReadPoint()) {
PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint());
}
return scan;
}

View File

@ -14554,6 +14554,15 @@ public final class ClientProtos {
*/
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder(
int index);
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
boolean hasMvccReadPoint();
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
long getMvccReadPoint();
}
/**
* <pre>
@ -14594,6 +14603,7 @@ public final class ClientProtos {
caching_ = 0;
allowPartialResults_ = false;
cfTimeRange_ = java.util.Collections.emptyList();
mvccReadPoint_ = 0L;
}
@java.lang.Override
@ -14753,6 +14763,11 @@ public final class ClientProtos {
input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.PARSER, extensionRegistry));
break;
}
case 160: {
bitField0_ |= 0x00010000;
mvccReadPoint_ = input.readUInt64();
break;
}
}
}
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@ -15153,6 +15168,21 @@ public final class ClientProtos {
return cfTimeRange_.get(index);
}
public static final int MVCC_READ_POINT_FIELD_NUMBER = 20;
private long mvccReadPoint_;
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00010000) == 0x00010000);
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
@ -15246,6 +15276,9 @@ public final class ClientProtos {
for (int i = 0; i < cfTimeRange_.size(); i++) {
output.writeMessage(19, cfTimeRange_.get(i));
}
if (((bitField0_ & 0x00010000) == 0x00010000)) {
output.writeUInt64(20, mvccReadPoint_);
}
unknownFields.writeTo(output);
}
@ -15330,6 +15363,10 @@ public final class ClientProtos {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeMessageSize(19, cfTimeRange_.get(i));
}
if (((bitField0_ & 0x00010000) == 0x00010000)) {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeUInt64Size(20, mvccReadPoint_);
}
size += unknownFields.getSerializedSize();
memoizedSize = size;
return size;
@ -15432,6 +15469,11 @@ public final class ClientProtos {
}
result = result && getCfTimeRangeList()
.equals(other.getCfTimeRangeList());
result = result && (hasMvccReadPoint() == other.hasMvccReadPoint());
if (hasMvccReadPoint()) {
result = result && (getMvccReadPoint()
== other.getMvccReadPoint());
}
result = result && unknownFields.equals(other.unknownFields);
return result;
}
@ -15525,6 +15567,11 @@ public final class ClientProtos {
hash = (37 * hash) + CF_TIME_RANGE_FIELD_NUMBER;
hash = (53 * hash) + getCfTimeRangeList().hashCode();
}
if (hasMvccReadPoint()) {
hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
getMvccReadPoint());
}
hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash;
return hash;
@ -15716,6 +15763,8 @@ public final class ClientProtos {
} else {
cfTimeRangeBuilder_.clear();
}
mvccReadPoint_ = 0L;
bitField0_ = (bitField0_ & ~0x00080000);
return this;
}
@ -15839,6 +15888,10 @@ public final class ClientProtos {
} else {
result.cfTimeRange_ = cfTimeRangeBuilder_.build();
}
if (((from_bitField0_ & 0x00080000) == 0x00080000)) {
to_bitField0_ |= 0x00010000;
}
result.mvccReadPoint_ = mvccReadPoint_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -16007,6 +16060,9 @@ public final class ClientProtos {
}
}
}
if (other.hasMvccReadPoint()) {
setMvccReadPoint(other.getMvccReadPoint());
}
this.mergeUnknownFields(other.unknownFields);
onChanged();
return this;
@ -17484,6 +17540,38 @@ public final class ClientProtos {
}
return cfTimeRangeBuilder_;
}
private long mvccReadPoint_ ;
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00080000) == 0x00080000);
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public Builder setMvccReadPoint(long value) {
bitField0_ |= 0x00080000;
mvccReadPoint_ = value;
onChanged();
return this;
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public Builder clearMvccReadPoint() {
bitField0_ = (bitField0_ & ~0x00080000);
mvccReadPoint_ = 0L;
onChanged();
return this;
}
public final Builder setUnknownFields(
final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields);
@ -19311,6 +19399,27 @@ public final class ClientProtos {
* <code>optional .hbase.pb.ScanMetrics scan_metrics = 10;</code>
*/
org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder getScanMetricsOrBuilder();
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
boolean hasMvccReadPoint();
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
long getMvccReadPoint();
}
/**
* <pre>
@ -19339,6 +19448,7 @@ public final class ClientProtos {
partialFlagPerResult_ = java.util.Collections.emptyList();
moreResultsInRegion_ = false;
heartbeatMessage_ = false;
mvccReadPoint_ = 0L;
}
@java.lang.Override
@ -19463,6 +19573,11 @@ public final class ClientProtos {
bitField0_ |= 0x00000040;
break;
}
case 88: {
bitField0_ |= 0x00000080;
mvccReadPoint_ = input.readUInt64();
break;
}
}
}
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@ -19821,6 +19936,33 @@ public final class ClientProtos {
return scanMetrics_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics.getDefaultInstance() : scanMetrics_;
}
public static final int MVCC_READ_POINT_FIELD_NUMBER = 11;
private long mvccReadPoint_;
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00000080) == 0x00000080);
}
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
@ -19863,6 +20005,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000040) == 0x00000040)) {
output.writeMessage(10, getScanMetrics());
}
if (((bitField0_ & 0x00000080) == 0x00000080)) {
output.writeUInt64(11, mvccReadPoint_);
}
unknownFields.writeTo(output);
}
@ -19918,6 +20063,10 @@ public final class ClientProtos {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeMessageSize(10, getScanMetrics());
}
if (((bitField0_ & 0x00000080) == 0x00000080)) {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
.computeUInt64Size(11, mvccReadPoint_);
}
size += unknownFields.getSerializedSize();
memoizedSize = size;
return size;
@ -19976,6 +20125,11 @@ public final class ClientProtos {
result = result && getScanMetrics()
.equals(other.getScanMetrics());
}
result = result && (hasMvccReadPoint() == other.hasMvccReadPoint());
if (hasMvccReadPoint()) {
result = result && (getMvccReadPoint()
== other.getMvccReadPoint());
}
result = result && unknownFields.equals(other.unknownFields);
return result;
}
@ -20032,6 +20186,11 @@ public final class ClientProtos {
hash = (37 * hash) + SCAN_METRICS_FIELD_NUMBER;
hash = (53 * hash) + getScanMetrics().hashCode();
}
if (hasMvccReadPoint()) {
hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
getMvccReadPoint());
}
hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash;
return hash;
@ -20186,6 +20345,8 @@ public final class ClientProtos {
scanMetricsBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000200);
mvccReadPoint_ = 0L;
bitField0_ = (bitField0_ & ~0x00000400);
return this;
}
@ -20261,6 +20422,10 @@ public final class ClientProtos {
} else {
result.scanMetrics_ = scanMetricsBuilder_.build();
}
if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
to_bitField0_ |= 0x00000080;
}
result.mvccReadPoint_ = mvccReadPoint_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -20370,6 +20535,9 @@ public final class ClientProtos {
if (other.hasScanMetrics()) {
mergeScanMetrics(other.getScanMetrics());
}
if (other.hasMvccReadPoint()) {
setMvccReadPoint(other.getMvccReadPoint());
}
this.mergeUnknownFields(other.unknownFields);
onChanged();
return this;
@ -21433,6 +21601,62 @@ public final class ClientProtos {
}
return scanMetricsBuilder_;
}
private long mvccReadPoint_ ;
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00000400) == 0x00000400);
}
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
public Builder setMvccReadPoint(long value) {
bitField0_ |= 0x00000400;
mvccReadPoint_ = value;
onChanged();
return this;
}
/**
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*/
public Builder clearMvccReadPoint() {
bitField0_ = (bitField0_ & ~0x00000400);
mvccReadPoint_ = 0L;
onChanged();
return this;
}
public final Builder setUnknownFields(
final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields);
@ -40434,7 +40658,7 @@ public final class ClientProtos {
"tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" +
"_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" +
"\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " +
"\001(\010\"\275\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
"\001(\010\"\331\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
"Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" +
"eBytesPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_ro" +
"w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" +
@ -40448,96 +40672,98 @@ public final class ClientProtos {
" \001(\0162\025.hbase.pb.Consistency:\006STRONG\022\017\n\007c" +
"aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " +
"\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" +
"lumnFamilyTimeRange\"\246\002\n\013ScanRequest\022)\n\006r",
"egion\030\001 \001(\0132\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\rclos" +
"e_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\031clien" +
"t_handles_heartbeats\030\010 \001(\010\022\032\n\022track_scan" +
"_metrics\030\t \001(\010\022\024\n\005renew\030\n \001(\010:\005false\"\232\002\n" +
"\014ScanResponse\022\030\n\020cells_per_result\030\001 \003(\r\022" +
"\022\n\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(" +
"\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030\005 \003(\0132\020.hbase.",
"pb.Result\022\r\n\005stale\030\006 \001(\010\022\037\n\027partial_flag" +
"_per_result\030\007 \003(\010\022\036\n\026more_results_in_reg" +
"ion\030\010 \001(\010\022\031\n\021heartbeat_message\030\t \001(\010\022+\n\014" +
"scan_metrics\030\n \001(\0132\025.hbase.pb.ScanMetric" +
"s\"\240\002\n\024BulkLoadHFileRequest\022)\n\006region\030\001 \002" +
"(\0132\031.hbase.pb.RegionSpecifier\022>\n\013family_" +
"path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReque" +
"st.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\022+\n" +
"\010fs_token\030\004 \001(\0132\031.hbase.pb.DelegationTok" +
"en\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tcopy_file\030\006 \001(",
"\010:\005false\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014" +
"\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n" +
"\006loaded\030\001 \002(\010\"V\n\017DelegationToken\022\022\n\niden" +
"tifier\030\001 \001(\014\022\020\n\010password\030\002 \001(\014\022\014\n\004kind\030\003" +
" \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026PrepareBulkLoad" +
"Request\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.T" +
"ableName\022)\n\006region\030\002 \001(\0132\031.hbase.pb.Regi" +
"onSpecifier\"-\n\027PrepareBulkLoadResponse\022\022" +
"\n\nbulk_token\030\001 \002(\t\"W\n\026CleanupBulkLoadReq" +
"uest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006region\030\002 \001(\013",
"2\031.hbase.pb.RegionSpecifier\"\031\n\027CleanupBu" +
"lkLoadResponse\"a\n\026CoprocessorServiceCall" +
"\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013m" +
"ethod_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030Cop" +
"rocessorServiceResult\022&\n\005value\030\001 \001(\0132\027.h" +
"base.pb.NameBytesPair\"v\n\031CoprocessorServ" +
"iceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re" +
"gionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb.C" +
"oprocessorServiceCall\"o\n\032CoprocessorServ" +
"iceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb.R",
"egionSpecifier\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.MutationPr" +
"oto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014servi" +
"ce_call\030\004 \001(\0132 .hbase.pb.CoprocessorServ" +
"iceCall\"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\017" +
"RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010" +
"\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compaction",
"Pressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadStat" +
"s\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpeci" +
"fier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLoad" +
"Stats\"\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\t" +
"exception\030\003 \001(\0132\027.hbase.pb.NameBytesPair" +
"\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Copr" +
"ocessorServiceResult\0220\n\tloadStats\030\005 \001(\0132" +
"\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Region" +
"ActionResult\0226\n\021resultOrException\030\001 \003(\0132",
"\033.hbase.pb.ResultOrException\022*\n\texceptio" +
"n\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mult" +
"iRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase.p" +
"b.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tco" +
"ndition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n\rM" +
"ultiResponse\0228\n\022regionActionResult\030\001 \003(\013" +
"2\034.hbase.pb.RegionActionResult\022\021\n\tproces" +
"sed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036.hb" +
"ase.pb.MultiRegionLoadStats*\'\n\013Consisten" +
"cy\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\263\005\n\rClientS",
"ervice\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025.hb" +
"ase.pb.GetResponse\022;\n\006Mutate\022\027.hbase.pb." +
"MutateRequest\032\030.hbase.pb.MutateResponse\022" +
"5\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase.p" +
"b.ScanResponse\022P\n\rBulkLoadHFile\022\036.hbase." +
"pb.BulkLoadHFileRequest\032\037.hbase.pb.BulkL" +
"oadHFileResponse\022V\n\017PrepareBulkLoad\022 .hb" +
"ase.pb.PrepareBulkLoadRequest\032!.hbase.pb" +
".PrepareBulkLoadResponse\022V\n\017CleanupBulkL" +
"oad\022 .hbase.pb.CleanupBulkLoadRequest\032!.",
"hbase.pb.CleanupBulkLoadResponse\022X\n\013Exec" +
"Service\022#.hbase.pb.CoprocessorServiceReq" +
"uest\032$.hbase.pb.CoprocessorServiceRespon" +
"se\022d\n\027ExecRegionServerService\022#.hbase.pb" +
".CoprocessorServiceRequest\032$.hbase.pb.Co" +
"processorServiceResponse\0228\n\005Multi\022\026.hbas" +
"e.pb.MultiRequest\032\027.hbase.pb.MultiRespon" +
"seBI\n1org.apache.hadoop.hbase.shaded.pro" +
"tobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
"lumnFamilyTimeRange\022\032\n\017mvcc_read_point\030\024",
" \001(\004:\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132" +
"\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\013" +
"2\016.hbase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016" +
"number_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 " +
"\001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_han" +
"dles_partials\030\007 \001(\010\022!\n\031client_handles_he" +
"artbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001" +
"(\010\022\024\n\005renew\030\n \001(\010:\005false\"\266\002\n\014ScanRespons" +
"e\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_i" +
"d\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001",
"(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n" +
"\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_result\030" +
"\007 \003(\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n" +
"\021heartbeat_message\030\t \001(\010\022+\n\014scan_metrics" +
"\030\n \001(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_re" +
"ad_point\030\013 \001(\004:\0010\"\240\002\n\024BulkLoadHFileReque" +
"st\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" +
"ifier\022>\n\013family_path\030\002 \003(\0132).hbase.pb.Bu" +
"lkLoadHFileRequest.FamilyPath\022\026\n\016assign_" +
"seq_num\030\003 \001(\010\022+\n\010fs_token\030\004 \001(\0132\031.hbase.",
"pb.DelegationToken\022\022\n\nbulk_token\030\005 \001(\t\022\030" +
"\n\tcopy_file\030\006 \001(\010:\005false\032*\n\nFamilyPath\022\016" +
"\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoad" +
"HFileResponse\022\016\n\006loaded\030\001 \002(\010\"V\n\017Delegat" +
"ionToken\022\022\n\nidentifier\030\001 \001(\014\022\020\n\010password" +
"\030\002 \001(\014\022\014\n\004kind\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n" +
"\026PrepareBulkLoadRequest\022\'\n\ntable_name\030\001 " +
"\002(\0132\023.hbase.pb.TableName\022)\n\006region\030\002 \001(\013" +
"2\031.hbase.pb.RegionSpecifier\"-\n\027PrepareBu" +
"lkLoadResponse\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026Cl",
"eanupBulkLoadRequest\022\022\n\nbulk_token\030\001 \002(\t" +
"\022)\n\006region\030\002 \001(\0132\031.hbase.pb.RegionSpecif" +
"ier\"\031\n\027CleanupBulkLoadResponse\"a\n\026Coproc" +
"essorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service" +
"_name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007requ" +
"est\030\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n" +
"\005value\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n" +
"\031CoprocessorServiceRequest\022)\n\006region\030\001 \002" +
"(\0132\031.hbase.pb.RegionSpecifier\022.\n\004call\030\002 " +
"\002(\0132 .hbase.pb.CoprocessorServiceCall\"o\n",
"\032CoprocessorServiceResponse\022)\n\006region\030\001 " +
"\002(\0132\031.hbase.pb.RegionSpecifier\022&\n\005value\030" +
"\002 \002(\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Actio" +
"n\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hba" +
"se.pb.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase" +
".pb.Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb" +
".CoprocessorServiceCall\"k\n\014RegionAction\022" +
")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
"er\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hba" +
"se.pb.Action\"c\n\017RegionLoadStats\022\027\n\014memst",
"oreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:" +
"\0010\022\035\n\022compactionPressure\030\003 \001(\005:\0010\"j\n\024Mul" +
"tiRegionLoadStats\022)\n\006region\030\001 \003(\0132\031.hbas" +
"e.pb.RegionSpecifier\022\'\n\004stat\030\002 \003(\0132\031.hba" +
"se.pb.RegionLoadStats\"\336\001\n\021ResultOrExcept" +
"ion\022\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hba" +
"se.pb.Result\022*\n\texception\030\003 \001(\0132\027.hbase." +
"pb.NameBytesPair\022:\n\016service_result\030\004 \001(\013" +
"2\".hbase.pb.CoprocessorServiceResult\0220\n\t" +
"loadStats\030\005 \001(\0132\031.hbase.pb.RegionLoadSta",
"tsB\002\030\001\"x\n\022RegionActionResult\0226\n\021resultOr" +
"Exception\030\001 \003(\0132\033.hbase.pb.ResultOrExcep" +
"tion\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameB" +
"ytesPair\"x\n\014MultiRequest\022,\n\014regionAction" +
"\030\001 \003(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceG" +
"roup\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb" +
".Condition\"\226\001\n\rMultiResponse\0228\n\022regionAc" +
"tionResult\030\001 \003(\0132\034.hbase.pb.RegionAction" +
"Result\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regionStati" +
"stics\030\003 \001(\0132\036.hbase.pb.MultiRegionLoadSt",
"ats*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELI" +
"NE\020\0012\263\005\n\rClientService\0222\n\003Get\022\024.hbase.pb" +
".GetRequest\032\025.hbase.pb.GetResponse\022;\n\006Mu" +
"tate\022\027.hbase.pb.MutateRequest\032\030.hbase.pb" +
".MutateResponse\0225\n\004Scan\022\025.hbase.pb.ScanR" +
"equest\032\026.hbase.pb.ScanResponse\022P\n\rBulkLo" +
"adHFile\022\036.hbase.pb.BulkLoadHFileRequest\032" +
"\037.hbase.pb.BulkLoadHFileResponse\022V\n\017Prep" +
"areBulkLoad\022 .hbase.pb.PrepareBulkLoadRe" +
"quest\032!.hbase.pb.PrepareBulkLoadResponse",
"\022V\n\017CleanupBulkLoad\022 .hbase.pb.CleanupBu" +
"lkLoadRequest\032!.hbase.pb.CleanupBulkLoad" +
"Response\022X\n\013ExecService\022#.hbase.pb.Copro" +
"cessorServiceRequest\032$.hbase.pb.Coproces" +
"sorServiceResponse\022d\n\027ExecRegionServerSe" +
"rvice\022#.hbase.pb.CoprocessorServiceReque" +
"st\032$.hbase.pb.CoprocessorServiceResponse" +
"\0228\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbas" +
"e.pb.MultiResponseBI\n1org.apache.hadoop." +
"hbase.shaded.protobuf.generatedB\014ClientP",
"rotosH\001\210\001\001\240\001\001"
};
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() {
@ -40639,7 +40865,7 @@ public final class ClientProtos {
internal_static_hbase_pb_Scan_fieldAccessorTable = new
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
internal_static_hbase_pb_Scan_descriptor,
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", });
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", });
internal_static_hbase_pb_ScanRequest_descriptor =
getDescriptor().getMessageTypes().get(12);
internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
@ -40651,7 +40877,7 @@ public final class ClientProtos {
internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
internal_static_hbase_pb_ScanResponse_descriptor,
new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", });
new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", "MvccReadPoint", });
internal_static_hbase_pb_BulkLoadHFileRequest_descriptor =
getDescriptor().getMessageTypes().get(14);
internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable = new

View File

@ -255,6 +255,7 @@ message Scan {
optional uint32 caching = 17;
optional bool allow_partial_results = 18;
repeated ColumnFamilyTimeRange cf_time_range = 19;
optional uint64 mvcc_read_point = 20 [default = 0];
}
/**
@ -317,17 +318,22 @@ message ScanResponse {
// reasons such as the size in bytes or quantity of results accumulated. This field
// will true when more results exist in the current region.
optional bool more_results_in_region = 8;
// This field is filled in if the server is sending back a heartbeat message.
// Heartbeat messages are sent back to the client to prevent the scanner from
// timing out. Seeing a heartbeat message communicates to the Client that the
// server would have continued to scan had the time limit not been reached.
optional bool heartbeat_message = 9;
// This field is filled in if the client has requested that scan metrics be tracked.
// The metrics tracked here are sent back to the client to be tracked together with
// The metrics tracked here are sent back to the client to be tracked together with
// the existing client side metrics.
optional ScanMetrics scan_metrics = 10;
// The mvcc read point which is used to open the scanner at server side. Client can
// make use of this mvcc_read_point when restarting a scanner to get a consistent view
// of a row.
optional uint64 mvcc_read_point = 11 [default = 0];
}
/**

View File

@ -14219,6 +14219,16 @@ public final class ClientProtos {
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder(
int index);
// optional uint64 mvcc_read_point = 20 [default = 0];
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
boolean hasMvccReadPoint();
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
long getMvccReadPoint();
}
/**
* Protobuf type {@code hbase.pb.Scan}
@ -14408,6 +14418,11 @@ public final class ClientProtos {
cfTimeRange_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.PARSER, extensionRegistry));
break;
}
case 160: {
bitField0_ |= 0x00010000;
mvccReadPoint_ = input.readUInt64();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -14841,6 +14856,22 @@ public final class ClientProtos {
return cfTimeRange_.get(index);
}
// optional uint64 mvcc_read_point = 20 [default = 0];
public static final int MVCC_READ_POINT_FIELD_NUMBER = 20;
private long mvccReadPoint_;
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00010000) == 0x00010000);
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
private void initFields() {
column_ = java.util.Collections.emptyList();
attribute_ = java.util.Collections.emptyList();
@ -14861,6 +14892,7 @@ public final class ClientProtos {
caching_ = 0;
allowPartialResults_ = false;
cfTimeRange_ = java.util.Collections.emptyList();
mvccReadPoint_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -14955,6 +14987,9 @@ public final class ClientProtos {
for (int i = 0; i < cfTimeRange_.size(); i++) {
output.writeMessage(19, cfTimeRange_.get(i));
}
if (((bitField0_ & 0x00010000) == 0x00010000)) {
output.writeUInt64(20, mvccReadPoint_);
}
getUnknownFields().writeTo(output);
}
@ -15040,6 +15075,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(19, cfTimeRange_.get(i));
}
if (((bitField0_ & 0x00010000) == 0x00010000)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(20, mvccReadPoint_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -15149,6 +15188,11 @@ public final class ClientProtos {
}
result = result && getCfTimeRangeList()
.equals(other.getCfTimeRangeList());
result = result && (hasMvccReadPoint() == other.hasMvccReadPoint());
if (hasMvccReadPoint()) {
result = result && (getMvccReadPoint()
== other.getMvccReadPoint());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -15238,6 +15282,10 @@ public final class ClientProtos {
hash = (37 * hash) + CF_TIME_RANGE_FIELD_NUMBER;
hash = (53 * hash) + getCfTimeRangeList().hashCode();
}
if (hasMvccReadPoint()) {
hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getMvccReadPoint());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -15421,6 +15469,8 @@ public final class ClientProtos {
} else {
cfTimeRangeBuilder_.clear();
}
mvccReadPoint_ = 0L;
bitField0_ = (bitField0_ & ~0x00080000);
return this;
}
@ -15548,6 +15598,10 @@ public final class ClientProtos {
} else {
result.cfTimeRange_ = cfTimeRangeBuilder_.build();
}
if (((from_bitField0_ & 0x00080000) == 0x00080000)) {
to_bitField0_ |= 0x00010000;
}
result.mvccReadPoint_ = mvccReadPoint_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -15690,6 +15744,9 @@ public final class ClientProtos {
}
}
}
if (other.hasMvccReadPoint()) {
setMvccReadPoint(other.getMvccReadPoint());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -17182,6 +17239,39 @@ public final class ClientProtos {
return cfTimeRangeBuilder_;
}
// optional uint64 mvcc_read_point = 20 [default = 0];
private long mvccReadPoint_ ;
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00080000) == 0x00080000);
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public Builder setMvccReadPoint(long value) {
bitField0_ |= 0x00080000;
mvccReadPoint_ = value;
onChanged();
return this;
}
/**
* <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
*/
public Builder clearMvccReadPoint() {
bitField0_ = (bitField0_ & ~0x00080000);
mvccReadPoint_ = 0L;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:hbase.pb.Scan)
}
@ -18972,6 +19062,28 @@ public final class ClientProtos {
* </pre>
*/
org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder getScanMetricsOrBuilder();
// optional uint64 mvcc_read_point = 11 [default = 0];
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
boolean hasMvccReadPoint();
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
long getMvccReadPoint();
}
/**
* Protobuf type {@code hbase.pb.ScanResponse}
@ -19123,6 +19235,11 @@ public final class ClientProtos {
bitField0_ |= 0x00000040;
break;
}
case 88: {
bitField0_ |= 0x00000080;
mvccReadPoint_ = input.readUInt64();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -19506,6 +19623,34 @@ public final class ClientProtos {
return scanMetrics_;
}
// optional uint64 mvcc_read_point = 11 [default = 0];
public static final int MVCC_READ_POINT_FIELD_NUMBER = 11;
private long mvccReadPoint_;
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00000080) == 0x00000080);
}
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
private void initFields() {
cellsPerResult_ = java.util.Collections.emptyList();
scannerId_ = 0L;
@ -19517,6 +19662,7 @@ public final class ClientProtos {
moreResultsInRegion_ = false;
heartbeatMessage_ = false;
scanMetrics_ = org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetrics.getDefaultInstance();
mvccReadPoint_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -19560,6 +19706,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000040) == 0x00000040)) {
output.writeMessage(10, scanMetrics_);
}
if (((bitField0_ & 0x00000080) == 0x00000080)) {
output.writeUInt64(11, mvccReadPoint_);
}
getUnknownFields().writeTo(output);
}
@ -19616,6 +19765,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(10, scanMetrics_);
}
if (((bitField0_ & 0x00000080) == 0x00000080)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(11, mvccReadPoint_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -19680,6 +19833,11 @@ public final class ClientProtos {
result = result && getScanMetrics()
.equals(other.getScanMetrics());
}
result = result && (hasMvccReadPoint() == other.hasMvccReadPoint());
if (hasMvccReadPoint()) {
result = result && (getMvccReadPoint()
== other.getMvccReadPoint());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -19733,6 +19891,10 @@ public final class ClientProtos {
hash = (37 * hash) + SCAN_METRICS_FIELD_NUMBER;
hash = (53 * hash) + getScanMetrics().hashCode();
}
if (hasMvccReadPoint()) {
hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getMvccReadPoint());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -19878,6 +20040,8 @@ public final class ClientProtos {
scanMetricsBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000200);
mvccReadPoint_ = 0L;
bitField0_ = (bitField0_ & ~0x00000400);
return this;
}
@ -19957,6 +20121,10 @@ public final class ClientProtos {
} else {
result.scanMetrics_ = scanMetricsBuilder_.build();
}
if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
to_bitField0_ |= 0x00000080;
}
result.mvccReadPoint_ = mvccReadPoint_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -20040,6 +20208,9 @@ public final class ClientProtos {
if (other.hasScanMetrics()) {
mergeScanMetrics(other.getScanMetrics());
}
if (other.hasMvccReadPoint()) {
setMvccReadPoint(other.getMvccReadPoint());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -21108,6 +21279,63 @@ public final class ClientProtos {
return scanMetricsBuilder_;
}
// optional uint64 mvcc_read_point = 11 [default = 0];
private long mvccReadPoint_ ;
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
public boolean hasMvccReadPoint() {
return ((bitField0_ & 0x00000400) == 0x00000400);
}
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
public long getMvccReadPoint() {
return mvccReadPoint_;
}
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
public Builder setMvccReadPoint(long value) {
bitField0_ |= 0x00000400;
mvccReadPoint_ = value;
onChanged();
return this;
}
/**
* <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
*
* <pre>
* The mvcc read point which is used to open the scanner at server side. Client can
* make use of this mvcc_read_point when restarting a scanner to get a consistent view
* of a row.
* </pre>
*/
public Builder clearMvccReadPoint() {
bitField0_ = (bitField0_ & ~0x00000400);
mvccReadPoint_ = 0L;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:hbase.pb.ScanResponse)
}
@ -39504,7 +39732,7 @@ public final class ClientProtos {
"tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" +
"_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" +
"\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " +
"\001(\010\"\275\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
"\001(\010\"\331\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
"Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" +
"eBytesPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_ro" +
"w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" +
@ -39518,96 +39746,98 @@ public final class ClientProtos {
" \001(\0162\025.hbase.pb.Consistency:\006STRONG\022\017\n\007c" +
"aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " +
"\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" +
"lumnFamilyTimeRange\"\246\002\n\013ScanRequest\022)\n\006r",
"egion\030\001 \001(\0132\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\rclos" +
"e_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\031clien" +
"t_handles_heartbeats\030\010 \001(\010\022\032\n\022track_scan" +
"_metrics\030\t \001(\010\022\024\n\005renew\030\n \001(\010:\005false\"\232\002\n" +
"\014ScanResponse\022\030\n\020cells_per_result\030\001 \003(\r\022" +
"\022\n\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(" +
"\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030\005 \003(\0132\020.hbase.",
"pb.Result\022\r\n\005stale\030\006 \001(\010\022\037\n\027partial_flag" +
"_per_result\030\007 \003(\010\022\036\n\026more_results_in_reg" +
"ion\030\010 \001(\010\022\031\n\021heartbeat_message\030\t \001(\010\022+\n\014" +
"scan_metrics\030\n \001(\0132\025.hbase.pb.ScanMetric" +
"s\"\240\002\n\024BulkLoadHFileRequest\022)\n\006region\030\001 \002" +
"(\0132\031.hbase.pb.RegionSpecifier\022>\n\013family_" +
"path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReque" +
"st.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\022+\n" +
"\010fs_token\030\004 \001(\0132\031.hbase.pb.DelegationTok" +
"en\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tcopy_file\030\006 \001(",
"\010:\005false\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014" +
"\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n" +
"\006loaded\030\001 \002(\010\"V\n\017DelegationToken\022\022\n\niden" +
"tifier\030\001 \001(\014\022\020\n\010password\030\002 \001(\014\022\014\n\004kind\030\003" +
" \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026PrepareBulkLoad" +
"Request\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.T" +
"ableName\022)\n\006region\030\002 \001(\0132\031.hbase.pb.Regi" +
"onSpecifier\"-\n\027PrepareBulkLoadResponse\022\022" +
"\n\nbulk_token\030\001 \002(\t\"W\n\026CleanupBulkLoadReq" +
"uest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006region\030\002 \001(\013",
"2\031.hbase.pb.RegionSpecifier\"\031\n\027CleanupBu" +
"lkLoadResponse\"a\n\026CoprocessorServiceCall" +
"\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013m" +
"ethod_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030Cop" +
"rocessorServiceResult\022&\n\005value\030\001 \001(\0132\027.h" +
"base.pb.NameBytesPair\"v\n\031CoprocessorServ" +
"iceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re" +
"gionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb.C" +
"oprocessorServiceCall\"o\n\032CoprocessorServ" +
"iceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb.R",
"egionSpecifier\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.MutationPr" +
"oto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014servi" +
"ce_call\030\004 \001(\0132 .hbase.pb.CoprocessorServ" +
"iceCall\"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\017" +
"RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010" +
"\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compaction",
"Pressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadStat" +
"s\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpeci" +
"fier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLoad" +
"Stats\"\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\t" +
"exception\030\003 \001(\0132\027.hbase.pb.NameBytesPair" +
"\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Copr" +
"ocessorServiceResult\0220\n\tloadStats\030\005 \001(\0132" +
"\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Region" +
"ActionResult\0226\n\021resultOrException\030\001 \003(\0132",
"\033.hbase.pb.ResultOrException\022*\n\texceptio" +
"n\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mult" +
"iRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase.p" +
"b.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tco" +
"ndition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n\rM" +
"ultiResponse\0228\n\022regionActionResult\030\001 \003(\013" +
"2\034.hbase.pb.RegionActionResult\022\021\n\tproces" +
"sed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036.hb" +
"ase.pb.MultiRegionLoadStats*\'\n\013Consisten" +
"cy\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\263\005\n\rClientS",
"ervice\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025.hb" +
"ase.pb.GetResponse\022;\n\006Mutate\022\027.hbase.pb." +
"MutateRequest\032\030.hbase.pb.MutateResponse\022" +
"5\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase.p" +
"b.ScanResponse\022P\n\rBulkLoadHFile\022\036.hbase." +
"pb.BulkLoadHFileRequest\032\037.hbase.pb.BulkL" +
"oadHFileResponse\022V\n\017PrepareBulkLoad\022 .hb" +
"ase.pb.PrepareBulkLoadRequest\032!.hbase.pb" +
".PrepareBulkLoadResponse\022V\n\017CleanupBulkL" +
"oad\022 .hbase.pb.CleanupBulkLoadRequest\032!.",
"hbase.pb.CleanupBulkLoadResponse\022X\n\013Exec" +
"Service\022#.hbase.pb.CoprocessorServiceReq" +
"uest\032$.hbase.pb.CoprocessorServiceRespon" +
"se\022d\n\027ExecRegionServerService\022#.hbase.pb" +
".CoprocessorServiceRequest\032$.hbase.pb.Co" +
"processorServiceResponse\0228\n\005Multi\022\026.hbas" +
"e.pb.MultiRequest\032\027.hbase.pb.MultiRespon" +
"seBB\n*org.apache.hadoop.hbase.protobuf.g" +
"eneratedB\014ClientProtosH\001\210\001\001\240\001\001"
"lumnFamilyTimeRange\022\032\n\017mvcc_read_point\030\024",
" \001(\004:\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132" +
"\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\013" +
"2\016.hbase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016" +
"number_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 " +
"\001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_han" +
"dles_partials\030\007 \001(\010\022!\n\031client_handles_he" +
"artbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001" +
"(\010\022\024\n\005renew\030\n \001(\010:\005false\"\266\002\n\014ScanRespons" +
"e\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_i" +
"d\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001",
"(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n" +
"\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_result\030" +
"\007 \003(\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n" +
"\021heartbeat_message\030\t \001(\010\022+\n\014scan_metrics" +
"\030\n \001(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_re" +
"ad_point\030\013 \001(\004:\0010\"\240\002\n\024BulkLoadHFileReque" +
"st\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" +
"ifier\022>\n\013family_path\030\002 \003(\0132).hbase.pb.Bu" +
"lkLoadHFileRequest.FamilyPath\022\026\n\016assign_" +
"seq_num\030\003 \001(\010\022+\n\010fs_token\030\004 \001(\0132\031.hbase.",
"pb.DelegationToken\022\022\n\nbulk_token\030\005 \001(\t\022\030" +
"\n\tcopy_file\030\006 \001(\010:\005false\032*\n\nFamilyPath\022\016" +
"\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoad" +
"HFileResponse\022\016\n\006loaded\030\001 \002(\010\"V\n\017Delegat" +
"ionToken\022\022\n\nidentifier\030\001 \001(\014\022\020\n\010password" +
"\030\002 \001(\014\022\014\n\004kind\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n" +
"\026PrepareBulkLoadRequest\022\'\n\ntable_name\030\001 " +
"\002(\0132\023.hbase.pb.TableName\022)\n\006region\030\002 \001(\013" +
"2\031.hbase.pb.RegionSpecifier\"-\n\027PrepareBu" +
"lkLoadResponse\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026Cl",
"eanupBulkLoadRequest\022\022\n\nbulk_token\030\001 \002(\t" +
"\022)\n\006region\030\002 \001(\0132\031.hbase.pb.RegionSpecif" +
"ier\"\031\n\027CleanupBulkLoadResponse\"a\n\026Coproc" +
"essorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service" +
"_name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007requ" +
"est\030\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n" +
"\005value\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n" +
"\031CoprocessorServiceRequest\022)\n\006region\030\001 \002" +
"(\0132\031.hbase.pb.RegionSpecifier\022.\n\004call\030\002 " +
"\002(\0132 .hbase.pb.CoprocessorServiceCall\"o\n",
"\032CoprocessorServiceResponse\022)\n\006region\030\001 " +
"\002(\0132\031.hbase.pb.RegionSpecifier\022&\n\005value\030" +
"\002 \002(\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Actio" +
"n\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hba" +
"se.pb.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase" +
".pb.Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb" +
".CoprocessorServiceCall\"k\n\014RegionAction\022" +
")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
"er\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hba" +
"se.pb.Action\"c\n\017RegionLoadStats\022\027\n\014memst",
"oreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:" +
"\0010\022\035\n\022compactionPressure\030\003 \001(\005:\0010\"j\n\024Mul" +
"tiRegionLoadStats\022)\n\006region\030\001 \003(\0132\031.hbas" +
"e.pb.RegionSpecifier\022\'\n\004stat\030\002 \003(\0132\031.hba" +
"se.pb.RegionLoadStats\"\336\001\n\021ResultOrExcept" +
"ion\022\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hba" +
"se.pb.Result\022*\n\texception\030\003 \001(\0132\027.hbase." +
"pb.NameBytesPair\022:\n\016service_result\030\004 \001(\013" +
"2\".hbase.pb.CoprocessorServiceResult\0220\n\t" +
"loadStats\030\005 \001(\0132\031.hbase.pb.RegionLoadSta",
"tsB\002\030\001\"x\n\022RegionActionResult\0226\n\021resultOr" +
"Exception\030\001 \003(\0132\033.hbase.pb.ResultOrExcep" +
"tion\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameB" +
"ytesPair\"x\n\014MultiRequest\022,\n\014regionAction" +
"\030\001 \003(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceG" +
"roup\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb" +
".Condition\"\226\001\n\rMultiResponse\0228\n\022regionAc" +
"tionResult\030\001 \003(\0132\034.hbase.pb.RegionAction" +
"Result\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regionStati" +
"stics\030\003 \001(\0132\036.hbase.pb.MultiRegionLoadSt",
"ats*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELI" +
"NE\020\0012\263\005\n\rClientService\0222\n\003Get\022\024.hbase.pb" +
".GetRequest\032\025.hbase.pb.GetResponse\022;\n\006Mu" +
"tate\022\027.hbase.pb.MutateRequest\032\030.hbase.pb" +
".MutateResponse\0225\n\004Scan\022\025.hbase.pb.ScanR" +
"equest\032\026.hbase.pb.ScanResponse\022P\n\rBulkLo" +
"adHFile\022\036.hbase.pb.BulkLoadHFileRequest\032" +
"\037.hbase.pb.BulkLoadHFileResponse\022V\n\017Prep" +
"areBulkLoad\022 .hbase.pb.PrepareBulkLoadRe" +
"quest\032!.hbase.pb.PrepareBulkLoadResponse",
"\022V\n\017CleanupBulkLoad\022 .hbase.pb.CleanupBu" +
"lkLoadRequest\032!.hbase.pb.CleanupBulkLoad" +
"Response\022X\n\013ExecService\022#.hbase.pb.Copro" +
"cessorServiceRequest\032$.hbase.pb.Coproces" +
"sorServiceResponse\022d\n\027ExecRegionServerSe" +
"rvice\022#.hbase.pb.CoprocessorServiceReque" +
"st\032$.hbase.pb.CoprocessorServiceResponse" +
"\0228\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbas" +
"e.pb.MultiResponseBB\n*org.apache.hadoop." +
"hbase.protobuf.generatedB\014ClientProtosH\001",
"\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -39697,7 +39927,7 @@ public final class ClientProtos {
internal_static_hbase_pb_Scan_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_Scan_descriptor,
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", });
new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", });
internal_static_hbase_pb_ScanRequest_descriptor =
getDescriptor().getMessageTypes().get(12);
internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
@ -39709,7 +39939,7 @@ public final class ClientProtos {
internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_ScanResponse_descriptor,
new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", });
new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", "MvccReadPoint", });
internal_static_hbase_pb_BulkLoadHFileRequest_descriptor =
getDescriptor().getMessageTypes().get(14);
internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable = new

View File

@ -255,6 +255,7 @@ message Scan {
optional uint32 caching = 17;
optional bool allow_partial_results = 18;
repeated ColumnFamilyTimeRange cf_time_range = 19;
optional uint64 mvcc_read_point = 20 [default = 0];
}
/**
@ -317,17 +318,22 @@ message ScanResponse {
// reasons such as the size in bytes or quantity of results accumulated. This field
// will true when more results exist in the current region.
optional bool more_results_in_region = 8;
// This field is filled in if the server is sending back a heartbeat message.
// Heartbeat messages are sent back to the client to prevent the scanner from
// timing out. Seeing a heartbeat message communicates to the Client that the
// server would have continued to scan had the time limit not been reached.
optional bool heartbeat_message = 9;
// This field is filled in if the client has requested that scan metrics be tracked.
// The metrics tracked here are sent back to the client to be tracked together with
// The metrics tracked here are sent back to the client to be tracked together with
// the existing client side metrics.
optional ScanMetrics scan_metrics = 10;
// The mvcc read point which is used to open the scanner at server side. Client can
// make use of this mvcc_read_point when restarting a scanner to get a consistent view
// of a row.
optional uint64 mvcc_read_point = 11 [default = 0];
}
/**

View File

@ -111,6 +111,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
@ -5739,8 +5740,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// synchronize on scannerReadPoints so that nobody calculates
// getSmallestReadPoint, before scannerReadPoints is updated.
IsolationLevel isolationLevel = scan.getIsolationLevel();
synchronized(scannerReadPoints) {
if (nonce == HConstants.NO_NONCE || rsServices == null
long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
synchronized (scannerReadPoints) {
if (mvccReadPoint > 0) {
this.readPt = mvccReadPoint;
} else if (nonce == HConstants.NO_NONCE || rsServices == null
|| rsServices.getNonceManager() == null) {
this.readPt = getReadPoint(isolationLevel);
} else {
@ -5748,7 +5752,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
scannerReadPoints.put(this, this.readPt);
}
initializeScanners(scan, additionalScanners);
}

View File

@ -2749,8 +2749,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
scannerName = String.valueOf(scannerId);
rsh = addScanner(scannerName, scanner, region);
ttl = this.scannerLeaseTimeoutPeriod;
builder.setMvccReadPoint(scanner.getMvccReadPoint());
}
assert scanner != null;
if (request.hasRenew() && request.getRenew()) {
rsh = scanners.get(scannerName);
lease = regionServer.leases.removeLease(scannerName);
@ -2868,7 +2868,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
} else {
throw new UnsupportedOperationException("We only do " +
"PayloadCarryingRpcControllers! FIX IF A PROBLEM: " + controller);
"HBaseRpcControllers! FIX IF A PROBLEM: " + controller);
}
}
// Use half of whichever timeout value was more restrictive... But don't allow

View File

@ -734,12 +734,9 @@ public class TestPartialResultsFromClientSide {
Table tmpTable = createTestTable(testName, rows, families, qualifiers, value);
Scan scan = new Scan();
scan.setMaxResultSize(1);
scan.setAllowPartialResults(true);
// Open scanner before deletes
ResultScanner scanner = tmpTable.getScanner(scan);
ResultScanner scanner =
tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
Delete delete1 = new Delete(rows[0]);
delete1.addColumn(families[0], qualifiers[0], 0);
@ -756,13 +753,13 @@ public class TestPartialResultsFromClientSide {
scannerCount == expectedCount);
// Minus 2 for the two cells that were deleted
scanner = tmpTable.getScanner(scan);
scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
scannerCount = countCellsFromScanner(scanner);
expectedCount = numRows * numFamilies * numQualifiers - 2;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
scannerCount == expectedCount);
scanner = tmpTable.getScanner(scan);
scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
// 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));
@ -779,7 +776,7 @@ public class TestPartialResultsFromClientSide {
scannerCount == expectedCount);
// Now the scanner should see the cells that were added by puts
scanner = tmpTable.getScanner(scan);
scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
scannerCount = countCellsFromScanner(scanner);
expectedCount = numRows * numFamilies * numQualifiers;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,

View File

@ -0,0 +1,134 @@
/**
* 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.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
@Category({ LargeTests.class, ClientTests.class })
public class TestMvccConsistentScanner {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static Connection CONN;
private static final byte[] CF = Bytes.toBytes("cf");
private static final byte[] CQ1 = Bytes.toBytes("cq1");
private static final byte[] CQ2 = Bytes.toBytes("cq2");
private static final byte[] CQ3 = Bytes.toBytes("cq3");
@Rule
public TestName testName = new TestName();
private TableName tableName;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
UTIL.startMiniCluster(2);
CONN = ConnectionFactory.createConnection(UTIL.getConfiguration());
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
CONN.close();
UTIL.shutdownMiniCluster();
}
@Before
public void setUp() throws IOException, InterruptedException {
tableName = TableName.valueOf(testName.getMethodName().replaceAll("[^0-9a-zA-Z]", "_"));
UTIL.createTable(tableName, CF);
UTIL.waitTableAvailable(tableName);
}
private void put(byte[] row, byte[] cq, byte[] value) throws IOException {
try (Table table = CONN.getTable(tableName)) {
table.put(new Put(row).addColumn(CF, cq, value));
}
}
private void move() throws IOException, InterruptedException {
HRegionInfo region =
UTIL.getHBaseCluster().getRegions(tableName).stream().findAny().get().getRegionInfo();
HRegionServer rs =
UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
.filter(r -> !r.getOnlineTables().contains(tableName)).findAny().get();
UTIL.getAdmin().move(region.getEncodedNameAsBytes(),
Bytes.toBytes(rs.getServerName().getServerName()));
while (UTIL.getRSForFirstRegionInTable(tableName) != rs) {
Thread.sleep(100);
}
}
@Test
public void testRowAtomic() throws IOException, InterruptedException {
byte[] row = Bytes.toBytes("row");
put(row, CQ1, Bytes.toBytes(1));
put(row, CQ2, Bytes.toBytes(2));
try (Table table = CONN.getTable(tableName);
ResultScanner scanner = table.getScanner(new Scan().setBatch(1).setCaching(1))) {
Result result = scanner.next();
assertEquals(1, result.rawCells().length);
assertEquals(1, Bytes.toInt(result.getValue(CF, CQ1)));
move();
put(row, CQ3, Bytes.toBytes(3));
result = scanner.next();
assertEquals(1, result.rawCells().length);
assertEquals(2, Bytes.toInt(result.getValue(CF, CQ2)));
assertNull(scanner.next());
}
}
@Test
public void testCrossRowAtomicInRegion() throws IOException, InterruptedException {
put(Bytes.toBytes("row1"), CQ1, Bytes.toBytes(1));
put(Bytes.toBytes("row2"), CQ1, Bytes.toBytes(2));
try (Table table = CONN.getTable(tableName);
ResultScanner scanner = table.getScanner(new Scan().setCaching(1))) {
Result result = scanner.next();
assertArrayEquals(Bytes.toBytes("row1"), result.getRow());
assertEquals(1, Bytes.toInt(result.getValue(CF, CQ1)));
move();
put(Bytes.toBytes("row3"), CQ1, Bytes.toBytes(3));
result = scanner.next();
assertArrayEquals(Bytes.toBytes("row2"), result.getRow());
assertEquals(2, Bytes.toInt(result.getValue(CF, CQ1)));
assertNull(scanner.next());
}
}
}

View File

@ -440,10 +440,8 @@ public class TestTags {
increment.add(new KeyValue(row2, f, q, 1234L, v));
increment.setAttribute("visibility", Bytes.toBytes("tag2"));
table.increment(increment);
Scan scan = new Scan();
scan.setStartRow(row2);
TestCoprocessorForTags.checkTagPresence = true;
scanner = table.getScanner(scan);
scanner = table.getScanner(new Scan().setStartRow(row2));
result = scanner.next();
kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
tags = TestCoprocessorForTags.tags;
@ -462,10 +460,8 @@ public class TestTags {
Append append = new Append(row3);
append.add(f, q, Bytes.toBytes("b"));
table.append(append);
scan = new Scan();
scan.setStartRow(row3);
TestCoprocessorForTags.checkTagPresence = true;
scanner = table.getScanner(scan);
scanner = table.getScanner(new Scan().setStartRow(row3));
result = scanner.next();
kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
tags = TestCoprocessorForTags.tags;
@ -479,7 +475,7 @@ public class TestTags {
append.setAttribute("visibility", Bytes.toBytes("tag2"));
table.append(append);
TestCoprocessorForTags.checkTagPresence = true;
scanner = table.getScanner(scan);
scanner = table.getScanner(new Scan().setStartRow(row3));
result = scanner.next();
kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
tags = TestCoprocessorForTags.tags;
@ -502,10 +498,8 @@ public class TestTags {
append.add(new KeyValue(row4, f, q, 1234L, v));
append.setAttribute("visibility", Bytes.toBytes("tag2"));
table.append(append);
scan = new Scan();
scan.setStartRow(row4);
TestCoprocessorForTags.checkTagPresence = true;
scanner = table.getScanner(scan);
scanner = table.getScanner(new Scan().setStartRow(row4));
result = scanner.next();
kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
tags = TestCoprocessorForTags.tags;

View File

@ -332,22 +332,16 @@ public class TestReplicationSink {
}
entries.add(builder.build());
ResultScanner scanRes = null;
try {
Scan scan = new Scan();
scanRes = table1.getScanner(scan);
try (ResultScanner scanner = table1.getScanner(new Scan())) {
// 6. Assert no existing data in table
assertEquals(0, scanRes.next(numRows).length);
// 7. Replicate the bulk loaded entry
SINK.replicateEntries(entries, CellUtil.createCellScanner(edit.getCells().iterator()),
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
scanRes = table1.getScanner(scan);
assertEquals(0, scanner.next(numRows).length);
}
// 7. Replicate the bulk loaded entry
SINK.replicateEntries(entries, CellUtil.createCellScanner(edit.getCells().iterator()),
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
try (ResultScanner scanner = table1.getScanner(new Scan())) {
// 8. Assert data is replicated
assertEquals(numRows, scanRes.next(numRows).length);
} finally {
if (scanRes != null) {
scanRes.close();
}
assertEquals(numRows, scanner.next(numRows).length);
}
}