diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractClientScanner.java index 7658faf..6296111 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractClientScanner.java @@ -30,11 +30,10 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics; @InterfaceAudience.Private public abstract class AbstractClientScanner implements ResultScanner { protected ScanMetrics scanMetrics; + protected boolean allowHeartbeatResults; - /** - * Check and initialize if application wants to collect scan metrics - */ - protected void initScanMetrics(Scan scan) { + protected AbstractClientScanner(Scan scan) { + allowHeartbeatResults = scan.getAllowHeartbeatResults(); // check if application wants to collect scan metrics if (scan.isScanMetricsEnabled()) { scanMetrics = new ScanMetrics(); @@ -69,7 +68,11 @@ public abstract class AbstractClientScanner implements ResultScanner { for(int i = 0; i < nbRows; i++) { Result next = next(); if (next != null) { - resultSets.add(next); + if (allowHeartbeatResults && next.isHeartbeat()) { + --i; // discard the heartbeat row + } else { + resultSets.add(next); + } } else { break; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java index a4514bf..9ffe201 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java @@ -111,6 +111,7 @@ public abstract class ClientScanner extends AbstractClientScanner { ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) throws IOException { + super(scan); if (LOG.isTraceEnabled()) { LOG.trace("Scan table=" + tableName + ", startRow=" + Bytes.toStringBinary(scan.getStartRow())); @@ -135,9 +136,6 @@ public abstract class ClientScanner extends AbstractClientScanner { 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(); @@ -499,6 +497,12 @@ public abstract class ClientScanner extends AbstractClientScanner { this.lastResult = rs; } } + // Add a heartbeat result to notify the caller of the timeout + if (allowHeartbeatResults && + callable.isHeartbeatMessage() && + callable.getNextCell() != null) { + cache.add(Result.create(new Cell[]{callable.getNextCell()}, null, false, false, true)); + } // Caller of this method just wants a Result. If we see a heartbeat message, it means // processing of the scan is taking a long time server side. Rather than continue to diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java index 702983b..5ad65a3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java @@ -94,6 +94,11 @@ public class Result implements CellScannable, CellScanner { * value can be seen here: {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE} */ private boolean partial = false; + /** + * Heartbeat result contains the first cell of the next row when the timeout is reached in the + * server side. + */ + private boolean heartbeat = false; // We're not using java serialization. Transient here is just a marker to say // that this is where we cache row if we're ever asked for it. private transient byte [] row = null; @@ -152,10 +157,15 @@ public class Result implements CellScannable, CellScanner { } public static Result create(List cells, Boolean exists, boolean stale, boolean partial) { + return create(cells, exists, stale, partial, false); + } + + public static Result create(List cells, Boolean exists, boolean stale, boolean partial, + boolean heartbeat) { if (exists != null){ - return new Result(null, exists, stale, partial); + return new Result(null, exists, stale, partial, heartbeat); } - return new Result(cells.toArray(new Cell[cells.size()]), null, stale, partial); + return new Result(cells.toArray(new Cell[cells.size()]), null, stale, partial, heartbeat); } /** @@ -172,18 +182,24 @@ public class Result implements CellScannable, CellScanner { } public static Result create(Cell[] cells, Boolean exists, boolean stale, boolean partial) { + return create(cells, exists, stale, partial, false); + } + + public static Result create(Cell[] cells, Boolean exists, boolean stale, boolean partial, + boolean heartbeat) { if (exists != null){ - return new Result(null, exists, stale, partial); + return new Result(null, exists, stale, partial, heartbeat); } - return new Result(cells, null, stale, partial); + return new Result(cells, null, stale, partial, heartbeat); } /** Private ctor. Use {@link #create(Cell[])}. */ - private Result(Cell[] cells, Boolean exists, boolean stale, boolean partial) { + private Result(Cell[] cells, Boolean exists, boolean stale, boolean partial, boolean heartbeat) { this.cells = cells; this.exists = exists; this.stale = stale; this.partial = partial; + this.heartbeat = heartbeat; this.readonly = false; } @@ -906,6 +922,14 @@ public class Result implements CellScannable, CellScanner { } /** + * Whether or not the result is a heartbeat result. + * @return Whether or not the result is a heartbeat result + */ + public boolean isHeartbeat() { + return heartbeat; + } + + /** * Add load information about the region to the information about the result * @param loadStats statistics about the current region from which this was returned * @deprecated use {@link #setStatistics(ClientProtos.RegionLoadStats)} instead 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 9d46bc7..9379f13 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 @@ -104,6 +104,13 @@ public class Scan extends Query { */ private boolean allowPartialResults = false; + /** + * Heartbeat {@link Result} is a fake result which contains the rowkey of the next row when + * timeout is reached in the server side. This flag controls whether return the heartbeat result + * during a scan. + */ + private boolean allowHeartbeatResults = false; + private int storeLimit = -1; private int storeOffset = 0; private boolean getScan; @@ -719,6 +726,24 @@ public class Scan extends Query { } /** + * Setting whether the caller wants to get the heartbeat result when the scan timeout is reached. + * @param allowHeartbeatResults + * @return this + */ + public Scan setAllowHeartbeatResults(boolean allowHeartbeatResults) { + this.allowHeartbeatResults = allowHeartbeatResults; + return this; + } + + /** + * @return true when the caller wants to get the heartbeat result when the scan timeout is + * reached. + */ + public boolean getAllowHeartbeatResults() { + return allowHeartbeatResults; + } + + /** * Set the value indicating whether loading CFs on demand should be allowed (cluster * default is false). On-demand CF loading doesn't load column families until necessary, e.g. * if you filter on one column, the other column family data will be loaded only for the rows 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 65f74c8..9745b70 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 @@ -87,6 +87,11 @@ public class ScannerCallable extends RegionServerCallable { * Heartbeat messages are identified by the flag {@link ScanResponse#getHeartbeatMessage()} */ protected boolean heartbeatMessage = false; + /** + * Saves whether or not the most recent response from the server has a next cell. + * The next cell is set in the heartbeat message when the scan breaks at row boundary. + */ + protected Cell nextCell = null; static { try { myAddress = DNS.getDefaultHost("default", "default"); @@ -114,6 +119,7 @@ public class ScannerCallable extends RegionServerCallable { ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory) { this(connection, tableName, scan, scanMetrics, rpcControllerFactory, 0); } + /** * * @param connection @@ -202,11 +208,14 @@ public class ScannerCallable extends RegionServerCallable { ScanRequest request = null; // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server setHeartbeatMessage(false); + setNextCell(null); try { incRPCcallsMetrics(); + // when client want to see the heartbeat result, the scanner can only break between rows + boolean handlePartials = !scan.getAllowHeartbeatResults(); request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, - this.scanMetrics != null); + this.scanMetrics != null, handlePartials, true, callTimeout); ScanResponse response = null; controller = controllerFactory.newController(); controller.setPriority(getTableName()); @@ -225,6 +234,7 @@ public class ScannerCallable extends RegionServerCallable { nextCallSeq++; long timestamp = System.currentTimeMillis(); setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage()); + setNextCell(response.hasNext() ? ProtobufUtil.toCell(response.getNext()) : null); // Results are returned via controller CellScanner cellScanner = controller.cellScanner(); rrs = ResponseConverter.getResults(cellScanner, response); @@ -320,6 +330,19 @@ public class ScannerCallable extends RegionServerCallable { this.heartbeatMessage = heartbeatMessage; } + /** + * @return Return the next cell when the most recent RPC response was a heartbeat message and + * the server is only allowed break at row boundary. If the next cell is beyond the + * scan range, null will be returned. + */ + protected Cell getNextCell() { + return nextCell; + } + + protected void setNextCell(Cell nextCell) { + this.nextCell = nextCell; + } + private void incRPCcallsMetrics() { if (this.scanMetrics == null) { return; 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 586db8c..f59ec59 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 @@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; @@ -262,6 +263,10 @@ class ScannerCallableWithReplicas implements RetryingCallable { return currentScannerCallable != null && currentScannerCallable.isHeartbeatMessage(); } + public Cell getNextCell() { + return currentScannerCallable == null ? null : currentScannerCallable.getNextCell(); + } + private void addCallsForCurrentReplica( ResultBoundedCompletionService> cs, RegionLocations rl) { RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable); 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 cc411a6..2904f26 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 @@ -507,6 +507,35 @@ public final class RequestConverter { } /** + * Create a protocol buffer ScanRequest for a scanner id + * + * @param scannerId + * @param numberOfRows + * @param closeScanner + * @param nextCallSeq + * @param trackMetrics + * @param clientHandlesPartials + * @param clientHandlesHeartbeats + * @param timeout + * @return a scan request + */ + public static ScanRequest buildScanRequest(final long scannerId, final int numberOfRows, + final boolean closeScanner, final long nextCallSeq, final boolean trackMetrics, + final boolean clientHandlesPartials, final boolean clientHandlesHeartbeats, + final long timeout) { + ScanRequest.Builder builder = ScanRequest.newBuilder(); + builder.setNumberOfRows(numberOfRows); + builder.setCloseScanner(closeScanner); + builder.setScannerId(scannerId); + builder.setNextCallSeq(nextCallSeq); + builder.setClientHandlesPartials(clientHandlesPartials); + builder.setClientHandlesHeartbeats(clientHandlesHeartbeats); + builder.setTimeout(timeout); + builder.setTrackScanMetrics(trackMetrics); + return builder.build(); + } + + /** * Create a protocol buffer bulk load request * * @param familyPaths diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml index 20ea50c..274e185 100644 --- a/hbase-protocol/pom.xml +++ b/hbase-protocol/pom.xml @@ -193,6 +193,7 @@ RowProcessor.proto RPC.proto SecureBulkLoad.proto + Snapshot.proto Tracing.proto VisibilityLabels.proto WAL.proto 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 c4b1eec..e02a5de 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 @@ -16321,6 +16321,16 @@ public final class ClientProtos { * optional bool track_scan_metrics = 9; */ boolean getTrackScanMetrics(); + + // optional uint64 timeout = 10; + /** + * optional uint64 timeout = 10; + */ + boolean hasTimeout(); + /** + * optional uint64 timeout = 10; + */ + long getTimeout(); } /** * Protobuf type {@code hbase.pb.ScanRequest} @@ -16447,6 +16457,11 @@ public final class ClientProtos { trackScanMetrics_ = input.readBool(); break; } + case 80: { + bitField0_ |= 0x00000200; + timeout_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -16643,6 +16658,22 @@ public final class ClientProtos { return trackScanMetrics_; } + // optional uint64 timeout = 10; + public static final int TIMEOUT_FIELD_NUMBER = 10; + private long timeout_; + /** + * optional uint64 timeout = 10; + */ + public boolean hasTimeout() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + /** + * optional uint64 timeout = 10; + */ + public long getTimeout() { + return timeout_; + } + private void initFields() { region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); @@ -16653,6 +16684,7 @@ public final class ClientProtos { clientHandlesPartials_ = false; clientHandlesHeartbeats_ = false; trackScanMetrics_ = false; + timeout_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -16705,6 +16737,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000100) == 0x00000100)) { output.writeBool(9, trackScanMetrics_); } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeUInt64(10, timeout_); + } getUnknownFields().writeTo(output); } @@ -16750,6 +16785,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeBoolSize(9, trackScanMetrics_); } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(10, timeout_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -16818,6 +16857,11 @@ public final class ClientProtos { result = result && (getTrackScanMetrics() == other.getTrackScanMetrics()); } + result = result && (hasTimeout() == other.hasTimeout()); + if (hasTimeout()) { + result = result && (getTimeout() + == other.getTimeout()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -16867,6 +16911,10 @@ public final class ClientProtos { hash = (37 * hash) + TRACK_SCAN_METRICS_FIELD_NUMBER; hash = (53 * hash) + hashBoolean(getTrackScanMetrics()); } + if (hasTimeout()) { + hash = (37 * hash) + TIMEOUT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getTimeout()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -17017,6 +17065,8 @@ public final class ClientProtos { bitField0_ = (bitField0_ & ~0x00000080); trackScanMetrics_ = false; bitField0_ = (bitField0_ & ~0x00000100); + timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000200); return this; } @@ -17089,6 +17139,10 @@ public final class ClientProtos { to_bitField0_ |= 0x00000100; } result.trackScanMetrics_ = trackScanMetrics_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000200; + } + result.timeout_ = timeout_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -17132,6 +17186,9 @@ public final class ClientProtos { if (other.hasTrackScanMetrics()) { setTrackScanMetrics(other.getTrackScanMetrics()); } + if (other.hasTimeout()) { + setTimeout(other.getTimeout()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -17636,6 +17693,39 @@ public final class ClientProtos { return this; } + // optional uint64 timeout = 10; + private long timeout_ ; + /** + * optional uint64 timeout = 10; + */ + public boolean hasTimeout() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + /** + * optional uint64 timeout = 10; + */ + public long getTimeout() { + return timeout_; + } + /** + * optional uint64 timeout = 10; + */ + public Builder setTimeout(long value) { + bitField0_ |= 0x00000200; + timeout_ = value; + onChanged(); + return this; + } + /** + * optional uint64 timeout = 10; + */ + public Builder clearTimeout() { + bitField0_ = (bitField0_ & ~0x00000200); + timeout_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest) } @@ -17910,6 +18000,41 @@ public final class ClientProtos { * */ org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder getScanMetricsOrBuilder(); + + // optional .hbase.pb.Cell next = 11; + /** + * optional .hbase.pb.Cell next = 11; + * + *
+     * This field is filled with the next cell (the first cell of the next row) of the
+     * scanner when server send heartbeat message after timeout. The cell can be used
+     * as start row of a new scan. This cell is not necessarily belongs to the valid
+     * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+     * 
+ */ + boolean hasNext(); + /** + * optional .hbase.pb.Cell next = 11; + * + *
+     * This field is filled with the next cell (the first cell of the next row) of the
+     * scanner when server send heartbeat message after timeout. The cell can be used
+     * as start row of a new scan. This cell is not necessarily belongs to the valid
+     * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+     * 
+ */ + org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell getNext(); + /** + * optional .hbase.pb.Cell next = 11; + * + *
+     * This field is filled with the next cell (the first cell of the next row) of the
+     * scanner when server send heartbeat message after timeout. The cell can be used
+     * as start row of a new scan. This cell is not necessarily belongs to the valid
+     * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+     * 
+ */ + org.apache.hadoop.hbase.protobuf.generated.CellProtos.CellOrBuilder getNextOrBuilder(); } /** * Protobuf type {@code hbase.pb.ScanResponse} @@ -18061,6 +18186,19 @@ public final class ClientProtos { bitField0_ |= 0x00000040; break; } + case 90: { + org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.Builder subBuilder = null; + if (((bitField0_ & 0x00000080) == 0x00000080)) { + subBuilder = next_.toBuilder(); + } + next_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(next_); + next_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000080; + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -18444,6 +18582,49 @@ public final class ClientProtos { return scanMetrics_; } + // optional .hbase.pb.Cell next = 11; + public static final int NEXT_FIELD_NUMBER = 11; + private org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell next_; + /** + * optional .hbase.pb.Cell next = 11; + * + *
+     * This field is filled with the next cell (the first cell of the next row) of the
+     * scanner when server send heartbeat message after timeout. The cell can be used
+     * as start row of a new scan. This cell is not necessarily belongs to the valid
+     * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+     * 
+ */ + public boolean hasNext() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+     * This field is filled with the next cell (the first cell of the next row) of the
+     * scanner when server send heartbeat message after timeout. The cell can be used
+     * as start row of a new scan. This cell is not necessarily belongs to the valid
+     * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+     * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell getNext() { + return next_; + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+     * This field is filled with the next cell (the first cell of the next row) of the
+     * scanner when server send heartbeat message after timeout. The cell can be used
+     * as start row of a new scan. This cell is not necessarily belongs to the valid
+     * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+     * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.CellProtos.CellOrBuilder getNextOrBuilder() { + return next_; + } + private void initFields() { cellsPerResult_ = java.util.Collections.emptyList(); scannerId_ = 0L; @@ -18455,6 +18636,7 @@ public final class ClientProtos { moreResultsInRegion_ = false; heartbeatMessage_ = false; scanMetrics_ = org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetrics.getDefaultInstance(); + next_ = org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -18498,6 +18680,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000040) == 0x00000040)) { output.writeMessage(10, scanMetrics_); } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeMessage(11, next_); + } getUnknownFields().writeTo(output); } @@ -18554,6 +18739,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(10, scanMetrics_); } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(11, next_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -18618,6 +18807,11 @@ public final class ClientProtos { result = result && getScanMetrics() .equals(other.getScanMetrics()); } + result = result && (hasNext() == other.hasNext()); + if (hasNext()) { + result = result && getNext() + .equals(other.getNext()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -18671,6 +18865,10 @@ public final class ClientProtos { hash = (37 * hash) + SCAN_METRICS_FIELD_NUMBER; hash = (53 * hash) + getScanMetrics().hashCode(); } + if (hasNext()) { + hash = (37 * hash) + NEXT_FIELD_NUMBER; + hash = (53 * hash) + getNext().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -18780,6 +18978,7 @@ public final class ClientProtos { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getResultsFieldBuilder(); getScanMetricsFieldBuilder(); + getNextFieldBuilder(); } } private static Builder create() { @@ -18816,6 +19015,12 @@ public final class ClientProtos { scanMetricsBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000200); + if (nextBuilder_ == null) { + next_ = org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.getDefaultInstance(); + } else { + nextBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000400); return this; } @@ -18895,6 +19100,14 @@ public final class ClientProtos { } else { result.scanMetrics_ = scanMetricsBuilder_.build(); } + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000080; + } + if (nextBuilder_ == null) { + result.next_ = next_; + } else { + result.next_ = nextBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -18978,6 +19191,9 @@ public final class ClientProtos { if (other.hasScanMetrics()) { mergeScanMetrics(other.getScanMetrics()); } + if (other.hasNext()) { + mergeNext(other.getNext()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -20046,6 +20262,186 @@ public final class ClientProtos { return scanMetricsBuilder_; } + // optional .hbase.pb.Cell next = 11; + private org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell next_ = org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.CellProtos.CellOrBuilder> nextBuilder_; + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public boolean hasNext() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell getNext() { + if (nextBuilder_ == null) { + return next_; + } else { + return nextBuilder_.getMessage(); + } + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public Builder setNext(org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell value) { + if (nextBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + next_ = value; + onChanged(); + } else { + nextBuilder_.setMessage(value); + } + bitField0_ |= 0x00000400; + return this; + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public Builder setNext( + org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.Builder builderForValue) { + if (nextBuilder_ == null) { + next_ = builderForValue.build(); + onChanged(); + } else { + nextBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000400; + return this; + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public Builder mergeNext(org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell value) { + if (nextBuilder_ == null) { + if (((bitField0_ & 0x00000400) == 0x00000400) && + next_ != org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.getDefaultInstance()) { + next_ = + org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.newBuilder(next_).mergeFrom(value).buildPartial(); + } else { + next_ = value; + } + onChanged(); + } else { + nextBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000400; + return this; + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public Builder clearNext() { + if (nextBuilder_ == null) { + next_ = org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.getDefaultInstance(); + onChanged(); + } else { + nextBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000400); + return this; + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.Builder getNextBuilder() { + bitField0_ |= 0x00000400; + onChanged(); + return getNextFieldBuilder().getBuilder(); + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.CellProtos.CellOrBuilder getNextOrBuilder() { + if (nextBuilder_ != null) { + return nextBuilder_.getMessageOrBuilder(); + } else { + return next_; + } + } + /** + * optional .hbase.pb.Cell next = 11; + * + *
+       * This field is filled with the next cell (the first cell of the next row) of the
+       * scanner when server send heartbeat message after timeout. The cell can be used
+       * as start row of a new scan. This cell is not necessarily belongs to the valid
+       * scan results, i.e. it can be a delete marker or won't pass the scan filter.
+       * 
+ */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.CellProtos.CellOrBuilder> + getNextFieldBuilder() { + if (nextBuilder_ == null) { + nextBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.CellProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.CellProtos.CellOrBuilder>( + next_, + getParentForChildren(), + isClean()); + next_ = null; + } + return nextBuilder_; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.ScanResponse) } @@ -33180,75 +33576,77 @@ public final class ClientProtos { "t\030\014 \001(\r\022&\n\036load_column_families_on_deman" + "d\030\r \001(\010\022\r\n\005small\030\016 \001(\010\022\027\n\010reversed\030\017 \001(\010" + ":\005false\0222\n\013consistency\030\020 \001(\0162\025.hbase.pb." + - "Consistency:\006STRONG\022\017\n\007caching\030\021 \001(\r\"\220\002\n" + + "Consistency:\006STRONG\022\017\n\007caching\030\021 \001(\r\"\241\002\n" + "\013ScanRequest\022)\n\006region\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_r" + "ows\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_partia", "ls\030\007 \001(\010\022!\n\031client_handles_heartbeats\030\010 " + - "\001(\010\022\032\n\022track_scan_metrics\030\t \001(\010\"\232\002\n\014Scan" + - "Response\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nsc" + - "anner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003" + - "ttl\030\004 \001(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Re" + - "sult\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\"\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.BulkLoadHFileRequest.Fa" + - "milyPath\022\026\n\016assign_seq_num\030\003 \001(\010\032*\n\nFami" + - "lyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025" + - "BulkLoadHFileResponse\022\016\n\006loaded\030\001 \002(\010\"a\n" + - "\026CoprocessorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014" + - "service_name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022" + - "\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorServiceRe" + - "sult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.NameBytes" + - "Pair\"v\n\031CoprocessorServiceRequest\022)\n\006reg", - "ion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022.\n\004" + - "call\030\002 \002(\0132 .hbase.pb.CoprocessorService" + - "Call\"o\n\032CoprocessorServiceResponse\022)\n\006re" + - "gion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022&\n" + - "\005value\030\002 \002(\0132\027.hbase.pb.NameBytesPair\"\226\001" + - "\n\006Action\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(" + - "\0132\027.hbase.pb.MutationProto\022\032\n\003get\030\003 \001(\0132" + - "\r.hbase.pb.Get\0226\n\014service_call\030\004 \001(\0132 .h" + - "base.pb.CoprocessorServiceCall\"k\n\014Region" + - "Action\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Region", - "Specifier\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(" + - "\0132\020.hbase.pb.Action\"D\n\017RegionLoadStats\022\027" + - "\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy" + - "\030\002 \001(\005:\0010\"\332\001\n\021ResultOrException\022\r\n\005index" + - "\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result" + - "\022*\n\texception\030\003 \001(\0132\027.hbase.pb.NameBytes" + - "Pair\022:\n\016service_result\030\004 \001(\0132\".hbase.pb." + - "CoprocessorServiceResult\022,\n\tloadStats\030\005 " + - "\001(\0132\031.hbase.pb.RegionLoadStats\"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\"\\\n\rMu" + - "ltiResponse\0228\n\022regionActionResult\030\001 \003(\0132" + - "\034.hbase.pb.RegionActionResult\022\021\n\tprocess" + - "ed\030\002 \001(\010*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010T" + - "IMELINE\020\0012\203\004\n\rClientService\0222\n\003Get\022\024.hba" + - "se.pb.GetRequest\032\025.hbase.pb.GetResponse\022", - ";\n\006Mutate\022\027.hbase.pb.MutateRequest\032\030.hba" + - "se.pb.MutateResponse\0225\n\004Scan\022\025.hbase.pb." + - "ScanRequest\032\026.hbase.pb.ScanResponse\022P\n\rB" + - "ulkLoadHFile\022\036.hbase.pb.BulkLoadHFileReq" + - "uest\032\037.hbase.pb.BulkLoadHFileResponse\022X\n" + - "\013ExecService\022#.hbase.pb.CoprocessorServi" + - "ceRequest\032$.hbase.pb.CoprocessorServiceR" + - "esponse\022d\n\027ExecRegionServerService\022#.hba" + - "se.pb.CoprocessorServiceRequest\032$.hbase." + - "pb.CoprocessorServiceResponse\0228\n\005Multi\022\026", - ".hbase.pb.MultiRequest\032\027.hbase.pb.MultiR" + - "esponseBB\n*org.apache.hadoop.hbase.proto" + - "buf.generatedB\014ClientProtosH\001\210\001\001\240\001\001" + "\001(\010\022\032\n\022track_scan_metrics\030\t \001(\010\022\017\n\007timeo" + + "ut\030\n \001(\004\"\270\002\n\014ScanResponse\022\030\n\020cells_per_r" + + "esult\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\027" + + "partial_flag_per_result\030\007 \003(\010\022\036\n\026more_re" + + "sults_in_region\030\010 \001(\010\022\031\n\021heartbeat_messa" + + "ge\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbase.p" + + "b.ScanMetrics\022\034\n\004next\030\013 \001(\0132\016.hbase.pb.C", + "ell\"\305\001\n\024BulkLoadHFileRequest\022)\n\006region\030\001" + + " \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013famil" + + "y_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReq" + + "uest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\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_name" + + "\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorSe" + + "rviceResult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.Na", + "meBytesPair\"v\n\031CoprocessorServiceRequest" + + "\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecif" + + "ier\022.\n\004call\030\002 \002(\0132 .hbase.pb.Coprocessor" + + "ServiceCall\"o\n\032CoprocessorServiceRespons" + + "e\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpeci" + + "fier\022&\n\005value\030\002 \002(\0132\027.hbase.pb.NameBytes" + + "Pair\"\226\001\n\006Action\022\r\n\005index\030\001 \001(\r\022)\n\010mutati" + + "on\030\002 \001(\0132\027.hbase.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" + + ".RegionSpecifier\022\016\n\006atomic\030\002 \001(\010\022 \n\006acti" + + "on\030\003 \003(\0132\020.hbase.pb.Action\"D\n\017RegionLoad" + + "Stats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rheapOc" + + "cupancy\030\002 \001(\005:\0010\"\332\001\n\021ResultOrException\022\r" + + "\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb" + + ".Result\022*\n\texception\030\003 \001(\0132\027.hbase.pb.Na" + + "meBytesPair\022:\n\016service_result\030\004 \001(\0132\".hb" + + "ase.pb.CoprocessorServiceResult\022,\n\tloadS" + + "tats\030\005 \001(\0132\031.hbase.pb.RegionLoadStats\"x\n", + "\022RegionActionResult\0226\n\021resultOrException" + + "\030\001 \003(\0132\033.hbase.pb.ResultOrException\022*\n\te" + + "xception\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"" + + "x\n\014MultiRequest\022,\n\014regionAction\030\001 \003(\0132\026." + + "hbase.pb.RegionAction\022\022\n\nnonceGroup\030\002 \001(" + + "\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb.Conditio" + + "n\"\\\n\rMultiResponse\0228\n\022regionActionResult" + + "\030\001 \003(\0132\034.hbase.pb.RegionActionResult\022\021\n\t" + + "processed\030\002 \001(\010*\'\n\013Consistency\022\n\n\006STRONG" + + "\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientService\0222\n\003Ge", + "t\022\024.hbase.pb.GetRequest\032\025.hbase.pb.GetRe" + + "sponse\022;\n\006Mutate\022\027.hbase.pb.MutateReques" + + "t\032\030.hbase.pb.MutateResponse\0225\n\004Scan\022\025.hb" + + "ase.pb.ScanRequest\032\026.hbase.pb.ScanRespon" + + "se\022P\n\rBulkLoadHFile\022\036.hbase.pb.BulkLoadH" + + "FileRequest\032\037.hbase.pb.BulkLoadHFileResp" + + "onse\022X\n\013ExecService\022#.hbase.pb.Coprocess" + + "orServiceRequest\032$.hbase.pb.CoprocessorS" + + "erviceResponse\022d\n\027ExecRegionServerServic" + + "e\022#.hbase.pb.CoprocessorServiceRequest\032$", + ".hbase.pb.CoprocessorServiceResponse\0228\n\005" + + "Multi\022\026.hbase.pb.MultiRequest\032\027.hbase.pb" + + ".MultiResponseBB\n*org.apache.hadoop.hbas" + + "e.protobuf.generatedB\014ClientProtosH\001\210\001\001\240" + + "\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -33344,13 +33742,13 @@ public final class ClientProtos { 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", }); + new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Timeout", }); internal_static_hbase_pb_ScanResponse_descriptor = getDescriptor().getMessageTypes().get(13); 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", "Next", }); internal_static_hbase_pb_BulkLoadHFileRequest_descriptor = getDescriptor().getMessageTypes().get(14); internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java index fe18cae..0763365 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java @@ -17341,7 +17341,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.RowRange range_list = 4; */ - java.util.List + java.util.List getRangeListList(); /** * repeated .hbase.pb.RowRange range_list = 4; @@ -17354,7 +17354,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.RowRange range_list = 4; */ - java.util.List + java.util.List getRangeListOrBuilderList(); /** * repeated .hbase.pb.RowRange range_list = 4; @@ -17552,7 +17552,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.RowRange range_list = 4; */ - public java.util.List + public java.util.List getRangeListOrBuilderList() { return rangeList_; } @@ -17932,7 +17932,7 @@ public final class FilterProtos { rangeListBuilder_ = null; rangeList_ = other.rangeList_; bitField0_ = (bitField0_ & ~0x00000008); - rangeListBuilder_ = + rangeListBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? getRangeListFieldBuilder() : null; } else { @@ -17946,11 +17946,11 @@ public final class FilterProtos { public final boolean isInitialized() { if (!hasColumnFamily()) { - + return false; } if (!hasQualifier()) { - + return false; } return true; @@ -18314,7 +18314,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.RowRange range_list = 4; */ - public java.util.List + public java.util.List getRangeListOrBuilderList() { if (rangeListBuilder_ != null) { return rangeListBuilder_.getMessageOrBuilderList(); @@ -18340,12 +18340,12 @@ public final class FilterProtos { /** * repeated .hbase.pb.RowRange range_list = 4; */ - public java.util.List + public java.util.List getRangeListBuilderList() { return getRangeListFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder> + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder> getRangeListFieldBuilder() { if (rangeListBuilder_ == null) { rangeListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< @@ -18377,7 +18377,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.SQLPredicatePushDownColumnFilter column_filter_list = 1; */ - java.util.List + java.util.List getColumnFilterListList(); /** * repeated .hbase.pb.SQLPredicatePushDownColumnFilter column_filter_list = 1; @@ -18390,7 +18390,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.SQLPredicatePushDownColumnFilter column_filter_list = 1; */ - java.util.List + java.util.List getColumnFilterListOrBuilderList(); /** * repeated .hbase.pb.SQLPredicatePushDownColumnFilter column_filter_list = 1; @@ -18511,7 +18511,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.SQLPredicatePushDownColumnFilter column_filter_list = 1; */ - public java.util.List + public java.util.List getColumnFilterListOrBuilderList() { return columnFilterList_; } @@ -18799,7 +18799,7 @@ public final class FilterProtos { columnFilterListBuilder_ = null; columnFilterList_ = other.columnFilterList_; bitField0_ = (bitField0_ & ~0x00000001); - columnFilterListBuilder_ = + columnFilterListBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? getColumnFilterListFieldBuilder() : null; } else { @@ -18814,7 +18814,7 @@ public final class FilterProtos { public final boolean isInitialized() { for (int i = 0; i < getColumnFilterListCount(); i++) { if (!getColumnFilterList(i).isInitialized()) { - + return false; } } @@ -19035,7 +19035,7 @@ public final class FilterProtos { /** * repeated .hbase.pb.SQLPredicatePushDownColumnFilter column_filter_list = 1; */ - public java.util.List + public java.util.List getColumnFilterListOrBuilderList() { if (columnFilterListBuilder_ != null) { return columnFilterListBuilder_.getMessageOrBuilderList(); @@ -19061,12 +19061,12 @@ public final class FilterProtos { /** * repeated .hbase.pb.SQLPredicatePushDownColumnFilter column_filter_list = 1; */ - public java.util.List + public java.util.List getColumnFilterListBuilderList() { return getColumnFilterListFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SQLPredicatePushDownColumnFilter, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SQLPredicatePushDownColumnFilter.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SQLPredicatePushDownColumnFilterOrBuilder> + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SQLPredicatePushDownColumnFilter, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SQLPredicatePushDownColumnFilter.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SQLPredicatePushDownColumnFilterOrBuilder> getColumnFilterListFieldBuilder() { if (columnFilterListBuilder_ == null) { columnFilterListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java index 8dbb5ad..9805d50 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java @@ -11,13 +11,13 @@ public final class SnapshotProtos { public interface SnapshotFileInfoOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required .SnapshotFileInfo.Type type = 1; + // required .hbase.pb.SnapshotFileInfo.Type type = 1; /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ boolean hasType(); /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType(); @@ -67,7 +67,7 @@ public final class SnapshotProtos { getWalNameBytes(); } /** - * Protobuf type {@code SnapshotFileInfo} + * Protobuf type {@code hbase.pb.SnapshotFileInfo} */ public static final class SnapshotFileInfo extends com.google.protobuf.GeneratedMessage @@ -157,12 +157,12 @@ public final class SnapshotProtos { } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Builder.class); } @@ -183,7 +183,7 @@ public final class SnapshotProtos { } /** - * Protobuf enum {@code SnapshotFileInfo.Type} + * Protobuf enum {@code hbase.pb.SnapshotFileInfo.Type} */ public enum Type implements com.google.protobuf.ProtocolMessageEnum { @@ -261,21 +261,21 @@ public final class SnapshotProtos { this.value = value; } - // @@protoc_insertion_point(enum_scope:SnapshotFileInfo.Type) + // @@protoc_insertion_point(enum_scope:hbase.pb.SnapshotFileInfo.Type) } private int bitField0_; - // required .SnapshotFileInfo.Type type = 1; + // required .hbase.pb.SnapshotFileInfo.Type type = 1; public static final int TYPE_FIELD_NUMBER = 1; private org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type type_; /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ public boolean hasType() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType() { return type_; @@ -613,19 +613,19 @@ public final class SnapshotProtos { return builder; } /** - * Protobuf type {@code SnapshotFileInfo} + * Protobuf type {@code hbase.pb.SnapshotFileInfo} */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfoOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Builder.class); } @@ -667,7 +667,7 @@ public final class SnapshotProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor; } public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo getDefaultInstanceForType() { @@ -767,22 +767,22 @@ public final class SnapshotProtos { } private int bitField0_; - // required .SnapshotFileInfo.Type type = 1; + // required .hbase.pb.SnapshotFileInfo.Type type = 1; private org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type type_ = org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type.HFILE; /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ public boolean hasType() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType() { return type_; } /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ public Builder setType(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type value) { if (value == null) { @@ -794,7 +794,7 @@ public final class SnapshotProtos { return this; } /** - * required .SnapshotFileInfo.Type type = 1; + * required .hbase.pb.SnapshotFileInfo.Type type = 1; */ public Builder clearType() { bitField0_ = (bitField0_ & ~0x00000001); @@ -1025,7 +1025,7 @@ public final class SnapshotProtos { return this; } - // @@protoc_insertion_point(builder_scope:SnapshotFileInfo) + // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotFileInfo) } static { @@ -1033,7 +1033,7 @@ public final class SnapshotProtos { defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:SnapshotFileInfo) + // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotFileInfo) } public interface SnapshotRegionManifestOrBuilder @@ -1049,47 +1049,47 @@ public final class SnapshotProtos { */ int getVersion(); - // required .RegionInfo region_info = 2; + // required .hbase.pb.RegionInfo region_info = 2; /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ boolean hasRegionInfo(); /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(); /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(); - // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ java.util.List getFamilyFilesList(); /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index); /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ int getFamilyFilesCount(); /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ java.util.List getFamilyFilesOrBuilderList(); /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder( int index); } /** - * Protobuf type {@code SnapshotRegionManifest} + * Protobuf type {@code hbase.pb.SnapshotRegionManifest} */ public static final class SnapshotRegionManifest extends com.google.protobuf.GeneratedMessage @@ -1182,12 +1182,12 @@ public final class SnapshotProtos { } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder.class); } @@ -1225,17 +1225,17 @@ public final class SnapshotProtos { com.google.protobuf.ByteString getNameBytes(); - // optional .Reference reference = 2; + // optional .hbase.pb.Reference reference = 2; /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ boolean hasReference(); /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference(); /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder(); @@ -1258,7 +1258,7 @@ public final class SnapshotProtos { long getFileSize(); } /** - * Protobuf type {@code SnapshotRegionManifest.StoreFile} + * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile} */ public static final class StoreFile extends com.google.protobuf.GeneratedMessage @@ -1345,12 +1345,12 @@ public final class SnapshotProtos { } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder.class); } @@ -1414,23 +1414,23 @@ public final class SnapshotProtos { } } - // optional .Reference reference = 2; + // optional .hbase.pb.Reference reference = 2; public static final int REFERENCE_FIELD_NUMBER = 2; private org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference reference_; /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public boolean hasReference() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference() { return reference_; } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder() { return reference_; @@ -1652,19 +1652,19 @@ public final class SnapshotProtos { return builder; } /** - * Protobuf type {@code SnapshotRegionManifest.StoreFile} + * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile} */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder.class); } @@ -1709,7 +1709,7 @@ public final class SnapshotProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor; } public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getDefaultInstanceForType() { @@ -1882,18 +1882,18 @@ public final class SnapshotProtos { return this; } - // optional .Reference reference = 2; + // optional .hbase.pb.Reference reference = 2; private org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference reference_ = org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder> referenceBuilder_; /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public boolean hasReference() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference() { if (referenceBuilder_ == null) { @@ -1903,7 +1903,7 @@ public final class SnapshotProtos { } } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public Builder setReference(org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference value) { if (referenceBuilder_ == null) { @@ -1919,7 +1919,7 @@ public final class SnapshotProtos { return this; } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public Builder setReference( org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder builderForValue) { @@ -1933,7 +1933,7 @@ public final class SnapshotProtos { return this; } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public Builder mergeReference(org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference value) { if (referenceBuilder_ == null) { @@ -1952,7 +1952,7 @@ public final class SnapshotProtos { return this; } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public Builder clearReference() { if (referenceBuilder_ == null) { @@ -1965,7 +1965,7 @@ public final class SnapshotProtos { return this; } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder getReferenceBuilder() { bitField0_ |= 0x00000002; @@ -1973,7 +1973,7 @@ public final class SnapshotProtos { return getReferenceFieldBuilder().getBuilder(); } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ public org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder() { if (referenceBuilder_ != null) { @@ -1983,7 +1983,7 @@ public final class SnapshotProtos { } } /** - * optional .Reference reference = 2; + * optional .hbase.pb.Reference reference = 2; */ private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder> @@ -2048,7 +2048,7 @@ public final class SnapshotProtos { return this; } - // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest.StoreFile) + // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest.StoreFile) } static { @@ -2056,7 +2056,7 @@ public final class SnapshotProtos { defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:SnapshotRegionManifest.StoreFile) + // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest.StoreFile) } public interface FamilyFilesOrBuilder @@ -2072,33 +2072,33 @@ public final class SnapshotProtos { */ com.google.protobuf.ByteString getFamilyName(); - // repeated .SnapshotRegionManifest.StoreFile store_files = 2; + // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ java.util.List getStoreFilesList(); /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index); /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ int getStoreFilesCount(); /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ java.util.List getStoreFilesOrBuilderList(); /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder( int index); } /** - * Protobuf type {@code SnapshotRegionManifest.FamilyFiles} + * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles} */ public static final class FamilyFiles extends com.google.protobuf.GeneratedMessage @@ -2178,12 +2178,12 @@ public final class SnapshotProtos { } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder.class); } @@ -2220,36 +2220,36 @@ public final class SnapshotProtos { return familyName_; } - // repeated .SnapshotRegionManifest.StoreFile store_files = 2; + // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; public static final int STORE_FILES_FIELD_NUMBER = 2; private java.util.List storeFiles_; /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public java.util.List getStoreFilesList() { return storeFiles_; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public java.util.List getStoreFilesOrBuilderList() { return storeFiles_; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public int getStoreFilesCount() { return storeFiles_.size(); } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index) { return storeFiles_.get(index); } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder( int index) { @@ -2428,19 +2428,19 @@ public final class SnapshotProtos { return builder; } /** - * Protobuf type {@code SnapshotRegionManifest.FamilyFiles} + * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles} */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder.class); } @@ -2483,7 +2483,7 @@ public final class SnapshotProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor; } public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getDefaultInstanceForType() { @@ -2633,7 +2633,7 @@ public final class SnapshotProtos { return this; } - // repeated .SnapshotRegionManifest.StoreFile store_files = 2; + // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; private java.util.List storeFiles_ = java.util.Collections.emptyList(); private void ensureStoreFilesIsMutable() { @@ -2647,7 +2647,7 @@ public final class SnapshotProtos { org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> storeFilesBuilder_; /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public java.util.List getStoreFilesList() { if (storeFilesBuilder_ == null) { @@ -2657,7 +2657,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public int getStoreFilesCount() { if (storeFilesBuilder_ == null) { @@ -2667,7 +2667,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index) { if (storeFilesBuilder_ == null) { @@ -2677,7 +2677,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder setStoreFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) { @@ -2694,7 +2694,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder setStoreFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) { @@ -2708,7 +2708,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder addStoreFiles(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) { if (storeFilesBuilder_ == null) { @@ -2724,7 +2724,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder addStoreFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) { @@ -2741,7 +2741,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder addStoreFiles( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) { @@ -2755,7 +2755,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder addStoreFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) { @@ -2769,7 +2769,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder addAllStoreFiles( java.lang.Iterable values) { @@ -2783,7 +2783,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder clearStoreFiles() { if (storeFilesBuilder_ == null) { @@ -2796,7 +2796,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public Builder removeStoreFiles(int index) { if (storeFilesBuilder_ == null) { @@ -2809,14 +2809,14 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder getStoreFilesBuilder( int index) { return getStoreFilesFieldBuilder().getBuilder(index); } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder( int index) { @@ -2826,7 +2826,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public java.util.List getStoreFilesOrBuilderList() { @@ -2837,14 +2837,14 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder addStoreFilesBuilder() { return getStoreFilesFieldBuilder().addBuilder( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.getDefaultInstance()); } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder addStoreFilesBuilder( int index) { @@ -2852,7 +2852,7 @@ public final class SnapshotProtos { index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.getDefaultInstance()); } /** - * repeated .SnapshotRegionManifest.StoreFile store_files = 2; + * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2; */ public java.util.List getStoreFilesBuilderList() { @@ -2873,7 +2873,7 @@ public final class SnapshotProtos { return storeFilesBuilder_; } - // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest.FamilyFiles) + // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest.FamilyFiles) } static { @@ -2881,7 +2881,7 @@ public final class SnapshotProtos { defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:SnapshotRegionManifest.FamilyFiles) + // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest.FamilyFiles) } private int bitField0_; @@ -2901,58 +2901,58 @@ public final class SnapshotProtos { return version_; } - // required .RegionInfo region_info = 2; + // required .hbase.pb.RegionInfo region_info = 2; public static final int REGION_INFO_FIELD_NUMBER = 2; private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_; /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public boolean hasRegionInfo() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() { return regionInfo_; } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() { return regionInfo_; } - // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; public static final int FAMILY_FILES_FIELD_NUMBER = 3; private java.util.List familyFiles_; /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public java.util.List getFamilyFilesList() { return familyFiles_; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public java.util.List getFamilyFilesOrBuilderList() { return familyFiles_; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public int getFamilyFilesCount() { return familyFiles_.size(); } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index) { return familyFiles_.get(index); } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder( int index) { @@ -3152,19 +3152,19 @@ public final class SnapshotProtos { return builder; } /** - * Protobuf type {@code SnapshotRegionManifest} + * Protobuf type {@code hbase.pb.SnapshotRegionManifest} */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder.class); } @@ -3214,7 +3214,7 @@ public final class SnapshotProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor; } public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getDefaultInstanceForType() { @@ -3376,18 +3376,18 @@ public final class SnapshotProtos { return this; } - // required .RegionInfo region_info = 2; + // required .hbase.pb.RegionInfo region_info = 2; private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_; /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public boolean hasRegionInfo() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() { if (regionInfoBuilder_ == null) { @@ -3397,7 +3397,7 @@ public final class SnapshotProtos { } } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public Builder setRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) { if (regionInfoBuilder_ == null) { @@ -3413,7 +3413,7 @@ public final class SnapshotProtos { return this; } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public Builder setRegionInfo( org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) { @@ -3427,7 +3427,7 @@ public final class SnapshotProtos { return this; } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public Builder mergeRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) { if (regionInfoBuilder_ == null) { @@ -3446,7 +3446,7 @@ public final class SnapshotProtos { return this; } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public Builder clearRegionInfo() { if (regionInfoBuilder_ == null) { @@ -3459,7 +3459,7 @@ public final class SnapshotProtos { return this; } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() { bitField0_ |= 0x00000002; @@ -3467,7 +3467,7 @@ public final class SnapshotProtos { return getRegionInfoFieldBuilder().getBuilder(); } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() { if (regionInfoBuilder_ != null) { @@ -3477,7 +3477,7 @@ public final class SnapshotProtos { } } /** - * required .RegionInfo region_info = 2; + * required .hbase.pb.RegionInfo region_info = 2; */ private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> @@ -3493,7 +3493,7 @@ public final class SnapshotProtos { return regionInfoBuilder_; } - // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; private java.util.List familyFiles_ = java.util.Collections.emptyList(); private void ensureFamilyFilesIsMutable() { @@ -3507,7 +3507,7 @@ public final class SnapshotProtos { org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> familyFilesBuilder_; /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public java.util.List getFamilyFilesList() { if (familyFilesBuilder_ == null) { @@ -3517,7 +3517,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public int getFamilyFilesCount() { if (familyFilesBuilder_ == null) { @@ -3527,7 +3527,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index) { if (familyFilesBuilder_ == null) { @@ -3537,7 +3537,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder setFamilyFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) { @@ -3554,7 +3554,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder setFamilyFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) { @@ -3568,7 +3568,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder addFamilyFiles(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) { if (familyFilesBuilder_ == null) { @@ -3584,7 +3584,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder addFamilyFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) { @@ -3601,7 +3601,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder addFamilyFiles( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) { @@ -3615,7 +3615,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder addFamilyFiles( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) { @@ -3629,7 +3629,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder addAllFamilyFiles( java.lang.Iterable values) { @@ -3643,7 +3643,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder clearFamilyFiles() { if (familyFilesBuilder_ == null) { @@ -3656,7 +3656,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public Builder removeFamilyFiles(int index) { if (familyFilesBuilder_ == null) { @@ -3669,14 +3669,14 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder getFamilyFilesBuilder( int index) { return getFamilyFilesFieldBuilder().getBuilder(index); } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder( int index) { @@ -3686,7 +3686,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public java.util.List getFamilyFilesOrBuilderList() { @@ -3697,14 +3697,14 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder addFamilyFilesBuilder() { return getFamilyFilesFieldBuilder().addBuilder( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.getDefaultInstance()); } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder addFamilyFilesBuilder( int index) { @@ -3712,7 +3712,7 @@ public final class SnapshotProtos { index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.getDefaultInstance()); } /** - * repeated .SnapshotRegionManifest.FamilyFiles family_files = 3; + * repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3; */ public java.util.List getFamilyFilesBuilderList() { @@ -3733,7 +3733,7 @@ public final class SnapshotProtos { return familyFilesBuilder_; } - // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest) + // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest) } static { @@ -3741,53 +3741,53 @@ public final class SnapshotProtos { defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:SnapshotRegionManifest) + // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest) } public interface SnapshotDataManifestOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required .TableSchema table_schema = 1; + // required .hbase.pb.TableSchema table_schema = 1; /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ boolean hasTableSchema(); /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema(); /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder(); - // repeated .SnapshotRegionManifest region_manifests = 2; + // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ java.util.List getRegionManifestsList(); /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index); /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ int getRegionManifestsCount(); /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ java.util.List getRegionManifestsOrBuilderList(); /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder( int index); } /** - * Protobuf type {@code SnapshotDataManifest} + * Protobuf type {@code hbase.pb.SnapshotDataManifest} */ public static final class SnapshotDataManifest extends com.google.protobuf.GeneratedMessage @@ -3875,12 +3875,12 @@ public final class SnapshotProtos { } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.Builder.class); } @@ -3901,58 +3901,58 @@ public final class SnapshotProtos { } private int bitField0_; - // required .TableSchema table_schema = 1; + // required .hbase.pb.TableSchema table_schema = 1; public static final int TABLE_SCHEMA_FIELD_NUMBER = 1; private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_; /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public boolean hasTableSchema() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() { return tableSchema_; } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() { return tableSchema_; } - // repeated .SnapshotRegionManifest region_manifests = 2; + // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; public static final int REGION_MANIFESTS_FIELD_NUMBER = 2; private java.util.List regionManifests_; /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public java.util.List getRegionManifestsList() { return regionManifests_; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public java.util.List getRegionManifestsOrBuilderList() { return regionManifests_; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public int getRegionManifestsCount() { return regionManifests_.size(); } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index) { return regionManifests_.get(index); } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder( int index) { @@ -4135,19 +4135,19 @@ public final class SnapshotProtos { return builder; } /** - * Protobuf type {@code SnapshotDataManifest} + * Protobuf type {@code hbase.pb.SnapshotDataManifest} */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifestOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable .ensureFieldAccessorsInitialized( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.Builder.class); } @@ -4195,7 +4195,7 @@ public final class SnapshotProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor; } public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest getDefaultInstanceForType() { @@ -4317,18 +4317,18 @@ public final class SnapshotProtos { } private int bitField0_; - // required .TableSchema table_schema = 1; + // required .hbase.pb.TableSchema table_schema = 1; private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_; /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public boolean hasTableSchema() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() { if (tableSchemaBuilder_ == null) { @@ -4338,7 +4338,7 @@ public final class SnapshotProtos { } } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public Builder setTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) { if (tableSchemaBuilder_ == null) { @@ -4354,7 +4354,7 @@ public final class SnapshotProtos { return this; } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public Builder setTableSchema( org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) { @@ -4368,7 +4368,7 @@ public final class SnapshotProtos { return this; } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public Builder mergeTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) { if (tableSchemaBuilder_ == null) { @@ -4387,7 +4387,7 @@ public final class SnapshotProtos { return this; } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public Builder clearTableSchema() { if (tableSchemaBuilder_ == null) { @@ -4400,7 +4400,7 @@ public final class SnapshotProtos { return this; } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getTableSchemaBuilder() { bitField0_ |= 0x00000001; @@ -4408,7 +4408,7 @@ public final class SnapshotProtos { return getTableSchemaFieldBuilder().getBuilder(); } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() { if (tableSchemaBuilder_ != null) { @@ -4418,7 +4418,7 @@ public final class SnapshotProtos { } } /** - * required .TableSchema table_schema = 1; + * required .hbase.pb.TableSchema table_schema = 1; */ private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> @@ -4434,7 +4434,7 @@ public final class SnapshotProtos { return tableSchemaBuilder_; } - // repeated .SnapshotRegionManifest region_manifests = 2; + // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; private java.util.List regionManifests_ = java.util.Collections.emptyList(); private void ensureRegionManifestsIsMutable() { @@ -4448,7 +4448,7 @@ public final class SnapshotProtos { org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> regionManifestsBuilder_; /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public java.util.List getRegionManifestsList() { if (regionManifestsBuilder_ == null) { @@ -4458,7 +4458,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public int getRegionManifestsCount() { if (regionManifestsBuilder_ == null) { @@ -4468,7 +4468,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index) { if (regionManifestsBuilder_ == null) { @@ -4478,7 +4478,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder setRegionManifests( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) { @@ -4495,7 +4495,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder setRegionManifests( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) { @@ -4509,7 +4509,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder addRegionManifests(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) { if (regionManifestsBuilder_ == null) { @@ -4525,7 +4525,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder addRegionManifests( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) { @@ -4542,7 +4542,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder addRegionManifests( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) { @@ -4556,7 +4556,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder addRegionManifests( int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) { @@ -4570,7 +4570,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder addAllRegionManifests( java.lang.Iterable values) { @@ -4584,7 +4584,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder clearRegionManifests() { if (regionManifestsBuilder_ == null) { @@ -4597,7 +4597,7 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public Builder removeRegionManifests(int index) { if (regionManifestsBuilder_ == null) { @@ -4610,14 +4610,14 @@ public final class SnapshotProtos { return this; } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder getRegionManifestsBuilder( int index) { return getRegionManifestsFieldBuilder().getBuilder(index); } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder( int index) { @@ -4627,7 +4627,7 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public java.util.List getRegionManifestsOrBuilderList() { @@ -4638,14 +4638,14 @@ public final class SnapshotProtos { } } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder addRegionManifestsBuilder() { return getRegionManifestsFieldBuilder().addBuilder( org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.getDefaultInstance()); } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder addRegionManifestsBuilder( int index) { @@ -4653,7 +4653,7 @@ public final class SnapshotProtos { index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.getDefaultInstance()); } /** - * repeated .SnapshotRegionManifest region_manifests = 2; + * repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2; */ public java.util.List getRegionManifestsBuilderList() { @@ -4674,7 +4674,7 @@ public final class SnapshotProtos { return regionManifestsBuilder_; } - // @@protoc_insertion_point(builder_scope:SnapshotDataManifest) + // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotDataManifest) } static { @@ -4682,34 +4682,34 @@ public final class SnapshotProtos { defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:SnapshotDataManifest) + // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotDataManifest) } private static com.google.protobuf.Descriptors.Descriptor - internal_static_SnapshotFileInfo_descriptor; + internal_static_hbase_pb_SnapshotFileInfo_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_SnapshotFileInfo_fieldAccessorTable; + internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_SnapshotRegionManifest_descriptor; + internal_static_hbase_pb_SnapshotRegionManifest_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_SnapshotRegionManifest_fieldAccessorTable; + internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_SnapshotRegionManifest_StoreFile_descriptor; + internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable; + internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_SnapshotRegionManifest_FamilyFiles_descriptor; + internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable; + internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_SnapshotDataManifest_descriptor; + internal_static_hbase_pb_SnapshotDataManifest_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_SnapshotDataManifest_fieldAccessorTable; + internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -4719,58 +4719,60 @@ public final class SnapshotProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\016Snapshot.proto\032\010FS.proto\032\013HBase.proto\"" + - "\211\001\n\020SnapshotFileInfo\022$\n\004type\030\001 \002(\0162\026.Sna" + - "pshotFileInfo.Type\022\r\n\005hfile\030\003 \001(\t\022\022\n\nwal" + - "_server\030\004 \001(\t\022\020\n\010wal_name\030\005 \001(\t\"\032\n\004Type\022" + - "\t\n\005HFILE\020\001\022\007\n\003WAL\020\002\"\257\002\n\026SnapshotRegionMa" + - "nifest\022\017\n\007version\030\001 \001(\005\022 \n\013region_info\030\002" + - " \002(\0132\013.RegionInfo\0229\n\014family_files\030\003 \003(\0132" + - "#.SnapshotRegionManifest.FamilyFiles\032K\n\t" + - "StoreFile\022\014\n\004name\030\001 \002(\t\022\035\n\treference\030\002 \001" + - "(\0132\n.Reference\022\021\n\tfile_size\030\003 \001(\004\032Z\n\013Fam", - "ilyFiles\022\023\n\013family_name\030\001 \002(\014\0226\n\013store_f" + - "iles\030\002 \003(\0132!.SnapshotRegionManifest.Stor" + - "eFile\"m\n\024SnapshotDataManifest\022\"\n\014table_s" + - "chema\030\001 \002(\0132\014.TableSchema\0221\n\020region_mani" + - "fests\030\002 \003(\0132\027.SnapshotRegionManifestBD\n*" + - "org.apache.hadoop.hbase.protobuf.generat" + - "edB\016SnapshotProtosH\001\210\001\001\240\001\001" + "\n\016Snapshot.proto\022\010hbase.pb\032\010FS.proto\032\013HB" + + "ase.proto\"\222\001\n\020SnapshotFileInfo\022-\n\004type\030\001" + + " \002(\0162\037.hbase.pb.SnapshotFileInfo.Type\022\r\n" + + "\005hfile\030\003 \001(\t\022\022\n\nwal_server\030\004 \001(\t\022\020\n\010wal_" + + "name\030\005 \001(\t\"\032\n\004Type\022\t\n\005HFILE\020\001\022\007\n\003WAL\020\002\"\323" + + "\002\n\026SnapshotRegionManifest\022\017\n\007version\030\001 \001" + + "(\005\022)\n\013region_info\030\002 \002(\0132\024.hbase.pb.Regio" + + "nInfo\022B\n\014family_files\030\003 \003(\0132,.hbase.pb.S" + + "napshotRegionManifest.FamilyFiles\032T\n\tSto" + + "reFile\022\014\n\004name\030\001 \002(\t\022&\n\treference\030\002 \001(\0132", + "\023.hbase.pb.Reference\022\021\n\tfile_size\030\003 \001(\004\032" + + "c\n\013FamilyFiles\022\023\n\013family_name\030\001 \002(\014\022?\n\013s" + + "tore_files\030\002 \003(\0132*.hbase.pb.SnapshotRegi" + + "onManifest.StoreFile\"\177\n\024SnapshotDataMani" + + "fest\022+\n\014table_schema\030\001 \002(\0132\025.hbase.pb.Ta" + + "bleSchema\022:\n\020region_manifests\030\002 \003(\0132 .hb" + + "ase.pb.SnapshotRegionManifestBD\n*org.apa" + + "che.hadoop.hbase.protobuf.generatedB\016Sna" + + "pshotProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { public com.google.protobuf.ExtensionRegistry assignDescriptors( com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; - internal_static_SnapshotFileInfo_descriptor = + internal_static_hbase_pb_SnapshotFileInfo_descriptor = getDescriptor().getMessageTypes().get(0); - internal_static_SnapshotFileInfo_fieldAccessorTable = new + internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_SnapshotFileInfo_descriptor, + internal_static_hbase_pb_SnapshotFileInfo_descriptor, new java.lang.String[] { "Type", "Hfile", "WalServer", "WalName", }); - internal_static_SnapshotRegionManifest_descriptor = + internal_static_hbase_pb_SnapshotRegionManifest_descriptor = getDescriptor().getMessageTypes().get(1); - internal_static_SnapshotRegionManifest_fieldAccessorTable = new + internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_SnapshotRegionManifest_descriptor, + internal_static_hbase_pb_SnapshotRegionManifest_descriptor, new java.lang.String[] { "Version", "RegionInfo", "FamilyFiles", }); - internal_static_SnapshotRegionManifest_StoreFile_descriptor = - internal_static_SnapshotRegionManifest_descriptor.getNestedTypes().get(0); - internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable = new + internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor = + internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(0); + internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_SnapshotRegionManifest_StoreFile_descriptor, + internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor, new java.lang.String[] { "Name", "Reference", "FileSize", }); - internal_static_SnapshotRegionManifest_FamilyFiles_descriptor = - internal_static_SnapshotRegionManifest_descriptor.getNestedTypes().get(1); - internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable = new + internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor = + internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(1); + internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_SnapshotRegionManifest_FamilyFiles_descriptor, + internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor, new java.lang.String[] { "FamilyName", "StoreFiles", }); - internal_static_SnapshotDataManifest_descriptor = + internal_static_hbase_pb_SnapshotDataManifest_descriptor = getDescriptor().getMessageTypes().get(2); - internal_static_SnapshotDataManifest_fieldAccessorTable = new + internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_SnapshotDataManifest_descriptor, + internal_static_hbase_pb_SnapshotDataManifest_descriptor, new java.lang.String[] { "TableSchema", "RegionManifests", }); return null; } diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index 101854d..6c392a1 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -274,6 +274,7 @@ message ScanRequest { optional bool client_handles_partials = 7; optional bool client_handles_heartbeats = 8; optional bool track_scan_metrics = 9; + optional uint64 timeout = 10; } /** @@ -323,6 +324,12 @@ message ScanResponse { // 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; + + // This field is filled with the next cell (the first cell of the next row) of the + // scanner when server send heartbeat message after timeout. The cell can be used + // as start row of a new scan. This cell is not necessarily belongs to the valid + // scan results, i.e. it can be a delete marker or won't pass the scan filter. + optional Cell next = 11; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java index 1ec085f..b23387e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java @@ -49,6 +49,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner { public ClientSideRegionScanner(Configuration conf, FileSystem fs, Path rootDir, HTableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics) throws IOException { + super(scan); // region is immutable, set isolation level scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED); @@ -60,9 +61,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner { this.scanner = region.getScanner(scan); values = new ArrayList(); - if (scanMetrics == null) { - initScanMetrics(scan); - } else { + if (scanMetrics != null) { this.scanMetrics = scanMetrics; } region.startRegionOperation(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java index f817e70..320b236 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java @@ -114,6 +114,7 @@ public class TableSnapshotScanner extends AbstractClientScanner { */ public TableSnapshotScanner(Configuration conf, Path rootDir, Path restoreDir, String snapshotName, Scan scan) throws IOException { + super(scan); this.conf = conf; this.snapshotName = snapshotName; this.rootDir = rootDir; @@ -151,8 +152,6 @@ public class TableSnapshotScanner extends AbstractClientScanner { // sort for regions according to startKey. Collections.sort(regions); - initScanMetrics(scan); - RestoreSnapshotHelper.copySnapshotForScanner(conf, fs, rootDir, restoreDir, snapshotName); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 2c145b4..99361df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -5787,6 +5787,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (!stopRow) continue; } + if (!stopRow && !scannerContext.partialResultFormed() && + scannerContext.hasTimeLimit(LimitScope.BETWEEN_ROWS)) { + // Peek the next cell and send back in the heartbeat message + Cell nextKv = this.storeHeap.peek(); + if (nextKv != null && !isStopRow(nextKv)) { + scannerContext.setNextCell(nextKv); + } else { + stopRow = true; + } + } if (stopRow) { return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } else { 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 3c0f50a..aa6118e 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 @@ -2467,6 +2467,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler, boolean stale = (region.getRegionInfo().getReplicaId() != 0); boolean clientHandlesPartials = request.hasClientHandlesPartials() && request.getClientHandlesPartials(); + // Heartbeat messages occur when the processing of the ScanRequest exceeds a + // certain time threshold on the server. When the time threshold is exceeded, the + // server stops the scan and sends back whatever Results it has accumulated within + // that time period (may be empty). If client handle partials, the result can break + // in the middle of a row, else break after finish scanning the current row. boolean clientHandlesHeartbeats = request.hasClientHandlesHeartbeats() && request.getClientHandlesHeartbeats(); @@ -2480,15 +2485,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan; boolean moreRows = false; - // Heartbeat messages occur when the processing of the ScanRequest is exceeds a - // certain time threshold on the server. When the time threshold is exceeded, the - // server stops the scan and sends back whatever Results it has accumulated within - // that time period (may be empty). Since heartbeat messages have the potential to - // create partial Results (in the event that the timeout occurs in the middle of a - // row), we must only generate heartbeat messages when the client can handle both - // heartbeats AND partials - boolean allowHeartbeatMessages = clientHandlesHeartbeats && allowPartialResults; - // Default value of timeLimit is negative to indicate no timeLimit should be // enforced. long timeLimit = -1; @@ -2496,25 +2492,30 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // Set the time limit to be half of the more restrictive timeout value (one of the // timeout values must be positive). In the event that both values are positive, the // more restrictive of the two is used to calculate the limit. - if (allowHeartbeatMessages && (scannerLeaseTimeoutPeriod > 0 || rpcTimeout > 0)) { - long timeLimitDelta; - if (scannerLeaseTimeoutPeriod > 0 && rpcTimeout > 0) { - timeLimitDelta = Math.min(scannerLeaseTimeoutPeriod, rpcTimeout); - } else { - timeLimitDelta = - scannerLeaseTimeoutPeriod > 0 ? scannerLeaseTimeoutPeriod : rpcTimeout; + if (clientHandlesHeartbeats && + (request.hasTimeout() && request.getTimeout() > 0 || + scannerLeaseTimeoutPeriod > 0 || rpcTimeout > 0)) { + long timeLimitDelta = Long.MAX_VALUE; + if (request.hasTimeout() && request.getTimeout() > 0) { + timeLimitDelta = request.getTimeout(); + } + if (scannerLeaseTimeoutPeriod > 0 && scannerLeaseTimeoutPeriod < timeLimitDelta) { + timeLimitDelta = scannerLeaseTimeoutPeriod; + } + if (rpcTimeout > 0 && rpcTimeout < timeLimitDelta) { + timeLimitDelta = rpcTimeout; } // Use half of whichever timeout value was more restrictive... But don't allow // the time limit to be less than the allowable minimum (could cause an - // immediatate timeout before scanning any data). + // immediate timeout before scanning any data). timeLimitDelta = Math.max(timeLimitDelta / 2, minimumScanTimeLimitDelta); timeLimit = System.currentTimeMillis() + timeLimitDelta; } final LimitScope sizeScope = allowPartialResults ? LimitScope.BETWEEN_CELLS : LimitScope.BETWEEN_ROWS; - final LimitScope timeScope = - allowHeartbeatMessages ? LimitScope.BETWEEN_CELLS : LimitScope.BETWEEN_ROWS; + final LimitScope timeScope = clientHandlesHeartbeats && clientHandlesPartials ? + LimitScope.BETWEEN_CELLS : LimitScope.BETWEEN_ROWS; boolean trackMetrics = request.hasTrackScanMetrics() && request.getTrackScanMetrics(); @@ -2565,6 +2566,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler, if (moreRows) { // Heartbeat messages occur when the time limit has been reached. builder.setHeartbeatMessage(timeLimitReached); + if (timeScope == LimitScope.BETWEEN_ROWS) { + Cell nextCell = scannerContext.getNextCell(); + if (nextCell != null) { + builder.setNext(ProtobufUtil.toCell(nextCell)); + } + } } break; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java index a927789..c2ea78f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java @@ -21,6 +21,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -67,6 +68,12 @@ public class ScannerContext { private static final NextState DEFAULT_STATE = NextState.MORE_VALUES; /** + * The next position of the scanner. This will be set only when the scanner timeout and client + * expect this field (when the client does not accept partial results). + */ + Cell nextCell; + + /** * Used as an indication to invocations of {@link InternalScanner#next(java.util.List)} and * {@link RegionScanner#next(java.util.List)} that, if true, the progress tracked within this * {@link ScannerContext} instance should be considered while evaluating the limits. Useful for @@ -220,6 +227,14 @@ public class ScannerContext { return state; } + void setNextCell(Cell nextCell) { + this.nextCell = nextCell; + } + + Cell getNextCell() { + return this.nextCell; + } + /** * @return true when a partial result is formed. A partial result is formed when a limit is * reached in the middle of a row. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java index 14c71d2..91b4166 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java @@ -240,11 +240,17 @@ public class TestScannerHeartbeatMessages { public Void call() throws Exception { // Configure the scan so that it can read the entire table in a single RPC. We want to test // the case where a scan stops on the server side due to a time limit - Scan scan = new Scan(); - scan.setMaxResultSize(Long.MAX_VALUE); - scan.setCaching(Integer.MAX_VALUE); + Scan baseScan = new Scan(); + baseScan.setMaxResultSize(Long.MAX_VALUE); + baseScan.setCaching(Integer.MAX_VALUE); - testEquivalenceOfScanWithHeartbeats(scan, DEFAULT_ROW_SLEEP_TIME, -1, false); + // Copy the scan before each test. When a scan object is used by a scanner, some of its + // fields may be changed such as start row + Scan scanCopy = new Scan(baseScan); + testEquivalenceOfScanWithHeartbeats(scanCopy, DEFAULT_ROW_SLEEP_TIME, -1, false); + scanCopy = new Scan(baseScan); + scanCopy.setAllowHeartbeatResults(true); + testEquivalenceOfScanWithHeartbeats(scanCopy, DEFAULT_ROW_SLEEP_TIME, -1, false); return null; } }; @@ -270,6 +276,9 @@ public class TestScannerHeartbeatMessages { testEquivalenceOfScanWithHeartbeats(scanCopy, -1, DEFAULT_CF_SLEEP_TIME, false); scanCopy = new Scan(baseScan); testEquivalenceOfScanWithHeartbeats(scanCopy, -1, DEFAULT_CF_SLEEP_TIME, true); + scanCopy = new Scan(baseScan); + scanCopy.setAllowHeartbeatResults(true); + testEquivalenceOfScanWithHeartbeats(scanCopy, -1, DEFAULT_CF_SLEEP_TIME, true); return null; } }; @@ -293,11 +302,17 @@ public class TestScannerHeartbeatMessages { Result r1 = null; Result r2 = null; + int heartbeats = 0; while ((r1 = scanner.next()) != null) { // Enforce the specified sleep conditions during calls to the heartbeat scanner configureSleepTime(rowSleepTime, cfSleepTime, sleepBeforeCf); - r2 = scannerWithHeartbeats.next(); + do { + r2 = scannerWithHeartbeats.next(); + if (r2 != null && r2.isHeartbeat()) { + ++heartbeats; + } + } while (r2 != null && r2.isHeartbeat()); disableSleeping(); assertTrue(r2 != null); @@ -309,6 +324,8 @@ public class TestScannerHeartbeatMessages { } assertTrue(scannerWithHeartbeats.next() == null); + assertTrue("heartbeats = " + heartbeats, + scan.getAllowHeartbeatResults() ? heartbeats > 0 : heartbeats == 0); scanner.close(); scannerWithHeartbeats.close(); }