From 24bbbbc83d3c5409133f649e25c69f29656e32ed Mon Sep 17 00:00:00 2001 From: Jonathan Lawlor Date: Wed, 4 Feb 2015 14:07:35 -0800 Subject: [PATCH] HBASE-11544: [Ergonomics] hbase.client.scanner.caching is dogged and will try to return batch even if it means OOME --- .../apache/hadoop/hbase/client/ClientScanner.java | 210 ++++- .../org/apache/hadoop/hbase/client/Result.java | 78 +- .../hadoop/hbase/client/ReversedClientScanner.java | 7 +- .../java/org/apache/hadoop/hbase/client/Scan.java | 34 + .../apache/hadoop/hbase/protobuf/ProtobufUtil.java | 6 +- .../hadoop/hbase/protobuf/RequestConverter.java | 3 + .../hadoop/hbase/protobuf/ResponseConverter.java | 5 +- .../java/org/apache/hadoop/hbase/HConstants.java | 2 +- hbase-common/src/main/resources/hbase-default.xml | 2 +- .../hbase/protobuf/generated/ClientProtos.java | 983 ++++++++++++++++----- hbase-protocol/src/main/protobuf/Client.proto | 26 +- .../hadoop/hbase/rest/ScannerInstanceResource.java | 3 +- .../hbase/client/ClientSideRegionScanner.java | 7 +- .../hadoop/hbase/client/TableSnapshotScanner.java | 4 +- .../apache/hadoop/hbase/regionserver/HRegion.java | 184 +++- .../hadoop/hbase/regionserver/InternalScanner.java | 13 +- .../hadoop/hbase/regionserver/KeyValueHeap.java | 8 +- .../hadoop/hbase/regionserver/RSRpcServices.java | 49 +- .../hadoop/hbase/regionserver/RegionScanner.java | 81 +- .../hadoop/hbase/regionserver/StoreScanner.java | 34 +- .../hbase/TestPartialResultsFromClientSide.java | 789 +++++++++++++++++ .../coprocessor/TestCoprocessorInterface.java | 22 +- .../coprocessor/TestRegionObserverInterface.java | 13 +- .../hbase/regionserver/TestStripeCompactor.java | 27 +- .../compactions/TestStripeCompactionPolicy.java | 9 +- 25 files changed, 2230 insertions(+), 369 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java 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 110b039..ca0fa72 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,9 +17,13 @@ */ package org.apache.hadoop.hbase.client; +import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosestRowBefore; + import java.io.IOException; import java.io.InterruptedIOException; +import java.util.ArrayList; import java.util.LinkedList; +import java.util.List; import java.util.concurrent.ExecutorService; import org.apache.commons.logging.Log; @@ -44,8 +48,6 @@ import org.apache.hadoop.hbase.util.Bytes; import com.google.common.annotations.VisibleForTesting; -import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosestRowBefore; - /** * Implements the scanner interface for the HBase client. * If there are multiple regions in a table, this scanner will iterate @@ -61,10 +63,19 @@ public class ClientScanner extends AbstractClientScanner { protected HRegionInfo currentRegion = null; protected ScannerCallableWithReplicas callable = null; protected final LinkedList cache = new LinkedList(); + /** + * A list of partial results that have been returned from the server. This list should only + * contain results if this scanner does not have enough partial results to form the complete + * result. + */ + protected final LinkedList partialResults = new LinkedList(); protected final int caching; protected long lastNext; // Keep lastResult returned successfully in case we have to reset scanner. protected Result lastResult = null; + // Keep a flag to tell us whether or not a partial result was returned from the last + // RPC request + protected boolean partialResultReturned = false; protected final long maxScannerResultSize; private final ClusterConnection connection; private final TableName tableName; @@ -337,21 +348,26 @@ public class ClientScanner extends AbstractClientScanner { return null; } if (cache.size() == 0) { - Result [] values = null; + Result[] values = null; long remainingResultSize = maxScannerResultSize; int countdown = this.caching; + // We need to reset it if it's a new callable that was created // with a countdown in nextScanner callable.setCaching(this.caching); - // This flag is set when we want to skip the result returned. We do + // 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; + boolean retryAfterOutOfOrderException = true; do { try { - // Server returns a null values if scanning is to stop. Else, + // Reset flag before each RPC request + partialResultReturned = false; + + // 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); + // 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 @@ -362,9 +378,13 @@ public class ClientScanner extends AbstractClientScanner { this.currentRegion = callable.getHRegionInfo(); continue; } - retryAfterOutOfOrderException = true; + retryAfterOutOfOrderException = true; } catch (DoNotRetryIOException e) { - // DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us + // 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. + partialResults.clear(); + + // 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 (e instanceof UnknownScannerException) { long timeout = lastNext + scannerTimeout; @@ -373,9 +393,9 @@ public class ClientScanner extends AbstractClientScanner { // id against the new region server; reset the scanner. if (timeout < System.currentTimeMillis()) { long elapsed = System.currentTimeMillis() - lastNext; - ScannerTimeoutException ex = new ScannerTimeoutException( - elapsed + "ms passed since the last invocation, " + - "timeout is currently set to " + scannerTimeout); + ScannerTimeoutException ex = + new ScannerTimeoutException(elapsed + "ms passed since the last invocation, " + + "timeout is currently set to " + scannerTimeout); ex.initCause(e); throw ex; } @@ -383,9 +403,9 @@ public class ClientScanner extends AbstractClientScanner { // 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) { + if ((cause != null && cause instanceof NotServingRegionException) + || (cause != null && cause instanceof RegionServerStoppedException) + || e instanceof OutOfOrderScannerNextException) { // 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. @@ -401,9 +421,9 @@ public class ClientScanner extends AbstractClientScanner { // scanner starts at the correct row. Otherwise we may see previously // returned rows again. // (ScannerCallable by now has "relocated" the correct region) - if(scan.isReversed()){ + if (scan.isReversed()) { scan.setStartRow(createClosestRowBefore(lastResult.getRow())); - }else { + } else { scan.setStartRow(Bytes.add(lastResult.getRow(), new byte[1])); } } @@ -412,8 +432,8 @@ public class ClientScanner extends AbstractClientScanner { 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); + throw new DoNotRetryIOException("Failed after retry of " + + "OutOfOrderScannerNextException: was there a rpc timeout?", e); } } // Clear region. @@ -421,16 +441,18 @@ public class ClientScanner extends AbstractClientScanner { // 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; + // This continue will take us to while at end of loop where we will set up new scanner. continue; } long currentTime = System.currentTimeMillis(); if (this.scanMetrics != null) { - this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime-lastNext); + this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext); } lastNext = currentTime; - if (values != null && values.length > 0) { - for (Result rs : values) { + List resultsToAddToCache = getResultsToAddToCache(values); + if (!resultsToAddToCache.isEmpty()) { + for (Result rs : resultsToAddToCache) { cache.add(rs); // We don't make Iterator here for (Cell cell : rs.rawCells()) { @@ -441,19 +463,157 @@ public class ClientScanner extends AbstractClientScanner { } } // Values == null means server-side filter has determined we must STOP - } while (remainingResultSize > 0 && countdown > 0 && - possiblyNextScanner(countdown, values == null)); + } while (remainingResultSize > 0 && countdown > 0 + && (partialResultReturned || possiblyNextScanner(countdown, values == null))); } - + if (cache.size() > 0) { return cache.poll(); } - + // if we exhausted this scanner before calling close, write out the scan metrics writeScanMetrics(); return null; } + @VisibleForTesting + public int getCacheSize() { + return cache != null ? cache.size() : 0; + } + + /** + * This method ensures all of our book keeping regarding partial results is kept up to date. This + * method should be called once we know that the results we received back from the RPC request do + * not contain errors. We return a list of results that should be added to the cache. In general, + * this list will contain all NON-partial results from the input array (unless the client has + * specified that they are okay with receiving partial results) + * @return the list of results that should be added to the cache. + * @throws IOException + */ + protected List getResultsToAddToCache(Result[] resultsFromServer) throws IOException { + List resultsToAddToCache = new ArrayList(); + + final boolean isBatchSet = scan != null && scan.getBatch() > 0; + final boolean allowPartials = scan != null && scan.getAllowPartialResults(); + + // If the caller has indicated in their scan that they are okay with seeing partial results, + // then simply add all results to the list. Note that since scan batching also returns results + // for a row in pieces we treat batch being set as equivalent to allowing partials. The + // implication of treating batching as equivalent to partial results is that it is possible + // the caller will receive a result back where the number of cells in the result is less than + // the batch size even though it may not be the last group of cells for that row. + if (allowPartials || isBatchSet) { + addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length); + return resultsToAddToCache; + } + + // If no results were returned it indicates that we have the all the partial results necessary + // to construct the complete result. + if (resultsFromServer == null || resultsFromServer.length == 0) { + if (!partialResults.isEmpty()) { + resultsToAddToCache.add(Result.createCompleteResult(partialResults)); + partialResults.clear(); + } + + return resultsToAddToCache; + } + + // In every RPC response there should be at most a single partial result. Furthermore, if + // there is a partial result, it is guaranteed to be in the last position of the array. + Result last = resultsFromServer[resultsFromServer.length - 1]; + Result partial = last.isPartial() ? last : null; + partialResultReturned = partial != null; + + if (LOG.isTraceEnabled()) { + LOG.trace("partialResultReturned: " + partialResultReturned); + LOG.trace("number of results returned from RPC: " + resultsFromServer.length); + LOG.trace("partial != null: " + (partial != null)); + LOG.trace("number of partial results so far: " + partialResults.size()); + } + + // There are four possibilities cases that can occur while handling partial results + // + // 1. (partial != null && partialResults.isEmpty()) + // This is the first partial result that we have received. It should be added to + // the list of partialResults and await the next RPC request at which point another + // portion of the complete result will be received + // + // 2. (partial != null && !partialResults.isEmpty()) + // a. values.length == 1 + // Since partialResults contains some elements, it means that we are expecting to receive + // the remainder of the complete result within this RPC response. The fact that a partial result + // was returned and it's the ONLY result returned indicates that we are still receiving + // fragments of the complete result. The Result can be completely formed only when we have + // received all of the fragments and thus in this case we simply add the partial result to + // our list. + // + // b. values.length > 1 + // More than one result has been returned from the server. The fact that we are accumulating + // partials in partialList and we just received more than one result back from the server + // indicates that the FIRST result we received from the server must be the final fragment that + // can be used to complete our result. What this means is that the partial that we received is + // a partial result for a different row, and at this point we should combine the existing + // partials into a complete result, clear the partialList, and begin accumulating partials for + // a new row + // + // 3. (partial == null && !partialResults.isEmpty()) + // No partial was received but we are accumulating partials in our list. That means the final + // fragment of the complete result will be the first Result in values[]. We use it to create the + // complete Result, clear the list, and add it to the list of Results that must be added to the + // cache. All other Results in values[] are added after the complete result to maintain proper + // ordering + // + // 4. (partial == null && partialResults.isEmpty()) + // Business as usual. We are not accumulating partial results and there wasn't a partial result + // in the RPC response. This means that all of the results we received from the server are + // complete and can be added directly to the cache + if (partial != null && partialResults.isEmpty()) { + partialResults.add(partial); + + // Exclude the last result, it's a partial + addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length - 1); + } else if (partial != null && !partialResults.isEmpty()) { + if (resultsFromServer.length > 1) { + Result finalResult = resultsFromServer[0]; + partialResults.add(finalResult); + resultsToAddToCache.add(Result.createCompleteResult(partialResults)); + partialResults.clear(); + + // Exclude first result, it was used to form our complete result + // Exclude last result, it's a partial result + addResultsToList(resultsToAddToCache, resultsFromServer, 1, resultsFromServer.length - 1); + } + partialResults.add(partial); + } else if (partial == null && !partialResults.isEmpty()) { + Result finalResult = resultsFromServer[0]; + partialResults.add(finalResult); + resultsToAddToCache.add(Result.createCompleteResult(partialResults)); + partialResults.clear(); + + // Exclude the first result, it was used to form our complete result + addResultsToList(resultsToAddToCache, resultsFromServer, 1, resultsFromServer.length); + } else { // partial == null && partialResults.isEmpty() -- business as usual + addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length); + } + + return resultsToAddToCache; + } + + /** + * Helper method for adding results between the indices [start, end) to the outputList + * @param outputList the list that results will be added to + * @param inputArray the array that results are taken from + * @param start beginning index (inclusive) + * @param end ending index (exclusive) + */ + private void addResultsToList(List outputList, Result[] inputArray, int start, int end) { + if (inputArray == null || start < 0 || end > inputArray.length) return; + + for (int i = start; i < end; i++) { + outputList.add(inputArray[i]); + } + } + @Override public void close() { if (!scanMetricsPublished) writeScanMetrics(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java index c418e47..16be9ce 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client; +import java.io.IOException; import java.nio.BufferOverflowException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -80,6 +81,15 @@ public class Result implements CellScannable, CellScanner { private Cell[] cells; private Boolean exists; // if the query was just to check existence. private boolean stale = false; + + /** + * Partial results do not contain the full row's worth of cells. The result had to be returned in + * parts because the size of the cells in the row exceeded the RPC result size on the server. + * Partial results must be combined client side with results representing the remainder of the + * row's cells to form the complete result. Partial results and RPC result size allow us to avoid + * OOME on the server when servicing requests for large rows. + */ + private boolean partial = false; // We're not using java serialization. Transient here is just a marker to say // that this is where we cache row if we're ever asked for it. private transient byte [] row = null; @@ -115,7 +125,7 @@ public class Result implements CellScannable, CellScanner { * @param cells List of cells */ public static Result create(List cells) { - return new Result(cells.toArray(new Cell[cells.size()]), null, false); + return create(cells, null); } public static Result create(List cells, Boolean exists) { @@ -123,10 +133,14 @@ public class Result implements CellScannable, CellScanner { } public static Result create(List cells, Boolean exists, boolean stale) { + return create(cells, exists, stale, false); + } + + public static Result create(List cells, Boolean exists, boolean stale, boolean partial) { if (exists != null){ - return new Result(null, exists, stale); + return new Result(null, exists, stale, partial); } - return new Result(cells.toArray(new Cell[cells.size()]), null, stale); + return new Result(cells.toArray(new Cell[cells.size()]), null, stale, partial); } /** @@ -135,21 +149,26 @@ public class Result implements CellScannable, CellScanner { * @param cells array of cells */ public static Result create(Cell[] cells) { - return new Result(cells, null, false); + return create(cells, null, false); } public static Result create(Cell[] cells, Boolean exists, boolean stale) { + return create(cells, exists, stale, false); + } + + public static Result create(Cell[] cells, Boolean exists, boolean stale, boolean partial) { if (exists != null){ - return new Result(null, exists, stale); + return new Result(null, exists, stale, partial); } - return new Result(cells, null, stale); + return new Result(cells, null, stale, partial); } /** Private ctor. Use {@link #create(Cell[])}. */ - private Result(Cell[] cells, Boolean exists, boolean stale) { + private Result(Cell[] cells, Boolean exists, boolean stale, boolean partial) { this.cells = cells; this.exists = exists; this.stale = stale; + this.partial = partial; } /** @@ -746,6 +765,37 @@ public class Result implements CellScannable, CellScanner { } /** + * Forms a single result from the partial results in the partialResults list. This method is + * useful for reconstructing partial results on the client side. + * @param partialResults list of partial results + * @return The complete result that is formed by combining all of the partial results together + * @throws IOException A complete result cannot be formed because the results in the partial list + * come from different rows + */ + public static Result createCompleteResult(List partialResults) + throws IOException { + List cells = new ArrayList(); + boolean stale = false; + byte[] prevRow = null; + byte[] currentRow = null; + + for (Result r : partialResults) { + currentRow = r.getRow(); + if (prevRow != null && !Bytes.equals(prevRow, currentRow)) { + throw new IOException("Cannot form complete result. Rows of partial results do not match." + + " Partial Results: " + partialResults); + } + prevRow = currentRow; + stale = stale || r.isStale(); + for (Cell c : r.rawCells()) { + cells.add(c); + } + } + + return Result.create(cells, null, stale); + } + + /** * Get total size of raw cells * @param result * @return Total size. @@ -805,6 +855,20 @@ public class Result implements CellScannable, CellScanner { } /** + * Whether or not the result is a partial result. Partial results contain a subset of the cells + * for a row and should be combined with a result representing the remaining cells in that row to + * form a complete (non-partial) result. + * @return Whether or not the result is a partial result + */ + public boolean isPartial() { + return partial; + } + + public void setPartial(boolean partial) { + this.partial = partial; + } + + /** * Add load information about the region to the information about the result * @param loadStats statistics about the current region from which this was returned */ 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 0f244e0..0ab7f31 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 @@ -139,9 +139,10 @@ public class ReversedClientScanner extends ClientScanner { new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, locateStartRow, this.rpcControllerFactory); s.setCaching(nbRows); - ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(getTable(), getConnection(), - s, pool, primaryOperationTimeout, scan, - getRetries(), getScannerTimeout(), caching, getConf(), caller); + ScannerCallableWithReplicas sr = + new ScannerCallableWithReplicas(getTable(), getConnection(), s, pool, + primaryOperationTimeout, scan, getRetries(), getScannerTimeout(), caching, getConf(), + caller); return sr; } 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 bfcfa20..3bb8548 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 @@ -115,6 +115,19 @@ public class Scan extends Query { private int maxVersions = 1; private int batch = -1; + /** + * Partial {@link Result}s are {@link Result}s must be combined to form a complete {@link Result}. + * The {@link Result}s had to be returned in fragments (i.e. as partials) because the size of the + * cells in the row exceeded max result size on the server. Typically partial results will be + * combined client side into complete results before being delivered to the caller. However, if + * this flag is set, the caller is indicating that they do not mind seeing partial results (i.e. + * they understand that the results returned from the Scanner may only represent part of a + * particular row). In such a case, any attempt to combine the partials into a complete result on + * the client side will be skipped, and the caller will be able to see the exact results returned + * from the server. + */ + private boolean allowPartialResults = false; + private int storeLimit = -1; private int storeOffset = 0; private boolean getScan; @@ -692,6 +705,27 @@ public class Scan extends Query { } /** + * Setting whether the caller wants to see the partial results that may be returned from the + * server. By default this value is false and the complete results will be assembled client side + * before being delivered to the caller. + * @param allowPartialResults + * @return this + */ + public Scan setAllowPartialResults(final boolean allowPartialResults) { + this.allowPartialResults = allowPartialResults; + return this; + } + + /** + * @return true when the constructor of this scan understands that the results they will see may + * only represent a partial portion of a row. The entire row would be retrieved by + * subsequent calls to {@link ResultScanner#next()} + */ + public boolean getAllowPartialResults() { + return allowPartialResults; + } + + /** * Set the value indicating whether loading CFs on demand should be allowed (cluster * default is false). On-demand CF loading doesn't load column families until necessary, e.g. * if you filter on one column, the other column family data will be loaded only for the rows diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 038b148..bad3cb4 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 @@ -37,7 +37,6 @@ import java.util.Map.Entry; import java.util.NavigableSet; import java.util.concurrent.TimeUnit; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Delete; @@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.TimeRange; -import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; @@ -1285,6 +1284,7 @@ public final class ProtobufUtil { } builder.setStale(result.isStale()); + builder.setPartial(result.isPartial()); return builder.build(); } @@ -1343,7 +1343,7 @@ public final class ProtobufUtil { for (CellProtos.Cell c : values) { cells.add(toCell(c)); } - return Result.create(cells, null, proto.getStale()); + return Result.create(cells, null, proto.getStale(), proto.getPartial()); } /** 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 d23aa02..566ec63 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 @@ -486,6 +486,7 @@ public final class RequestConverter { builder.setCloseScanner(closeScanner); builder.setRegion(region); builder.setScan(ProtobufUtil.toScan(scan)); + builder.setClientHandlesPartials(true); return builder.build(); } @@ -503,6 +504,7 @@ public final class RequestConverter { builder.setNumberOfRows(numberOfRows); builder.setCloseScanner(closeScanner); builder.setScannerId(scannerId); + builder.setClientHandlesPartials(true); return builder.build(); } @@ -522,6 +524,7 @@ public final class RequestConverter { builder.setCloseScanner(closeScanner); builder.setScannerId(scannerId); builder.setNextCallSeq(nextCallSeq); + builder.setClientHandlesPartials(true); return builder.build(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java index 1d42a82..59ba1c3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java @@ -339,6 +339,9 @@ public final class ResponseConverter { // Cells are out in cellblocks. Group them up again as Results. How many to read at a // time will be found in getCellsLength -- length here is how many Cells in the i'th Result int noOfCells = response.getCellsPerResult(i); + boolean isPartial = + response.getPartialFlagPerResultCount() > i ? + response.getPartialFlagPerResult(i) : false; List cells = new ArrayList(noOfCells); for (int j = 0; j < noOfCells; j++) { try { @@ -361,7 +364,7 @@ public final class ResponseConverter { } cells.add(cellScanner.current()); } - results[i] = Result.create(cells, null, response.getStale()); + results[i] = Result.create(cells, null, response.getStale(), isPartial); } else { // Result is pure pb. results[i] = ProtobufUtil.toResult(response.getResults(i)); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 40f67f3..53beb4e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -698,7 +698,7 @@ public final class HConstants { /** * Default value for {@link #HBASE_CLIENT_SCANNER_CACHING} */ - public static final int DEFAULT_HBASE_CLIENT_SCANNER_CACHING = 100; + public static final int DEFAULT_HBASE_CLIENT_SCANNER_CACHING = Integer.MAX_VALUE; /** * Parameter name for number of rows that will be fetched when calling next on diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index af6822b..27b0046 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -527,7 +527,7 @@ possible configurations would overwhelm and obscure the important. hbase.client.scanner.caching - 100 + 2147483647 Number of rows that will be fetched when calling next on a scanner if it is not served from (local, client) memory. Higher caching values will enable faster scanners but will eat up more memory 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 afd67a1..6c27aba 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 @@ -4247,6 +4247,30 @@ public final class ClientProtos { * */ boolean getStale(); + + // optional bool partial = 5 [default = false]; + /** + * optional bool partial = 5 [default = false]; + * + *
+     * Whether or not the entire result could be returned. Results will be split when
+     * the RPC chunk size limit is reached. Partial results contain only a subset of the
+     * cells for a row and must be combined with a result containing the remaining cells
+     * to form a complete result
+     * 
+ */ + boolean hasPartial(); + /** + * optional bool partial = 5 [default = false]; + * + *
+     * Whether or not the entire result could be returned. Results will be split when
+     * the RPC chunk size limit is reached. Partial results contain only a subset of the
+     * cells for a row and must be combined with a result containing the remaining cells
+     * to form a complete result
+     * 
+ */ + boolean getPartial(); } /** * Protobuf type {@code Result} @@ -4322,6 +4346,11 @@ public final class ClientProtos { stale_ = input.readBool(); break; } + case 40: { + bitField0_ |= 0x00000008; + partial_ = input.readBool(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -4510,11 +4539,42 @@ public final class ClientProtos { return stale_; } + // optional bool partial = 5 [default = false]; + public static final int PARTIAL_FIELD_NUMBER = 5; + private boolean partial_; + /** + * optional bool partial = 5 [default = false]; + * + *
+     * Whether or not the entire result could be returned. Results will be split when
+     * the RPC chunk size limit is reached. Partial results contain only a subset of the
+     * cells for a row and must be combined with a result containing the remaining cells
+     * to form a complete result
+     * 
+ */ + public boolean hasPartial() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional bool partial = 5 [default = false]; + * + *
+     * Whether or not the entire result could be returned. Results will be split when
+     * the RPC chunk size limit is reached. Partial results contain only a subset of the
+     * cells for a row and must be combined with a result containing the remaining cells
+     * to form a complete result
+     * 
+ */ + public boolean getPartial() { + return partial_; + } + private void initFields() { cell_ = java.util.Collections.emptyList(); associatedCellCount_ = 0; exists_ = false; stale_ = false; + partial_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -4540,6 +4600,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000004) == 0x00000004)) { output.writeBool(4, stale_); } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBool(5, partial_); + } getUnknownFields().writeTo(output); } @@ -4565,6 +4628,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeBoolSize(4, stale_); } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(5, partial_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -4605,6 +4672,11 @@ public final class ClientProtos { result = result && (getStale() == other.getStale()); } + result = result && (hasPartial() == other.hasPartial()); + if (hasPartial()) { + result = result && (getPartial() + == other.getPartial()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -4634,6 +4706,10 @@ public final class ClientProtos { hash = (37 * hash) + STALE_FIELD_NUMBER; hash = (53 * hash) + hashBoolean(getStale()); } + if (hasPartial()) { + hash = (37 * hash) + PARTIAL_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getPartial()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -4756,6 +4832,8 @@ public final class ClientProtos { bitField0_ = (bitField0_ & ~0x00000004); stale_ = false; bitField0_ = (bitField0_ & ~0x00000008); + partial_ = false; + bitField0_ = (bitField0_ & ~0x00000010); return this; } @@ -4805,6 +4883,10 @@ public final class ClientProtos { to_bitField0_ |= 0x00000004; } result.stale_ = stale_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000008; + } + result.partial_ = partial_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -4856,6 +4938,9 @@ public final class ClientProtos { if (other.hasStale()) { setStale(other.getStale()); } + if (other.hasPartial()) { + setPartial(other.getPartial()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -5384,6 +5469,67 @@ public final class ClientProtos { return this; } + // optional bool partial = 5 [default = false]; + private boolean partial_ ; + /** + * optional bool partial = 5 [default = false]; + * + *
+       * Whether or not the entire result could be returned. Results will be split when
+       * the RPC chunk size limit is reached. Partial results contain only a subset of the
+       * cells for a row and must be combined with a result containing the remaining cells
+       * to form a complete result
+       * 
+ */ + public boolean hasPartial() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional bool partial = 5 [default = false]; + * + *
+       * Whether or not the entire result could be returned. Results will be split when
+       * the RPC chunk size limit is reached. Partial results contain only a subset of the
+       * cells for a row and must be combined with a result containing the remaining cells
+       * to form a complete result
+       * 
+ */ + public boolean getPartial() { + return partial_; + } + /** + * optional bool partial = 5 [default = false]; + * + *
+       * Whether or not the entire result could be returned. Results will be split when
+       * the RPC chunk size limit is reached. Partial results contain only a subset of the
+       * cells for a row and must be combined with a result containing the remaining cells
+       * to form a complete result
+       * 
+ */ + public Builder setPartial(boolean value) { + bitField0_ |= 0x00000010; + partial_ = value; + onChanged(); + return this; + } + /** + * optional bool partial = 5 [default = false]; + * + *
+       * Whether or not the entire result could be returned. Results will be split when
+       * the RPC chunk size limit is reached. Partial results contain only a subset of the
+       * cells for a row and must be combined with a result containing the remaining cells
+       * to form a complete result
+       * 
+ */ + public Builder clearPartial() { + bitField0_ = (bitField0_ & ~0x00000010); + partial_ = false; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:Result) } @@ -16277,6 +16423,16 @@ public final class ClientProtos { * optional uint64 next_call_seq = 6; */ long getNextCallSeq(); + + // optional bool client_handles_partials = 7; + /** + * optional bool client_handles_partials = 7; + */ + boolean hasClientHandlesPartials(); + /** + * optional bool client_handles_partials = 7; + */ + boolean getClientHandlesPartials(); } /** * Protobuf type {@code ScanRequest} @@ -16388,6 +16544,11 @@ public final class ClientProtos { nextCallSeq_ = input.readUInt64(); break; } + case 56: { + bitField0_ |= 0x00000040; + clientHandlesPartials_ = input.readBool(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -16536,6 +16697,22 @@ public final class ClientProtos { return nextCallSeq_; } + // optional bool client_handles_partials = 7; + public static final int CLIENT_HANDLES_PARTIALS_FIELD_NUMBER = 7; + private boolean clientHandlesPartials_; + /** + * optional bool client_handles_partials = 7; + */ + public boolean hasClientHandlesPartials() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + /** + * optional bool client_handles_partials = 7; + */ + public boolean getClientHandlesPartials() { + return clientHandlesPartials_; + } + private void initFields() { region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); @@ -16543,6 +16720,7 @@ public final class ClientProtos { numberOfRows_ = 0; closeScanner_ = false; nextCallSeq_ = 0L; + clientHandlesPartials_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -16586,6 +16764,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000020) == 0x00000020)) { output.writeUInt64(6, nextCallSeq_); } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeBool(7, clientHandlesPartials_); + } getUnknownFields().writeTo(output); } @@ -16619,6 +16800,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeUInt64Size(6, nextCallSeq_); } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(7, clientHandlesPartials_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -16672,6 +16857,11 @@ public final class ClientProtos { result = result && (getNextCallSeq() == other.getNextCallSeq()); } + result = result && (hasClientHandlesPartials() == other.hasClientHandlesPartials()); + if (hasClientHandlesPartials()) { + result = result && (getClientHandlesPartials() + == other.getClientHandlesPartials()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -16709,6 +16899,10 @@ public final class ClientProtos { hash = (37 * hash) + NEXT_CALL_SEQ_FIELD_NUMBER; hash = (53 * hash) + hashLong(getNextCallSeq()); } + if (hasClientHandlesPartials()) { + hash = (37 * hash) + CLIENT_HANDLES_PARTIALS_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getClientHandlesPartials()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -16853,6 +17047,8 @@ public final class ClientProtos { bitField0_ = (bitField0_ & ~0x00000010); nextCallSeq_ = 0L; bitField0_ = (bitField0_ & ~0x00000020); + clientHandlesPartials_ = false; + bitField0_ = (bitField0_ & ~0x00000040); return this; } @@ -16913,6 +17109,10 @@ public final class ClientProtos { to_bitField0_ |= 0x00000020; } result.nextCallSeq_ = nextCallSeq_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.clientHandlesPartials_ = clientHandlesPartials_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -16947,6 +17147,9 @@ public final class ClientProtos { if (other.hasNextCallSeq()) { setNextCallSeq(other.getNextCallSeq()); } + if (other.hasClientHandlesPartials()) { + setClientHandlesPartials(other.getClientHandlesPartials()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -17352,6 +17555,39 @@ public final class ClientProtos { return this; } + // optional bool client_handles_partials = 7; + private boolean clientHandlesPartials_ ; + /** + * optional bool client_handles_partials = 7; + */ + public boolean hasClientHandlesPartials() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + /** + * optional bool client_handles_partials = 7; + */ + public boolean getClientHandlesPartials() { + return clientHandlesPartials_; + } + /** + * optional bool client_handles_partials = 7; + */ + public Builder setClientHandlesPartials(boolean value) { + bitField0_ |= 0x00000040; + clientHandlesPartials_ = value; + onChanged(); + return this; + } + /** + * optional bool client_handles_partials = 7; + */ + public Builder clearClientHandlesPartials() { + bitField0_ = (bitField0_ & ~0x00000040); + clientHandlesPartials_ = false; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:ScanRequest) } @@ -17410,39 +17646,83 @@ public final class ClientProtos { */ int getCellsPerResult(int index); - // optional uint64 scanner_id = 2; + // repeated bool partial_flag_per_result = 2; + /** + * repeated bool partial_flag_per_result = 2; + * + *
+     * This field is filled in if we are doing cellblocks. In the event that a row
+     * could not fit all of its cells into a single RPC chunk, the results will be
+     * returned as partials, and reconstructed into a complete result on the client 
+     * side. This field is a list of flags indicating whether or not the result
+     * that the cells belong to is a partial result. For example, if this field
+     * has false, false, true in it, then we know that on the client side, we need to 
+     * make another RPC request since the last result was only a partial.
+     * 
+ */ + java.util.List getPartialFlagPerResultList(); + /** + * repeated bool partial_flag_per_result = 2; + * + *
+     * This field is filled in if we are doing cellblocks. In the event that a row
+     * could not fit all of its cells into a single RPC chunk, the results will be
+     * returned as partials, and reconstructed into a complete result on the client 
+     * side. This field is a list of flags indicating whether or not the result
+     * that the cells belong to is a partial result. For example, if this field
+     * has false, false, true in it, then we know that on the client side, we need to 
+     * make another RPC request since the last result was only a partial.
+     * 
+ */ + int getPartialFlagPerResultCount(); + /** + * repeated bool partial_flag_per_result = 2; + * + *
+     * This field is filled in if we are doing cellblocks. In the event that a row
+     * could not fit all of its cells into a single RPC chunk, the results will be
+     * returned as partials, and reconstructed into a complete result on the client 
+     * side. This field is a list of flags indicating whether or not the result
+     * that the cells belong to is a partial result. For example, if this field
+     * has false, false, true in it, then we know that on the client side, we need to 
+     * make another RPC request since the last result was only a partial.
+     * 
+ */ + boolean getPartialFlagPerResult(int index); + + // optional uint64 scanner_id = 3; /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ boolean hasScannerId(); /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ long getScannerId(); - // optional bool more_results = 3; + // optional bool more_results = 4; /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ boolean hasMoreResults(); /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ boolean getMoreResults(); - // optional uint32 ttl = 4; + // optional uint32 ttl = 5; /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ boolean hasTtl(); /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ int getTtl(); - // repeated .Result results = 5; + // repeated .Result results = 6; /** - * repeated .Result results = 5; + * repeated .Result results = 6; * *
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17453,7 +17733,7 @@ public final class ClientProtos {
     java.util.List 
         getResultsList();
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17463,7 +17743,7 @@ public final class ClientProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getResults(int index);
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17473,7 +17753,7 @@ public final class ClientProtos {
      */
     int getResultsCount();
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17484,7 +17764,7 @@ public final class ClientProtos {
     java.util.List 
         getResultsOrBuilderList();
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17495,13 +17775,13 @@ public final class ClientProtos {
     org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultsOrBuilder(
         int index);
 
-    // optional bool stale = 6;
+    // optional bool stale = 7;
     /**
-     * optional bool stale = 6;
+     * optional bool stale = 7;
      */
     boolean hasStale();
     /**
-     * optional bool stale = 6;
+     * optional bool stale = 7;
      */
     boolean getStale();
   }
@@ -17584,29 +17864,50 @@ public final class ClientProtos {
               break;
             }
             case 16: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                partialFlagPerResult_ = new java.util.ArrayList();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              partialFlagPerResult_.add(input.readBool());
+              break;
+            }
+            case 18: {
+              int length = input.readRawVarint32();
+              int limit = input.pushLimit(length);
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002) && input.getBytesUntilLimit() > 0) {
+                partialFlagPerResult_ = new java.util.ArrayList();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              while (input.getBytesUntilLimit() > 0) {
+                partialFlagPerResult_.add(input.readBool());
+              }
+              input.popLimit(limit);
+              break;
+            }
+            case 24: {
               bitField0_ |= 0x00000001;
               scannerId_ = input.readUInt64();
               break;
             }
-            case 24: {
+            case 32: {
               bitField0_ |= 0x00000002;
               moreResults_ = input.readBool();
               break;
             }
-            case 32: {
+            case 40: {
               bitField0_ |= 0x00000004;
               ttl_ = input.readUInt32();
               break;
             }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+            case 50: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
                 results_ = new java.util.ArrayList();
-                mutable_bitField0_ |= 0x00000010;
+                mutable_bitField0_ |= 0x00000020;
               }
               results_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.PARSER, extensionRegistry));
               break;
             }
-            case 48: {
+            case 56: {
               bitField0_ |= 0x00000008;
               stale_ = input.readBool();
               break;
@@ -17622,7 +17923,10 @@ public final class ClientProtos {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
           cellsPerResult_ = java.util.Collections.unmodifiableList(cellsPerResult_);
         }
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          partialFlagPerResult_ = java.util.Collections.unmodifiableList(partialFlagPerResult_);
+        }
+        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
           results_ = java.util.Collections.unmodifiableList(results_);
         }
         this.unknownFields = unknownFields.build();
@@ -17710,59 +18014,112 @@ public final class ClientProtos {
       return cellsPerResult_.get(index);
     }
 
-    // optional uint64 scanner_id = 2;
-    public static final int SCANNER_ID_FIELD_NUMBER = 2;
+    // repeated bool partial_flag_per_result = 2;
+    public static final int PARTIAL_FLAG_PER_RESULT_FIELD_NUMBER = 2;
+    private java.util.List partialFlagPerResult_;
+    /**
+     * repeated bool partial_flag_per_result = 2;
+     *
+     * 
+     * This field is filled in if we are doing cellblocks. In the event that a row
+     * could not fit all of its cells into a single RPC chunk, the results will be
+     * returned as partials, and reconstructed into a complete result on the client 
+     * side. This field is a list of flags indicating whether or not the result
+     * that the cells belong to is a partial result. For example, if this field
+     * has false, false, true in it, then we know that on the client side, we need to 
+     * make another RPC request since the last result was only a partial.
+     * 
+ */ + public java.util.List + getPartialFlagPerResultList() { + return partialFlagPerResult_; + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+     * This field is filled in if we are doing cellblocks. In the event that a row
+     * could not fit all of its cells into a single RPC chunk, the results will be
+     * returned as partials, and reconstructed into a complete result on the client 
+     * side. This field is a list of flags indicating whether or not the result
+     * that the cells belong to is a partial result. For example, if this field
+     * has false, false, true in it, then we know that on the client side, we need to 
+     * make another RPC request since the last result was only a partial.
+     * 
+ */ + public int getPartialFlagPerResultCount() { + return partialFlagPerResult_.size(); + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+     * This field is filled in if we are doing cellblocks. In the event that a row
+     * could not fit all of its cells into a single RPC chunk, the results will be
+     * returned as partials, and reconstructed into a complete result on the client 
+     * side. This field is a list of flags indicating whether or not the result
+     * that the cells belong to is a partial result. For example, if this field
+     * has false, false, true in it, then we know that on the client side, we need to 
+     * make another RPC request since the last result was only a partial.
+     * 
+ */ + public boolean getPartialFlagPerResult(int index) { + return partialFlagPerResult_.get(index); + } + + // optional uint64 scanner_id = 3; + public static final int SCANNER_ID_FIELD_NUMBER = 3; private long scannerId_; /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ public boolean hasScannerId() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ public long getScannerId() { return scannerId_; } - // optional bool more_results = 3; - public static final int MORE_RESULTS_FIELD_NUMBER = 3; + // optional bool more_results = 4; + public static final int MORE_RESULTS_FIELD_NUMBER = 4; private boolean moreResults_; /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ public boolean hasMoreResults() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ public boolean getMoreResults() { return moreResults_; } - // optional uint32 ttl = 4; - public static final int TTL_FIELD_NUMBER = 4; + // optional uint32 ttl = 5; + public static final int TTL_FIELD_NUMBER = 5; private int ttl_; /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ public boolean hasTtl() { return ((bitField0_ & 0x00000004) == 0x00000004); } /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ public int getTtl() { return ttl_; } - // repeated .Result results = 5; - public static final int RESULTS_FIELD_NUMBER = 5; + // repeated .Result results = 6; + public static final int RESULTS_FIELD_NUMBER = 6; private java.util.List results_; /** - * repeated .Result results = 5; + * repeated .Result results = 6; * *
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17774,7 +18131,7 @@ public final class ClientProtos {
       return results_;
     }
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17787,7 +18144,7 @@ public final class ClientProtos {
       return results_;
     }
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17799,7 +18156,7 @@ public final class ClientProtos {
       return results_.size();
     }
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17811,7 +18168,7 @@ public final class ClientProtos {
       return results_.get(index);
     }
     /**
-     * repeated .Result results = 5;
+     * repeated .Result results = 6;
      *
      * 
      * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -17824,17 +18181,17 @@ public final class ClientProtos {
       return results_.get(index);
     }
 
-    // optional bool stale = 6;
-    public static final int STALE_FIELD_NUMBER = 6;
+    // optional bool stale = 7;
+    public static final int STALE_FIELD_NUMBER = 7;
     private boolean stale_;
     /**
-     * optional bool stale = 6;
+     * optional bool stale = 7;
      */
     public boolean hasStale() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * optional bool stale = 6;
+     * optional bool stale = 7;
      */
     public boolean getStale() {
       return stale_;
@@ -17842,6 +18199,7 @@ public final class ClientProtos {
 
     private void initFields() {
       cellsPerResult_ = java.util.Collections.emptyList();
+      partialFlagPerResult_ = java.util.Collections.emptyList();
       scannerId_ = 0L;
       moreResults_ = false;
       ttl_ = 0;
@@ -17863,20 +18221,23 @@ public final class ClientProtos {
       for (int i = 0; i < cellsPerResult_.size(); i++) {
         output.writeUInt32(1, cellsPerResult_.get(i));
       }
+      for (int i = 0; i < partialFlagPerResult_.size(); i++) {
+        output.writeBool(2, partialFlagPerResult_.get(i));
+      }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeUInt64(2, scannerId_);
+        output.writeUInt64(3, scannerId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBool(3, moreResults_);
+        output.writeBool(4, moreResults_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt32(4, ttl_);
+        output.writeUInt32(5, ttl_);
       }
       for (int i = 0; i < results_.size(); i++) {
-        output.writeMessage(5, results_.get(i));
+        output.writeMessage(6, results_.get(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBool(6, stale_);
+        output.writeBool(7, stale_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -17896,25 +18257,31 @@ public final class ClientProtos {
         size += dataSize;
         size += 1 * getCellsPerResultList().size();
       }
+      {
+        int dataSize = 0;
+        dataSize = 1 * getPartialFlagPerResultList().size();
+        size += dataSize;
+        size += 1 * getPartialFlagPerResultList().size();
+      }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(2, scannerId_);
+          .computeUInt64Size(3, scannerId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(3, moreResults_);
+          .computeBoolSize(4, moreResults_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(4, ttl_);
+          .computeUInt32Size(5, ttl_);
       }
       for (int i = 0; i < results_.size(); i++) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(5, results_.get(i));
+          .computeMessageSize(6, results_.get(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(6, stale_);
+          .computeBoolSize(7, stale_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -17941,6 +18308,8 @@ public final class ClientProtos {
       boolean result = true;
       result = result && getCellsPerResultList()
           .equals(other.getCellsPerResultList());
+      result = result && getPartialFlagPerResultList()
+          .equals(other.getPartialFlagPerResultList());
       result = result && (hasScannerId() == other.hasScannerId());
       if (hasScannerId()) {
         result = result && (getScannerId()
@@ -17980,6 +18349,10 @@ public final class ClientProtos {
         hash = (37 * hash) + CELLS_PER_RESULT_FIELD_NUMBER;
         hash = (53 * hash) + getCellsPerResultList().hashCode();
       }
+      if (getPartialFlagPerResultCount() > 0) {
+        hash = (37 * hash) + PARTIAL_FLAG_PER_RESULT_FIELD_NUMBER;
+        hash = (53 * hash) + getPartialFlagPerResultList().hashCode();
+      }
       if (hasScannerId()) {
         hash = (37 * hash) + SCANNER_ID_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getScannerId());
@@ -18118,20 +18491,22 @@ public final class ClientProtos {
         super.clear();
         cellsPerResult_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
-        scannerId_ = 0L;
+        partialFlagPerResult_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000002);
-        moreResults_ = false;
+        scannerId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
-        ttl_ = 0;
+        moreResults_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
+        ttl_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000010);
         if (resultsBuilder_ == null) {
           results_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
+          bitField0_ = (bitField0_ & ~0x00000020);
         } else {
           resultsBuilder_.clear();
         }
         stale_ = false;
-        bitField0_ = (bitField0_ & ~0x00000020);
+        bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
@@ -18165,28 +18540,33 @@ public final class ClientProtos {
           bitField0_ = (bitField0_ & ~0x00000001);
         }
         result.cellsPerResult_ = cellsPerResult_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          partialFlagPerResult_ = java.util.Collections.unmodifiableList(partialFlagPerResult_);
+          bitField0_ = (bitField0_ & ~0x00000002);
+        }
+        result.partialFlagPerResult_ = partialFlagPerResult_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000001;
         }
         result.scannerId_ = scannerId_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000002;
         }
         result.moreResults_ = moreResults_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
           to_bitField0_ |= 0x00000004;
         }
         result.ttl_ = ttl_;
         if (resultsBuilder_ == null) {
-          if (((bitField0_ & 0x00000010) == 0x00000010)) {
+          if (((bitField0_ & 0x00000020) == 0x00000020)) {
             results_ = java.util.Collections.unmodifiableList(results_);
-            bitField0_ = (bitField0_ & ~0x00000010);
+            bitField0_ = (bitField0_ & ~0x00000020);
           }
           result.results_ = results_;
         } else {
           result.results_ = resultsBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
           to_bitField0_ |= 0x00000008;
         }
         result.stale_ = stale_;
@@ -18216,6 +18596,16 @@ public final class ClientProtos {
           }
           onChanged();
         }
+        if (!other.partialFlagPerResult_.isEmpty()) {
+          if (partialFlagPerResult_.isEmpty()) {
+            partialFlagPerResult_ = other.partialFlagPerResult_;
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            ensurePartialFlagPerResultIsMutable();
+            partialFlagPerResult_.addAll(other.partialFlagPerResult_);
+          }
+          onChanged();
+        }
         if (other.hasScannerId()) {
           setScannerId(other.getScannerId());
         }
@@ -18229,7 +18619,7 @@ public final class ClientProtos {
           if (!other.results_.isEmpty()) {
             if (results_.isEmpty()) {
               results_ = other.results_;
-              bitField0_ = (bitField0_ & ~0x00000010);
+              bitField0_ = (bitField0_ & ~0x00000020);
             } else {
               ensureResultsIsMutable();
               results_.addAll(other.results_);
@@ -18242,7 +18632,7 @@ public final class ClientProtos {
               resultsBuilder_.dispose();
               resultsBuilder_ = null;
               results_ = other.results_;
-              bitField0_ = (bitField0_ & ~0x00000010);
+              bitField0_ = (bitField0_ & ~0x00000020);
               resultsBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getResultsFieldBuilder() : null;
@@ -18417,112 +18807,248 @@ public final class ClientProtos {
         return this;
       }
 
-      // optional uint64 scanner_id = 2;
+      // repeated bool partial_flag_per_result = 2;
+      private java.util.List partialFlagPerResult_ = java.util.Collections.emptyList();
+      private void ensurePartialFlagPerResultIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          partialFlagPerResult_ = new java.util.ArrayList(partialFlagPerResult_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      /**
+       * repeated bool partial_flag_per_result = 2;
+       *
+       * 
+       * This field is filled in if we are doing cellblocks. In the event that a row
+       * could not fit all of its cells into a single RPC chunk, the results will be
+       * returned as partials, and reconstructed into a complete result on the client 
+       * side. This field is a list of flags indicating whether or not the result
+       * that the cells belong to is a partial result. For example, if this field
+       * has false, false, true in it, then we know that on the client side, we need to 
+       * make another RPC request since the last result was only a partial.
+       * 
+ */ + public java.util.List + getPartialFlagPerResultList() { + return java.util.Collections.unmodifiableList(partialFlagPerResult_); + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+       * This field is filled in if we are doing cellblocks. In the event that a row
+       * could not fit all of its cells into a single RPC chunk, the results will be
+       * returned as partials, and reconstructed into a complete result on the client 
+       * side. This field is a list of flags indicating whether or not the result
+       * that the cells belong to is a partial result. For example, if this field
+       * has false, false, true in it, then we know that on the client side, we need to 
+       * make another RPC request since the last result was only a partial.
+       * 
+ */ + public int getPartialFlagPerResultCount() { + return partialFlagPerResult_.size(); + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+       * This field is filled in if we are doing cellblocks. In the event that a row
+       * could not fit all of its cells into a single RPC chunk, the results will be
+       * returned as partials, and reconstructed into a complete result on the client 
+       * side. This field is a list of flags indicating whether or not the result
+       * that the cells belong to is a partial result. For example, if this field
+       * has false, false, true in it, then we know that on the client side, we need to 
+       * make another RPC request since the last result was only a partial.
+       * 
+ */ + public boolean getPartialFlagPerResult(int index) { + return partialFlagPerResult_.get(index); + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+       * This field is filled in if we are doing cellblocks. In the event that a row
+       * could not fit all of its cells into a single RPC chunk, the results will be
+       * returned as partials, and reconstructed into a complete result on the client 
+       * side. This field is a list of flags indicating whether or not the result
+       * that the cells belong to is a partial result. For example, if this field
+       * has false, false, true in it, then we know that on the client side, we need to 
+       * make another RPC request since the last result was only a partial.
+       * 
+ */ + public Builder setPartialFlagPerResult( + int index, boolean value) { + ensurePartialFlagPerResultIsMutable(); + partialFlagPerResult_.set(index, value); + onChanged(); + return this; + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+       * This field is filled in if we are doing cellblocks. In the event that a row
+       * could not fit all of its cells into a single RPC chunk, the results will be
+       * returned as partials, and reconstructed into a complete result on the client 
+       * side. This field is a list of flags indicating whether or not the result
+       * that the cells belong to is a partial result. For example, if this field
+       * has false, false, true in it, then we know that on the client side, we need to 
+       * make another RPC request since the last result was only a partial.
+       * 
+ */ + public Builder addPartialFlagPerResult(boolean value) { + ensurePartialFlagPerResultIsMutable(); + partialFlagPerResult_.add(value); + onChanged(); + return this; + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+       * This field is filled in if we are doing cellblocks. In the event that a row
+       * could not fit all of its cells into a single RPC chunk, the results will be
+       * returned as partials, and reconstructed into a complete result on the client 
+       * side. This field is a list of flags indicating whether or not the result
+       * that the cells belong to is a partial result. For example, if this field
+       * has false, false, true in it, then we know that on the client side, we need to 
+       * make another RPC request since the last result was only a partial.
+       * 
+ */ + public Builder addAllPartialFlagPerResult( + java.lang.Iterable values) { + ensurePartialFlagPerResultIsMutable(); + super.addAll(values, partialFlagPerResult_); + onChanged(); + return this; + } + /** + * repeated bool partial_flag_per_result = 2; + * + *
+       * This field is filled in if we are doing cellblocks. In the event that a row
+       * could not fit all of its cells into a single RPC chunk, the results will be
+       * returned as partials, and reconstructed into a complete result on the client 
+       * side. This field is a list of flags indicating whether or not the result
+       * that the cells belong to is a partial result. For example, if this field
+       * has false, false, true in it, then we know that on the client side, we need to 
+       * make another RPC request since the last result was only a partial.
+       * 
+ */ + public Builder clearPartialFlagPerResult() { + partialFlagPerResult_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + + // optional uint64 scanner_id = 3; private long scannerId_ ; /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ public boolean hasScannerId() { - return ((bitField0_ & 0x00000002) == 0x00000002); + return ((bitField0_ & 0x00000004) == 0x00000004); } /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ public long getScannerId() { return scannerId_; } /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ public Builder setScannerId(long value) { - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; scannerId_ = value; onChanged(); return this; } /** - * optional uint64 scanner_id = 2; + * optional uint64 scanner_id = 3; */ public Builder clearScannerId() { - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000004); scannerId_ = 0L; onChanged(); return this; } - // optional bool more_results = 3; + // optional bool more_results = 4; private boolean moreResults_ ; /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ public boolean hasMoreResults() { - return ((bitField0_ & 0x00000004) == 0x00000004); + return ((bitField0_ & 0x00000008) == 0x00000008); } /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ public boolean getMoreResults() { return moreResults_; } /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ public Builder setMoreResults(boolean value) { - bitField0_ |= 0x00000004; + bitField0_ |= 0x00000008; moreResults_ = value; onChanged(); return this; } /** - * optional bool more_results = 3; + * optional bool more_results = 4; */ public Builder clearMoreResults() { - bitField0_ = (bitField0_ & ~0x00000004); + bitField0_ = (bitField0_ & ~0x00000008); moreResults_ = false; onChanged(); return this; } - // optional uint32 ttl = 4; + // optional uint32 ttl = 5; private int ttl_ ; /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ public boolean hasTtl() { - return ((bitField0_ & 0x00000008) == 0x00000008); + return ((bitField0_ & 0x00000010) == 0x00000010); } /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ public int getTtl() { return ttl_; } /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ public Builder setTtl(int value) { - bitField0_ |= 0x00000008; + bitField0_ |= 0x00000010; ttl_ = value; onChanged(); return this; } /** - * optional uint32 ttl = 4; + * optional uint32 ttl = 5; */ public Builder clearTtl() { - bitField0_ = (bitField0_ & ~0x00000008); + bitField0_ = (bitField0_ & ~0x00000010); ttl_ = 0; onChanged(); return this; } - // repeated .Result results = 5; + // repeated .Result results = 6; private java.util.List results_ = java.util.Collections.emptyList(); private void ensureResultsIsMutable() { - if (!((bitField0_ & 0x00000010) == 0x00000010)) { + if (!((bitField0_ & 0x00000020) == 0x00000020)) { results_ = new java.util.ArrayList(results_); - bitField0_ |= 0x00000010; + bitField0_ |= 0x00000020; } } @@ -18530,7 +19056,7 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder> resultsBuilder_; /** - * repeated .Result results = 5; + * repeated .Result results = 6; * *
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18546,7 +19072,7 @@ public final class ClientProtos {
         }
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18562,7 +19088,7 @@ public final class ClientProtos {
         }
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18578,7 +19104,7 @@ public final class ClientProtos {
         }
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18601,7 +19127,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18621,7 +19147,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18643,7 +19169,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18666,7 +19192,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18686,7 +19212,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18706,7 +19232,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18726,7 +19252,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18737,7 +19263,7 @@ public final class ClientProtos {
       public Builder clearResults() {
         if (resultsBuilder_ == null) {
           results_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
+          bitField0_ = (bitField0_ & ~0x00000020);
           onChanged();
         } else {
           resultsBuilder_.clear();
@@ -18745,7 +19271,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18764,7 +19290,7 @@ public final class ClientProtos {
         return this;
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18777,7 +19303,7 @@ public final class ClientProtos {
         return getResultsFieldBuilder().getBuilder(index);
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18793,7 +19319,7 @@ public final class ClientProtos {
         }
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18810,7 +19336,7 @@ public final class ClientProtos {
         }
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18823,7 +19349,7 @@ public final class ClientProtos {
             org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance());
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18837,7 +19363,7 @@ public final class ClientProtos {
             index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance());
       }
       /**
-       * repeated .Result results = 5;
+       * repeated .Result results = 6;
        *
        * 
        * If cells are not carried in an accompanying cellblock, then they are pb'd here.
@@ -18856,7 +19382,7 @@ public final class ClientProtos {
           resultsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
               org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder>(
                   results_,
-                  ((bitField0_ & 0x00000010) == 0x00000010),
+                  ((bitField0_ & 0x00000020) == 0x00000020),
                   getParentForChildren(),
                   isClean());
           results_ = null;
@@ -18864,34 +19390,34 @@ public final class ClientProtos {
         return resultsBuilder_;
       }
 
-      // optional bool stale = 6;
+      // optional bool stale = 7;
       private boolean stale_ ;
       /**
-       * optional bool stale = 6;
+       * optional bool stale = 7;
        */
       public boolean hasStale() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
+        return ((bitField0_ & 0x00000040) == 0x00000040);
       }
       /**
-       * optional bool stale = 6;
+       * optional bool stale = 7;
        */
       public boolean getStale() {
         return stale_;
       }
       /**
-       * optional bool stale = 6;
+       * optional bool stale = 7;
        */
       public Builder setStale(boolean value) {
-        bitField0_ |= 0x00000020;
+        bitField0_ |= 0x00000040;
         stale_ = value;
         onChanged();
         return this;
       }
       /**
-       * optional bool stale = 6;
+       * optional bool stale = 7;
        */
       public Builder clearStale() {
-        bitField0_ = (bitField0_ & ~0x00000020);
+        bitField0_ = (bitField0_ & ~0x00000040);
         stale_ = false;
         onChanged();
         return this;
@@ -31980,105 +32506,108 @@ public final class ClientProtos {
       "\024\n\014store_offset\030\t \001(\r\022\035\n\016existence_only\030" +
       "\n \001(\010:\005false\022!\n\022closest_row_before\030\013 \001(\010" +
       ":\005false\022)\n\013consistency\030\014 \001(\0162\014.Consisten" +
-      "cy:\006STRONG\"b\n\006Result\022\023\n\004cell\030\001 \003(\0132\005.Cel" +
+      "cy:\006STRONG\"z\n\006Result\022\023\n\004cell\030\001 \003(\0132\005.Cel" +
       "l\022\035\n\025associated_cell_count\030\002 \001(\005\022\016\n\006exis" +
-      "ts\030\003 \001(\010\022\024\n\005stale\030\004 \001(\010:\005false\"A\n\nGetReq" +
-      "uest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\021" +
-      "\n\003get\030\002 \002(\0132\004.Get\"&\n\013GetResponse\022\027\n\006resu" +
-      "lt\030\001 \001(\0132\007.Result\"\200\001\n\tCondition\022\013\n\003row\030\001" +
-      " \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022",
-      "\"\n\014compare_type\030\004 \002(\0162\014.CompareType\022\037\n\nc" +
-      "omparator\030\005 \002(\0132\013.Comparator\"\265\006\n\rMutatio" +
-      "nProto\022\013\n\003row\030\001 \001(\014\0220\n\013mutate_type\030\002 \001(\016" +
-      "2\033.MutationProto.MutationType\0220\n\014column_" +
-      "value\030\003 \003(\0132\032.MutationProto.ColumnValue\022" +
-      "\021\n\ttimestamp\030\004 \001(\004\022!\n\tattribute\030\005 \003(\0132\016." +
-      "NameBytesPair\022:\n\ndurability\030\006 \001(\0162\031.Muta" +
-      "tionProto.Durability:\013USE_DEFAULT\022\036\n\ntim" +
-      "e_range\030\007 \001(\0132\n.TimeRange\022\035\n\025associated_" +
-      "cell_count\030\010 \001(\005\022\r\n\005nonce\030\t \001(\004\032\347\001\n\013Colu",
-      "mnValue\022\016\n\006family\030\001 \002(\014\022B\n\017qualifier_val" +
-      "ue\030\002 \003(\0132).MutationProto.ColumnValue.Qua" +
-      "lifierValue\032\203\001\n\016QualifierValue\022\021\n\tqualif" +
-      "ier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022\021\n\ttimestamp\030\003 " +
-      "\001(\004\022.\n\013delete_type\030\004 \001(\0162\031.MutationProto" +
-      ".DeleteType\022\014\n\004tags\030\005 \001(\014\"W\n\nDurability\022" +
-      "\017\n\013USE_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_" +
-      "WAL\020\002\022\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014Mu" +
-      "tationType\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n" +
-      "\003PUT\020\002\022\n\n\006DELETE\020\003\"p\n\nDeleteType\022\026\n\022DELE",
-      "TE_ONE_VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERS" +
-      "IONS\020\001\022\021\n\rDELETE_FAMILY\020\002\022\031\n\025DELETE_FAMI" +
-      "LY_VERSION\020\003\"\207\001\n\rMutateRequest\022 \n\006region" +
-      "\030\001 \002(\0132\020.RegionSpecifier\022 \n\010mutation\030\002 \002" +
-      "(\0132\016.MutationProto\022\035\n\tcondition\030\003 \001(\0132\n." +
-      "Condition\022\023\n\013nonce_group\030\004 \001(\004\"<\n\016Mutate" +
-      "Response\022\027\n\006result\030\001 \001(\0132\007.Result\022\021\n\tpro" +
-      "cessed\030\002 \001(\010\"\271\003\n\004Scan\022\027\n\006column\030\001 \003(\0132\007." +
-      "Column\022!\n\tattribute\030\002 \003(\0132\016.NameBytesPai" +
-      "r\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027",
-      "\n\006filter\030\005 \001(\0132\007.Filter\022\036\n\ntime_range\030\006 " +
-      "\001(\0132\n.TimeRange\022\027\n\014max_versions\030\007 \001(\r:\0011" +
-      "\022\032\n\014cache_blocks\030\010 \001(\010:\004true\022\022\n\nbatch_si" +
-      "ze\030\t \001(\r\022\027\n\017max_result_size\030\n \001(\004\022\023\n\013sto" +
-      "re_limit\030\013 \001(\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036" +
-      "load_column_families_on_demand\030\r \001(\010\022\r\n\005" +
-      "small\030\016 \001(\010\022\027\n\010reversed\030\017 \001(\010:\005false\022)\n\013" +
-      "consistency\030\020 \001(\0162\014.Consistency:\006STRONG\022" +
-      "\017\n\007caching\030\021 \001(\r\"\236\001\n\013ScanRequest\022 \n\006regi" +
-      "on\030\001 \001(\0132\020.RegionSpecifier\022\023\n\004scan\030\002 \001(\013",
-      "2\005.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\rne" +
-      "xt_call_seq\030\006 \001(\004\"\210\001\n\014ScanResponse\022\030\n\020ce" +
-      "lls_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\030\n\007r" +
-      "esults\030\005 \003(\0132\007.Result\022\r\n\005stale\030\006 \001(\010\"\263\001\n" +
-      "\024BulkLoadHFileRequest\022 \n\006region\030\001 \002(\0132\020." +
-      "RegionSpecifier\0225\n\013family_path\030\002 \003(\0132 .B" +
-      "ulkLoadHFileRequest.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\025BulkLoadHFileRespo" +
-      "nse\022\016\n\006loaded\030\001 \002(\010\"a\n\026CoprocessorServic" +
-      "eCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t" +
-      "\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"9" +
-      "\n\030CoprocessorServiceResult\022\035\n\005value\030\001 \001(" +
-      "\0132\016.NameBytesPair\"d\n\031CoprocessorServiceR" +
-      "equest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier" +
-      "\022%\n\004call\030\002 \002(\0132\027.CoprocessorServiceCall\"" +
-      "]\n\032CoprocessorServiceResponse\022 \n\006region\030" +
-      "\001 \002(\0132\020.RegionSpecifier\022\035\n\005value\030\002 \002(\0132\016",
-      ".NameBytesPair\"{\n\006Action\022\r\n\005index\030\001 \001(\r\022" +
-      " \n\010mutation\030\002 \001(\0132\016.MutationProto\022\021\n\003get" +
-      "\030\003 \001(\0132\004.Get\022-\n\014service_call\030\004 \001(\0132\027.Cop" +
-      "rocessorServiceCall\"Y\n\014RegionAction\022 \n\006r" +
-      "egion\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006atomic\030" +
-      "\002 \001(\010\022\027\n\006action\030\003 \003(\0132\007.Action\"D\n\017Region" +
-      "LoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rhe" +
-      "apOccupancy\030\002 \001(\005:\0010\"\266\001\n\021ResultOrExcepti" +
-      "on\022\r\n\005index\030\001 \001(\r\022\027\n\006result\030\002 \001(\0132\007.Resu" +
-      "lt\022!\n\texception\030\003 \001(\0132\016.NameBytesPair\0221\n",
-      "\016service_result\030\004 \001(\0132\031.CoprocessorServi" +
-      "ceResult\022#\n\tloadStats\030\005 \001(\0132\020.RegionLoad" +
-      "Stats\"f\n\022RegionActionResult\022-\n\021resultOrE" +
-      "xception\030\001 \003(\0132\022.ResultOrException\022!\n\tex" +
-      "ception\030\002 \001(\0132\016.NameBytesPair\"f\n\014MultiRe" +
-      "quest\022#\n\014regionAction\030\001 \003(\0132\r.RegionActi" +
-      "on\022\022\n\nnonceGroup\030\002 \001(\004\022\035\n\tcondition\030\003 \001(" +
-      "\0132\n.Condition\"S\n\rMultiResponse\022/\n\022region" +
-      "ActionResult\030\001 \003(\0132\023.RegionActionResult\022" +
-      "\021\n\tprocessed\030\002 \001(\010*\'\n\013Consistency\022\n\n\006STR",
-      "ONG\020\000\022\014\n\010TIMELINE\020\0012\205\003\n\rClientService\022 \n" +
-      "\003Get\022\013.GetRequest\032\014.GetResponse\022)\n\006Mutat" +
-      "e\022\016.MutateRequest\032\017.MutateResponse\022#\n\004Sc" +
-      "an\022\014.ScanRequest\032\r.ScanResponse\022>\n\rBulkL" +
-      "oadHFile\022\025.BulkLoadHFileRequest\032\026.BulkLo" +
-      "adHFileResponse\022F\n\013ExecService\022\032.Coproce" +
-      "ssorServiceRequest\032\033.CoprocessorServiceR" +
-      "esponse\022R\n\027ExecRegionServerService\022\032.Cop" +
-      "rocessorServiceRequest\032\033.CoprocessorServ" +
-      "iceResponse\022&\n\005Multi\022\r.MultiRequest\032\016.Mu",
-      "ltiResponseBB\n*org.apache.hadoop.hbase.p" +
-      "rotobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
+      "ts\030\003 \001(\010\022\024\n\005stale\030\004 \001(\010:\005false\022\026\n\007partia" +
+      "l\030\005 \001(\010:\005false\"A\n\nGetRequest\022 \n\006region\030\001" +
+      " \002(\0132\020.RegionSpecifier\022\021\n\003get\030\002 \002(\0132\004.Ge" +
+      "t\"&\n\013GetResponse\022\027\n\006result\030\001 \001(\0132\007.Resul" +
+      "t\"\200\001\n\tCondition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002",
+      " \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\"\n\014compare_type\030" +
+      "\004 \002(\0162\014.CompareType\022\037\n\ncomparator\030\005 \002(\0132" +
+      "\013.Comparator\"\265\006\n\rMutationProto\022\013\n\003row\030\001 " +
+      "\001(\014\0220\n\013mutate_type\030\002 \001(\0162\033.MutationProto" +
+      ".MutationType\0220\n\014column_value\030\003 \003(\0132\032.Mu" +
+      "tationProto.ColumnValue\022\021\n\ttimestamp\030\004 \001" +
+      "(\004\022!\n\tattribute\030\005 \003(\0132\016.NameBytesPair\022:\n" +
+      "\ndurability\030\006 \001(\0162\031.MutationProto.Durabi" +
+      "lity:\013USE_DEFAULT\022\036\n\ntime_range\030\007 \001(\0132\n." +
+      "TimeRange\022\035\n\025associated_cell_count\030\010 \001(\005",
+      "\022\r\n\005nonce\030\t \001(\004\032\347\001\n\013ColumnValue\022\016\n\006famil" +
+      "y\030\001 \002(\014\022B\n\017qualifier_value\030\002 \003(\0132).Mutat" +
+      "ionProto.ColumnValue.QualifierValue\032\203\001\n\016" +
+      "QualifierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005val" +
+      "ue\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004\022.\n\013delete_ty" +
+      "pe\030\004 \001(\0162\031.MutationProto.DeleteType\022\014\n\004t" +
+      "ags\030\005 \001(\014\"W\n\nDurability\022\017\n\013USE_DEFAULT\020\000" +
+      "\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_WAL\020\002\022\014\n\010SYNC_WA" +
+      "L\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014MutationType\022\n\n\006AP" +
+      "PEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE",
+      "\020\003\"p\n\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020\000" +
+      "\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELETE" +
+      "_FAMILY\020\002\022\031\n\025DELETE_FAMILY_VERSION\020\003\"\207\001\n" +
+      "\rMutateRequest\022 \n\006region\030\001 \002(\0132\020.RegionS" +
+      "pecifier\022 \n\010mutation\030\002 \002(\0132\016.MutationPro" +
+      "to\022\035\n\tcondition\030\003 \001(\0132\n.Condition\022\023\n\013non" +
+      "ce_group\030\004 \001(\004\"<\n\016MutateResponse\022\027\n\006resu" +
+      "lt\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\271\003\n" +
+      "\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tattrib" +
+      "ute\030\002 \003(\0132\016.NameBytesPair\022\021\n\tstart_row\030\003",
+      " \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007" +
+      ".Filter\022\036\n\ntime_range\030\006 \001(\0132\n.TimeRange\022" +
+      "\027\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blocks" +
+      "\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017max_" +
+      "result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(\r\022\024" +
+      "\n\014store_offset\030\014 \001(\r\022&\n\036load_column_fami" +
+      "lies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\022\027\n\010r" +
+      "eversed\030\017 \001(\010:\005false\022)\n\013consistency\030\020 \001(" +
+      "\0162\014.Consistency:\006STRONG\022\017\n\007caching\030\021 \001(\r" +
+      "\"\277\001\n\013ScanRequest\022 \n\006region\030\001 \001(\0132\020.Regio",
+      "nSpecifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\022\n\nscann" +
+      "er_id\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rc" +
+      "lose_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\"\251\001\n\014S" +
+      "canResponse\022\030\n\020cells_per_result\030\001 \003(\r\022\037\n" +
+      "\027partial_flag_per_result\030\002 \003(\010\022\022\n\nscanne" +
+      "r_id\030\003 \001(\004\022\024\n\014more_results\030\004 \001(\010\022\013\n\003ttl\030" +
+      "\005 \001(\r\022\030\n\007results\030\006 \003(\0132\007.Result\022\r\n\005stale" +
+      "\030\007 \001(\010\"\263\001\n\024BulkLoadHFileRequest\022 \n\006regio" +
+      "n\030\001 \002(\0132\020.RegionSpecifier\0225\n\013family_path",
+      "\030\002 \003(\0132 .BulkLoadHFileRequest.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\025BulkLoad" +
+      "HFileResponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026Coproce" +
+      "ssorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_" +
+      "name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007reque" +
+      "st\030\004 \002(\014\"9\n\030CoprocessorServiceResult\022\035\n\005" +
+      "value\030\001 \001(\0132\016.NameBytesPair\"d\n\031Coprocess" +
+      "orServiceRequest\022 \n\006region\030\001 \002(\0132\020.Regio" +
+      "nSpecifier\022%\n\004call\030\002 \002(\0132\027.CoprocessorSe",
+      "rviceCall\"]\n\032CoprocessorServiceResponse\022" +
+      " \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\035\n\005val" +
+      "ue\030\002 \002(\0132\016.NameBytesPair\"{\n\006Action\022\r\n\005in" +
+      "dex\030\001 \001(\r\022 \n\010mutation\030\002 \001(\0132\016.MutationPr" +
+      "oto\022\021\n\003get\030\003 \001(\0132\004.Get\022-\n\014service_call\030\004" +
+      " \001(\0132\027.CoprocessorServiceCall\"Y\n\014RegionA" +
+      "ction\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022" +
+      "\016\n\006atomic\030\002 \001(\010\022\027\n\006action\030\003 \003(\0132\007.Action" +
+      "\"D\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(" +
+      "\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\"\266\001\n\021Resul",
+      "tOrException\022\r\n\005index\030\001 \001(\r\022\027\n\006result\030\002 " +
+      "\001(\0132\007.Result\022!\n\texception\030\003 \001(\0132\016.NameBy" +
+      "tesPair\0221\n\016service_result\030\004 \001(\0132\031.Coproc" +
+      "essorServiceResult\022#\n\tloadStats\030\005 \001(\0132\020." +
+      "RegionLoadStats\"f\n\022RegionActionResult\022-\n" +
+      "\021resultOrException\030\001 \003(\0132\022.ResultOrExcep" +
+      "tion\022!\n\texception\030\002 \001(\0132\016.NameBytesPair\"" +
+      "f\n\014MultiRequest\022#\n\014regionAction\030\001 \003(\0132\r." +
+      "RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022\035\n\tcond" +
+      "ition\030\003 \001(\0132\n.Condition\"S\n\rMultiResponse",
+      "\022/\n\022regionActionResult\030\001 \003(\0132\023.RegionAct" +
+      "ionResult\022\021\n\tprocessed\030\002 \001(\010*\'\n\013Consiste" +
+      "ncy\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\205\003\n\rClient" +
+      "Service\022 \n\003Get\022\013.GetRequest\032\014.GetRespons" +
+      "e\022)\n\006Mutate\022\016.MutateRequest\032\017.MutateResp" +
+      "onse\022#\n\004Scan\022\014.ScanRequest\032\r.ScanRespons" +
+      "e\022>\n\rBulkLoadHFile\022\025.BulkLoadHFileReques" +
+      "t\032\026.BulkLoadHFileResponse\022F\n\013ExecService" +
+      "\022\032.CoprocessorServiceRequest\032\033.Coprocess" +
+      "orServiceResponse\022R\n\027ExecRegionServerSer",
+      "vice\022\032.CoprocessorServiceRequest\032\033.Copro" +
+      "cessorServiceResponse\022&\n\005Multi\022\r.MultiRe" +
+      "quest\032\016.MultiResponseBB\n*org.apache.hado" +
+      "op.hbase.protobuf.generatedB\014ClientProto" +
+      "sH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -32114,7 +32643,7 @@ public final class ClientProtos {
           internal_static_Result_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_Result_descriptor,
-              new java.lang.String[] { "Cell", "AssociatedCellCount", "Exists", "Stale", });
+              new java.lang.String[] { "Cell", "AssociatedCellCount", "Exists", "Stale", "Partial", });
           internal_static_GetRequest_descriptor =
             getDescriptor().getMessageTypes().get(5);
           internal_static_GetRequest_fieldAccessorTable = new
@@ -32174,13 +32703,13 @@ public final class ClientProtos {
           internal_static_ScanRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ScanRequest_descriptor,
-              new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", });
+              new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", });
           internal_static_ScanResponse_descriptor =
             getDescriptor().getMessageTypes().get(13);
           internal_static_ScanResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ScanResponse_descriptor,
-              new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", });
+              new java.lang.String[] { "CellsPerResult", "PartialFlagPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", });
           internal_static_BulkLoadHFileRequest_descriptor =
             getDescriptor().getMessageTypes().get(14);
           internal_static_BulkLoadHFileRequest_fieldAccessorTable = new
diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto
index 606ca8d..6803898 100644
--- a/hbase-protocol/src/main/protobuf/Client.proto
+++ b/hbase-protocol/src/main/protobuf/Client.proto
@@ -105,6 +105,12 @@ message Result {
 
   // Whether or not the results are coming from possibly stale data 
   optional bool stale = 4 [default = false];
+
+  // Whether or not the entire result could be returned. Results will be split when
+  // the RPC chunk size limit is reached. Partial results contain only a subset of the
+  // cells for a row and must be combined with a result containing the remaining cells
+  // to form a complete result
+  optional bool partial = 5 [default = false];
 }
 
 /**
@@ -268,6 +274,7 @@ message ScanRequest {
   optional uint32 number_of_rows = 4;
   optional bool close_scanner = 5;
   optional uint64 next_call_seq = 6;
+  optional bool client_handles_partials = 7;
 }
 
 /**
@@ -283,14 +290,23 @@ message ScanResponse {
   // has 3, 3, 3 in it, then we know that on the client, we are to make
   // three Results each of three Cells each.
   repeated uint32 cells_per_result = 1;
-  optional uint64 scanner_id = 2;
-  optional bool more_results = 3;
-  optional uint32 ttl = 4;
+  
+  // This field is filled in if we are doing cellblocks. In the event that a row
+  // could not fit all of its cells into a single RPC chunk, the results will be
+  // returned as partials, and reconstructed into a complete result on the client 
+  // side. This field is a list of flags indicating whether or not the result
+  // that the cells belong to is a partial result. For example, if this field
+  // has false, false, true in it, then we know that on the client side, we need to 
+  // make another RPC request since the last result was only a partial.
+  repeated bool partial_flag_per_result = 2;
+  optional uint64 scanner_id = 3;
+  optional bool more_results = 4;
+  optional uint32 ttl = 5;
   // If cells are not carried in an accompanying cellblock, then they are pb'd here.
   // This field is mutually exclusive with cells_per_result (since the Cells will
   // be inside the pb'd Result)
-  repeated Result results = 5;
-  optional bool stale = 6;
+  repeated Result results = 6;
+  optional bool stale = 7;
 }
 
 /**
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
index ffb2fae..ee700bb 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
@@ -33,11 +33,10 @@ import javax.ws.rs.core.UriInfo;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
index 2bab21b..1688616 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -30,6 +29,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -72,7 +72,10 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
   public Result next() throws IOException {
     values.clear();
 
-    scanner.nextRaw(values, -1); // pass -1 as limit so that we see the whole row.
+    // negative values indicate no limits
+    final long remainingResultSize = -1;
+    final int batchLimit = -1;
+    scanner.nextRaw(values, batchLimit, remainingResultSize);
     if (values.isEmpty()) {
       //we are done
       return null;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
index baf2aa6..a371e3e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
@@ -26,14 +26,14 @@ import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 79cb7db..98fb66e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -139,8 +139,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescripto
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.WriteEntry;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
-import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
+import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -4951,8 +4951,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
      * If the joined heap data gathering is interrupted due to scan limits, this will
      * contain the row for which we are populating the values.*/
     protected Cell joinedContinuationRow = null;
-    // KeyValue indicating that limit is reached when scanning
-    private final KeyValue KV_LIMIT = new KeyValue();
+    // KeyValue indicating that batch limit is reached when scanning
+    private final KeyValue KV_BATCH_LIMIT = new KeyValue();
+    /**
+     * KeyValue indicating that the result size limit is reached when scanning. If the result size
+     * limit is reached, a partial {@link Result} will be returned to the client and it will be the
+     * client's responsibility to form another RPC request to complete the result
+     */
+    private final KeyValue KV_SIZE_LIMIT = new KeyValue();
     protected final byte[] stopRow;
     private final FilterWrapper filter;
     private int batch;
@@ -5037,6 +5043,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
       return this.readPt;
     }
 
+    @Override
+    public int getBatch() {
+      return this.batch;
+    }
+
     /**
      * Reset both the filter and the old filter.
      *
@@ -5057,6 +5068,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
 
     @Override
     public synchronized boolean next(List outResults, int limit) throws IOException {
+      return next(outResults, limit, -1);
+    }
+
+    @Override
+    public synchronized boolean next(List outResults, int limit, long remainingResultSize)
+        throws IOException {
       if (this.filterClosed) {
         throw new UnknownScannerException("Scanner was closed (timed out?) " +
             "after we renewed it. Could be caused by a very slow scanner " +
@@ -5065,48 +5082,55 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
       startRegionOperation(Operation.SCAN);
       readRequestsCount.increment();
       try {
-        return nextRaw(outResults, limit);
+        return nextRaw(outResults, limit, remainingResultSize).moreRows();
       } finally {
         closeRegionOperation(Operation.SCAN);
       }
     }
 
     @Override
-    public boolean nextRaw(List outResults)
-        throws IOException {
+    public NextRawState nextRaw(List outResults) throws IOException {
       return nextRaw(outResults, batch);
     }
 
     @Override
-    public boolean nextRaw(List outResults, int limit) throws IOException {
+    public NextRawState nextRaw(List outResults, int limit)
+        throws IOException {
+      return nextRaw(outResults, limit, -1);
+    }
+
+    @Override
+    public NextRawState nextRaw(List outResults, int batchLimit, long remainingResultSize)
+        throws IOException {
       if (storeHeap == null) {
         // scanner is closed
         throw new UnknownScannerException("Scanner was closed");
       }
-      boolean returnResult;
+      NextRawState state;
       if (outResults.isEmpty()) {
         // Usually outResults is empty. This is true when next is called
         // to handle scan or get operation.
-        returnResult = nextInternal(outResults, limit);
+        state = nextInternal(outResults, batchLimit, remainingResultSize);
       } else {
         List tmpList = new ArrayList();
-        returnResult = nextInternal(tmpList, limit);
+        state = nextInternal(tmpList, batchLimit, remainingResultSize);
         outResults.addAll(tmpList);
       }
       resetFilters();
       if (isFilterDoneInternal()) {
-        returnResult = false;
+        state = NextRawState.NO_MORE_ROWS;
       }
-      return returnResult;
+      return state;
     }
 
-    private void populateFromJoinedHeap(List results, int limit)
+    private void populateFromJoinedHeap(List results, int limit, long remainingResultSize)
         throws IOException {
       assert joinedContinuationRow != null;
-      Cell kv = populateResult(results, this.joinedHeap, limit,
+      Cell kv =
+          populateResult(results, this.joinedHeap, limit, remainingResultSize,
           joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
           joinedContinuationRow.getRowLength());
-      if (kv != KV_LIMIT) {
+      if (kv != KV_BATCH_LIMIT && kv != KV_SIZE_LIMIT) {
         // We are done with this row, reset the continuation.
         joinedContinuationRow = null;
       }
@@ -5116,28 +5140,78 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
     }
 
     /**
-     * Fetches records with currentRow into results list, until next row or limit (if not -1).
+     * Fetches records with currentRow into results list, until next row, batchLimit (if not -1) is
+     * reached, or remainingResultSize (if not -1) is reaced
      * @param heap KeyValueHeap to fetch data from.It must be positioned on correct row before call.
-     * @param limit Max amount of KVs to place in result list, -1 means no limit.
+     * @param remainingResultSize The remaining space within our result size limit. A negative value
+     *          indicate no limit
+     * @param batchLimit Max amount of KVs to place in result list, -1 means no limit.
      * @param currentRow Byte array with key we are fetching.
      * @param offset offset for currentRow
      * @param length length for currentRow
      * @return KV_LIMIT if limit reached, next KeyValue otherwise.
      */
-    private Cell populateResult(List results, KeyValueHeap heap, int limit,
-        byte[] currentRow, int offset, short length) throws IOException {
+    private Cell populateResult(List results, KeyValueHeap heap, int batchLimit,
+        long remainingResultSize, byte[] currentRow, int offset, short length) throws IOException {
       Cell nextKv;
+      boolean moreCellsInRow = false;
       do {
-        heap.next(results, limit - results.size());
-        if (limit > 0 && results.size() == limit) {
-          return KV_LIMIT;
+        int remainingBatchLimit = batchLimit - results.size();
+        heap.next(results, remainingBatchLimit, 
+          remainingResultSize - calculateResultSize(results));
+        if (batchLimit > 0 && results.size() == batchLimit) {
+          return KV_BATCH_LIMIT;
         }
+
         nextKv = heap.peek();
-      } while (nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length));
+        moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length);
+        if (moreCellsInRow && sizeLimitReached(remainingResultSize, results)) {
+          return KV_SIZE_LIMIT;
+        }
+      } while (moreCellsInRow);
 
       return nextKv;
     }
 
+    /**
+     * Based on the nextKv in the heap, and the current row, decide whether or not there are more
+     * cells to be read in the heap. If the row of the nextKv in the heap matches the current row
+     * then there are more cells to be read in the row.
+     * @param nextKv
+     * @param currentRow
+     * @param offset
+     * @param length
+     * @return true When there are more cells in the row to be read
+     */
+    private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset, 
+        short length) {
+      return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length);
+    }
+
+    /**
+     * @param remainingResultSize
+     * @param results
+     * @return true When a maxResultSize limit has been specified and we have reached that limit
+     */
+    private boolean sizeLimitReached(final long remainingResultSize, List results) {
+      return remainingResultSize > 0 && calculateResultSize(results) >= remainingResultSize;
+    }
+
+    /**
+     * @param results List of cells we want to calculate size of
+     * @return aggregate size of results as estimated by {@link CellUtil#estimatedHeapSizeOf(Cell)}
+     */
+    private long calculateResultSize(List results) {
+      if (results == null || results.isEmpty()) return 0;
+
+      long size = 0;
+      for (Cell c : results) {
+        size += CellUtil.estimatedHeapSizeOf(c);
+      }
+
+      return size;
+    }
+
     /*
      * @return True if a filter rules the scanner is over, done.
      */
@@ -5150,8 +5224,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
       return this.filter != null && this.filter.filterAllRemaining();
     }
 
-    private boolean nextInternal(List results, int limit)
-    throws IOException {
+    private NextRawState nextInternal(List results, int batchLimit, long remainingResultSize)
+        throws IOException {
       if (!results.isEmpty()) {
         throw new IllegalArgumentException("First parameter should be an empty list");
       }
@@ -5187,36 +5261,62 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
           offset = current.getRowOffset();
           length = current.getRowLength();
         }
+
         boolean stopRow = isStopRow(currentRow, offset, length);
+        boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow();
+
+        // If filter#hasFilterRow is true, partial results are not allowed since allowing them
+        // would prevent the filters from being evaluated. Thus, if it is true, change the
+        // remainingResultSize to -1 so that the entire row's worth of cells are fetched.
+        if (hasFilterRow && remainingResultSize > 0) {
+          remainingResultSize = -1;
+          if (LOG.isWarnEnabled()) {
+            LOG.warn("filter#hasFilterRow is true which prevents partial results from being "
+                + " formed. The remainingResultSize of: " + remainingResultSize + " will not "
+                + " be considered when fetching the cells for this row.");
+          }
+        }
+
         // Check if we were getting data from the joinedHeap and hit the limit.
         // If not, then it's main path - getting results from storeHeap.
         if (joinedContinuationRow == null) {
           // First, check if we are at a stop row. If so, there are no more results.
           if (stopRow) {
-            if (filter != null && filter.hasFilterRow()) {
+            if (hasFilterRow) {
               filter.filterRowCells(results);
             }
-            return false;
+            return NextRawState.NO_MORE_ROWS;
           }
 
           // Check if rowkey filter wants to exclude this row. If so, loop to next.
           // Technically, if we hit limits before on this row, we don't need this call.
           if (filterRowKey(currentRow, offset, length)) {
             boolean moreRows = nextRow(currentRow, offset, length);
-            if (!moreRows) return false;
+            if (!moreRows) return NextRawState.NO_MORE_ROWS;
             results.clear();
             continue;
           }
 
-          Cell nextKv = populateResult(results, this.storeHeap, limit, currentRow, offset,
-              length);
+          Cell nextKv =
+              populateResult(results, this.storeHeap, batchLimit, remainingResultSize, currentRow,
+                offset, length);
           // Ok, we are good, let's try to get some results from the main heap.
-          if (nextKv == KV_LIMIT) {
-            if (this.filter != null && filter.hasFilterRow()) {
+          if (nextKv == KV_BATCH_LIMIT) {
+            if (hasFilterRow) {
               throw new IncompatibleFilterException(
                 "Filter whose hasFilterRow() returns true is incompatible with scan with limit!");
             }
-            return true; // We hit the limit.
+            return NextRawState.BATCH_LIMIT_REACHED; // We hit the batch limit.
+          } else if (nextKv == KV_SIZE_LIMIT) {
+            if (hasFilterRow) {
+              // We try to guard against this case above when remainingResultSize is set to -1 if
+              // hasFilterRow is true. In the even that the guard doesn't work, an exception must be
+              // thrown
+              throw new IncompatibleFilterException(
+                  "Filter whose hasFilterRows() returns true is incompatible with scans that"
+                      + " return partial results");
+            }
+            return NextRawState.SIZE_LIMIT_REACHED; // We hit the size limit.
           }
 
           stopRow = nextKv == null ||
@@ -5227,19 +5327,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
           // We have the part of the row necessary for filtering (all of it, usually).
           // First filter with the filterRow(List).
           FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;
-          if (filter != null && filter.hasFilterRow()) {
+          if (hasFilterRow) {
             ret = filter.filterRowCellsWithRet(results);
           }
 
           if ((isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE) || filterRow()) {
             results.clear();
             boolean moreRows = nextRow(currentRow, offset, length);
-            if (!moreRows) return false;
+            if (!moreRows) return NextRawState.NO_MORE_ROWS;
 
             // This row was totally filtered out, if this is NOT the last row,
             // we should continue on. Otherwise, nothing else to do.
             if (!stopRow) continue;
-            return false;
+            return NextRawState.NO_MORE_ROWS;
           }
 
           // Ok, we are done with storeHeap for this row.
@@ -5257,18 +5357,20 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
                     currentRow, offset, length));
             if (mayHaveData) {
               joinedContinuationRow = current;
-              populateFromJoinedHeap(results, limit);
+              populateFromJoinedHeap(results, batchLimit, remainingResultSize
+                  - calculateResultSize(results));
             }
           }
         } else {
           // Populating from the joined heap was stopped by limits, populate some more.
-          populateFromJoinedHeap(results, limit);
+          populateFromJoinedHeap(results, batchLimit, remainingResultSize
+              - calculateResultSize(results));
         }
 
         // We may have just called populateFromJoinedMap and hit the limits. If that is
         // the case, we need to call it again on the next next() invocation.
         if (joinedContinuationRow != null) {
-          return true;
+          return NextRawState.MORE_ROWS;
         }
 
         // Finally, we are done with both joinedHeap and storeHeap.
@@ -5276,12 +5378,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
         // the case when SingleColumnValueExcludeFilter is used.
         if (results.isEmpty()) {
           boolean moreRows = nextRow(currentRow, offset, length);
-          if (!moreRows) return false;
+          if (!moreRows) return NextRawState.NO_MORE_ROWS;
           if (!stopRow) continue;
         }
 
         // We are done. Return the result.
-        return !stopRow;
+        return stopRow ? NextRawState.NO_MORE_ROWS : NextRawState.MORE_ROWS;
       }
     }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java
index 41708c0..b193c67 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java
@@ -22,8 +22,8 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * Internal scanners differ from client-side scanners in that they operate on
@@ -60,6 +60,17 @@ public interface InternalScanner extends Closeable {
   boolean next(List result, int limit) throws IOException;
 
   /**
+   * Grab the next row's worth of values with a limit on the number of values to return as well as a
+   * restriction on the size of the list of values that are returned.
+   * @param result return output array
+   * @param limit limit on row count to get
+   * @param remainingResultSize limit on the size of the result being returned
+   * @return true if more rows exist after this one, false if scanner is done
+   * @throws IOException e
+   */
+  boolean next(List result, int limit, long remainingResultSize) throws IOException;
+
+  /**
    * Closes the scanner and releases any resources it has allocated
    * @throws IOException
    */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
index 23834d3..86347ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
@@ -24,9 +24,9 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.PriorityQueue;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * Implements a heap merge across any number of KeyValueScanners.
@@ -132,11 +132,15 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
    * @return true if there are more keys, false if all scanners are done
    */
   public boolean next(List result, int limit) throws IOException {
+    return next(result, limit, -1);
+  }
+
+  public boolean next(List result, int limit, long remainingResultSize) throws IOException {
     if (this.current == null) {
       return false;
     }
     InternalScanner currentAsInternal = (InternalScanner)this.current;
-    boolean mayContainMoreRows = currentAsInternal.next(result, limit);
+    boolean mayContainMoreRows = currentAsInternal.next(result, limit, remainingResultSize);
     Cell pee = this.current.peek();
     /*
      * By definition, any InternalScanner must return false only when it has no
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 62c3305..9e608cc 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
@@ -150,6 +150,7 @@ import org.apache.hadoop.hbase.quotas.OperationQuota;
 import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
 import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
+import org.apache.hadoop.hbase.regionserver.RegionScanner.NextRawState;
 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -338,6 +339,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     if (isClientCellBlockSupport()) {
       for (Result res : results) {
         builder.addCellsPerResult(res.size());
+        builder.addPartialFlagPerResult(res.isPartial());
       }
       ((PayloadCarryingRpcController)controller).
         setCellScanner(CellUtil.createCellScanner(results));
@@ -2042,6 +2044,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       RegionScannerHolder rsh = null;
       boolean moreResults = true;
       boolean closeScanner = false;
+      boolean isSmallScan = false;
       ScanResponse.Builder builder = ScanResponse.newBuilder();
       if (request.hasCloseScanner()) {
         closeScanner = request.getCloseScanner();
@@ -2073,6 +2076,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         if (!isLoadingCfsOnDemandSet) {
           scan.setLoadColumnFamiliesOnDemand(region.isLoadingCfsOnDemandDefault());
         }
+
+        isSmallScan = scan.isSmall();
         region.prepareScanner(scan);
         if (region.getCoprocessorHost() != null) {
           scanner = region.getCoprocessorHost().preScannerOpen(scan);
@@ -2113,9 +2118,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           // Remove lease while its being processed in server; protects against case
           // where processing of request takes > lease expiration time.
           lease = regionServer.leases.removeLease(scannerName);
-          List results = new ArrayList(rows);
-          long currentScanResultSize = 0;
+          List results = new ArrayList();
           long totalCellSize = 0;
+          long currentScanResultSize = 0;
 
           boolean done = false;
           // Call coprocessor. Get region info from scanner.
@@ -2125,8 +2130,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             if (!results.isEmpty()) {
               for (Result r : results) {
                 for (Cell cell : r.rawCells()) {
-                  currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
                   totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
+                  currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
                 }
               }
             }
@@ -2146,23 +2151,47 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
               int i = 0;
               synchronized(scanner) {
                 boolean stale = (region.getRegionInfo().getReplicaId() != 0);
+                boolean clientHandlesPartials =
+                    request.hasClientHandlesPartials() && request.getClientHandlesPartials();
+
+                // On the server side we must ensure that the correct ordering of partial results is
+                // returned to the client to allow them to properly reconstruct the partial results.
+                // If the coprocessor host is adding to the result list, we cannot guarantee the
+                // correct ordering of partial results and so we prevent partial results from being
+                // formed.
+                boolean serverGuaranteesOrderOfPartials = currentScanResultSize == 0;
+                boolean enforceMaxResultSizeAtCellLevel =
+                    clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan;
+
                 while (i < rows) {
-                  // Stop collecting results if maxScannerResultSize is set and we have exceeded it
-                  if ((maxScannerResultSize < Long.MAX_VALUE) &&
-                      (currentScanResultSize >= maxResultSize)) {
+                  // Stop collecting results if we have exceeded maxResultSize
+                  if (currentScanResultSize >= maxResultSize) {
                     break;
                   }
+
+                  // A negative remainingResultSize communicates that there is no limit on the size
+                  // of the results.
+                  final long remainingResultSize =
+                      enforceMaxResultSizeAtCellLevel ? maxResultSize - currentScanResultSize
+                          : -1;
+
                   // Collect values to be returned here
-                  boolean moreRows = scanner.nextRaw(values);
+                  NextRawState state =
+                      scanner.nextRaw(values, scanner.getBatch(), remainingResultSize);
+
                   if (!values.isEmpty()) {
                     for (Cell cell : values) {
-                      currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
                       totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
+                      currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
                     }
-                    results.add(Result.create(values, null, stale));
+                    // The size limit was reached. This means there are more cells remaining in
+                    // the row but we had to stop because we exceeded our max result size. This
+                    // indicates that we are returning a partial result
+                    final boolean partial = state == NextRawState.SIZE_LIMIT_REACHED;
+                    results.add(Result.create(values, null, stale, partial));
                     i++;
                   }
-                  if (!moreRows) {
+                  if (!state.moreRows()) {
                     break;
                   }
                   values.clear();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
index ec68dc7..e55eb74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
@@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * RegionScanner describes iterators over rows in an HRegion.
@@ -66,27 +66,72 @@ public interface RegionScanner extends InternalScanner {
    * @return The Scanner's MVCC readPt see {@link MultiVersionConsistencyControl}
    */
   long getMvccReadPoint();
+  
+  /**
+   * @return The limit on the number of cells to retrieve on each call to next(). See
+   *         {@link org.apache.hadoop.hbase.client.Scan#setBatch(int)}
+   */
+  int getBatch();
 
   /**
-   * Grab the next row's worth of values with the default limit on the number of values
-   * to return.
+   * Enumeration of possible states that scanner can be in after a call to
+   * {@link RegionScanner#nextRaw(List)}. The enumeration is necessary so that we can communicate
+   * the state of the scanner to the caller of nextRaw
+   * 

+ * {@link NextRawState#moreRows()} can be used to determine whether or not this state indicates + * that the scanner has more rows to be scanned + */ + public enum NextRawState { + MORE_ROWS(true), + NO_MORE_ROWS(false), + BATCH_LIMIT_REACHED(true), + SIZE_LIMIT_REACHED(true); + + private boolean moreRows; + NextRawState(final boolean moreRows) { + this.moreRows = moreRows; + } + + /** + * @return true when the state indicates that there are more rows left to be scanned + */ + public boolean moreRows() { + return this.moreRows; + } + } + + /** + * Grab the next row's worth of values with the default limit on the number of values to return. * This is a special internal method to be called from coprocessor hooks to avoid expensive setup. - * Caller must set the thread's readpoint, start and close a region operation, an synchronize on the scanner object. - * Caller should maintain and update metrics. - * See {@link #nextRaw(List, int)} + * Caller must set the thread's readpoint, start and close a region operation, an synchronize on + * the scanner object. Caller should maintain and update metrics. See + * {@link #nextRaw(List, int, long)} * @param result return output array - * @return true if more rows exist after this one, false if scanner is done + * @return a state where {@link NextRawState#moreRows()} is true when more rows exist, false when + * scanner is done. * @throws IOException e */ - boolean nextRaw(List result) throws IOException; + NextRawState nextRaw(List result) throws IOException; /** - * Grab the next row's worth of values with a limit on the number of values - * to return. + * Grab the next row's worth of values with the default limit on the number of values to return. * This is a special internal method to be called from coprocessor hooks to avoid expensive setup. - * Caller must set the thread's readpoint, start and close a region operation, an synchronize on the scanner object. - * Example: - *

+   * Caller must set the thread's readpoint, start and close a region operation, an synchronize on
+   * the scanner object. Caller should maintain and update metrics. See
+   * {@link #nextRaw(List, int, long)}
+   * @param result return output array
+   * @param limit limit on row count to get
+   * @return a state where {@link NextRawState#moreRows()} is true when more rows exist, false when
+   *         scanner is done.
+   * @throws IOException e
+   */
+  NextRawState nextRaw(List result, int limit) throws IOException;
+  
+  /**
+   * Grab the next row's worth of values with a limit on the number of values to return as well as a
+   * limit on the heap size of those values. This is a special internal method to be called from
+   * coprocessor hooks to avoid expensive setup. Caller must set the thread's readpoint, start and
+   * close a region operation, an synchronize on the scanner object. Example: 
    * HRegion region = ...;
    * RegionScanner scanner = ...
    * MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
@@ -103,8 +148,12 @@ public interface RegionScanner extends InternalScanner {
    * 
* @param result return output array * @param limit limit on row count to get - * @return true if more rows exist after this one, false if scanner is done + * @param remainingResultSize the space remaining within the restriction on the result size. + * Negative values indicate no limit + * @return a state where {@link NextRawState#moreRows()} is true when more rows exist, false when + * scanner is done. * @throws IOException e */ - boolean nextRaw(List result, int limit) throws IOException; + NextRawState nextRaw(List result, int limit, final long remainingResultSize) + throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 9db116e..ea54f54 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -30,7 +30,6 @@ import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.executor.ExecutorService; @@ -449,6 +449,20 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner */ @Override public boolean next(List outResult, int limit) throws IOException { + // -1 means no limit + return next(outResult, limit, -1); + } + + /** + * Get the next row of values from this Store. + * @param outResult + * @param limit + * @param remainingResultSize + * @return true if there are more rows, false if scanner is done + */ + @Override + public boolean next(List outResult, int limit, long remainingResultSize) + throws IOException { lock.lock(); try { if (checkReseek()) { @@ -473,10 +487,15 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner byte[] row = peeked.getRowArray(); int offset = peeked.getRowOffset(); short length = peeked.getRowLength(); - if (limit < 0 || matcher.row == null || !Bytes.equals(row, offset, length, matcher.row, - matcher.rowOffset, matcher.rowLength)) { - this.countPerRow = 0; - matcher.setRow(row, offset, length); + + // If limit < 0 and remainingResultSize < 0 we can skip the row comparison because we know + // the row has changed. Else it is possible we are still traversing the same row so we + // must perform the row comparison. + if ((limit < 0 && remainingResultSize < 0) || matcher.row == null + || !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, + matcher.rowLength)) { + this.countPerRow = 0; + matcher.setRow(row, offset, length); } Cell cell; @@ -487,6 +506,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner int count = 0; long totalBytesRead = 0; + long totalHeapSize = 0; LOOP: while((cell = this.heap.peek()) != null) { if (prevCell != cell) ++kvsScanned; // Do object compare - we set prevKV from the same heap. @@ -522,6 +542,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner outResult.add(cell); count++; totalBytesRead += CellUtil.estimatedSerializedSizeOf(cell); + totalHeapSize += CellUtil.estimatedHeapSizeOf(cell); if (totalBytesRead > maxRowSize) { throw new RowTooBigException("Max row size allowed: " + maxRowSize + ", but the row is bigger than that."); @@ -542,6 +563,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner if (limit > 0 && (count == limit)) { break LOOP; } + if (remainingResultSize > 0 && (totalHeapSize >= remainingResultSize)) { + break LOOP; + } continue; case DONE: 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 new file mode 100644 index 0000000..f45f67c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java @@ -0,0 +1,789 @@ +/** + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.client.ClientScanner; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +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.filter.RandomRowFilter; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * These tests are focused on testing how partial results appear to a client. Partial results are + * {@link Result}s that contain only a portion of a row's complete list of cells. Partial results + * are formed when the server breaches its maximum result size when trying to service a client's RPC + * request. It is the responsibility of the scanner on the client side to recognize when partial + * results have been returned and to take action to form the complete results. + *

+ * Unless the flag {@link Scan#setAllowPartialResults(boolean)} has been set to true, the caller of + * {@link ResultScanner#next()} should never see partial results. + */ +@Category(MediumTests.class) +public class TestPartialResultsFromClientSide { + private static final Log LOG = LogFactory.getLog(TestPartialResultsFromClientSide.class); + + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + private static Table TABLE = null; + + /** + * Table configuration + */ + private static TableName TABLE_NAME = TableName.valueOf("testTable"); + + private static int NUM_ROWS = 5; + private static byte[] ROW = Bytes.toBytes("testRow"); + private static byte[][] ROWS = HTestConst.makeNAscii(ROW, NUM_ROWS); + + // Should keep this value below 10 to keep generation of expected kv's simple. If above 10 then + // table/row/cf1/... will be followed by table/row/cf10/... instead of table/row/cf2/... which + // breaks the simple generation of expected kv's + private static int NUM_FAMILIES = 10; + private static byte[] FAMILY = Bytes.toBytes("testFamily"); + private static byte[][] FAMILIES = HTestConst.makeNAscii(FAMILY, NUM_FAMILIES); + + private static int NUM_QUALIFIERS = 10; + private static byte[] QUALIFIER = Bytes.toBytes("testQualifier"); + private static byte[][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, NUM_QUALIFIERS); + + private static int VALUE_SIZE = 1024; + private static byte[] VALUE = Bytes.createMaxByteArray(VALUE_SIZE); + + private static int NUM_COLS = NUM_FAMILIES * NUM_QUALIFIERS; + + // Approximation of how large the heap size of cells in our table. Should be accessed through + // getCellHeapSize(). + private static long CELL_HEAP_SIZE = -1; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.startMiniCluster(3); + TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE); + } + + static Table createTestTable(TableName name, byte[][] rows, byte[][] families, + byte[][] qualifiers, byte[] cellValue) throws IOException { + Table ht = TEST_UTIL.createTable(name, families); + List puts = createPuts(rows, families, qualifiers, cellValue); + ht.put(puts); + + return ht; + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * Ensure that the expected key values appear in a result returned from a scanner that is + * combining partial results into complete results + * @throws Exception + */ + @Test + public void testExpectedValuesOfPartialResults() throws Exception { + testExpectedValuesOfPartialResults(false); + testExpectedValuesOfPartialResults(true); + } + + public void testExpectedValuesOfPartialResults(boolean reversed) throws Exception { + Scan partialScan = new Scan(); + partialScan.setMaxVersions(); + // Max result size of 1 ensures that each RPC request will return a single cell. The scanner + // will need to reconstruct the results into a complete result before returning to the caller + partialScan.setMaxResultSize(1); + partialScan.setReversed(reversed); + ResultScanner partialScanner = TABLE.getScanner(partialScan); + + final int startRow = reversed ? ROWS.length - 1 : 0; + final int endRow = reversed ? -1 : ROWS.length; + final int loopDelta = reversed ? -1 : 1; + String message; + + for (int row = startRow; row != endRow; row = row + loopDelta) { + message = "Ensuring the expected keyValues are present for row " + row; + List expectedKeyValues = createKeyValuesForRow(ROWS[row], FAMILIES, QUALIFIERS, VALUE); + Result result = partialScanner.next(); + assertFalse(result.isPartial()); + verifyResult(result, expectedKeyValues, message); + } + + partialScanner.close(); + } + + /** + * Ensure that we only see Results marked as partial when the allowPartial flag is set + * @throws Exception + */ + @Test + public void testAllowPartialResults() throws Exception { + Scan scan = new Scan(); + scan.setAllowPartialResults(true); + scan.setMaxResultSize(1); + ResultScanner scanner = TABLE.getScanner(scan); + Result result = scanner.next(); + + assertTrue(result != null); + assertTrue(result.isPartial()); + assertTrue(result.rawCells() != null); + assertTrue(result.rawCells().length == 1); + + scanner.close(); + + scan.setAllowPartialResults(false); + scanner = TABLE.getScanner(scan); + result = scanner.next(); + + assertTrue(result != null); + assertTrue(!result.isPartial()); + assertTrue(result.rawCells() != null); + assertTrue(result.rawCells().length == NUM_COLS); + + scanner.close(); + } + + /** + * Ensure that the results returned from a scanner that retrieves all results in a single RPC call + * matches the results that are returned from a scanner that must incrementally combine partial + * results into complete results. A variety of scan configurations can be tested + * @throws Exception + */ + @Test + public void testEquivalenceOfScanResults() throws Exception { + Scan oneShotScan = new Scan(); + oneShotScan.setMaxResultSize(Long.MAX_VALUE); + + Scan partialScan = new Scan(oneShotScan); + partialScan.setMaxResultSize(1); + + testEquivalenceOfScanResults(TABLE, oneShotScan, partialScan); + } + + public void testEquivalenceOfScanResults(Table table, Scan scan1, Scan scan2) throws Exception { + ResultScanner scanner1 = table.getScanner(scan1); + ResultScanner scanner2 = table.getScanner(scan2); + + Result r1 = null; + Result r2 = null; + int count = 0; + + while ((r1 = scanner1.next()) != null) { + r2 = scanner2.next(); + + assertTrue(r2 != null); + compareResults(r1, r2, "Comparing result #" + count); + count++; + } + + assertTrue(scanner2.next() == null); + + scanner1.close(); + scanner2.close(); + } + + /** + * Order of cells in partial results matches the ordering of cells from complete results + * @throws Exception + */ + @Test + public void testOrderingOfCellsInPartialResults() throws Exception { + Scan scan = new Scan(); + + for (int col = 1; col <= NUM_COLS; col++) { + scan.setMaxResultSize(getResultSizeForNumberOfCells(col)); + testOrderingOfCellsInPartialResults(scan); + + // Test again with a reversed scanner + scan.setReversed(true); + testOrderingOfCellsInPartialResults(scan); + } + } + + public void testOrderingOfCellsInPartialResults(final Scan basePartialScan) throws Exception { + // Scan that retrieves results in pieces (partials). By setting allowPartialResults to be true + // the results will NOT be reconstructed and instead the caller will see the partial results + // returned by the server + Scan partialScan = new Scan(basePartialScan); + partialScan.setAllowPartialResults(true); + ResultScanner partialScanner = TABLE.getScanner(partialScan); + + // Scan that retrieves all table results in single RPC request + Scan oneShotScan = new Scan(basePartialScan); + oneShotScan.setMaxResultSize(Long.MAX_VALUE); + oneShotScan.setCaching(ROWS.length); + ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan); + + Result oneShotResult = oneShotScanner.next(); + Result partialResult = null; + int iterationCount = 0; + + while (oneShotResult != null && oneShotResult.rawCells() != null) { + List aggregatePartialCells = new ArrayList(); + do { + partialResult = partialScanner.next(); + assertTrue("Partial Result is null. iteration: " + iterationCount, partialResult != null); + assertTrue("Partial cells are null. iteration: " + iterationCount, + partialResult.rawCells() != null); + + for (Cell c : partialResult.rawCells()) { + aggregatePartialCells.add(c); + } + } while (partialResult.isPartial()); + + assertTrue("Number of cells differs. iteration: " + iterationCount, + oneShotResult.rawCells().length == aggregatePartialCells.size()); + final Cell[] oneShotCells = oneShotResult.rawCells(); + for (int cell = 0; cell < oneShotCells.length; cell++) { + Cell oneShotCell = oneShotCells[cell]; + Cell partialCell = aggregatePartialCells.get(cell); + + assertTrue("One shot cell was null", oneShotCell != null); + assertTrue("Partial cell was null", partialCell != null); + assertTrue("Cell differs. oneShotCell:" + oneShotCell + " partialCell:" + partialCell, + oneShotCell.equals(partialCell)); + } + + oneShotResult = oneShotScanner.next(); + iterationCount++; + } + + assertTrue(partialScanner.next() == null); + + partialScanner.close(); + oneShotScanner.close(); + } + + /** + * Setting the max result size allows us to control how many cells we expect to see on each call + * to next on the scanner. Test a variety of different sizes for correctness + * @throws Exception + */ + @Test + public void testExpectedNumberOfCellsPerPartialResult() throws Exception { + Scan scan = new Scan(); + testExpectedNumberOfCellsPerPartialResult(scan); + + scan.setReversed(true); + testExpectedNumberOfCellsPerPartialResult(scan); + } + + public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan) throws Exception { + for (int expectedCells = 1; expectedCells <= NUM_COLS; expectedCells++) { + testExpectedNumberOfCellsPerPartialResult(baseScan, expectedCells); + } + } + + public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan, int expectedNumberOfCells) + throws Exception { + + if (LOG.isInfoEnabled()) LOG.info("groupSize:" + expectedNumberOfCells); + + // Use the cellHeapSize to set maxResultSize such that we know how many cells to expect back + // from the call. The returned results should NOT exceed expectedNumberOfCells but may be less + // than it in cases where expectedNumberOfCells is not an exact multiple of the number of + // columns in the table. + Scan scan = new Scan(baseScan); + scan.setAllowPartialResults(true); + scan.setMaxResultSize(getResultSizeForNumberOfCells(expectedNumberOfCells)); + + ResultScanner scanner = TABLE.getScanner(scan); + Result result = null; + byte[] prevRow = null; + while ((result = scanner.next()) != null) { + assertTrue(result.rawCells() != null); + + // Cases when cell count won't equal expectedNumberOfCells: + // 1. Returned result is the final result needed to form the complete result for that row + // 2. It is the first result we have seen for that row and thus may have been fetched as + // the last group of cells that fit inside the maxResultSize + assertTrue( + "Result's cell count differed from expected number. result: " + result, + result.rawCells().length == expectedNumberOfCells || !result.isPartial() + || !Bytes.equals(prevRow, result.getRow())); + prevRow = result.getRow(); + } + + scanner.close(); + } + + /** + * @return The approximate heap size of a cell in the test table. All cells should have + * approximately the same heap size, so the value is cached to avoid repeating the + * calculation + * @throws Exception + */ + private long getCellHeapSize() throws Exception { + if (CELL_HEAP_SIZE == -1) { + // Do a partial scan that will return a single result with a single cell + Scan scan = new Scan(); + scan.setMaxResultSize(1); + scan.setAllowPartialResults(true); + ResultScanner scanner = TABLE.getScanner(scan); + + Result result = scanner.next(); + + assertTrue(result != null); + assertTrue(result.rawCells() != null); + assertTrue(result.rawCells().length == 1); + + CELL_HEAP_SIZE = CellUtil.estimatedHeapSizeOf(result.rawCells()[0]); + if (LOG.isInfoEnabled()) LOG.info("Cell heap size: " + CELL_HEAP_SIZE); + scanner.close(); + } + + return CELL_HEAP_SIZE; + } + + /** + * @param numberOfCells + * @return the result size that should be used in {@link Scan#setMaxResultSize(long)} if you want + * the server to return exactly numberOfCells cells + * @throws Exception + */ + private long getResultSizeForNumberOfCells(int numberOfCells) throws Exception { + return getCellHeapSize() * numberOfCells; + } + + /** + * Test various combinations of batching and partial results for correctness + */ + @Test + public void testPartialResultsAndBatch() throws Exception { + for (int batch = 1; batch <= NUM_COLS / 4; batch++) { + for (int cellsPerPartial = 1; cellsPerPartial <= NUM_COLS / 4; cellsPerPartial++) { + testPartialResultsAndBatch(batch, cellsPerPartial); + } + } + } + + public void testPartialResultsAndBatch(final int batch, final int cellsPerPartialResult) + throws Exception { + if (LOG.isInfoEnabled()) { + LOG.info("batch: " + batch + " cellsPerPartialResult: " + cellsPerPartialResult); + } + + Scan scan = new Scan(); + scan.setMaxResultSize(getResultSizeForNumberOfCells(cellsPerPartialResult)); + scan.setBatch(batch); + ResultScanner scanner = TABLE.getScanner(scan); + Result result = scanner.next(); + + while ((result = scanner.next()) != null) { + assertTrue(result.rawCells() != null); + + if (result.isPartial()) { + final String error = + "Cells:" + result.rawCells().length + " Batch size:" + batch + + " cellsPerPartialResult:" + cellsPerPartialResult; + assertTrue(error, result.rawCells().length <= Math.min(batch, cellsPerPartialResult)); + } else { + assertTrue(result.rawCells().length <= batch); + } + } + + scanner.close(); + } + + /** + * Test the method {@link Result#createCompleteResult(List, Result)} + * @throws Exception + */ + @Test + public void testPartialResultsReassembly() throws Exception { + Scan scan = new Scan(); + testPartialResultsReassembly(scan); + scan.setReversed(true); + testPartialResultsReassembly(scan); + } + + public void testPartialResultsReassembly(Scan scanBase) throws Exception { + Scan partialScan = new Scan(scanBase); + partialScan.setMaxResultSize(1); + partialScan.setAllowPartialResults(true); + ResultScanner partialScanner = TABLE.getScanner(partialScan); + + Scan oneShotScan = new Scan(scanBase); + oneShotScan.setMaxResultSize(Long.MAX_VALUE); + ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan); + + ArrayList partials = new ArrayList<>(); + for (int i = 0; i < NUM_ROWS; i++) { + Result partialResult = null; + Result completeResult = null; + Result oneShotResult = null; + partials.clear(); + + do { + partialResult = partialScanner.next(); + partials.add(partialResult); + } while (partialResult.isPartial()); + + completeResult = Result.createCompleteResult(partials); + oneShotResult = oneShotScanner.next(); + + compareResults(completeResult, oneShotResult, null); + } + + assertTrue(oneShotScanner.next() == null); + assertTrue(partialScanner.next() == null); + + oneShotScanner.close(); + partialScanner.close(); + } + + /** + * When reconstructing the complete result from its partials we ensure that the row of each + * partial result is the same. If one of the rows differs, an exception is thrown. + */ + @Test + public void testExceptionThrownOnMismatchedPartialResults() throws IOException { + assertTrue(NUM_ROWS >= 2); + + ArrayList partials = new ArrayList<>(); + Scan scan = new Scan(); + scan.setMaxResultSize(Long.MAX_VALUE); + ResultScanner scanner = TABLE.getScanner(scan); + Result r1 = scanner.next(); + partials.add(r1); + Result r2 = scanner.next(); + partials.add(r2); + + assertFalse(Bytes.equals(r1.getRow(), r2.getRow())); + + try { + Result.createCompleteResult(partials); + fail("r1 and r2 are from different rows. It should not be possible to combine them into" + + " a single result"); + } catch (IOException e) { + } + + scanner.close(); + } + + /** + * When a scan has a filter where {@link org.apache.hadoop.hbase.filter.Filter#hasFilterRow()} is + * true, the scanner should not return partial results. The scanner cannot return partial results + * because the entire row needs to be read for the include/exclude decision to be made + */ + @Test + public void testNoPartialResultsWhenFilterPresent() throws Exception { + Scan scan = new Scan(); + scan.setMaxResultSize(1); + scan.setAllowPartialResults(true); + // If a filter hasFilter() is true then partial results should not be returned else filter + // application server side would break. + scan.setFilter(new RandomRowFilter(1.0f)); + ResultScanner scanner = TABLE.getScanner(scan); + + Result r = null; + while ((r = scanner.next()) != null) { + assertFalse(r.isPartial()); + } + + scanner.close(); + } + + /** + * Examine the interaction between the maxResultSize and caching. If the caching limit is reached + * before the maxResultSize limit, we should not see partial results. On the other hand, if the + * maxResultSize limit is reached before the caching limit, it is likely that partial results will + * be seen. + * @throws Exception + */ + @Test + public void testPartialResultsAndCaching() throws Exception { + for (int caching = 1; caching <= NUM_ROWS; caching++) { + for (int maxResultRows = 0; maxResultRows <= NUM_ROWS; maxResultRows++) { + testPartialResultsAndCaching(maxResultRows, caching); + } + } + } + + /** + * @param resultSizeRowLimit The row limit that will be enforced through maxResultSize + * @param cachingRowLimit The row limit that will be enforced through caching + * @throws Exception + */ + public void testPartialResultsAndCaching(int resultSizeRowLimit, int cachingRowLimit) + throws Exception { + Scan scan = new Scan(); + scan.setAllowPartialResults(true); + + // The number of cells specified in the call to getResultSizeForNumberOfCells is offset to + // ensure that the result size we specify is not an exact multiple of the number of cells + // in a row. This ensures that partial results will be returned when the result size limit + // is reached before the caching limit. + int cellOffset = NUM_COLS / 3; + long maxResultSize = getResultSizeForNumberOfCells(resultSizeRowLimit * NUM_COLS + cellOffset); + scan.setMaxResultSize(maxResultSize); + scan.setCaching(cachingRowLimit); + + ResultScanner scanner = TABLE.getScanner(scan); + ClientScanner clientScanner = (ClientScanner) scanner; + Result r = null; + + // Approximate the number of rows we expect will fit into the specified max rsult size. If this + // approximation is less than caching, then we expect that the max result size limit will be + // hit before the caching limit and thus partial results may be seen + boolean expectToSeePartialResults = resultSizeRowLimit < cachingRowLimit; + while ((r = clientScanner.next()) != null) { + assertTrue(!r.isPartial() || expectToSeePartialResults); + if (r.isPartial()) System.out.println("Partial!!: resultSizeRowLimit:" + resultSizeRowLimit + + " cachingRowLimit: " + cachingRowLimit); + } + + scanner.close(); + } + + /** + * Small scans should not return partial results because it would prevent small scans from + * retrieving all of the necessary results in a single RPC request which is what makese small + * scans useful. Thus, ensure that even when {@link Scan#getAllowPartialResults()} is true, small + * scans do not return partial results + * @throws Exception + */ + @Test + public void testSmallScansDoNotAllowPartials() throws Exception { + Scan scan = new Scan(); + testSmallScansDoNotAllowPartials(scan); + scan.setReversed(true); + testSmallScansDoNotAllowPartials(scan); + } + + public void testSmallScansDoNotAllowPartials(Scan baseScan) throws Exception { + Scan scan = new Scan(baseScan); + scan.setAllowPartialResults(true); + scan.setSmall(true); + scan.setMaxResultSize(1); + + ResultScanner scanner = TABLE.getScanner(scan); + Result r = null; + + while ((r = scanner.next()) != null) { + assertFalse(r.isPartial()); + } + + scanner.close(); + } + + /** + * Make puts to put the input value into each combination of row, family, and qualifier + * @param rows + * @param families + * @param qualifiers + * @param value + * @return + * @throws IOException + */ + static ArrayList createPuts(byte[][] rows, byte[][] families, byte[][] qualifiers, + byte[] value) throws IOException { + Put put; + ArrayList puts = new ArrayList<>(); + + for (int row = 0; row < rows.length; row++) { + put = new Put(rows[row]); + for (int fam = 0; fam < families.length; fam++) { + for (int qual = 0; qual < qualifiers.length; qual++) { + KeyValue kv = new KeyValue(rows[row], families[fam], qualifiers[qual], qual, value); + put.add(kv); + } + } + puts.add(put); + } + + return puts; + } + + /** + * Make key values to represent each possible combination of family and qualifier in the specified + * row. + * @param row + * @param families + * @param qualifiers + * @param value + * @return + */ + static ArrayList createKeyValuesForRow(byte[] row, byte[][] families, byte[][] qualifiers, + byte[] value) { + ArrayList outList = new ArrayList<>(); + for (int fam = 0; fam < families.length; fam++) { + for (int qual = 0; qual < qualifiers.length; qual++) { + outList.add(new KeyValue(row, families[fam], qualifiers[qual], qual, value)); + } + } + return outList; + } + + /** + * Verifies that result contains all the key values within expKvList. Fails the test otherwise + * @param result + * @param expKvList + * @param msg + */ + static void verifyResult(Result result, List expKvList, String msg) { + if (LOG.isInfoEnabled()) { + LOG.info(msg); + LOG.info("Expected count: " + expKvList.size()); + LOG.info("Actual count: " + result.size()); + } + + if (expKvList.size() == 0) return; + + int i = 0; + for (Cell kv : result.rawCells()) { + if (i >= expKvList.size()) { + break; // we will check the size later + } + + Cell kvExp = expKvList.get(i++); + assertTrue("Not equal. get kv: " + kv.toString() + " exp kv: " + kvExp.toString(), + kvExp.equals(kv)); + } + + assertEquals(expKvList.size(), result.size()); + } + + /** + * Compares two results and fails the test if the results are different + * @param r1 + * @param r2 + * @param message + */ + static void compareResults(Result r1, Result r2, final String message) { + if (LOG.isInfoEnabled()) { + if (message != null) LOG.info(message); + LOG.info("r1: " + r1); + LOG.info("r2: " + r2); + } + + final String failureMessage = "Results r1:" + r1 + " r2:" + r2 + " are not equivalent"; + if (r1 == null && r2 == null) fail(failureMessage); + else if (r1 == null || r2 == null) fail(failureMessage); + + try { + Result.compareResults(r1, r2); + } catch (Exception e) { + fail(failureMessage); + } + } + + @Test + public void testReadPointAndPartialResults() throws Exception { + TableName testName = TableName.valueOf("testReadPointAndPartialResults"); + int numRows = 5; + int numFamilies = 5; + int numQualifiers = 5; + byte[][] rows = HTestConst.makeNAscii(Bytes.toBytes("testRow"), numRows); + byte[][] families = HTestConst.makeNAscii(Bytes.toBytes("testFamily"), numFamilies); + byte[][] qualifiers = HTestConst.makeNAscii(Bytes.toBytes("testQualifier"), numQualifiers); + byte[] value = Bytes.createMaxByteArray(100); + + Table tmpTable = createTestTable(testName, rows, families, qualifiers, value); + + Scan scan = new Scan(); + scan.setMaxResultSize(1); + scan.setAllowPartialResults(true); + + // Open scanner before deletes + ResultScanner scanner = tmpTable.getScanner(scan); + + Delete delete1 = new Delete(rows[0]); + delete1.addColumn(families[0], qualifiers[0], 0); + tmpTable.delete(delete1); + + Delete delete2 = new Delete(rows[1]); + delete2.addColumn(families[1], qualifiers[1], 1); + tmpTable.delete(delete2); + + // Should see all cells because scanner was opened prior to deletes + int scannerCount = countCellsFromScanner(scanner); + int expectedCount = numRows * numFamilies * numQualifiers; + assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount, + scannerCount == expectedCount); + + // Minus 2 for the two cells that were deleted + scanner = tmpTable.getScanner(scan); + scannerCount = countCellsFromScanner(scanner); + expectedCount = numRows * numFamilies * numQualifiers - 2; + assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount, + scannerCount == expectedCount); + + scanner = tmpTable.getScanner(scan); + // 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)); + tmpTable.put(put1); + + Put put2 = new Put(rows[1]); + put2.add(new KeyValue(rows[1], families[1], qualifiers[1], 2, value)); + tmpTable.put(put2); + + // Scanner opened prior to puts. Cell count shouldn't have changed + scannerCount = countCellsFromScanner(scanner); + expectedCount = numRows * numFamilies * numQualifiers - 2; + assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount, + scannerCount == expectedCount); + + // Now the scanner should see the cells that were added by puts + scanner = tmpTable.getScanner(scan); + scannerCount = countCellsFromScanner(scanner); + expectedCount = numRows * numFamilies * numQualifiers; + assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount, + scannerCount == expectedCount); + + TEST_UTIL.deleteTable(testName); + } + + /** + * Exhausts the scanner by calling next repetitively. Once completely exhausted, close scanner and + * return total cell count + * @param scanner + * @return + * @throws Exception + */ + private int countCellsFromScanner(ResultScanner scanner) throws Exception { + Result result = null; + int numCells = 0; + while ((result = scanner.next()) != null) { + numCells += result.rawCells().length; + } + + scanner.close(); + return numCells; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java index ce76e8a..825f694 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java @@ -95,18 +95,28 @@ public class TestCoprocessorInterface { } @Override - public boolean nextRaw(List result) + public boolean next(List result, int limit, long remainingResultSize) throws IOException { + return delegate.next(result, limit, remainingResultSize); + } + + @Override + public NextRawState nextRaw(List result) throws IOException { return delegate.nextRaw(result); } @Override - public boolean nextRaw(List result, int limit) - throws IOException { + public NextRawState nextRaw(List result, int limit) throws IOException { return delegate.nextRaw(result, limit); } @Override + public NextRawState nextRaw(List result, int limit, long remainingResultSize) + throws IOException { + return delegate.nextRaw(result, limit, remainingResultSize); + } + + @Override public void close() throws IOException { delegate.close(); } @@ -135,6 +145,12 @@ public class TestCoprocessorInterface { public long getMvccReadPoint() { return delegate.getMvccReadPoint(); } + + @Override + public int getBatch() { + return delegate.getBatch(); + } + } public static class CoprocessorImpl extends BaseRegionObserver { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java index bf5227b..4862b92 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java @@ -438,12 +438,17 @@ public class TestRegionObserverInterface { } @Override - public boolean next(List results, int limit) - throws IOException{ + public boolean next(List results, int limit) throws IOException { + return next(results, limit, -1); + } + + @Override + public boolean next(List results, int limit, long remainingResultSize) + throws IOException { List internalResults = new ArrayList(); boolean hasMore; do { - hasMore = scanner.next(internalResults, limit); + hasMore = scanner.next(internalResults, limit, remainingResultSize); if (!internalResults.isEmpty()) { long row = Bytes.toLong(CellUtil.cloneValue(internalResults.get(0))); if (row % 2 == 0) { @@ -772,4 +777,4 @@ public class TestRegionObserverInterface { writer.close(); } } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java index 86d670c..2c0b4ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java @@ -17,11 +17,20 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; -import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY; -import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY; import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY; +import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY; +import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; @@ -38,14 +47,14 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -237,6 +246,12 @@ public class TestStripeCompactor { public boolean next(List result, int limit) throws IOException { return next(result); } + + @Override + public boolean next(List result, int limit, long remainingResultSize) throws IOException { + return next(result); + } + @Override public void close() throws IOException {} } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java index f3b7be4..25e1cd7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java @@ -49,8 +49,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.BloomType; @@ -66,6 +64,8 @@ import org.apache.hadoop.hbase.regionserver.StripeStoreFileManager; import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher; import org.apache.hadoop.hbase.regionserver.TestStripeCompactor.StoreFileWritersCapture; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy.StripeInformationProvider; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ConcatenatedLists; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -786,6 +786,11 @@ public class TestStripeCompactionPolicy { } @Override + public boolean next(List result, int limit, long remainingResultSize) throws IOException { + return next(result); + } + + @Override public void close() throws IOException { } } -- 1.9.3 (Apple Git-50)