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 fb2bc4bcd1c..40b50029771 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.client; +import static org.apache.hadoop.hbase.client.ConnectionUtils.*; import com.google.common.annotations.VisibleForTesting; import java.io.IOException; @@ -27,6 +28,7 @@ import java.util.LinkedList; import java.util.List; import java.util.concurrent.ExecutorService; +import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -42,6 +44,7 @@ import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults; import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; import org.apache.hadoop.hbase.exceptions.ScannerResetException; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; @@ -57,9 +60,7 @@ import org.apache.hadoop.hbase.util.Bytes; @InterfaceAudience.Private public class ClientScanner extends AbstractClientScanner { private static final Log LOG = LogFactory.getLog(ClientScanner.class); - // A byte array in which all elements are the max byte, and it is used to - // construct closest front row - static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9); + protected Scan scan; protected boolean closed = false; // Current region scanner is against. Gets cleared if current region goes @@ -156,12 +157,6 @@ public class ClientScanner extends AbstractClientScanner { this.rpcControllerFactory = controllerFactory; this.conf = conf; - initializeScannerInConstruction(); - } - - protected void initializeScannerInConstruction() throws IOException { - // initialize the scanner - nextScanner(this.caching, false); } protected ClusterConnection getConnection() { @@ -242,31 +237,30 @@ public class ClientScanner extends AbstractClientScanner { this.callable = null; } } - /* + + /** * Gets a scanner for the next region. If this.currentRegion != null, then we will move to the - * endrow of this.currentRegion. Else we will get scanner at the scan.getStartRow(). We will go no - * further, just tidy up outstanding scanners, if currentRegion != null and - * done is true. - * @param nbRows - * @param done Server-side says we're done scanning. + * endrow of this.currentRegion. Else we will get scanner at the scan.getStartRow(). + * @param nbRows the caching option of the scan + * @return the results fetched when open scanner, or null which means terminate the scan. */ - protected boolean nextScanner(int nbRows, final boolean done) throws IOException { + protected Result[] nextScanner(int nbRows) throws IOException { // Close the previous scanner if it's open closeScanner(); // Where to start the next scanner byte[] localStartKey; - // if we're at end of table, close and return false to stop iterating + // if we're at end of table, close and return null to stop iterating if (this.currentRegion != null) { byte[] endKey = this.currentRegion.getEndKey(); - if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) - || checkScanStopRow(endKey) || done) { + if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) || + checkScanStopRow(endKey)) { close(); if (LOG.isTraceEnabled()) { LOG.trace("Finished " + this.currentRegion); } - return false; + return null; } localStartKey = endKey; // clear mvcc read point if we are going to switch regions @@ -287,16 +281,23 @@ public class ClientScanner extends AbstractClientScanner { callable = getScannerCallable(localStartKey, nbRows); // Open a scanner on the region server starting at the // beginning of the region - call(callable, caller, scannerTimeout); + Result[] rrs = call(callable, caller, scannerTimeout); this.currentRegion = callable.getHRegionInfo(); if (this.scanMetrics != null) { this.scanMetrics.countOfRegions.incrementAndGet(); } + if (rrs != null && rrs.length == 0 && callable.moreResultsForScan() == MoreResults.NO) { + // no results for the scan, return null to terminate the scan. + closed = true; + callable = null; + currentRegion = null; + return null; + } + return rrs; } catch (IOException e) { - close(); + closeScanner(); throw e; } - return true; } @VisibleForTesting @@ -304,9 +305,8 @@ public class ClientScanner extends AbstractClientScanner { return callable.isAnyRPCcancelled(); } - Result[] call(ScannerCallableWithReplicas callable, - RpcRetryingCaller caller, int scannerTimeout) - throws IOException, RuntimeException { + private Result[] call(ScannerCallableWithReplicas callable, RpcRetryingCaller caller, + int scannerTimeout) throws IOException { if (Thread.interrupted()) { throw new InterruptedIOException(); } @@ -369,22 +369,19 @@ public class ClientScanner extends AbstractClientScanner { return cache != null ? cache.size() : 0; } + private boolean scanExhausted(Result[] values) { + // This means the server tells us the whole scan operation is done. Usually decided by filter or + // limit. + return values == null || callable.moreResultsForScan() == MoreResults.NO; + } + private boolean regionExhausted(Result[] values) { - // This means the server tells us the whole scan operation is done. Usually decided by filter. - if (values == null) { - return true; - } - // Not a heartbeat message and we get nothing, this means the region is exhausted - if (values.length == 0 && !callable.isHeartbeatMessage()) { - return true; - } - // Server tells us that it has no more results for this region. Notice that this flag is get - // from the ScanResponse.getMoreResultsInRegion, not ScanResponse.getMoreResults. If the latter - // one is false then we will get a null values and quit in the first condition of this method. - if (callable.hasMoreResultsContext() && !callable.getServerHasMoreResults()) { - return true; - } - return false; + // 1. Not a heartbeat message and we get nothing, this means the region is exhausted. And in the + // old time we always return empty result for a open scanner operation so we add a check here to + // keep compatible with the old logic. Should remove the isOpenScanner in the future. + // 2. Server tells us that it has no more results for this region. + return (values.length == 0 && !callable.isHeartbeatMessage() && !callable.isOpenScanner()) + || callable.moreResultsInRegion() == MoreResults.NO; } private void closeScannerIfExhausted(boolean exhausted) throws IOException { @@ -400,16 +397,87 @@ public class ClientScanner extends AbstractClientScanner { } } + private Result[] nextScannerWithRetries(int nbRows) throws IOException { + for (;;) { + try { + return nextScanner(nbRows); + } catch (DoNotRetryIOException e) { + handleScanError(e, null); + } + } + } + + private void handleScanError(DoNotRetryIOException e, + MutableBoolean retryAfterOutOfOrderException) throws DoNotRetryIOException { + // An exception was thrown which makes any partial results that we were collecting + // invalid. The scanner will need to be reset to the beginning of a row. + clearPartialResults(); + // DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us + // to reset the scanner and come back in again. + + // If exception is any but the list below throw it back to the client; else setup + // the scanner and retry. + Throwable cause = e.getCause(); + if ((cause != null && cause instanceof NotServingRegionException) || + (cause != null && cause instanceof RegionServerStoppedException) || + e instanceof OutOfOrderScannerNextException || e instanceof UnknownScannerException || + e instanceof ScannerResetException) { + // Pass. It is easier writing the if loop test as list of what is allowed rather than + // as a list of what is not allowed... so if in here, it means we do not throw. + } else { + throw e; + } + + // Else, its signal from depths of ScannerCallable that we need to reset the scanner. + if (this.lastResult != null) { + // The region has moved. We need to open a brand new scanner at the new location. + // Reset the startRow to the row we've seen last so that the new scanner starts at + // the correct row. Otherwise we may see previously returned rows again. + // (ScannerCallable by now has "relocated" the correct region) + if (!this.lastResult.isPartial() && scan.getBatch() < 0) { + if (scan.isReversed()) { + scan.setStartRow(createClosestRowBefore(lastResult.getRow())); + } else { + scan.setStartRow(createClosestRowAfter(lastResult.getRow())); + } + } else { + // we need rescan this row because we only loaded partial row before + scan.setStartRow(lastResult.getRow()); + } + } + if (e instanceof OutOfOrderScannerNextException) { + if (retryAfterOutOfOrderException != null) { + if (retryAfterOutOfOrderException.isTrue()) { + retryAfterOutOfOrderException.setValue(false); + } else { + // TODO: Why wrap this in a DNRIOE when it already is a DNRIOE? + throw new DoNotRetryIOException( + "Failed after retry of OutOfOrderScannerNextException: was there a rpc timeout?", e); + } + } + } + // Clear region. + this.currentRegion = null; + // Set this to zero so we don't try and do an rpc and close on remote server when + // the exception we got was UnknownScanner or the Server is going down. + callable = null; + } + /** * Contact the servers to load more {@link Result}s in the cache. */ protected void loadCache() throws IOException { + // check if scanner was closed during previous prefetch + if (closed) { + return; + } Result[] values = null; long remainingResultSize = maxScannerResultSize; int countdown = this.caching; // This is possible if we just stopped at the boundary of a region in the previous call. if (callable == null) { - if (!nextScanner(countdown, false)) { + values = nextScannerWithRetries(countdown); + if (values == null) { return; } } @@ -417,80 +485,39 @@ public class ClientScanner extends AbstractClientScanner { callable.setCaching(this.caching); // This flag is set when we want to skip the result returned. We do // this when we reset scanner because it split under us. - boolean retryAfterOutOfOrderException = true; + MutableBoolean retryAfterOutOfOrderException = new MutableBoolean(true); for (;;) { try { // Server returns a null values if scanning is to stop. Else, // returns an empty array if scanning is to go on and we've just // exhausted current region. - values = call(callable, caller, scannerTimeout); + // now we will also fetch data when openScanner, so do not make a next call again if values + // is already non-null. + if (values == null) { + values = call(callable, caller, scannerTimeout); + } // When the replica switch happens, we need to do certain operations again. // The callable will openScanner with the right startkey but we need to pick up // from there. Bypass the rest of the loop and let the catch-up happen in the beginning // of the loop as it happens for the cases where we see exceptions. - // Since only openScanner would have happened, values would be null - if (values == null && callable.switchedToADifferentReplica()) { + if (callable.switchedToADifferentReplica()) { // Any accumulated partial results are no longer valid since the callable will // openScanner with the correct startkey and we must pick up from there clearPartialResults(); this.currentRegion = callable.getHRegionInfo(); - continue; - } - retryAfterOutOfOrderException = true; - } catch (DoNotRetryIOException | NeedUnmanagedConnectionException e) { - // An exception was thrown which makes any partial results that we were collecting - // invalid. The scanner will need to be reset to the beginning of a row. - clearPartialResults(); - // DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us - // to reset the scanner and come back in again. - - // If exception is any but the list below throw it back to the client; else setup - // the scanner and retry. - Throwable cause = e.getCause(); - if ((cause != null && cause instanceof NotServingRegionException) || - (cause != null && cause instanceof RegionServerStoppedException) || - e instanceof OutOfOrderScannerNextException || - e instanceof UnknownScannerException || - e instanceof ScannerResetException) { - // Pass. It is easier writing the if loop test as list of what is allowed rather than - // as a list of what is not allowed... so if in here, it means we do not throw. - } else { - throw e; - } - - // Else, its signal from depths of ScannerCallable that we need to reset the scanner. - if (this.lastResult != null) { - // The region has moved. We need to open a brand new scanner at the new location. - // Reset the startRow to the row we've seen last so that the new scanner starts at - // the correct row. Otherwise we may see previously returned rows again. - // (ScannerCallable by now has "relocated" the correct region) - if (!this.lastResult.isPartial() && scan.getBatch() < 0) { - if (scan.isReversed()) { - scan.setStartRow(createClosestRowBefore(lastResult.getRow())); - } else { - scan.setStartRow(Bytes.add(lastResult.getRow(), new byte[1])); - } - } else { - // we need rescan this row because we only loaded partial row before - scan.setStartRow(lastResult.getRow()); + // Now we will also fetch data when openScanner so usually we should not get a null + // result, but at some places we still use null to indicate the scan is terminated, so add + // a sanity check here. Should be removed later. + if (values == null) { + continue; } } - if (e instanceof OutOfOrderScannerNextException) { - if (retryAfterOutOfOrderException) { - retryAfterOutOfOrderException = false; - } else { - // TODO: Why wrap this in a DNRIOE when it already is a DNRIOE? - throw new DoNotRetryIOException("Failed after retry of " + - "OutOfOrderScannerNextException: was there a rpc timeout?", e); - } - } - // Clear region. - this.currentRegion = null; - // Set this to zero so we don't try and do an rpc and close on remote server when - // the exception we got was UnknownScanner or the Server is going down. - callable = null; + retryAfterOutOfOrderException.setValue(true); + } catch (DoNotRetryIOException e) { + handleScanError(e, retryAfterOutOfOrderException); // reopen the scanner - if (!nextScanner(countdown, false)) { + values = nextScannerWithRetries(countdown); + if (values == null) { break; } continue; @@ -523,8 +550,18 @@ public class ClientScanner extends AbstractClientScanner { this.lastCellLoadedToCache = null; } } + if (scan.getLimit() > 0) { + int limit = scan.getLimit() - resultsToAddToCache.size(); + assert limit >= 0; + scan.setLimit(limit); + } } - boolean exhausted = regionExhausted(values); + if (scanExhausted(values)) { + closeScanner(); + closed = true; + break; + } + boolean regionExhausted = regionExhausted(values); if (callable.isHeartbeatMessage()) { if (!cache.isEmpty()) { // Caller of this method just wants a Result. If we see a heartbeat message, it means @@ -542,12 +579,12 @@ public class ClientScanner extends AbstractClientScanner { } if (countdown <= 0) { // we have enough result. - closeScannerIfExhausted(exhausted); + closeScannerIfExhausted(regionExhausted); break; } if (remainingResultSize <= 0) { if (!cache.isEmpty()) { - closeScannerIfExhausted(exhausted); + closeScannerIfExhausted(regionExhausted); break; } else { // we have reached the max result size but we still can not find anything to return to the @@ -556,17 +593,21 @@ public class ClientScanner extends AbstractClientScanner { } } // we are done with the current region - if (exhausted) { + if (regionExhausted) { if (!partialResults.isEmpty()) { // XXX: continue if there are partial results. But in fact server should not set // hasMoreResults to false if there are partial results. LOG.warn("Server tells us there is no more results for this region but we still have" + " partialResults, this should not happen, retry on the current scanner anyway"); + values = null; // reset values for the next call continue; } - if (!nextScanner(countdown, values == null)) { + values = nextScannerWithRetries(countdown); + if (values == null) { break; } + } else { + values = null; // reset values for the next call } } } @@ -738,46 +779,24 @@ public class ClientScanner extends AbstractClientScanner { } } - @Override - public void close() { - if (!scanMetricsPublished) writeScanMetrics(); - if (callable != null) { - callable.setClose(); - try { - call(callable, caller, scannerTimeout); - } catch (UnknownScannerException e) { - // We used to catch this error, interpret, and rethrow. However, we - // have since decided that it's not nice for a scanner's close to - // throw exceptions. Chances are it was just due to lease time out. - } catch (IOException e) { - /* An exception other than UnknownScanner is unexpected. */ - LOG.warn("scanner failed to close. Exception follows: " + e); - } - callable = null; + @Override + public void close() { + if (!scanMetricsPublished) writeScanMetrics(); + if (callable != null) { + callable.setClose(); + try { + call(callable, caller, scannerTimeout); + } catch (UnknownScannerException e) { + // We used to catch this error, interpret, and rethrow. However, we + // have since decided that it's not nice for a scanner's close to + // throw exceptions. Chances are it was just due to lease time out. + } catch (IOException e) { + /* An exception other than UnknownScanner is unexpected. */ + LOG.warn("scanner failed to close. Exception follows: " + e); } - closed = true; - } - - /** - * Create the closest row before the specified row - * @param row - * @return a new byte array which is the closest front row of the specified one - */ - protected static byte[] createClosestRowBefore(byte[] row) { - if (row == null) { - throw new IllegalArgumentException("The passed row is empty"); - } - if (Bytes.equals(row, HConstants.EMPTY_BYTE_ARRAY)) { - return MAX_BYTE_ARRAY; - } - if (row[row.length - 1] == 0) { - return Arrays.copyOf(row, row.length - 1); - } else { - byte[] closestFrontRow = Arrays.copyOf(row, row.length); - closestFrontRow[row.length - 1] = (byte) ((closestFrontRow[row.length - 1] & 0xff) - 1); - closestFrontRow = Bytes.add(closestFrontRow, MAX_BYTE_ARRAY); - return closestFrontRow; + callable = null; } + closed = true; } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java deleted file mode 100644 index bd5575a8d34..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java +++ /dev/null @@ -1,345 +0,0 @@ -/** - * Copyright The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package org.apache.hadoop.hbase.client; - - -import com.google.protobuf.ServiceException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallable; -import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.util.Bytes; - -import com.google.common.annotations.VisibleForTesting; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.concurrent.ExecutorService; - -/** - *

- * Client scanner for small reversed scan. Generally, only one RPC is called to fetch the - * scan results, unless the results cross multiple regions or the row count of - * results exceed the caching. - *

- * For small scan, it will get better performance than {@link ReversedClientScanner} - */ -@InterfaceAudience.Private -public class ClientSmallReversedScanner extends ReversedClientScanner { - private static final Log LOG = LogFactory.getLog(ClientSmallReversedScanner.class); - private ScannerCallableWithReplicas smallReversedScanCallable = null; - private SmallReversedScannerCallableFactory callableFactory; - - /** - * Create a new ReversibleClientScanner for the specified table. Take note that the passed - * {@link Scan} 's start row maybe changed changed. - * - * @param conf - * The {@link Configuration} to use. - * @param scan - * {@link Scan} to use in this scanner - * @param tableName - * The table that we wish to rangeGet - * @param connection - * Connection identifying the cluster - * @param rpcFactory - * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads - * @param pool - * Threadpool for RPC threads - * @param primaryOperationTimeout - * Call timeout - * @throws IOException - * If the remote call fails - */ - public ClientSmallReversedScanner(final Configuration conf, final Scan scan, - final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) - throws IOException { - this(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, - primaryOperationTimeout, new SmallReversedScannerCallableFactory()); - } - - /** - * Create a new ReversibleClientScanner for the specified table. Take note that the passed - * {@link Scan}'s start row may be changed. - * - * @param conf - * The {@link Configuration} to use. - * @param scan - * {@link Scan} to use in this scanner - * @param tableName - * The table that we wish to rangeGet - * @param connection - * Connection identifying the cluster - * @param rpcFactory - * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads - * @param pool - * Threadpool for RPC threads - * @param primaryOperationTimeout - * Call timeout - * @param callableFactory - * Factory used to create the {@link SmallScannerCallable} - * @throws IOException - * If the remote call fails - */ - @VisibleForTesting - ClientSmallReversedScanner(final Configuration conf, final Scan scan, final TableName tableName, - ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout, - SmallReversedScannerCallableFactory callableFactory) throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, - primaryOperationTimeout); - this.callableFactory = callableFactory; - } - - /** - * Gets a scanner for following scan. Move to next region or continue from the last result or - * start from the start row. - * - * @param nbRows - * @param done - * true if Server-side says we're done scanning. - * @param currentRegionDone - * true if scan is over on current region - * @return true if has next scanner - * @throws IOException - */ - private boolean nextScanner(int nbRows, final boolean done, - boolean currentRegionDone) throws IOException { - // Where to start the next getter - byte[] localStartKey; - int cacheNum = nbRows; - boolean regionChanged = true; - boolean isFirstRegionToLocate = false; - // if we're at end of table, close and return false to stop iterating - if (this.currentRegion != null && currentRegionDone) { - byte[] startKey = this.currentRegion.getStartKey(); - if (startKey == null - || Bytes.equals(startKey, HConstants.EMPTY_BYTE_ARRAY) - || checkScanStopRow(startKey) || done) { - close(); - if (LOG.isDebugEnabled()) { - LOG.debug("Finished with small scan at " + this.currentRegion); - } - return false; - } - // We take the row just under to get to the previous region. - localStartKey = createClosestRowBefore(startKey); - if (LOG.isDebugEnabled()) { - LOG.debug("Finished with region " + this.currentRegion); - } - } else if (this.lastResult != null) { - regionChanged = false; - localStartKey = createClosestRowBefore(lastResult.getRow()); - } else { - localStartKey = this.scan.getStartRow(); - isFirstRegionToLocate = true; - } - - if (!isFirstRegionToLocate - && (localStartKey == null || Bytes.equals(localStartKey, HConstants.EMPTY_BYTE_ARRAY))) { - // when non-firstRegion & localStartKey is empty bytes, no more rowKey should scan. - // otherwise, maybe infinity results with RowKey=0x00 will return. - return false; - } - - if (LOG.isTraceEnabled()) { - LOG.trace("Advancing internal small scanner to startKey at '" - + Bytes.toStringBinary(localStartKey) + "'"); - } - - smallReversedScanCallable = - callableFactory.getCallable(getConnection(), getTable(), scan, getScanMetrics(), - localStartKey, cacheNum, rpcControllerFactory, getPool(), getPrimaryOperationTimeout(), - getRetries(), getScannerTimeout(), getConf(), caller, isFirstRegionToLocate); - - if (this.scanMetrics != null && regionChanged) { - this.scanMetrics.countOfRegions.incrementAndGet(); - } - return true; - } - - @Override - public Result next() throws IOException { - // If the scanner is closed and there's nothing left in the cache, next is a - // no-op. - if (cache.size() == 0 && this.closed) { - return null; - } - if (cache.size() == 0) { - loadCache(); - } - - if (cache.size() > 0) { - return cache.poll(); - } - // if we exhausted this scanner before calling close, write out the scan - // metrics - writeScanMetrics(); - return null; - } - - @Override - protected void loadCache() throws IOException { - Result[] values = null; - long remainingResultSize = maxScannerResultSize; - int countdown = this.caching; - boolean currentRegionDone = false; - // Values == null means server-side filter has determined we must STOP - while (remainingResultSize > 0 && countdown > 0 - && nextScanner(countdown, values == null, currentRegionDone)) { - // Server returns a null values if scanning is to stop. Else, - // returns an empty array if scanning is to go on and we've just - // exhausted current region. - // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, - // we do a callWithRetries - values = this.caller.callWithoutRetries(smallReversedScanCallable, scannerTimeout); - this.currentRegion = smallReversedScanCallable.getHRegionInfo(); - long currentTime = System.currentTimeMillis(); - if (this.scanMetrics != null) { - this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - - lastNext); - } - lastNext = currentTime; - if (values != null && values.length > 0) { - for (int i = 0; i < values.length; i++) { - Result rs = values[i]; - cache.add(rs); - // We don't make Iterator here - for (Cell cell : rs.rawCells()) { - remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell); - } - countdown--; - this.lastResult = rs; - } - } - if (smallReversedScanCallable.hasMoreResultsContext()) { - currentRegionDone = !smallReversedScanCallable.getServerHasMoreResults(); - } else { - currentRegionDone = countdown > 0; - } - } - } - - @Override - protected void initializeScannerInConstruction() throws IOException { - // No need to initialize the scanner when constructing instance, do it when - // calling next(). Do nothing here. - } - - @Override - public void close() { - if (!scanMetricsPublished) writeScanMetrics(); - closed = true; - } - - @VisibleForTesting - protected void setScannerCallableFactory(SmallReversedScannerCallableFactory callableFactory) { - this.callableFactory = callableFactory; - } - - /** - * A reversed ScannerCallable which supports backward small scanning. - */ - static class SmallReversedScannerCallable extends ReversedScannerCallable { - - public SmallReversedScannerCallable(ClusterConnection connection, TableName table, Scan scan, - ScanMetrics scanMetrics, byte[] locateStartRow, RpcControllerFactory controllerFactory, - int caching, int replicaId) { - super(connection, table, scan, scanMetrics, locateStartRow, controllerFactory, replicaId); - this.setCaching(caching); - } - - @Override - public Result[] call(int timeout) throws IOException { - if (this.closed) return null; - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - ClientProtos.ScanRequest request = RequestConverter.buildScanRequest( - getLocation().getRegionInfo().getRegionName(), getScan(), getCaching(), true); - ClientProtos.ScanResponse response = null; - controller = controllerFactory.newController(); - try { - controller.setPriority(getTableName()); - controller.setCallTimeout(timeout); - response = getStub().scan(controller, request); - Result[] results = ResponseConverter.getResults(controller.cellScanner(), response); - if (response.hasMoreResultsInRegion()) { - setHasMoreResultsContext(true); - setServerHasMoreResults(response.getMoreResultsInRegion()); - } else { - setHasMoreResultsContext(false); - } - // We need to update result metrics since we are overriding call() - updateResultsMetrics(results); - return results; - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - - @Override - public ScannerCallable getScannerCallableForReplica(int id) { - return new SmallReversedScannerCallable(getConnection(), getTableName(), getScan(), - scanMetrics, locateStartRow, controllerFactory, getCaching(), id); - } - } - - protected static class SmallReversedScannerCallableFactory { - - public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table, - Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout, - int retries, int scannerTimeout, Configuration conf, RpcRetryingCaller caller, - boolean isFirstRegionToLocate) { - byte[] locateStartRow = null; - if (isFirstRegionToLocate - && (localStartKey == null || Bytes.equals(localStartKey, HConstants.EMPTY_BYTE_ARRAY))) { - // HBASE-16886: if not setting startRow, then we will use a range [MAX_BYTE_ARRAY, +oo) to - // locate a region list, and the last one in region list is the region where our scan start. - locateStartRow = ClientScanner.MAX_BYTE_ARRAY; - } - - scan.setStartRow(localStartKey); - SmallReversedScannerCallable s = new SmallReversedScannerCallable(connection, table, scan, - scanMetrics, locateStartRow, controllerFactory, cacheNum, 0); - ScannerCallableWithReplicas scannerCallableWithReplicas = - new ScannerCallableWithReplicas(table, connection, s, pool, primaryOperationTimeout, scan, - retries, scannerTimeout, cacheNum, conf, caller); - return scannerCallableWithReplicas; - } - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java deleted file mode 100644 index b1554fdc094..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java +++ /dev/null @@ -1,317 +0,0 @@ -/** - * Copyright The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.client; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.concurrent.ExecutorService; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; -import org.apache.hadoop.hbase.util.Bytes; - -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ServiceException; - -/** - * Client scanner for small scan. Generally, only one RPC is called to fetch the - * scan results, unless the results cross multiple regions or the row count of - * results excess the caching. - * - * For small scan, it will get better performance than {@link ClientScanner} - */ -@InterfaceAudience.Private -public class ClientSmallScanner extends ClientScanner { - private static final Log LOG = LogFactory.getLog(ClientSmallScanner.class); - private ScannerCallableWithReplicas smallScanCallable = null; - private SmallScannerCallableFactory callableFactory; - - /** - * Create a new ShortClientScanner for the specified table. Take note that the passed {@link Scan} - * 's start row maybe changed changed. - * - * @param conf - * The {@link Configuration} to use. - * @param scan - * {@link Scan} to use in this scanner - * @param tableName - * The table that we wish to rangeGet - * @param connection - * Connection identifying the cluster - * @param rpcFactory - * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads - * @param pool - * Threadpool for RPC threads - * @param primaryOperationTimeout - * Call timeout - * @throws IOException - * If the remote call fails - */ - public ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName, - ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) - throws IOException { - this(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, - primaryOperationTimeout, new SmallScannerCallableFactory()); - } - - /** - * Create a new ShortClientScanner for the specified table. Take note that the passed {@link Scan} - * 's start row maybe changed changed. Intended for unit tests to provide their own - * {@link SmallScannerCallableFactory} implementation/mock. - * - * @param conf - * The {@link Configuration} to use. - * @param scan - * {@link Scan} to use in this scanner - * @param tableName - * The table that we wish to rangeGet - * @param connection - * Connection identifying the cluster - * @param rpcFactory - * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads - * @param pool - * Threadpool for RPC threads - * @param primaryOperationTimeout - * Call timeout - * @param callableFactory - * Factory used to create the {@link SmallScannerCallable} - * @throws IOException - */ - @VisibleForTesting - ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName, - ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout, - SmallScannerCallableFactory callableFactory) throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, - primaryOperationTimeout); - this.callableFactory = callableFactory; - } - - @Override - protected void initializeScannerInConstruction() throws IOException { - // No need to initialize the scanner when constructing instance, do it when - // calling next(). Do nothing here. - } - - /** - * Gets a scanner for following scan. Move to next region or continue from the - * last result or start from the start row. - * @param nbRows - * @param done true if Server-side says we're done scanning. - * @param currentRegionDone true if scan is over on current region - * @return true if has next scanner - * @throws IOException - */ - private boolean nextScanner(int nbRows, final boolean done, - boolean currentRegionDone) throws IOException { - // Where to start the next getter - byte[] localStartKey; - int cacheNum = nbRows; - boolean regionChanged = true; - // if we're at end of table, close and return false to stop iterating - if (this.currentRegion != null && currentRegionDone) { - byte[] endKey = this.currentRegion.getEndKey(); - if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) - || checkScanStopRow(endKey) || done) { - close(); - if (LOG.isTraceEnabled()) { - LOG.trace("Finished with small scan at " + this.currentRegion); - } - return false; - } - localStartKey = endKey; - if (LOG.isTraceEnabled()) { - LOG.trace("Finished with region " + this.currentRegion); - } - } else if (this.lastResult != null) { - regionChanged = false; - localStartKey = Bytes.add(lastResult.getRow(), new byte[1]); - } else { - localStartKey = this.scan.getStartRow(); - } - - if (LOG.isTraceEnabled()) { - LOG.trace("Advancing internal small scanner to startKey at '" - + Bytes.toStringBinary(localStartKey) + "'"); - } - smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan, - getScanMetrics(), localStartKey, cacheNum, rpcControllerFactory, getPool(), - getPrimaryOperationTimeout(), getRetries(), getScannerTimeout(), getConf(), caller); - if (this.scanMetrics != null && regionChanged) { - this.scanMetrics.countOfRegions.incrementAndGet(); - } - return true; - } - - static class SmallScannerCallable extends ScannerCallable { - public SmallScannerCallable( - ClusterConnection connection, TableName table, Scan scan, - ScanMetrics scanMetrics, RpcControllerFactory controllerFactory, int caching, int id) { - super(connection, table, scan, scanMetrics, controllerFactory, id); - this.setCaching(caching); - } - - @Override - public Result[] call(int timeout) throws IOException { - if (this.closed) return null; - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - ScanRequest request = RequestConverter.buildScanRequest(getLocation() - .getRegionInfo().getRegionName(), getScan(), getCaching(), true); - ScanResponse response = null; - controller = controllerFactory.newController(); - try { - controller.setPriority(getTableName()); - controller.setCallTimeout(timeout); - response = getStub().scan(controller, request); - Result[] results = ResponseConverter.getResults(controller.cellScanner(), - response); - if (response.hasMoreResultsInRegion()) { - setHasMoreResultsContext(true); - setServerHasMoreResults(response.getMoreResultsInRegion()); - } else { - setHasMoreResultsContext(false); - } - // We need to update result metrics since we are overriding call() - updateResultsMetrics(results); - return results; - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - - @Override - public ScannerCallable getScannerCallableForReplica(int id) { - return new SmallScannerCallable((ClusterConnection)connection, tableName, getScan(), - scanMetrics, controllerFactory, getCaching(), id); - } - } - - @Override - public Result next() throws IOException { - // If the scanner is closed and there's nothing left in the cache, next is a - // no-op. - if (cache.size() == 0 && this.closed) { - return null; - } - if (cache.size() == 0) { - loadCache(); - } - - if (cache.size() > 0) { - return cache.poll(); - } - // if we exhausted this scanner before calling close, write out the scan - // metrics - writeScanMetrics(); - return null; - } - - @Override - protected void loadCache() throws IOException { - Result[] values = null; - long remainingResultSize = maxScannerResultSize; - int countdown = this.caching; - boolean currentRegionDone = false; - // Values == null means server-side filter has determined we must STOP - while (remainingResultSize > 0 && countdown > 0 - && nextScanner(countdown, values == null, currentRegionDone)) { - // Server returns a null values if scanning is to stop. Else, - // returns an empty array if scanning is to go on and we've just - // exhausted current region. - // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, - // we do a callWithRetries - values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout); - this.currentRegion = smallScanCallable.getHRegionInfo(); - long currentTime = System.currentTimeMillis(); - if (this.scanMetrics != null) { - this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - - lastNext); - } - lastNext = currentTime; - if (values != null && values.length > 0) { - for (int i = 0; i < values.length; i++) { - Result rs = values[i]; - cache.add(rs); - // We don't make Iterator here - for (Cell cell : rs.rawCells()) { - remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell); - } - countdown--; - this.lastResult = rs; - } - } - if (smallScanCallable.hasMoreResultsContext()) { - // If the server has more results, the current region is not done - currentRegionDone = !smallScanCallable.getServerHasMoreResults(); - } else { - // not guaranteed to get the context in older versions, fall back to checking countdown - currentRegionDone = countdown > 0; - } - } - } - - public void close() { - if (!scanMetricsPublished) writeScanMetrics(); - closed = true; - } - - @VisibleForTesting - protected void setScannerCallableFactory(SmallScannerCallableFactory callableFactory) { - this.callableFactory = callableFactory; - } - - @InterfaceAudience.Private - protected static class SmallScannerCallableFactory { - - public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table, - Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout, - int retries, int scannerTimeout, Configuration conf, RpcRetryingCaller caller) { - scan.setStartRow(localStartKey); - SmallScannerCallable s = new SmallScannerCallable( - connection, table, scan, scanMetrics, controllerFactory, cacheNum, 0); - ScannerCallableWithReplicas scannerCallableWithReplicas = - new ScannerCallableWithReplicas(table, connection, - s, pool, primaryOperationTimeout, scan, retries, - scannerTimeout, cacheNum, conf, caller); - return scannerCallableWithReplicas; - } - - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java index 5f5badf586c..ab6cb8d518a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java @@ -1257,8 +1257,7 @@ class ConnectionManager { Scan s = new Scan(); s.setReversed(true); s.setStartRow(metaKey); - s.setSmall(true); - s.setCaching(1); + s.setOneRowLimit(); if (this.useMetaReplicas) { s.setConsistency(Consistency.TIMELINE); } @@ -1286,15 +1285,11 @@ class ConnectionManager { long pauseBase = this.pause; try { Result regionInfoRow = null; - ReversedClientScanner rcs = null; - try { - rcs = new ClientSmallReversedScanner(conf, s, TableName.META_TABLE_NAME, this, - rpcCallerFactory, rpcControllerFactory, getMetaLookupPool(), 0); + s.resetMvccReadPoint(); + try (ReversedClientScanner rcs = + new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory, + rpcControllerFactory, getMetaLookupPool(), 0)) { regionInfoRow = rcs.next(); - } finally { - if (rcs != null) { - rcs.close(); - } } if (regionInfoRow == null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java index c1e6d23d3e1..e7b41142e39 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java @@ -17,14 +17,16 @@ */ package org.apache.hadoop.hbase.client; +import com.google.common.annotations.VisibleForTesting; + import java.io.IOException; +import java.util.Arrays; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadLocalRandom; import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -32,8 +34,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; - -import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hbase.util.Bytes; /** * Utility used by client connections. @@ -195,4 +196,36 @@ public class ConnectionUtils { return false; } } + + // A byte array in which all elements are the max byte, and it is used to + // construct closest front row + static final byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9); + + /** + * Create the closest row after the specified row + */ + static byte[] createClosestRowAfter(byte[] row) { + return Arrays.copyOf(row, row.length + 1); + } + + /** + * Create the closest row before the specified row + * @deprecated in fact, we do not know the closest row before the given row, the result is only a + * row very close to the current row. Avoid using this method in the future. + */ + @Deprecated + static byte[] createClosestRowBefore(byte[] row) { + if (row.length == 0) { + return MAX_BYTE_ARRAY; + } + if (row[row.length - 1] == 0) { + return Arrays.copyOf(row, row.length - 1); + } else { + byte[] nextRow = new byte[row.length + MAX_BYTE_ARRAY.length]; + System.arraycopy(row, 0, nextRow, 0, row.length - 1); + nextRow[row.length - 1] = (byte) ((row[row.length - 1] & 0xFF) - 1); + System.arraycopy(MAX_BYTE_ARRAY, 0, nextRow, row.length, MAX_BYTE_ARRAY.length); + return nextRow; + } + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 83e7217afcb..1e3a900c2f0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -795,21 +795,9 @@ public class HTable implements HTableInterface, RegionLocator { } if (scan.isReversed()) { - if (scan.isSmall()) { - return new ClientSmallReversedScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } else { - return new ReversedClientScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } - } - - if (scan.isSmall()) { - return new ClientSmallScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + return new ReversedClientScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, this.rpcControllerFactory, + pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); } else { return new ClientScanner(getConfiguration(), scan, getName(), this.connection, this.rpcCallerFactory, this.rpcControllerFactory, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java index ca998ae26b5..edb66c5a1d7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java @@ -18,15 +18,18 @@ */ package org.apache.hadoop.hbase.client; +import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowBefore; + import java.io.IOException; import java.util.concurrent.ExecutorService; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ExceptionUtil; @@ -58,8 +61,7 @@ public class ReversedClientScanner extends ClientScanner { } @Override - protected boolean nextScanner(int nbRows, final boolean done) - throws IOException { + protected Result[] nextScanner(int nbRows) throws IOException { // Close the previous scanner if it's open closeScanner(); @@ -69,16 +71,17 @@ public class ReversedClientScanner extends ClientScanner { // if we're at start of table, close and return false to stop iterating if (this.currentRegion != null) { byte[] startKey = this.currentRegion.getStartKey(); - if (startKey == null - || Bytes.equals(startKey, HConstants.EMPTY_BYTE_ARRAY) - || checkScanStopRow(startKey) || done) { + if (startKey == null || Bytes.equals(startKey, HConstants.EMPTY_BYTE_ARRAY) + || checkScanStopRow(startKey)) { close(); if (LOG.isDebugEnabled()) { LOG.debug("Finished " + this.currentRegion); } - return false; + return null; } localStartKey = startKey; + // clear mvcc read point if we are going to switch regions + scan.resetMvccReadPoint(); if (LOG.isDebugEnabled()) { LOG.debug("Finished " + this.currentRegion); } @@ -109,17 +112,21 @@ public class ReversedClientScanner extends ClientScanner { // beginning of the region // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, // we do a callWithRetries - this.caller.callWithoutRetries(callable, scannerTimeout); + Result[] rrs = this.caller.callWithoutRetries(callable, scannerTimeout); this.currentRegion = callable.getHRegionInfo(); if (this.scanMetrics != null) { this.scanMetrics.countOfRegions.incrementAndGet(); } + if (rrs != null && rrs.length == 0 && callable.moreResultsForScan() == MoreResults.NO) { + // no results for the scan, return null to terminate the scan. + return null; + } + return rrs; } catch (IOException e) { ExceptionUtil.rethrowIfInterrupt(e); close(); throw e; } - return true; } protected ScannerCallableWithReplicas getScannerCallable(byte[] localStartKey, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java index e169f7a018e..840af97f90c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java @@ -44,7 +44,7 @@ public class ReversedScannerCallable extends ScannerCallable { /** * The start row for locating regions. In reversed scanner, may locate the * regions for a range of keys when doing - * {@link ReversedClientScanner#nextScanner(int, boolean)} + * {@link ReversedClientScanner#nextScanner(int)} */ protected final byte[] locateStartRow; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java index 128e7e1c1a7..84f1ca92e2d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java @@ -174,6 +174,17 @@ public class Scan extends Query { */ private long mvccReadPoint = -1L; + /** + * The number of rows we want for this scan. We will terminate the scan if the number of return + * rows reaches this value. + */ + private int limit = -1; + + /** + * Control whether to use pread at server side. + */ + private ReadType readType = ReadType.DEFAULT; + /** * Create a Scan operation across all rows. */ @@ -253,6 +264,7 @@ public class Scan extends Query { setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax()); } this.mvccReadPoint = scan.getMvccReadPoint(); + this.limit = scan.getLimit(); } /** @@ -1013,6 +1025,62 @@ public class Scan extends Query { return ProtobufUtil.toScanMetrics(bytes); } + /** + * @return the limit of rows for this scan + */ + public int getLimit() { + return limit; + } + + /** + * Set the limit of rows for this scan. We will terminate the scan if the number of returned rows + * reaches this value. + *

+ * This condition will be tested at last, after all other conditions such as stopRow, filter, etc. + *

+ * Can not be used together with batch and allowPartial. + * @param limit the limit of rows for this scan + * @return this + */ + public Scan setLimit(int limit) { + this.limit = limit; + return this; + } + + /** + * Call this when you only want to get one row. It will set {@code limit} to {@code 1}, and also + * set {@code readType} to {@link ReadType#PREAD}. + * @return this + */ + public Scan setOneRowLimit() { + return setLimit(1).setReadType(ReadType.PREAD); + } + + @InterfaceAudience.Public + @InterfaceStability.Unstable + public enum ReadType { + DEFAULT, STREAM, PREAD + } + + /** + * @return the read type for this scan + */ + public ReadType getReadType() { + return readType; + } + + /** + * Set the read type for this scan. + *

+ * Notice that we may choose to use pread even if you specific {@link ReadType#STREAM} here. For + * example, we will always use pread if this is a get scan. + * @return this + */ + public Scan setReadType(ReadType readType) { + this.readType = readType; + return this; + } + /** * Get the mvcc read point used to open a scanner. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java index 2dee7ceaa54..55be6da5fab 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java @@ -18,6 +18,9 @@ package org.apache.hadoop.hbase.client; +import com.google.protobuf.ServiceException; +import com.google.protobuf.TextFormat; + import java.io.IOException; import java.io.InterruptedIOException; import java.net.UnknownHostException; @@ -26,10 +29,8 @@ import java.util.Map.Entry; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; @@ -37,10 +38,10 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownScannerException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.exceptions.ScannerResetException; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; @@ -51,12 +52,8 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.DNS; -import com.google.protobuf.ServiceException; -import com.google.protobuf.TextFormat; - /** * Scanner operations such as create, next, etc. * Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as @@ -82,9 +79,15 @@ public class ScannerCallable extends RegionServerCallable { private int logCutOffLatency = 1000; private static String myAddress; protected final int id; - protected boolean serverHasMoreResultsContext; - protected boolean serverHasMoreResults; + enum MoreResults { + YES, NO, UNKNOWN + } + + private MoreResults moreResultsInRegion; + private MoreResults moreResultsForScan; + + private boolean openScanner; /** * Saves whether or not the most recent response from the server was a heartbeat message. * Heartbeat messages are identified by the flag {@link ScanResponse#getHeartbeatMessage()} @@ -136,6 +139,7 @@ public class ScannerCallable extends RegionServerCallable { logScannerActivity = conf.getBoolean(LOG_SCANNER_ACTIVITY, false); logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000); this.controllerFactory = rpcControllerFactory; + this.controller = rpcControllerFactory.newController(); } PayloadCarryingRpcController getController() { @@ -189,135 +193,124 @@ public class ScannerCallable extends RegionServerCallable { } } + private ScanResponse next() throws IOException { + // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server + setHeartbeatMessage(false); + incRPCcallsMetrics(); + ScanRequest request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, + this.scanMetrics != null, renew, scan.getLimit()); + try { + ScanResponse response = getStub().scan(controller, request); + nextCallSeq++; + return response; + } catch (Exception e) { + IOException ioe = ProtobufUtil.handleRemoteException(e); + if (logScannerActivity) { + LOG.info("Got exception making request " + TextFormat.shortDebugString(request) + " to " + + getLocation(), + e); + } + if (logScannerActivity) { + if (ioe instanceof UnknownScannerException) { + try { + HRegionLocation location = + getConnection().relocateRegion(getTableName(), scan.getStartRow()); + LOG.info("Scanner=" + scannerId + " expired, current region location is " + + location.toString()); + } catch (Throwable t) { + LOG.info("Failed to relocate region", t); + } + } else if (ioe instanceof ScannerResetException) { + LOG.info("Scanner=" + scannerId + " has received an exception, and the server " + + "asked us to reset the scanner state.", + ioe); + } + } + // The below convertion of exceptions into DoNotRetryExceptions is a little strange. + // Why not just have these exceptions implment DNRIOE you ask? Well, usually we want + // ServerCallable#withRetries to just retry when it gets these exceptions. In here in + // a scan when doing a next in particular, we want to break out and get the scanner to + // reset itself up again. Throwing a DNRIOE is how we signal this to happen (its ugly, + // yeah and hard to follow and in need of a refactor). + if (ioe instanceof NotServingRegionException) { + // Throw a DNRE so that we break out of cycle of calling NSRE + // when what we need is to open scanner against new location. + // Attach NSRE to signal client that it needs to re-setup scanner. + if (this.scanMetrics != null) { + this.scanMetrics.countOfNSRE.incrementAndGet(); + } + throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); + } else if (ioe instanceof RegionServerStoppedException) { + // Throw a DNRE so that we break out of cycle of the retries and instead go and + // open scanner against new location. + throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); + } else { + // The outer layers will retry + throw ioe; + } + } + } + + private void setAlreadyClosed() { + this.scannerId = -1L; + this.closed = true; + } @Override - public Result [] call(int callTimeout) throws IOException { + public Result[] call(int callTimeout) throws IOException { if (Thread.interrupted()) { throw new InterruptedIOException(); } - - if (controller == null) { - controller = controllerFactory.newController(); - controller.setPriority(getTableName()); - controller.setCallTimeout(callTimeout); - } - if (closed) { - if (scannerId != -1) { - close(); + close(); + return null; + } + controller.reset(); + controller.setPriority(getTableName()); + controller.setCallTimeout(callTimeout); + ScanResponse response; + if (this.scannerId == -1L) { + this.openScanner = true; + response = openScanner(); + } else { + this.openScanner = false; + response = next(); + } + long timestamp = System.currentTimeMillis(); + setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage()); + Result[] rrs = ResponseConverter.getResults(controller.cellScanner(), response); + if (logScannerActivity) { + long now = System.currentTimeMillis(); + if (now - timestamp > logCutOffLatency) { + int rows = rrs == null ? 0 : rrs.length; + LOG.info("Took " + (now - timestamp) + "ms to fetch " + rows + " rows from scanner=" + + scannerId); + } + } + updateServerSideMetrics(response); + // moreResults is only used for the case where a filter exhausts all elements + if (response.hasMoreResults()) { + if (response.getMoreResults()) { + setMoreResultsForScan(MoreResults.YES); + } else { + setMoreResultsForScan(MoreResults.NO); + setAlreadyClosed(); } } else { - if (scannerId == -1L) { - this.scannerId = openScanner(); - } else { - Result [] rrs = null; - ScanRequest request = null; - // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server - setHeartbeatMessage(false); - try { - incRPCcallsMetrics(); - request = - RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, - this.scanMetrics != null, renew); - ScanResponse response = null; - try { - response = getStub().scan(controller, request); - // Client and RS maintain a nextCallSeq number during the scan. Every next() call - // from client to server will increment this number in both sides. Client passes this - // number along with the request and at RS side both the incoming nextCallSeq and its - // nextCallSeq will be matched. In case of a timeout this increment at the client side - // should not happen. If at the server side fetching of next batch of data was over, - // there will be mismatch in the nextCallSeq number. Server will throw - // OutOfOrderScannerNextException and then client will reopen the scanner with startrow - // as the last successfully retrieved row. - // See HBASE-5974 - nextCallSeq++; - long timestamp = System.currentTimeMillis(); - setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage()); - // Results are returned via controller - CellScanner cellScanner = controller.cellScanner(); - rrs = ResponseConverter.getResults(cellScanner, response); - if (logScannerActivity) { - long now = System.currentTimeMillis(); - if (now - timestamp > logCutOffLatency) { - int rows = rrs == null ? 0 : rrs.length; - LOG.info("Took " + (now-timestamp) + "ms to fetch " - + rows + " rows from scanner=" + scannerId); - } - } - updateServerSideMetrics(response); - // moreResults is only used for the case where a filter exhausts all elements - if (response.hasMoreResults() && !response.getMoreResults()) { - scannerId = -1L; - closed = true; - // Implied that no results were returned back, either. - return null; - } - // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due - // to size or quantity of results in the response. - if (response.hasMoreResultsInRegion()) { - // Set what the RS said - setHasMoreResultsContext(true); - setServerHasMoreResults(response.getMoreResultsInRegion()); - } else { - // Server didn't respond whether it has more results or not. - setHasMoreResultsContext(false); - } - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - updateResultsMetrics(rrs); - } catch (IOException e) { - if (logScannerActivity) { - LOG.info("Got exception making request " + TextFormat.shortDebugString(request) - + " to " + getLocation(), e); - } - IOException ioe = e; - if (e instanceof RemoteException) { - ioe = RemoteExceptionHandler.decodeRemoteException((RemoteException)e); - } - if (logScannerActivity) { - if (ioe instanceof UnknownScannerException) { - try { - HRegionLocation location = - getConnection().relocateRegion(getTableName(), scan.getStartRow()); - LOG.info("Scanner=" + scannerId - + " expired, current region location is " + location.toString()); - } catch (Throwable t) { - LOG.info("Failed to relocate region", t); - } - } else if (ioe instanceof ScannerResetException) { - LOG.info("Scanner=" + scannerId + " has received an exception, and the server " - + "asked us to reset the scanner state.", ioe); - } - } - // The below convertion of exceptions into DoNotRetryExceptions is a little strange. - // Why not just have these exceptions implment DNRIOE you ask? Well, usually we want - // ServerCallable#withRetries to just retry when it gets these exceptions. In here in - // a scan when doing a next in particular, we want to break out and get the scanner to - // reset itself up again. Throwing a DNRIOE is how we signal this to happen (its ugly, - // yeah and hard to follow and in need of a refactor). - if (ioe instanceof NotServingRegionException) { - // Throw a DNRE so that we break out of cycle of calling NSRE - // when what we need is to open scanner against new location. - // Attach NSRE to signal client that it needs to re-setup scanner. - if (this.scanMetrics != null) { - this.scanMetrics.countOfNSRE.incrementAndGet(); - } - throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); - } else if (ioe instanceof RegionServerStoppedException) { - // Throw a DNRE so that we break out of cycle of the retries and instead go and - // open scanner against new location. - throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); - } else { - // The outer layers will retry - throw ioe; - } - } - return rrs; - } + setMoreResultsForScan(MoreResults.UNKNOWN); } - return null; + if (response.hasMoreResultsInRegion()) { + if (response.getMoreResultsInRegion()) { + setMoreResultsInRegion(MoreResults.YES); + } else { + setMoreResultsInRegion(MoreResults.NO); + setAlreadyClosed(); + } + } else { + setMoreResultsInRegion(MoreResults.UNKNOWN); + } + updateResultsMetrics(rrs); + return rrs; } /** @@ -326,11 +319,11 @@ public class ScannerCallable extends RegionServerCallable { * scan request exceeds a certain time threshold. Heartbeats allow the server to avoid * timeouts during long running scan operations. */ - protected boolean isHeartbeatMessage() { + boolean isHeartbeatMessage() { return heartbeatMessage; } - protected void setHeartbeatMessage(boolean heartbeatMessage) { + private void setHeartbeatMessage(boolean heartbeatMessage) { this.heartbeatMessage = heartbeatMessage; } @@ -397,12 +390,10 @@ public class ScannerCallable extends RegionServerCallable { this.scannerId = -1L; } - protected long openScanner() throws IOException { + private ScanResponse openScanner() throws IOException { incRPCcallsMetrics(); - ScanRequest request = - RequestConverter.buildScanRequest( - getLocation().getRegionInfo().getRegionName(), - this.scan, 0, false); + ScanRequest request = RequestConverter.buildScanRequest( + getLocation().getRegionInfo().getRegionName(), this.scan, this.caching, false); try { ScanResponse response = getStub().scan(controller, request); long id = response.getScannerId(); @@ -413,9 +404,10 @@ public class ScannerCallable extends RegionServerCallable { if (response.hasMvccReadPoint()) { this.scan.setMvccReadPoint(response.getMvccReadPoint()); } - return id; - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); + this.scannerId = id; + return response; + } catch (Exception e) { + throw ProtobufUtil.handleRemoteException(e); } } @@ -480,27 +472,31 @@ public class ScannerCallable extends RegionServerCallable { /** * Should the client attempt to fetch more results from this region - * @return True if the client should attempt to fetch more results, false otherwise. */ - protected boolean getServerHasMoreResults() { - assert serverHasMoreResultsContext; - return this.serverHasMoreResults; + MoreResults moreResultsInRegion() { + return moreResultsInRegion; } - protected void setServerHasMoreResults(boolean serverHasMoreResults) { - this.serverHasMoreResults = serverHasMoreResults; + void setMoreResultsInRegion(MoreResults moreResults) { + this.moreResultsInRegion = moreResults; } /** - * Did the server respond with information about whether more results might exist. - * Not guaranteed to respond with older server versions - * @return True if the server responded with information about more results. + * Should the client attempt to fetch more results for the whole scan. */ - protected boolean hasMoreResultsContext() { - return serverHasMoreResultsContext; + MoreResults moreResultsForScan() { + return moreResultsForScan; } - protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) { - this.serverHasMoreResultsContext = serverHasMoreResultsContext; + void setMoreResultsForScan(MoreResults moreResults) { + this.moreResultsForScan = moreResults; + } + + /** + * Whether the previous call is openScanner. This is used to keep compatible with the old + * implementation that we always returns empty result for openScanner. + */ + boolean isOpenScanner() { + return openScanner; } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java index a030e67efc6..46c8f9c0c27 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java @@ -18,7 +18,10 @@ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.ClientScanner.createClosestRowBefore; +import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter; +import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowBefore; + +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.io.InterruptedIOException; @@ -30,22 +33,18 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults; import org.apache.hadoop.hbase.util.Pair; -import com.google.common.annotations.VisibleForTesting; - /** * This class has the logic for handling scanners for regions with and without replicas. * 1. A scan is attempted on the default (primary) region @@ -115,20 +114,16 @@ class ScannerCallableWithReplicas implements RetryingCallable { return currentScannerCallable.getHRegionInfo(); } - public boolean getServerHasMoreResults() { - return currentScannerCallable.getServerHasMoreResults(); + public MoreResults moreResultsInRegion() { + return currentScannerCallable.moreResultsInRegion(); } - public void setServerHasMoreResults(boolean serverHasMoreResults) { - currentScannerCallable.setServerHasMoreResults(serverHasMoreResults); + public MoreResults moreResultsForScan() { + return currentScannerCallable.moreResultsForScan(); } - public boolean hasMoreResultsContext() { - return currentScannerCallable.hasMoreResultsContext(); - } - - public void setHasMoreResultsContext(boolean serverHasMoreResultsContext) { - currentScannerCallable.setHasMoreResultsContext(serverHasMoreResultsContext); + public boolean isOpenScanner() { + return currentScannerCallable.isOpenScanner(); } @Override @@ -342,7 +337,7 @@ class ScannerCallableWithReplicas implements RetryingCallable { if (callable.getScan().isReversed()) { callable.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow())); } else { - callable.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1])); + callable.getScan().setStartRow(createClosestRowAfter(this.lastResult.getRow())); } } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 715d1066b20..b3ade65342c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -20,6 +20,20 @@ package org.apache.hadoop.hbase.protobuf; import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.protobuf.ByteString; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Message; +import com.google.protobuf.Parser; +import com.google.protobuf.RpcChannel; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; +import com.google.protobuf.TextFormat; + import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; @@ -47,6 +61,7 @@ import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -128,12 +143,12 @@ import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; import org.apache.hadoop.hbase.protobuf.generated.WALProtos; +import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor; import org.apache.hadoop.hbase.quotas.QuotaScope; import org.apache.hadoop.hbase.quotas.QuotaType; @@ -158,20 +173,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.security.token.Token; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ListMultimap; -import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.Message; -import com.google.protobuf.Parser; -import com.google.protobuf.RpcChannel; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; -import com.google.protobuf.ServiceException; -import com.google.protobuf.TextFormat; - /** * Protobufs utility. */ @@ -322,17 +323,32 @@ public final class ProtobufUtil { * a new IOException that wraps the unexpected ServiceException. */ public static IOException getRemoteException(ServiceException se) { - Throwable e = se.getCause(); - if (e == null) { - return new IOException(se); + return makeIOExceptionOfException(se); + } + + /** + * Like {@link #getRemoteException(ServiceException)} but more generic, able to handle more than + * just {@link ServiceException}. Prefer this method to + * {@link #getRemoteException(ServiceException)} because trying to + * contain direct protobuf references. + * @param e + */ + public static IOException handleRemoteException(Exception e) { + return makeIOExceptionOfException(e); + } + + private static IOException makeIOExceptionOfException(Exception e) { + Throwable t = e; + if (e instanceof ServiceException) { + t = e.getCause(); } - if (ExceptionUtil.isInterrupt(e)) { - return ExceptionUtil.asInterrupt(e); + if (ExceptionUtil.isInterrupt(t)) { + return ExceptionUtil.asInterrupt(t); } - if (e instanceof RemoteException) { - e = ((RemoteException) e).unwrapRemoteException(); + if (t instanceof RemoteException) { + t = ((RemoteException) t).unwrapRemoteException(); } - return e instanceof IOException ? (IOException) e : new IOException(se); + return t instanceof IOException ? (IOException) t : new HBaseIOException(t); } /** @@ -908,6 +924,32 @@ public final class ProtobufUtil { return get; } + public static ClientProtos.Scan.ReadType toReadType(Scan.ReadType readType) { + switch (readType) { + case DEFAULT: + return ClientProtos.Scan.ReadType.DEFAULT; + case STREAM: + return ClientProtos.Scan.ReadType.STREAM; + case PREAD: + return ClientProtos.Scan.ReadType.PREAD; + default: + throw new IllegalArgumentException("Unknown ReadType: " + readType); + } + } + + public static Scan.ReadType toReadType(ClientProtos.Scan.ReadType readType) { + switch (readType) { + case DEFAULT: + return Scan.ReadType.DEFAULT; + case STREAM: + return Scan.ReadType.STREAM; + case PREAD: + return Scan.ReadType.PREAD; + default: + throw new IllegalArgumentException("Unknown ReadType: " + readType); + } + } + /** * Convert a client Scan to a protocol buffer Scan * @@ -1005,6 +1047,9 @@ public final class ProtobufUtil { if (mvccReadPoint > 0) { scanBuilder.setMvccReadPoint(mvccReadPoint); } + if (scan.getReadType() != Scan.ReadType.DEFAULT) { + scanBuilder.setReadType(toReadType(scan.getReadType())); + } return scanBuilder.build(); } @@ -1095,6 +1140,11 @@ public final class ProtobufUtil { if (proto.hasMvccReadPoint()) { PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint()); } + if (scan.isSmall()) { + scan.setReadType(Scan.ReadType.PREAD); + } else if (proto.hasReadType()) { + scan.setReadType(toReadType(proto.getReadType())); + } return scan; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java index 3b9d366754f..cac4e97e4d5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java @@ -498,6 +498,9 @@ public final class RequestConverter { builder.setClientHandlesPartials(true); builder.setClientHandlesHeartbeats(true); builder.setTrackScanMetrics(scan.isScanMetricsEnabled()); + if (scan.getLimit() > 0) { + builder.setLimitOfRows(scan.getLimit()); + } return builder.build(); } @@ -532,7 +535,7 @@ public final class RequestConverter { */ public static ScanRequest buildScanRequest(final long scannerId, final int numberOfRows, final boolean closeScanner, final long nextCallSeq, final boolean trackMetrics, - final boolean renew) { + final boolean renew, int limitOfRows) { ScanRequest.Builder builder = ScanRequest.newBuilder(); builder.setNumberOfRows(numberOfRows); builder.setCloseScanner(closeScanner); @@ -542,6 +545,9 @@ public final class RequestConverter { builder.setClientHandlesHeartbeats(true); builder.setTrackScanMetrics(trackMetrics); builder.setRenew(renew); + if (limitOfRows > 0) { + builder.setLimitOfRows(limitOfRows); + } return builder.build(); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java index 828d435500e..88c32ac15fd 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java @@ -17,9 +17,11 @@ */ package org.apache.hadoop.hbase.client; +import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -39,6 +41,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; @@ -87,6 +90,7 @@ public class TestClientScanner { private boolean rpcFinished = false; private boolean rpcFinishedFired = false; + private boolean initialized = false; public MockClientScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, @@ -97,9 +101,13 @@ public class TestClientScanner { } @Override - protected boolean nextScanner(int nbRows, final boolean done) throws IOException { + protected Result[] nextScanner(int nbRows) throws IOException { + if (!initialized) { + initialized = true; + return super.nextScanner(nbRows); + } if (!rpcFinished) { - return super.nextScanner(nbRows, done); + return super.nextScanner(nbRows); } // Enforce that we don't short-circuit more than once @@ -108,7 +116,7 @@ public class TestClientScanner { " short-circuit was triggered."); } rpcFinishedFired = true; - return false; + return null; } @Override @@ -150,14 +158,13 @@ public class TestClientScanner { ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize - case 2: // detect no more results - case 3: // close + count++; + callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.UNKNOWN); + return results; + case 1: // detect no more results + case 2: // close count++; return null; - case 1: - count++; - callable.setHasMoreResultsContext(false); - return results; default: throw new RuntimeException("Expected only 2 invocations"); } @@ -213,15 +220,13 @@ public class TestClientScanner { ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize - case 2: // close + count++; + // if we set no here the implementation will trigger a close + callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES); + return results; + case 1: // close count++; return null; - case 1: - count++; - callable.setHasMoreResultsContext(true); - // if we set false here the implementation will trigger a close - callable.setServerHasMoreResults(true); - return results; default: throw new RuntimeException("Expected only 2 invocations"); } @@ -237,16 +242,11 @@ public class TestClientScanner { try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt()); - InOrder inOrder = Mockito.inOrder(caller); scanner.loadCache(); - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( + inOrder.verify(caller, Mockito.times(1)).callWithoutRetries( Mockito.any(RetryingCallable.class), Mockito.anyInt()); assertEquals(1, scanner.cache.size()); @@ -281,15 +281,13 @@ public class TestClientScanner { ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize - case 2: // close + count++; + // if we set no here the implementation will trigger a close + callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES); + return results; + case 1: // close count++; return null; - case 1: - count++; - callable.setHasMoreResultsContext(true); - // if we set false here the implementation will trigger a close - callable.setServerHasMoreResults(true); - return results; default: throw new RuntimeException("Expected only 2 invocations"); } @@ -305,18 +303,11 @@ public class TestClientScanner { try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt()); - InOrder inOrder = Mockito.inOrder(caller); scanner.loadCache(); - // Ensures that possiblyNextScanner isn't called at the end which would trigger - // another call to callWithoutRetries - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( + inOrder.verify(caller, Mockito.times(1)).callWithoutRetries( Mockito.any(RetryingCallable.class), Mockito.anyInt()); assertEquals(3, scanner.cache.size()); @@ -363,14 +354,12 @@ public class TestClientScanner { ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize - case 2: // close + count++; + callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO); + return results; + case 1: // close count++; return null; - case 1: - count++; - callable.setHasMoreResultsContext(true); - callable.setServerHasMoreResults(false); - return results; default: throw new RuntimeException("Expected only 2 invocations"); } @@ -385,18 +374,13 @@ public class TestClientScanner { try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt()); - scanner.setRpcFinished(true); InOrder inOrder = Mockito.inOrder(caller); scanner.loadCache(); - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( + inOrder.verify(caller, Mockito.times(1)).callWithoutRetries( Mockito.any(RetryingCallable.class), Mockito.anyInt()); assertEquals(1, scanner.cache.size()); @@ -435,22 +419,19 @@ public class TestClientScanner { ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize - case 3: // close count++; - return null; + callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES); + return results1; case 1: - count++; - callable.setHasMoreResultsContext(true); - callable.setServerHasMoreResults(true); - return results1; - case 2: count++; // The server reports back false WRT more results - callable.setHasMoreResultsContext(true); - callable.setServerHasMoreResults(false); + callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO); return results2; + case 2: // close + count++; + return null; default: - throw new RuntimeException("Expected only 2 invocations"); + throw new RuntimeException("Expected only 3 invocations"); } } }); @@ -461,17 +442,12 @@ public class TestClientScanner { try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt()); - InOrder inOrder = Mockito.inOrder(caller); scanner.setRpcFinished(true); scanner.loadCache(); - inOrder.verify(caller, Mockito.times(3)).callWithoutRetries( + inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( Mockito.any(RetryingCallable.class), Mockito.anyInt()); assertEquals(2, scanner.cache.size()); @@ -516,8 +492,8 @@ public class TestClientScanner { iter.next(); } fail("Should have failed with RetriesExhaustedException"); - } catch (RetriesExhaustedException expected) { - + } catch (RuntimeException expected) { + assertThat(expected.getCause(), instanceOf(RetriesExhaustedException.class)); } } @@ -552,7 +528,5 @@ public class TestClientScanner { } }; } - } - } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java deleted file mode 100644 index 57b52e6c801..00000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java +++ /dev/null @@ -1,349 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.Type; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ClientSmallReversedScanner.SmallReversedScannerCallableFactory; -import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -/** - * Test the ClientSmallReversedScanner. - */ -@Category(SmallTests.class) -public class TestClientSmallReversedScanner { - - Scan scan; - ExecutorService pool; - Configuration conf; - - ClusterConnection clusterConn; - RpcRetryingCallerFactory rpcFactory; - RpcControllerFactory controllerFactory; - RpcRetryingCaller caller; - - @Before - @SuppressWarnings({"deprecation", "unchecked"}) - public void setup() throws IOException { - clusterConn = Mockito.mock(ClusterConnection.class); - rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class); - controllerFactory = Mockito.mock(RpcControllerFactory.class); - pool = Executors.newSingleThreadExecutor(); - scan = new Scan(); - conf = new Configuration(); - Mockito.when(clusterConn.getConfiguration()).thenReturn(conf); - // Mock out the RpcCaller - caller = Mockito.mock(RpcRetryingCaller.class); - // Return the mock from the factory - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - } - - @After - public void teardown() { - if (null != pool) { - pool.shutdownNow(); - } - } - - /** - * Create a simple Answer which returns true the first time, and false every time after. - */ - private Answer createTrueThenFalseAnswer() { - return new Answer() { - boolean first = true; - - @Override - public Boolean answer(InvocationOnMock invocation) { - if (first) { - first = false; - return true; - } - return false; - } - }; - } - - private SmallReversedScannerCallableFactory getFactory( - final ScannerCallableWithReplicas callableWithReplicas) { - return new SmallReversedScannerCallableFactory() { - @Override - public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table, - Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum, - RpcControllerFactory controllerFactory, ExecutorService pool, - int primaryOperationTimeout, int retries, int scannerTimeout, Configuration conf, - RpcRetryingCaller caller, boolean isFirstRegionToLocate) { - return callableWithReplicas; - } - }; - } - - @Test - public void testContextPresent() throws Exception { - final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum); - - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - // Mock out the RpcCaller - @SuppressWarnings("unchecked") - RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); - // Return the mock from the factory - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - - // Intentionally leave a "default" caching size in the Scan. No matter the value, we - // should continue based on the server context - - SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas); - - try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, - Integer.MAX_VALUE)) { - - csrs.setScannerCallableFactory(factory); - - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenAnswer(new Answer() { - int count = 0; - - @Override - public Result[] answer(InvocationOnMock invocation) { - Result[] results; - if (0 == count) { - results = new Result[] {Result.create(new Cell[] {kv3}), - Result.create(new Cell[] {kv2})}; - } else if (1 == count) { - results = new Result[] {Result.create(new Cell[] {kv1})}; - } else { - results = new Result[0]; - } - count++; - return results; - } - }); - - // Pass back the context always - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true); - // Only have more results the first time - Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenAnswer( - createTrueThenFalseAnswer()); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - csrs.loadCache(); - - List results = csrs.cache; - Iterator iter = results.iterator(); - assertEquals(3, results.size()); - for (int i = 3; i >= 1 && iter.hasNext(); i--) { - Result result = iter.next(); - byte[] row = result.getRow(); - assertEquals("row" + i, new String(row, StandardCharsets.UTF_8)); - assertEquals(1, result.getMap().size()); - } - assertTrue(csrs.closed); - } - } - - @Test - public void testNoContextFewerRecords() throws Exception { - final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum); - - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - // While the server returns 2 records per batch, we expect more records. - scan.setCaching(2); - - SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas); - - try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, - Integer.MAX_VALUE)) { - - csrs.setScannerCallableFactory(factory); - - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenAnswer(new Answer() { - int count = 0; - - @Override - public Result[] answer(InvocationOnMock invocation) { - Result[] results; - if (0 == count) { - results = new Result[] {Result.create(new Cell[] {kv3}), - Result.create(new Cell[] {kv2})}; - } else if (1 == count) { - // Return fewer records than expected (2) - results = new Result[] {Result.create(new Cell[] {kv1})}; - } else { - throw new RuntimeException("Should not fetch a third batch from the server"); - } - count++; - return results; - } - }); - - // Server doesn't return the context - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false); - // getServerHasMoreResults shouldn't be called when hasMoreResultsContext returns false - Mockito.when(callableWithReplicas.getServerHasMoreResults()) - .thenThrow(new RuntimeException("Should not be called")); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - csrs.loadCache(); - - List results = csrs.cache; - Iterator iter = results.iterator(); - assertEquals(2, results.size()); - for (int i = 3; i >= 2 && iter.hasNext(); i--) { - Result result = iter.next(); - byte[] row = result.getRow(); - assertEquals("row" + i, new String(row, StandardCharsets.UTF_8)); - assertEquals(1, result.getMap().size()); - } - - // "consume" the Results - results.clear(); - - csrs.loadCache(); - - assertEquals(1, results.size()); - Result result = results.get(0); - assertEquals("row1", new String(result.getRow(), StandardCharsets.UTF_8)); - assertEquals(1, result.getMap().size()); - - assertTrue(csrs.closed); - } - } - - @Test - public void testNoContextNoRecords() throws Exception { - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - // While the server return 2 records per RPC, we expect there to be more records. - scan.setCaching(2); - - SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas); - - try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, - Integer.MAX_VALUE)) { - - csrs.setScannerCallableFactory(factory); - - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenReturn(new Result[0]); - - // Server doesn't return the context - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false); - // Only have more results the first time - Mockito.when(callableWithReplicas.getServerHasMoreResults()) - .thenThrow(new RuntimeException("Should not be called")); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - csrs.loadCache(); - - assertEquals(0, csrs.cache.size()); - assertTrue(csrs.closed); - } - } - - @Test - public void testContextNoRecords() throws Exception { - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas); - - try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, - Integer.MAX_VALUE)) { - - csrs.setScannerCallableFactory(factory); - - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenReturn(new Result[0]); - - // Server doesn't return the context - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true); - // Only have more results the first time - Mockito.when(callableWithReplicas.getServerHasMoreResults()) - .thenReturn(false); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - csrs.loadCache(); - - assertEquals(0, csrs.cache.size()); - assertTrue(csrs.closed); - } - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java deleted file mode 100644 index 90bf4bbd3ab..00000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java +++ /dev/null @@ -1,339 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.Type; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory; -import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -/** - * Test the ClientSmallScanner. - */ -@Category(SmallTests.class) -public class TestClientSmallScanner { - - Scan scan; - ExecutorService pool; - Configuration conf; - - ClusterConnection clusterConn; - RpcRetryingCallerFactory rpcFactory; - RpcControllerFactory controllerFactory; - RpcRetryingCaller caller; - - @Before - @SuppressWarnings({"deprecation", "unchecked"}) - public void setup() throws IOException { - clusterConn = Mockito.mock(ClusterConnection.class); - rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class); - controllerFactory = Mockito.mock(RpcControllerFactory.class); - pool = Executors.newSingleThreadExecutor(); - scan = new Scan(); - conf = new Configuration(); - Mockito.when(clusterConn.getConfiguration()).thenReturn(conf); - // Mock out the RpcCaller - caller = Mockito.mock(RpcRetryingCaller.class); - // Return the mock from the factory - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - } - - @After - public void teardown() { - if (null != pool) { - pool.shutdownNow(); - } - } - - /** - * Create a simple Answer which returns true the first time, and false every time after. - */ - private Answer createTrueThenFalseAnswer() { - return new Answer() { - boolean first = true; - - @Override - public Boolean answer(InvocationOnMock invocation) { - if (first) { - first = false; - return true; - } - return false; - } - }; - } - - private SmallScannerCallableFactory getFactory( - final ScannerCallableWithReplicas callableWithReplicas) { - return new SmallScannerCallableFactory() { - @Override - public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table, - Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum, - RpcControllerFactory controllerFactory, ExecutorService pool, - int primaryOperationTimeout, int retries, int scannerTimeout, Configuration conf, - RpcRetryingCaller caller) { - return callableWithReplicas; - } - }; - } - - @Test - public void testContextPresent() throws Exception { - final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum); - - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - // Mock out the RpcCaller - @SuppressWarnings("unchecked") - RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); - // Return the mock from the factory - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - - SmallScannerCallableFactory factory = getFactory(callableWithReplicas); - - // Intentionally leave a "default" caching size in the Scan. No matter the value, we - // should continue based on the server context - - try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - css.setScannerCallableFactory(factory); - - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenAnswer(new Answer() { - int count = 0; - - @Override - public Result[] answer(InvocationOnMock invocation) { - Result[] results; - if (0 == count) { - results = new Result[] {Result.create(new Cell[] {kv1}), - Result.create(new Cell[] {kv2})}; - } else if (1 == count) { - results = new Result[] {Result.create(new Cell[] {kv3})}; - } else { - results = new Result[0]; - } - count++; - return results; - } - }); - - // Pass back the context always - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true); - // Only have more results the first time - Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenAnswer( - createTrueThenFalseAnswer()); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - css.loadCache(); - - List results = css.cache; - assertEquals(3, results.size()); - for (int i = 1; i <= 3; i++) { - Result result = results.get(i - 1); - byte[] row = result.getRow(); - assertEquals("row" + i, new String(row, StandardCharsets.UTF_8)); - assertEquals(1, result.getMap().size()); - } - - assertTrue(css.closed); - } - } - - @Test - public void testNoContextFewerRecords() throws Exception { - final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1, - Type.Maximum); - - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - // While the server returns 2 records per batch, we expect more records. - scan.setCaching(2); - SmallScannerCallableFactory factory = getFactory(callableWithReplicas); - - try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - css.setScannerCallableFactory(factory); - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenAnswer(new Answer() { - int count = 0; - - @Override - public Result[] answer(InvocationOnMock invocation) { - Result[] results; - if (0 == count) { - results = new Result[] {Result.create(new Cell[] {kv1}), - Result.create(new Cell[] {kv2})}; - } else if (1 == count) { - // Return fewer records than expected (2) - results = new Result[] {Result.create(new Cell[] {kv3})}; - } else { - throw new RuntimeException("Should not fetch a third batch from the server"); - } - count++; - return results; - } - }); - - // Server doesn't return the context - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false); - // Only have more results the first time - Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenThrow( - new RuntimeException("Should not be called")); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - css.loadCache(); - - List results = css.cache; - assertEquals(2, results.size()); - for (int i = 1; i <= 2; i++) { - Result result = results.get(i - 1); - byte[] row = result.getRow(); - assertEquals("row" + i, new String(row, StandardCharsets.UTF_8)); - assertEquals(1, result.getMap().size()); - } - - // "consume" the results we verified - results.clear(); - - css.loadCache(); - - assertEquals(1, results.size()); - Result result = results.get(0); - assertEquals("row3", new String(result.getRow(), StandardCharsets.UTF_8)); - assertEquals(1, result.getMap().size()); - assertTrue(css.closed); - } - } - - @Test - public void testNoContextNoRecords() throws Exception { - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - // While the server return 2 records per RPC, we expect there to be more records. - scan.setCaching(2); - - SmallScannerCallableFactory factory = getFactory(callableWithReplicas); - - try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - css.setScannerCallableFactory(factory); - - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenReturn(new Result[0]); - - // Server doesn't return the context - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false); - // Only have more results the first time - Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenThrow( - new RuntimeException("Should not be called")); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - css.loadCache(); - - assertEquals(0, css.cache.size()); - assertTrue(css.closed); - } - } - - @Test - public void testContextNoRecords() throws Exception { - ScannerCallableWithReplicas callableWithReplicas = Mockito - .mock(ScannerCallableWithReplicas.class); - - SmallScannerCallableFactory factory = getFactory(callableWithReplicas); - - try (ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - css.setScannerCallableFactory(factory); - - // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenReturn(new Result[0]); - - // Server doesn't return the context - Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true); - // Only have more results the first time - Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenReturn(false); - - // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right - HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class); - Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo); - // Trigger the "no more data" branch for #nextScanner(...) - Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY); - - css.loadCache(); - - assertEquals(0, css.cache.size()); - assertTrue(css.closed); - } - } -} diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index d1f28ba7c12..cefdd386490 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -14239,6 +14239,36 @@ public final class ClientProtos { * optional uint64 mvcc_read_point = 20 [default = 0]; */ long getMvccReadPoint(); + + // optional bool include_start_row = 21 [default = true]; + /** + * optional bool include_start_row = 21 [default = true]; + */ + boolean hasIncludeStartRow(); + /** + * optional bool include_start_row = 21 [default = true]; + */ + boolean getIncludeStartRow(); + + // optional bool include_stop_row = 22 [default = false]; + /** + * optional bool include_stop_row = 22 [default = false]; + */ + boolean hasIncludeStopRow(); + /** + * optional bool include_stop_row = 22 [default = false]; + */ + boolean getIncludeStopRow(); + + // optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + boolean hasReadType(); + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType(); } /** * Protobuf type {@code hbase.pb.Scan} @@ -14433,6 +14463,27 @@ public final class ClientProtos { mvccReadPoint_ = input.readUInt64(); break; } + case 168: { + bitField0_ |= 0x00020000; + includeStartRow_ = input.readBool(); + break; + } + case 176: { + bitField0_ |= 0x00040000; + includeStopRow_ = input.readBool(); + break; + } + case 184: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType value = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(23, rawValue); + } else { + bitField0_ |= 0x00080000; + readType_ = value; + } + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -14481,6 +14532,97 @@ public final class ClientProtos { return PARSER; } + /** + * Protobuf enum {@code hbase.pb.Scan.ReadType} + */ + public enum ReadType + implements com.google.protobuf.ProtocolMessageEnum { + /** + * DEFAULT = 0; + */ + DEFAULT(0, 0), + /** + * STREAM = 1; + */ + STREAM(1, 1), + /** + * PREAD = 2; + */ + PREAD(2, 2), + ; + + /** + * DEFAULT = 0; + */ + public static final int DEFAULT_VALUE = 0; + /** + * STREAM = 1; + */ + public static final int STREAM_VALUE = 1; + /** + * PREAD = 2; + */ + public static final int PREAD_VALUE = 2; + + + public final int getNumber() { return value; } + + public static ReadType valueOf(int value) { + switch (value) { + case 0: return DEFAULT; + case 1: return STREAM; + case 2: return PREAD; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public ReadType findValueByNumber(int number) { + return ReadType.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDescriptor().getEnumTypes().get(0); + } + + private static final ReadType[] VALUES = values(); + + public static ReadType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private ReadType(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:hbase.pb.Scan.ReadType) + } + private int bitField0_; // repeated .hbase.pb.Column column = 1; public static final int COLUMN_FIELD_NUMBER = 1; @@ -14882,6 +15024,54 @@ public final class ClientProtos { return mvccReadPoint_; } + // optional bool include_start_row = 21 [default = true]; + public static final int INCLUDE_START_ROW_FIELD_NUMBER = 21; + private boolean includeStartRow_; + /** + * optional bool include_start_row = 21 [default = true]; + */ + public boolean hasIncludeStartRow() { + return ((bitField0_ & 0x00020000) == 0x00020000); + } + /** + * optional bool include_start_row = 21 [default = true]; + */ + public boolean getIncludeStartRow() { + return includeStartRow_; + } + + // optional bool include_stop_row = 22 [default = false]; + public static final int INCLUDE_STOP_ROW_FIELD_NUMBER = 22; + private boolean includeStopRow_; + /** + * optional bool include_stop_row = 22 [default = false]; + */ + public boolean hasIncludeStopRow() { + return ((bitField0_ & 0x00040000) == 0x00040000); + } + /** + * optional bool include_stop_row = 22 [default = false]; + */ + public boolean getIncludeStopRow() { + return includeStopRow_; + } + + // optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + public static final int READTYPE_FIELD_NUMBER = 23; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType readType_; + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + public boolean hasReadType() { + return ((bitField0_ & 0x00080000) == 0x00080000); + } + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType() { + return readType_; + } + private void initFields() { column_ = java.util.Collections.emptyList(); attribute_ = java.util.Collections.emptyList(); @@ -14903,6 +15093,9 @@ public final class ClientProtos { allowPartialResults_ = false; cfTimeRange_ = java.util.Collections.emptyList(); mvccReadPoint_ = 0L; + includeStartRow_ = true; + includeStopRow_ = false; + readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -15000,6 +15193,15 @@ public final class ClientProtos { if (((bitField0_ & 0x00010000) == 0x00010000)) { output.writeUInt64(20, mvccReadPoint_); } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + output.writeBool(21, includeStartRow_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + output.writeBool(22, includeStopRow_); + } + if (((bitField0_ & 0x00080000) == 0x00080000)) { + output.writeEnum(23, readType_.getNumber()); + } getUnknownFields().writeTo(output); } @@ -15089,6 +15291,18 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeUInt64Size(20, mvccReadPoint_); } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(21, includeStartRow_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(22, includeStopRow_); + } + if (((bitField0_ & 0x00080000) == 0x00080000)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(23, readType_.getNumber()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -15203,6 +15417,21 @@ public final class ClientProtos { result = result && (getMvccReadPoint() == other.getMvccReadPoint()); } + result = result && (hasIncludeStartRow() == other.hasIncludeStartRow()); + if (hasIncludeStartRow()) { + result = result && (getIncludeStartRow() + == other.getIncludeStartRow()); + } + result = result && (hasIncludeStopRow() == other.hasIncludeStopRow()); + if (hasIncludeStopRow()) { + result = result && (getIncludeStopRow() + == other.getIncludeStopRow()); + } + result = result && (hasReadType() == other.hasReadType()); + if (hasReadType()) { + result = result && + (getReadType() == other.getReadType()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -15296,6 +15525,18 @@ public final class ClientProtos { hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER; hash = (53 * hash) + hashLong(getMvccReadPoint()); } + if (hasIncludeStartRow()) { + hash = (37 * hash) + INCLUDE_START_ROW_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getIncludeStartRow()); + } + if (hasIncludeStopRow()) { + hash = (37 * hash) + INCLUDE_STOP_ROW_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getIncludeStopRow()); + } + if (hasReadType()) { + hash = (37 * hash) + READTYPE_FIELD_NUMBER; + hash = (53 * hash) + hashEnum(getReadType()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -15481,6 +15722,12 @@ public final class ClientProtos { } mvccReadPoint_ = 0L; bitField0_ = (bitField0_ & ~0x00080000); + includeStartRow_ = true; + bitField0_ = (bitField0_ & ~0x00100000); + includeStopRow_ = false; + bitField0_ = (bitField0_ & ~0x00200000); + readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT; + bitField0_ = (bitField0_ & ~0x00400000); return this; } @@ -15612,6 +15859,18 @@ public final class ClientProtos { to_bitField0_ |= 0x00010000; } result.mvccReadPoint_ = mvccReadPoint_; + if (((from_bitField0_ & 0x00100000) == 0x00100000)) { + to_bitField0_ |= 0x00020000; + } + result.includeStartRow_ = includeStartRow_; + if (((from_bitField0_ & 0x00200000) == 0x00200000)) { + to_bitField0_ |= 0x00040000; + } + result.includeStopRow_ = includeStopRow_; + if (((from_bitField0_ & 0x00400000) == 0x00400000)) { + to_bitField0_ |= 0x00080000; + } + result.readType_ = readType_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -15757,6 +16016,15 @@ public final class ClientProtos { if (other.hasMvccReadPoint()) { setMvccReadPoint(other.getMvccReadPoint()); } + if (other.hasIncludeStartRow()) { + setIncludeStartRow(other.getIncludeStartRow()); + } + if (other.hasIncludeStopRow()) { + setIncludeStopRow(other.getIncludeStopRow()); + } + if (other.hasReadType()) { + setReadType(other.getReadType()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -17282,6 +17550,108 @@ public final class ClientProtos { return this; } + // optional bool include_start_row = 21 [default = true]; + private boolean includeStartRow_ = true; + /** + * optional bool include_start_row = 21 [default = true]; + */ + public boolean hasIncludeStartRow() { + return ((bitField0_ & 0x00100000) == 0x00100000); + } + /** + * optional bool include_start_row = 21 [default = true]; + */ + public boolean getIncludeStartRow() { + return includeStartRow_; + } + /** + * optional bool include_start_row = 21 [default = true]; + */ + public Builder setIncludeStartRow(boolean value) { + bitField0_ |= 0x00100000; + includeStartRow_ = value; + onChanged(); + return this; + } + /** + * optional bool include_start_row = 21 [default = true]; + */ + public Builder clearIncludeStartRow() { + bitField0_ = (bitField0_ & ~0x00100000); + includeStartRow_ = true; + onChanged(); + return this; + } + + // optional bool include_stop_row = 22 [default = false]; + private boolean includeStopRow_ ; + /** + * optional bool include_stop_row = 22 [default = false]; + */ + public boolean hasIncludeStopRow() { + return ((bitField0_ & 0x00200000) == 0x00200000); + } + /** + * optional bool include_stop_row = 22 [default = false]; + */ + public boolean getIncludeStopRow() { + return includeStopRow_; + } + /** + * optional bool include_stop_row = 22 [default = false]; + */ + public Builder setIncludeStopRow(boolean value) { + bitField0_ |= 0x00200000; + includeStopRow_ = value; + onChanged(); + return this; + } + /** + * optional bool include_stop_row = 22 [default = false]; + */ + public Builder clearIncludeStopRow() { + bitField0_ = (bitField0_ & ~0x00200000); + includeStopRow_ = false; + onChanged(); + return this; + } + + // optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT; + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + public boolean hasReadType() { + return ((bitField0_ & 0x00400000) == 0x00400000); + } + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType() { + return readType_; + } + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + public Builder setReadType(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00400000; + readType_ = value; + onChanged(); + return this; + } + /** + * optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT]; + */ + public Builder clearReadType() { + bitField0_ = (bitField0_ & ~0x00400000); + readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.Scan) } @@ -17403,6 +17773,24 @@ public final class ClientProtos { * optional bool renew = 10 [default = false]; */ boolean getRenew(); + + // optional uint32 limit_of_rows = 11 [default = 0]; + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *

+     * if we have returned limit_of_rows rows to client, then close the scanner.
+     * 
+ */ + boolean hasLimitOfRows(); + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *
+     * if we have returned limit_of_rows rows to client, then close the scanner.
+     * 
+ */ + int getLimitOfRows(); } /** * Protobuf type {@code hbase.pb.ScanRequest} @@ -17534,6 +17922,11 @@ public final class ClientProtos { renew_ = input.readBool(); break; } + case 88: { + bitField0_ |= 0x00000400; + limitOfRows_ = input.readUInt32(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -17746,6 +18139,30 @@ public final class ClientProtos { return renew_; } + // optional uint32 limit_of_rows = 11 [default = 0]; + public static final int LIMIT_OF_ROWS_FIELD_NUMBER = 11; + private int limitOfRows_; + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *
+     * if we have returned limit_of_rows rows to client, then close the scanner.
+     * 
+ */ + public boolean hasLimitOfRows() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *
+     * if we have returned limit_of_rows rows to client, then close the scanner.
+     * 
+ */ + public int getLimitOfRows() { + return limitOfRows_; + } + private void initFields() { region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); @@ -17757,6 +18174,7 @@ public final class ClientProtos { clientHandlesHeartbeats_ = false; trackScanMetrics_ = false; renew_ = false; + limitOfRows_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -17812,6 +18230,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000200) == 0x00000200)) { output.writeBool(10, renew_); } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeUInt32(11, limitOfRows_); + } getUnknownFields().writeTo(output); } @@ -17861,6 +18282,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeBoolSize(10, renew_); } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(11, limitOfRows_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -17934,6 +18359,11 @@ public final class ClientProtos { result = result && (getRenew() == other.getRenew()); } + result = result && (hasLimitOfRows() == other.hasLimitOfRows()); + if (hasLimitOfRows()) { + result = result && (getLimitOfRows() + == other.getLimitOfRows()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -17987,6 +18417,10 @@ public final class ClientProtos { hash = (37 * hash) + RENEW_FIELD_NUMBER; hash = (53 * hash) + hashBoolean(getRenew()); } + if (hasLimitOfRows()) { + hash = (37 * hash) + LIMIT_OF_ROWS_FIELD_NUMBER; + hash = (53 * hash) + getLimitOfRows(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -18139,6 +18573,8 @@ public final class ClientProtos { bitField0_ = (bitField0_ & ~0x00000100); renew_ = false; bitField0_ = (bitField0_ & ~0x00000200); + limitOfRows_ = 0; + bitField0_ = (bitField0_ & ~0x00000400); return this; } @@ -18215,6 +18651,10 @@ public final class ClientProtos { to_bitField0_ |= 0x00000200; } result.renew_ = renew_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000400; + } + result.limitOfRows_ = limitOfRows_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -18261,6 +18701,9 @@ public final class ClientProtos { if (other.hasRenew()) { setRenew(other.getRenew()); } + if (other.hasLimitOfRows()) { + setLimitOfRows(other.getLimitOfRows()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -18798,6 +19241,55 @@ public final class ClientProtos { return this; } + // optional uint32 limit_of_rows = 11 [default = 0]; + private int limitOfRows_ ; + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * 
+ */ + public boolean hasLimitOfRows() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * 
+ */ + public int getLimitOfRows() { + return limitOfRows_; + } + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * 
+ */ + public Builder setLimitOfRows(int value) { + bitField0_ |= 0x00000400; + limitOfRows_ = value; + onChanged(); + return this; + } + /** + * optional uint32 limit_of_rows = 11 [default = 0]; + * + *
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * 
+ */ + public Builder clearLimitOfRows() { + bitField0_ = (bitField0_ & ~0x00000400); + limitOfRows_ = 0; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest) } @@ -35899,8 +36391,8 @@ public final class ClientProtos { "\0132\027.hbase.pb.MutationProto\022&\n\tcondition\030" + "\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce_grou" + "p\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result\030\001 \001(" + - "\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 \001(\010\"\331" + - "\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb.Colum" + + "\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 \001(\010\"\377" + + "\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb.Colum" + "n\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.NameByte" + "sPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_row\030\004 \001" + "(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filter\022\'\n\n" + @@ -35915,81 +36407,86 @@ public final class ClientProtos { "g\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 \001(\010\0226" + "\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.ColumnF" + "amilyTimeRange\022\032\n\017mvcc_read_point\030\024 \001(\004:", - "\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132\031.hba" + - "se.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hb" + - "ase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016numbe" + - "r_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025" + - "\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_handles_" + - "partials\030\007 \001(\010\022!\n\031client_handles_heartbe" + - "ats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001(\010\022\024\n" + - "\005renew\030\n \001(\010:\005false\"\266\002\n\014ScanResponse\022\030\n\020" + - "cells_per_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001" + - "(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n", - "\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stal" + - "e\030\006 \001(\010\022\037\n\027partial_flag_per_result\030\007 \003(\010" + - "\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n\021hear" + - "tbeat_message\030\t \001(\010\022+\n\014scan_metrics\030\n \001(" + - "\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_read_po" + - "int\030\013 \001(\004:\0010\"\305\001\n\024BulkLoadHFileRequest\022)\n" + - "\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifier" + - "\022>\n\013family_path\030\002 \003(\0132).hbase.pb.BulkLoa" + - "dHFileRequest.FamilyPath\022\026\n\016assign_seq_n" + - "um\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014", - "\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n" + - "\006loaded\030\001 \002(\010\"a\n\026CoprocessorServiceCall\022" + - "\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013me" + - "thod_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030Copr" + - "ocessorServiceResult\022&\n\005value\030\001 \001(\0132\027.hb" + - "ase.pb.NameBytesPair\"v\n\031CoprocessorServi" + - "ceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Reg" + - "ionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb.Co" + - "processorServiceCall\"o\n\032CoprocessorServi" + - "ceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re", - "gionSpecifier\022&\n\005value\030\002 \002(\0132\027.hbase.pb." + - "NameBytesPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001(\r\022" + - ")\n\010mutation\030\002 \001(\0132\027.hbase.pb.MutationPro" + - "to\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014servic" + - "e_call\030\004 \001(\0132 .hbase.pb.CoprocessorServi" + - "ceCall\"k\n\014RegionAction\022)\n\006region\030\001 \002(\0132\031" + - ".hbase.pb.RegionSpecifier\022\016\n\006atomic\030\002 \001(" + - "\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Action\"c\n\017R" + - "egionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022" + - "\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compactionP", - "ressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadStats" + - "\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpecif" + - "ier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLoadS" + - "tats\"\336\001\n\021ResultOrException\022\r\n\005index\030\001 \001(" + - "\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*\n\te" + - "xception\030\003 \001(\0132\027.hbase.pb.NameBytesPair\022" + - ":\n\016service_result\030\004 \001(\0132\".hbase.pb.Copro" + - "cessorServiceResult\0220\n\tloadStats\030\005 \001(\0132\031" + - ".hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022RegionA" + - "ctionResult\0226\n\021resultOrException\030\001 \003(\0132\033", - ".hbase.pb.ResultOrException\022*\n\texception" + - "\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Multi" + - "Request\022,\n\014regionAction\030\001 \003(\0132\026.hbase.pb" + - ".RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcon" + - "dition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n\rMu" + - "ltiResponse\0228\n\022regionActionResult\030\001 \003(\0132" + - "\034.hbase.pb.RegionActionResult\022\021\n\tprocess" + - "ed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036.hba" + - "se.pb.MultiRegionLoadStats*\'\n\013Consistenc" + - "y\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientSe", - "rvice\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025.hba" + - "se.pb.GetResponse\022;\n\006Mutate\022\027.hbase.pb.M" + - "utateRequest\032\030.hbase.pb.MutateResponse\0225" + - "\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase.pb" + - ".ScanResponse\022P\n\rBulkLoadHFile\022\036.hbase.p" + - "b.BulkLoadHFileRequest\032\037.hbase.pb.BulkLo" + - "adHFileResponse\022X\n\013ExecService\022#.hbase.p" + - "b.CoprocessorServiceRequest\032$.hbase.pb.C" + - "oprocessorServiceResponse\022d\n\027ExecRegionS" + - "erverService\022#.hbase.pb.CoprocessorServi", - "ceRequest\032$.hbase.pb.CoprocessorServiceR" + - "esponse\0228\n\005Multi\022\026.hbase.pb.MultiRequest" + - "\032\027.hbase.pb.MultiResponseBB\n*org.apache." + - "hadoop.hbase.protobuf.generatedB\014ClientP" + - "rotosH\001\210\001\001\240\001\001" + "\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true\022\037\n\020i" + + "nclude_stop_row\030\026 \001(\010:\005false\0222\n\010readType" + + "\030\027 \001(\0162\027.hbase.pb.Scan.ReadType:\007DEFAULT" + + "\".\n\010ReadType\022\013\n\007DEFAULT\020\000\022\n\n\006STREAM\020\001\022\t\n" + + "\005PREAD\020\002\"\300\002\n\013ScanRequest\022)\n\006region\030\001 \001(\013" + + "2\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(" + + "\0132\016.hbase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n" + + "\016number_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005" + + " \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_ha" + + "ndles_partials\030\007 \001(\010\022!\n\031client_handles_h", + "eartbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t " + + "\001(\010\022\024\n\005renew\030\n \001(\010:\005false\022\030\n\rlimit_of_ro" + + "ws\030\013 \001(\r:\0010\"\266\002\n\014ScanResponse\022\030\n\020cells_pe" + + "r_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mo" + + "re_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results" + + "\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001(\010\022" + + "\037\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026more" + + "_results_in_region\030\010 \001(\010\022\031\n\021heartbeat_me" + + "ssage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbas" + + "e.pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(", + "\004:\0010\"\305\001\n\024BulkLoadHFileRequest\022)\n\006region\030" + + "\001 \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013fami" + + "ly_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileRe" + + "quest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010" + + "\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002" + + " \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n\006loaded\030" + + "\001 \002(\010\"a\n\026CoprocessorServiceCall\022\013\n\003row\030\001" + + " \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013method_nam" + + "e\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorS" + + "erviceResult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.N", + "ameBytesPair\"v\n\031CoprocessorServiceReques" + + "t\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpeci" + + "fier\022.\n\004call\030\002 \002(\0132 .hbase.pb.Coprocesso" + + "rServiceCall\"o\n\032CoprocessorServiceRespon" + + "se\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" + + "ifier\022&\n\005value\030\002 \002(\0132\027.hbase.pb.NameByte" + + "sPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001(\r\022)\n\010mutat" + + "ion\030\002 \001(\0132\027.hbase.pb.MutationProto\022\032\n\003ge" + + "t\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014service_call\030\004" + + " \001(\0132 .hbase.pb.CoprocessorServiceCall\"k", + "\n\014RegionAction\022)\n\006region\030\001 \002(\0132\031.hbase.p" + + "b.RegionSpecifier\022\016\n\006atomic\030\002 \001(\010\022 \n\006act" + + "ion\030\003 \003(\0132\020.hbase.pb.Action\"c\n\017RegionLoa" + + "dStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rheapO" + + "ccupancy\030\002 \001(\005:\0010\022\035\n\022compactionPressure\030" + + "\003 \001(\005:\0010\"j\n\024MultiRegionLoadStats\022)\n\006regi" + + "on\030\001 \003(\0132\031.hbase.pb.RegionSpecifier\022\'\n\004s" + + "tat\030\002 \003(\0132\031.hbase.pb.RegionLoadStats\"\336\001\n" + + "\021ResultOrException\022\r\n\005index\030\001 \001(\r\022 \n\006res" + + "ult\030\002 \001(\0132\020.hbase.pb.Result\022*\n\texception", + "\030\003 \001(\0132\027.hbase.pb.NameBytesPair\022:\n\016servi" + + "ce_result\030\004 \001(\0132\".hbase.pb.CoprocessorSe" + + "rviceResult\0220\n\tloadStats\030\005 \001(\0132\031.hbase.p" + + "b.RegionLoadStatsB\002\030\001\"x\n\022RegionActionRes" + + "ult\0226\n\021resultOrException\030\001 \003(\0132\033.hbase.p" + + "b.ResultOrException\022*\n\texception\030\002 \001(\0132\027" + + ".hbase.pb.NameBytesPair\"x\n\014MultiRequest\022" + + ",\n\014regionAction\030\001 \003(\0132\026.hbase.pb.RegionA" + + "ction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcondition\030\003" + + " \001(\0132\023.hbase.pb.Condition\"\226\001\n\rMultiRespo", + "nse\0228\n\022regionActionResult\030\001 \003(\0132\034.hbase." + + "pb.RegionActionResult\022\021\n\tprocessed\030\002 \001(\010" + + "\0228\n\020regionStatistics\030\003 \001(\0132\036.hbase.pb.Mu" + + "ltiRegionLoadStats*\'\n\013Consistency\022\n\n\006STR" + + "ONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientService\0222\n" + + "\003Get\022\024.hbase.pb.GetRequest\032\025.hbase.pb.Ge" + + "tResponse\022;\n\006Mutate\022\027.hbase.pb.MutateReq" + + "uest\032\030.hbase.pb.MutateResponse\0225\n\004Scan\022\025" + + ".hbase.pb.ScanRequest\032\026.hbase.pb.ScanRes" + + "ponse\022P\n\rBulkLoadHFile\022\036.hbase.pb.BulkLo", + "adHFileRequest\032\037.hbase.pb.BulkLoadHFileR" + + "esponse\022X\n\013ExecService\022#.hbase.pb.Coproc" + + "essorServiceRequest\032$.hbase.pb.Coprocess" + + "orServiceResponse\022d\n\027ExecRegionServerSer" + + "vice\022#.hbase.pb.CoprocessorServiceReques" + + "t\032$.hbase.pb.CoprocessorServiceResponse\022" + + "8\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbase" + + ".pb.MultiResponseBB\n*org.apache.hadoop.h" + + "base.protobuf.generatedB\014ClientProtosH\001\210" + + "\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -36079,13 +36576,13 @@ public final class ClientProtos { internal_static_hbase_pb_Scan_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_Scan_descriptor, - new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", }); + new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", "IncludeStartRow", "IncludeStopRow", "ReadType", }); internal_static_hbase_pb_ScanRequest_descriptor = getDescriptor().getMessageTypes().get(12); internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ScanRequest_descriptor, - new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", }); + new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", "LimitOfRows", }); internal_static_hbase_pb_ScanResponse_descriptor = getDescriptor().getMessageTypes().get(13); internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index f330ea4829c..4fb64bd85c6 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -260,6 +260,15 @@ message Scan { optional bool allow_partial_results = 18; repeated ColumnFamilyTimeRange cf_time_range = 19; optional uint64 mvcc_read_point = 20 [default = 0]; + optional bool include_start_row = 21 [default = true]; + optional bool include_stop_row = 22 [default = false]; + + enum ReadType { + DEFAULT = 0; + STREAM = 1; + PREAD = 2; + } + optional ReadType readType = 23 [default = DEFAULT]; } /** @@ -284,6 +293,8 @@ message ScanRequest { optional bool client_handles_heartbeats = 8; optional bool track_scan_metrics = 9; optional bool renew = 10 [default = false]; + // if we have returned limit_of_rows rows to client, then close the scanner. + optional uint32 limit_of_rows = 11 [default = 0]; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java index b19b4825464..27518ca9087 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java @@ -63,27 +63,27 @@ public class SyncTable extends Configured implements Tool { static final String SOURCE_ZK_CLUSTER_CONF_KEY = "sync.table.source.zk.cluster"; static final String TARGET_ZK_CLUSTER_CONF_KEY = "sync.table.target.zk.cluster"; static final String DRY_RUN_CONF_KEY="sync.table.dry.run"; - + Path sourceHashDir; String sourceTableName; String targetTableName; - + String sourceZkCluster; String targetZkCluster; boolean dryRun; - + Counters counters; - + public SyncTable(Configuration conf) { super(conf); } - + public Job createSubmittableJob(String[] args) throws IOException { FileSystem fs = sourceHashDir.getFileSystem(getConf()); if (!fs.exists(sourceHashDir)) { throw new IOException("Source hash dir not found: " + sourceHashDir); } - + HashTable.TableHash tableHash = HashTable.TableHash.read(getConf(), sourceHashDir); LOG.info("Read source hash manifest: " + tableHash); LOG.info("Read " + tableHash.partitions.size() + " partition keys"); @@ -97,7 +97,7 @@ public class SyncTable extends Configured implements Tool { + " says numHashFiles=" + tableHash.numHashFiles + " but the number of partition keys" + " found in the partitions file is " + tableHash.partitions.size()); } - + Path dataDir = new Path(sourceHashDir, HashTable.HASH_DATA_DIR); int dataSubdirCount = 0; for (FileStatus file : fs.listStatus(dataDir)) { @@ -105,14 +105,14 @@ public class SyncTable extends Configured implements Tool { dataSubdirCount++; } } - + if (dataSubdirCount != tableHash.numHashFiles) { throw new RuntimeException("Hash data appears corrupt. The number of of hash files created" + " should be 1 more than the number of partition keys. However, the number of data dirs" + " found is " + dataSubdirCount + " but the number of partition keys" + " found in the partitions file is " + tableHash.partitions.size()); } - + Job job = Job.getInstance(getConf(),getConf().get("mapreduce.job.name", "syncTable_" + sourceTableName + "-" + targetTableName)); Configuration jobConf = job.getConfiguration(); @@ -127,12 +127,12 @@ public class SyncTable extends Configured implements Tool { jobConf.set(TARGET_ZK_CLUSTER_CONF_KEY, targetZkCluster); } jobConf.setBoolean(DRY_RUN_CONF_KEY, dryRun); - + TableMapReduceUtil.initTableMapperJob(targetTableName, tableHash.initScan(), SyncMapper.class, null, null, job); - + job.setNumReduceTasks(0); - + if (dryRun) { job.setOutputFormatClass(NullOutputFormat.class); } else { @@ -140,37 +140,37 @@ public class SyncTable extends Configured implements Tool { // because it sets up the TableOutputFormat. TableMapReduceUtil.initTableReducerJob(targetTableName, null, job, null, targetZkCluster, null, null); - + // would be nice to add an option for bulk load instead } - + return job; } - + public static class SyncMapper extends TableMapper { Path sourceHashDir; - + Connection sourceConnection; Connection targetConnection; Table sourceTable; Table targetTable; boolean dryRun; - + HashTable.TableHash sourceTableHash; HashTable.TableHash.Reader sourceHashReader; ImmutableBytesWritable currentSourceHash; ImmutableBytesWritable nextSourceKey; HashTable.ResultHasher targetHasher; - + Throwable mapperException; - + public static enum Counter {BATCHES, HASHES_MATCHED, HASHES_NOT_MATCHED, SOURCEMISSINGROWS, SOURCEMISSINGCELLS, TARGETMISSINGROWS, TARGETMISSINGCELLS, ROWSWITHDIFFS, DIFFERENTCELLVALUES, MATCHINGROWS, MATCHINGCELLS, EMPTY_BATCHES, RANGESMATCHED, RANGESNOTMATCHED}; - + @Override protected void setup(Context context) throws IOException { - + Configuration conf = context.getConfiguration(); sourceHashDir = new Path(conf.get(SOURCE_HASH_DIR_CONF_KEY)); sourceConnection = openConnection(conf, SOURCE_ZK_CLUSTER_CONF_KEY, null); @@ -179,23 +179,23 @@ public class SyncTable extends Configured implements Tool { sourceTable = openTable(sourceConnection, conf, SOURCE_TABLE_CONF_KEY); targetTable = openTable(targetConnection, conf, TARGET_TABLE_CONF_KEY); dryRun = conf.getBoolean(SOURCE_TABLE_CONF_KEY, false); - + sourceTableHash = HashTable.TableHash.read(conf, sourceHashDir); LOG.info("Read source hash manifest: " + sourceTableHash); LOG.info("Read " + sourceTableHash.partitions.size() + " partition keys"); - + TableSplit split = (TableSplit) context.getInputSplit(); ImmutableBytesWritable splitStartKey = new ImmutableBytesWritable(split.getStartRow()); - + sourceHashReader = sourceTableHash.newReader(conf, splitStartKey); findNextKeyHashPair(); - + // create a hasher, but don't start it right away // instead, find the first hash batch at or after the start row // and skip any rows that come before. they will be caught by the previous task targetHasher = new HashTable.ResultHasher(); } - + private static Connection openConnection(Configuration conf, String zkClusterConfKey, String configPrefix) throws IOException { @@ -204,12 +204,12 @@ public class SyncTable extends Configured implements Tool { zkCluster, configPrefix); return ConnectionFactory.createConnection(clusterConf); } - + private static Table openTable(Connection connection, Configuration conf, String tableNameConfKey) throws IOException { return connection.getTable(TableName.valueOf(conf.get(tableNameConfKey))); } - + /** * Attempt to read the next source key/hash pair. * If there are no more, set nextSourceKey to null @@ -223,7 +223,7 @@ public class SyncTable extends Configured implements Tool { nextSourceKey = null; } } - + @Override protected void map(ImmutableBytesWritable key, Result value, Context context) throws IOException, InterruptedException { @@ -232,7 +232,7 @@ public class SyncTable extends Configured implements Tool { while (nextSourceKey != null && key.compareTo(nextSourceKey) >= 0) { moveToNextBatch(context); } - + // next, add the scanned row (as long as we've reached the first batch) if (targetHasher.isBatchStarted()) { targetHasher.hashResult(value); @@ -247,7 +247,7 @@ public class SyncTable extends Configured implements Tool { /** * If there is an open hash batch, complete it and sync if there are diffs. - * Start a new batch, and seek to read the + * Start a new batch, and seek to read the */ private void moveToNextBatch(Context context) throws IOException, InterruptedException { if (targetHasher.isBatchStarted()) { @@ -255,7 +255,7 @@ public class SyncTable extends Configured implements Tool { } targetHasher.startBatch(nextSourceKey); currentSourceHash = sourceHashReader.getCurrentHash(); - + findNextKeyHashPair(); } @@ -276,28 +276,28 @@ public class SyncTable extends Configured implements Tool { context.getCounter(Counter.HASHES_MATCHED).increment(1); } else { context.getCounter(Counter.HASHES_NOT_MATCHED).increment(1); - + ImmutableBytesWritable stopRow = nextSourceKey == null ? new ImmutableBytesWritable(sourceTableHash.stopRow) : nextSourceKey; - + if (LOG.isDebugEnabled()) { LOG.debug("Hash mismatch. Key range: " + toHex(targetHasher.getBatchStartKey()) + " to " + toHex(stopRow) + " sourceHash: " + toHex(currentSourceHash) + " targetHash: " + toHex(targetHash)); } - + syncRange(context, targetHasher.getBatchStartKey(), stopRow); } } private static String toHex(ImmutableBytesWritable bytes) { return Bytes.toHex(bytes.get(), bytes.getOffset(), bytes.getLength()); } - + private static final CellScanner EMPTY_CELL_SCANNER = new CellScanner(Iterators.emptyIterator()); - + /** * Rescan the given range directly from the source and target tables. * Count and log differences, and if this is not a dry run, output Puts and Deletes @@ -305,17 +305,16 @@ public class SyncTable extends Configured implements Tool { */ private void syncRange(Context context, ImmutableBytesWritable startRow, ImmutableBytesWritable stopRow) throws IOException, InterruptedException { - Scan scan = sourceTableHash.initScan(); scan.setStartRow(startRow.copyBytes()); scan.setStopRow(stopRow.copyBytes()); - + ResultScanner sourceScanner = sourceTable.getScanner(scan); CellScanner sourceCells = new CellScanner(sourceScanner.iterator()); - ResultScanner targetScanner = targetTable.getScanner(scan); + ResultScanner targetScanner = targetTable.getScanner(new Scan(scan)); CellScanner targetCells = new CellScanner(targetScanner.iterator()); - + boolean rangeMatched = true; byte[] nextSourceRow = sourceCells.nextRow(); byte[] nextTargetRow = targetCells.nextRow(); @@ -327,7 +326,7 @@ public class SyncTable extends Configured implements Tool { LOG.info("Target missing row: " + Bytes.toHex(nextSourceRow)); } context.getCounter(Counter.TARGETMISSINGROWS).increment(1); - + rowMatched = syncRowCells(context, nextSourceRow, sourceCells, EMPTY_CELL_SCANNER); nextSourceRow = sourceCells.nextRow(); // advance only source to next row } else if (rowComparison > 0) { @@ -335,41 +334,41 @@ public class SyncTable extends Configured implements Tool { LOG.info("Source missing row: " + Bytes.toHex(nextTargetRow)); } context.getCounter(Counter.SOURCEMISSINGROWS).increment(1); - + rowMatched = syncRowCells(context, nextTargetRow, EMPTY_CELL_SCANNER, targetCells); nextTargetRow = targetCells.nextRow(); // advance only target to next row } else { // current row is the same on both sides, compare cell by cell rowMatched = syncRowCells(context, nextSourceRow, sourceCells, targetCells); - nextSourceRow = sourceCells.nextRow(); + nextSourceRow = sourceCells.nextRow(); nextTargetRow = targetCells.nextRow(); } - + if (!rowMatched) { rangeMatched = false; } } - + sourceScanner.close(); targetScanner.close(); - + context.getCounter(rangeMatched ? Counter.RANGESMATCHED : Counter.RANGESNOTMATCHED) .increment(1); } - + private static class CellScanner { private final Iterator results; - + private byte[] currentRow; private Result currentRowResult; private int nextCellInRow; - + private Result nextRowResult; - + public CellScanner(Iterator results) { this.results = results; } - + /** * Advance to the next row and return its row key. * Returns null iff there are no more rows. @@ -390,7 +389,7 @@ public class SyncTable extends Configured implements Tool { nextRowResult = null; } } - + if (nextRowResult == null) { // end of data, no more rows currentRowResult = null; @@ -398,7 +397,7 @@ public class SyncTable extends Configured implements Tool { return null; } } - + // advance to cached result for next row currentRowResult = nextRowResult; nextCellInRow = 0; @@ -406,7 +405,7 @@ public class SyncTable extends Configured implements Tool { nextRowResult = null; return currentRow; } - + /** * Returns the next Cell in the current row or null iff none remain. */ @@ -415,7 +414,7 @@ public class SyncTable extends Configured implements Tool { // nothing left in current row return null; } - + Cell nextCell = currentRowResult.rawCells()[nextCellInRow]; nextCellInRow++; if (nextCellInRow == currentRowResult.size()) { @@ -441,7 +440,7 @@ public class SyncTable extends Configured implements Tool { return nextCell; } } - + /** * Compare the cells for the given row from the source and target tables. * Count and log any differences. @@ -465,14 +464,14 @@ public class SyncTable extends Configured implements Tool { } context.getCounter(Counter.TARGETMISSINGCELLS).increment(1); matchingRow = false; - + if (!dryRun) { if (put == null) { put = new Put(rowKey); } put.add(sourceCell); } - + sourceCell = sourceCells.nextCellInRow(); } else if (cellKeyComparison > 0) { if (LOG.isDebugEnabled()) { @@ -480,7 +479,7 @@ public class SyncTable extends Configured implements Tool { } context.getCounter(Counter.SOURCEMISSINGCELLS).increment(1); matchingRow = false; - + if (!dryRun) { if (delete == null) { delete = new Delete(rowKey); @@ -489,7 +488,7 @@ public class SyncTable extends Configured implements Tool { delete.addColumn(CellUtil.cloneFamily(targetCell), CellUtil.cloneQualifier(targetCell), targetCell.getTimestamp()); } - + targetCell = targetCells.nextCellInRow(); } else { // the cell keys are equal, now check values @@ -507,7 +506,7 @@ public class SyncTable extends Configured implements Tool { } context.getCounter(Counter.DIFFERENTCELLVALUES).increment(1); matchingRow = false; - + if (!dryRun) { // overwrite target cell if (put == null) { @@ -519,7 +518,7 @@ public class SyncTable extends Configured implements Tool { sourceCell = sourceCells.nextCellInRow(); targetCell = targetCells.nextCellInRow(); } - + if (!dryRun && sourceTableHash.scanBatch > 0) { if (put != null && put.size() >= sourceTableHash.scanBatch) { context.write(new ImmutableBytesWritable(rowKey), put); @@ -531,7 +530,7 @@ public class SyncTable extends Configured implements Tool { } } } - + if (!dryRun) { if (put != null) { context.write(new ImmutableBytesWritable(rowKey), put); @@ -540,7 +539,7 @@ public class SyncTable extends Configured implements Tool { context.write(new ImmutableBytesWritable(rowKey), delete); } } - + if (matchingCells > 0) { context.getCounter(Counter.MATCHINGCELLS).increment(matchingCells); } @@ -580,21 +579,21 @@ public class SyncTable extends Configured implements Tool { if (c2 == null) { return -1; // target missing cell } - + int result = CellComparator.compareFamilies(c1, c2); if (result != 0) { return result; } - + result = CellComparator.compareQualifiers(c1, c2); if (result != 0) { return result; } - + // note timestamp comparison is inverted - more recent cells first return CellComparator.compareTimestamps(c1, c2); } - + @Override protected void cleanup(Context context) throws IOException, InterruptedException { @@ -605,7 +604,7 @@ public class SyncTable extends Configured implements Tool { mapperException = t; } } - + try { sourceTable.close(); targetTable.close(); @@ -618,7 +617,7 @@ public class SyncTable extends Configured implements Tool { LOG.error("Suppressing exception from closing tables", t); } } - + // propagate first exception if (mapperException != null) { Throwables.propagateIfInstanceOf(mapperException, IOException.class); @@ -638,7 +637,7 @@ public class SyncTable extends Configured implements Tool { && (nextSourceKey.compareTo(splitEndRow) < 0 || reachedEndOfTable)) { moveToNextBatch(context); } - + if (targetHasher.isBatchStarted()) { // need to complete the final open hash batch @@ -653,7 +652,7 @@ public class SyncTable extends Configured implements Tool { } else { scan.setStopRow(nextSourceKey.copyBytes()); } - + ResultScanner targetScanner = null; try { targetScanner = targetTable.getScanner(scan); @@ -671,7 +670,7 @@ public class SyncTable extends Configured implements Tool { } } } - + private static final int NUM_ARGS = 3; private static void printUsage(final String errorMsg) { if (errorMsg != null && errorMsg.length() > 0) { @@ -681,7 +680,7 @@ public class SyncTable extends Configured implements Tool { System.err.println("Usage: SyncTable [options] "); System.err.println(); System.err.println("Options:"); - + System.err.println(" sourcezkcluster ZK cluster key of the source table"); System.err.println(" (defaults to cluster in classpath's config)"); System.err.println(" targetzkcluster ZK cluster key of the target table"); @@ -703,7 +702,7 @@ public class SyncTable extends Configured implements Tool { + " --sourcezkcluster=zk1.example.com,zk2.example.com,zk3.example.com:2181:/hbase" + " hdfs://nn:9000/hashes/tableA tableA tableA"); } - + private boolean doCommandLine(final String[] args) { if (args.length < NUM_ARGS) { printUsage(null); @@ -713,37 +712,37 @@ public class SyncTable extends Configured implements Tool { sourceHashDir = new Path(args[args.length - 3]); sourceTableName = args[args.length - 2]; targetTableName = args[args.length - 1]; - + for (int i = 0; i < args.length - NUM_ARGS; i++) { String cmd = args[i]; if (cmd.equals("-h") || cmd.startsWith("--h")) { printUsage(null); return false; } - + final String sourceZkClusterKey = "--sourcezkcluster="; if (cmd.startsWith(sourceZkClusterKey)) { sourceZkCluster = cmd.substring(sourceZkClusterKey.length()); continue; } - + final String targetZkClusterKey = "--targetzkcluster="; if (cmd.startsWith(targetZkClusterKey)) { targetZkCluster = cmd.substring(targetZkClusterKey.length()); continue; } - + final String dryRunKey = "--dryrun="; if (cmd.startsWith(dryRunKey)) { dryRun = Boolean.parseBoolean(cmd.substring(dryRunKey.length())); continue; } - + printUsage("Invalid argument '" + cmd + "'"); return false; } - + } catch (Exception e) { e.printStackTrace(); printUsage("Can't start because " + e.getMessage()); @@ -751,7 +750,7 @@ public class SyncTable extends Configured implements Tool { } return true; } - + /** * Main entry point. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index c5a850b9c4e..7339d87041c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -274,11 +274,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, private final String scannerName; private final RegionScanner s; private final Region r; + private final boolean allowPartial; - public RegionScannerHolder(String scannerName, RegionScanner s, Region r) { + public RegionScannerHolder(String scannerName, RegionScanner s, Region r, boolean allowPartial) { this.scannerName = scannerName; this.s = s; this.r = r; + this.allowPartial = allowPartial; } public long getNextCallSeq() { @@ -1110,11 +1112,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return lastBlock; } - private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r) - throws LeaseStillHeldException { + private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r, + boolean allowPartial) throws LeaseStillHeldException { regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod, new ScannerListener(scannerName)); - RegionScannerHolder rsh = new RegionScannerHolder(scannerName, s, r); + RegionScannerHolder rsh = new RegionScannerHolder(scannerName, s, r, allowPartial); RegionScannerHolder existing = scanners.putIfAbsent(scannerName, rsh); assert existing == null : "scannerId must be unique within regionserver's whole lifecycle!"; return rsh; @@ -2460,8 +2462,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return rsh; } - private Pair newRegionScanner(ScanRequest request, - ScanResponse.Builder builder) throws IOException { + private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder) + throws IOException { Region region = getRegion(request.getRegion()); ClientProtos.Scan protoScan = request.getScan(); boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand(); @@ -2491,7 +2493,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, builder.setMvccReadPoint(scanner.getMvccReadPoint()); builder.setTtl(scannerLeaseTimeoutPeriod); String scannerName = String.valueOf(scannerId); - return Pair.newPair(addScanner(scannerName, scanner, region), scan.isSmall()); + return addScanner(scannerName, scanner, region, + !scan.isSmall() && !(request.hasLimitOfRows() && request.getLimitOfRows() > 0)); } private void checkScanNextCallSeq(ScanRequest request, RegionScannerHolder rsh) @@ -2548,9 +2551,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // return whether we have more results in region. private boolean scan(PayloadCarryingRpcController controller, ScanRequest request, - RegionScannerHolder rsh, boolean isSmallScan, long maxQuotaResultSize, int rows, - List results, ScanResponse.Builder builder, MutableObject lastBlock, - RpcCallContext context) throws IOException { + RegionScannerHolder rsh, long maxQuotaResultSize, int rows, List results, + ScanResponse.Builder builder, MutableObject lastBlock, RpcCallContext context) + throws IOException { Region region = rsh.r; RegionScanner scanner = rsh.s; long maxResultSize; @@ -2581,7 +2584,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // formed. boolean serverGuaranteesOrderOfPartials = results.isEmpty(); boolean allowPartialResults = - clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan; + clientHandlesPartials && serverGuaranteesOrderOfPartials && rsh.allowPartial; boolean moreRows = false; // Heartbeat messages occur when the processing of the ScanRequest is exceeds a @@ -2738,15 +2741,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler, rpcScanRequestCount.increment(); RegionScannerHolder rsh; ScanResponse.Builder builder = ScanResponse.newBuilder(); - boolean isSmallScan; try { if (request.hasScannerId()) { rsh = getRegionScanner(request); - isSmallScan = false; } else { - Pair pair = newRegionScanner(request, builder); - rsh = pair.getFirst(); - isSmallScan = pair.getSecond().booleanValue(); + rsh = newRegionScanner(request, builder); } } catch (IOException e) { if (e == SCANNER_ALREADY_CLOSED) { @@ -2805,6 +2804,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler, RegionScanner scanner = rsh.s; boolean moreResults = true; boolean moreResultsInRegion = true; + // this is the limit of rows for this scan, if we the number of rows reach this value, we will + // close the scanner. + int limitOfRows; + if (request.hasLimitOfRows()) { + limitOfRows = request.getLimitOfRows(); + rows = Math.min(rows, limitOfRows); + } else { + limitOfRows = -1; + } MutableObject lastBlock = new MutableObject(); boolean scannerClosed = false; try { @@ -2825,7 +2833,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } if (!done) { moreResultsInRegion = scan((PayloadCarryingRpcController) controller, request, rsh, - isSmallScan, maxQuotaResultSize, rows, results, builder, lastBlock, context); + maxQuotaResultSize, rows, results, builder, lastBlock, context); } } @@ -2837,6 +2845,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // with the old scan implementation where we just ignore the returned results if moreResults // is false. Can remove the isEmpty check after we get rid of the old implementation. moreResults = false; + } else if (limitOfRows > 0 && results.size() >= limitOfRows && + !results.get(results.size() - 1).isPartial()) { + // if we have reached the limit of rows + moreResults = false; } addResults(builder, results, (PayloadCarryingRpcController) controller, RegionReplicaUtil.isDefaultReplica(region.getRegionInfo())); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java index 4fa987508e3..419b7c507c8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java @@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.Scan.ReadType; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.tool.Canary.RegionTask.TaskType; import org.apache.hadoop.hbase.util.Bytes; @@ -365,7 +366,7 @@ public final class Canary implements Tool { scan.setFilter(new FirstKeyOnlyFilter()); scan.addFamily(column.getName()); scan.setMaxResultSize(1L); - scan.setSmall(true); + scan.setOneRowLimit(); } if (LOG.isDebugEnabled()) { @@ -500,7 +501,7 @@ public final class Canary implements Tool { scan.setFilter(new FirstKeyOnlyFilter()); scan.setCaching(1); scan.setMaxResultSize(1L); - scan.setSmall(true); + scan.setOneRowLimit(); stopWatch.start(); ResultScanner s = table.getScanner(scan); s.next(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index f3becfe7d5c..6b785984d65 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -2427,14 +2427,17 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * Return the number of rows in the given table. */ public int countRows(final Table table) throws IOException { - Scan scan = new Scan(); - ResultScanner results = table.getScanner(scan); - int count = 0; - for (@SuppressWarnings("unused") Result res : results) { - count++; + return countRows(table, new Scan()); + } + + public int countRows(final Table table, final Scan scan) throws IOException { + try (ResultScanner results = table.getScanner(scan)) { + int count = 0; + while (results.next() != null) { + count++; + } + return count; } - results.close(); - return count; } public int countRows(final Table table, final byte[]... families) throws IOException { @@ -2442,13 +2445,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { for (byte[] family: families) { scan.addFamily(family); } - ResultScanner results = table.getScanner(scan); - int count = 0; - for (@SuppressWarnings("unused") Result res : results) { - count++; - } - results.close(); - return count; + return countRows(table, scan); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java index 3bf91a4a1a9..df38b8e29ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java @@ -166,7 +166,7 @@ public class TestMetaTableAccessorNoCluster { public ScanResponse answer(InvocationOnMock invocation) throws Throwable { ((PayloadCarryingRpcController) invocation.getArguments()[0]).setCellScanner(CellUtil .createCellScanner(cellScannables)); - return builder.build(); + return builder.setScannerId(1234567890L).build(); } }).thenReturn(ScanResponse.newBuilder().setMoreResults(false).build()); // Associate a spied-upon HConnection with UTIL.getConfiguration. Need diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java index ed8e848cc03..181d55a3cb4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java @@ -607,7 +607,6 @@ public class TestPartialResultsFromClientSide { scan.setAllowPartialResults(true); scan.setSmall(true); scan.setMaxResultSize(1); - ResultScanner scanner = TABLE.getScanner(scan); Result r = null; @@ -731,11 +730,13 @@ public class TestPartialResultsFromClientSide { byte[] value = Bytes.createMaxByteArray(100); Table tmpTable = createTestTable(testName, rows, families, qualifiers, value); - // Open scanner before deletes ResultScanner scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true)); - + // now the openScanner will also fetch data and will be executed lazily, i.e, only openScanner + // when you call next, so here we need to make a next call to open scanner. The maxResultSize + // limit can make sure that we will not fetch all the data at once, so the test sill works. + int scannerCount = scanner.next().rawCells().length; Delete delete1 = new Delete(rows[0]); delete1.addColumn(families[0], qualifiers[0], 0); tmpTable.delete(delete1); @@ -745,7 +746,7 @@ public class TestPartialResultsFromClientSide { tmpTable.delete(delete2); // Should see all cells because scanner was opened prior to deletes - int scannerCount = countCellsFromScanner(scanner); + scannerCount += countCellsFromScanner(scanner); int expectedCount = numRows * numFamilies * numQualifiers; assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount, scannerCount == expectedCount); @@ -758,6 +759,7 @@ public class TestPartialResultsFromClientSide { scannerCount == expectedCount); scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true)); + scannerCount = scanner.next().rawCells().length; // Put in 2 new rows. The timestamps differ from the deleted rows Put put1 = new Put(rows[0]); put1.add(new KeyValue(rows[0], families[0], qualifiers[0], 1, value)); @@ -768,7 +770,7 @@ public class TestPartialResultsFromClientSide { tmpTable.put(put2); // Scanner opened prior to puts. Cell count shouldn't have changed - scannerCount = countCellsFromScanner(scanner); + scannerCount += countCellsFromScanner(scanner); expectedCount = numRows * numFamilies * numQualifiers - 2; assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount, scannerCount == expectedCount); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java index 4a3b15253df..803a81da3a2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java @@ -85,9 +85,11 @@ public class TestClientScannerRPCTimeout { public void testScannerNextRPCTimesout() throws Exception { final TableName TABLE_NAME = TableName.valueOf("testScannerNextRPCTimesout"); Table ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY); + byte[] r0 = Bytes.toBytes("row-0"); byte[] r1 = Bytes.toBytes("row-1"); byte[] r2 = Bytes.toBytes("row-2"); byte[] r3 = Bytes.toBytes("row-3"); + putToTable(ht, r0); putToTable(ht, r1); putToTable(ht, r2); putToTable(ht, r3); @@ -97,6 +99,9 @@ public class TestClientScannerRPCTimeout { scan.setCaching(1); ResultScanner scanner = ht.getScanner(scan); Result result = scanner.next(); + // fetched when openScanner + assertTrue("Expected row: row-0", Bytes.equals(r0, result.getRow())); + result = scanner.next(); assertTrue("Expected row: row-1", Bytes.equals(r1, result.getRow())); LOG.info("Got expected first row"); long t1 = System.currentTimeMillis(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 69fdf6a0bd9..713688c8170 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -6108,7 +6108,7 @@ public class TestFromClientSide { public void testReversedScanUnderMultiRegions() throws Exception { // Test Initialization. TableName TABLE = TableName.valueOf("testReversedScanUnderMultiRegions"); - byte[] maxByteArray = ReversedClientScanner.MAX_BYTE_ARRAY; + byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY; byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), Bytes.toBytes("006"), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java index 3333f54be0f..288872e7977 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java @@ -106,8 +106,8 @@ public class TestLeaseRenewal { Scan s = new Scan(); s.setCaching(1); ResultScanner rs = table.getScanner(s); - // make sure that calling renewLease does not impact the scan results - assertTrue(((AbstractClientScanner)rs).renewLease()); + // we haven't open the scanner yet so nothing happens + assertFalse(((AbstractClientScanner) rs).renewLease()); assertTrue(Arrays.equals(rs.next().getRow(), ANOTHERROW)); // renew the lease a few times, long enough to be sure // the lease would have expired otherwise diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index e2ca677db84..261915e8733 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -169,7 +169,7 @@ public class TestRegionServerMetrics { // By default, master doesn't host meta now. // Adding some meta related requests - requests += 3; + requests += 1; readRequests ++; metricsRegionServer.getRegionServerWrapper().forceRecompute(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java index 1d7ec464820..6febd45b20d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; -import junit.framework.Assert; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -199,6 +198,7 @@ public class TestScannerWithBulkload { HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); createTable(admin, tableName); Scan scan = createScan(); + scan.setCaching(1); final HTable table = init(admin, l, scan, tableName); // use bulkload final Path hfilePath = writeToHFile(l, "/temp/testBulkLoadWithParallelScan/", @@ -207,6 +207,7 @@ public class TestScannerWithBulkload { conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true); final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf); ResultScanner scanner = table.getScanner(scan); + Result result = scanner.next(); // Create a scanner and then do bulk load final CountDownLatch latch = new CountDownLatch(1); new Thread() { @@ -226,7 +227,6 @@ public class TestScannerWithBulkload { latch.await(); // By the time we do next() the bulk loaded files are also added to the kv // scanner - Result result = scanner.next(); scanAfterBulkLoad(scanner, result, "version1"); scanner.close(); table.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 79d65cda4b5..1c1ab398c93 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -947,7 +947,6 @@ public class TestAccessController extends SecureTestUtil { for (Result r = scanner.next(); r != null; r = scanner.next()) { // do nothing } - } catch (IOException e) { } finally { scanner.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java index d400fa644fc..ec8bc955f1d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java @@ -381,6 +381,7 @@ public class TestAccessController2 extends SecureTestUtil { Scan s1 = new Scan(); s1.addFamily(TEST_FAMILY_2); try (ResultScanner scanner1 = table.getScanner(s1);) { + scanner1.next(); } } return null; @@ -411,6 +412,7 @@ public class TestAccessController2 extends SecureTestUtil { Scan s1 = new Scan(); s1.addFamily(TEST_FAMILY_2); try (ResultScanner scanner1 = table.getScanner(s1);) { + scanner1.next(); } } return null; @@ -425,6 +427,7 @@ public class TestAccessController2 extends SecureTestUtil { Scan s1 = new Scan(); s1.addColumn(TEST_FAMILY, Q2); try (ResultScanner scanner1 = table.getScanner(s1);) { + scanner1.next(); } } return null;