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 05a780c8535..ccd8c2d9033 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 @@ -401,6 +401,9 @@ public class ClientScanner extends AbstractClientScanner { // happens for the cases where we see exceptions. Since only openScanner // would have happened, values would be null if (values == null && callable.switchedToADifferentReplica()) { + // Any accumulated partial results are no longer valid since the callable will + // openScanner with the correct startkey and we must pick up from there + clearPartialResults(); this.currentRegion = callable.getHRegionInfo(); continue; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java index ca6ab056201..7ba152bd8b5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java @@ -292,14 +292,7 @@ class ScannerCallableWithReplicas implements RetryingCallable { continue; //this was already scheduled earlier } ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id); - - if (this.lastResult != null) { - if(s.getScan().isReversed()){ - s.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow())); - }else { - s.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1])); - } - } + setStartRowForReplicaCallable(s); outstandingCallables.add(s); RetryingRPC retryingOnReplica = new RetryingRPC(s); cs.submit(retryingOnReplica, scannerTimeout, id); @@ -307,6 +300,31 @@ class ScannerCallableWithReplicas implements RetryingCallable { return max - min + 1; } + /** + * Set the start row for the replica callable based on the state of the last result received. + * @param callable The callable to set the start row on + */ + private void setStartRowForReplicaCallable(ScannerCallable callable) { + if (this.lastResult == null || callable == null) return; + + if (this.lastResult.isPartial()) { + // The last result was a partial result which means we have not received all of the cells + // for this row. Thus, use the last result's row as the start row. If a replica switch + // occurs, the scanner will ensure that any accumulated partial results are cleared, + // and the scan can resume from this row. + callable.getScan().setStartRow(this.lastResult.getRow()); + } else { + // The last result was not a partial result which means it contained all of the cells for + // that row (we no longer need any information from it). Set the start row to the next + // closest row that could be seen. + if (callable.getScan().isReversed()) { + callable.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow())); + } else { + callable.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1])); + } + } + } + @VisibleForTesting boolean isAnyRPCcancelled() { return someRPCcancelled; diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java index e0c3baed7e0..93f98acdfac 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.Bu import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.OperationStatus; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; @@ -137,7 +136,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor List> deleteRows = new ArrayList>(rowBatchSize); for (int i = 0; i < rowBatchSize; i++) { List results = new ArrayList(); - hasMore = NextState.hasMoreValues(scanner.next(results)); + hasMore = scanner.next(results); if (results.size() > 0) { deleteRows.add(results); } diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java index 2afd05e5ff2..4309cdc2e42 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.RpcCallback; @@ -81,7 +80,7 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService byte[] lastRow = null; long count = 0; do { - hasMore = NextState.hasMoreValues(scanner.next(results)); + hasMore = scanner.next(results); for (Cell kv : results) { byte[] currentRow = CellUtil.cloneRow(kv); if (lastRow == null || !Bytes.equals(lastRow, currentRow)) { @@ -120,7 +119,7 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService boolean hasMore = false; long count = 0; do { - hasMore = NextState.hasMoreValues(scanner.next(results)); + hasMore = scanner.next(results); for (Cell kv : results) { count++; } 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 a80a07e1204..5809983385f 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,8 +29,10 @@ 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.NoLimitScannerContext; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.mortbay.log.Log; @@ -72,10 +73,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner { public Result next() throws IOException { values.clear(); - // negative values indicate no limits - final long remainingResultSize = -1; - final int batchLimit = -1; - scanner.nextRaw(values, batchLimit, remainingResultSize); + scanner.nextRaw(values, NoLimitScannerContext.getInstance()); if (values.isEmpty()) { //we are done return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java index b6f834e8063..81c933bbd01 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateReque import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateResponse; import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import com.google.protobuf.ByteString; import com.google.protobuf.Message; @@ -92,7 +91,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { // qualifier can be null. boolean hasMoreRows = false; do { - hasMoreRows = NextState.hasMoreValues(scanner.next(results)); + hasMoreRows = scanner.next(results); int listSize = results.size(); for (int i = 0; i < listSize; i++) { temp = ci.getValue(colFamily, qualifier, results.get(i)); @@ -146,7 +145,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { } boolean hasMoreRows = false; do { - hasMoreRows = NextState.hasMoreValues(scanner.next(results)); + hasMoreRows = scanner.next(results); int listSize = results.size(); for (int i = 0; i < listSize; i++) { temp = ci.getValue(colFamily, qualifier, results.get(i)); @@ -200,7 +199,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { List results = new ArrayList(); boolean hasMoreRows = false; do { - hasMoreRows = NextState.hasMoreValues(scanner.next(results)); + hasMoreRows = scanner.next(results); int listSize = results.size(); for (int i = 0; i < listSize; i++) { temp = ci.getValue(colFamily, qualifier, results.get(i)); @@ -254,7 +253,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { scanner = env.getRegion().getScanner(scan); boolean hasMoreRows = false; do { - hasMoreRows = NextState.hasMoreValues(scanner.next(results)); + hasMoreRows = scanner.next(results); if (results.size() > 0) { counter++; } @@ -313,7 +312,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { do { results.clear(); - hasMoreRows = NextState.hasMoreValues(scanner.next(results)); + hasMoreRows = scanner.next(results); int listSize = results.size(); for (int i = 0; i < listSize; i++) { sumVal = ci.add(sumVal, ci.castToReturnType(ci.getValue(colFamily, @@ -374,7 +373,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { do { tempVal = null; - hasMoreRows = NextState.hasMoreValues(scanner.next(results)); + hasMoreRows = scanner.next(results); int listSize = results.size(); for (int i = 0; i < listSize; i++) { tempVal = ci.add(tempVal, ci.castToReturnType(ci.getValue(colFamily, @@ -441,7 +440,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { do { tempVal = null; tempWeight = null; - hasMoreRows = NextState.hasMoreValues(scanner.next(results)); + hasMoreRows = scanner.next(results); int listSize = results.size(); for (int i = 0; i < listSize; i++) { Cell kv = results.get(i); 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 4a8e7cc2fbb..e082698f720 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 @@ -141,8 +141,9 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.Stor import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.WriteEntry; +import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; +import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController; import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory; @@ -5175,7 +5176,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi protected Cell joinedContinuationRow = null; protected final byte[] stopRow; private final FilterWrapper filter; - private int batch; + private ScannerContext defaultScannerContext; protected int isScan; private boolean filterClosed = false; private long readPt; @@ -5198,7 +5199,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi this.filter = null; } - this.batch = scan.getBatch(); + /** + * By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default + * scanner context that can be used to enforce the batch limit in the event that a + * ScannerContext is not specified during an invocation of next/nextRaw + */ + defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build(); + if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) && !scan.isGetScan()) { this.stopRow = null; } else { @@ -5259,7 +5266,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi @Override public int getBatch() { - return this.batch; + return this.defaultScannerContext.getBatchLimit(); } /** @@ -5274,19 +5281,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } @Override - public NextState next(List outResults) + public boolean next(List outResults) throws IOException { // apply the batching limit by default - return next(outResults, batch); + return next(outResults, defaultScannerContext); } @Override - public NextState next(List outResults, int limit) throws IOException { - return next(outResults, limit, -1); - } - - @Override - public synchronized NextState next(List outResults, int limit, long remainingResultSize) + public synchronized boolean next(List outResults, ScannerContext scannerContext) throws IOException { if (this.filterClosed) { throw new UnknownScannerException("Scanner was closed (timed out?) " + @@ -5296,122 +5298,107 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi startRegionOperation(Operation.SCAN); readRequestsCount.increment(); try { - return nextRaw(outResults, limit, remainingResultSize); + return nextRaw(outResults, scannerContext); } finally { closeRegionOperation(Operation.SCAN); } } @Override - public NextState nextRaw(List outResults) throws IOException { - return nextRaw(outResults, batch); + public boolean nextRaw(List outResults) throws IOException { + // Use the RegionScanner's context by default + return nextRaw(outResults, defaultScannerContext); } @Override - public NextState nextRaw(List outResults, int limit) - throws IOException { - return nextRaw(outResults, limit, -1); - } - - @Override - public NextState nextRaw(List outResults, int batchLimit, long remainingResultSize) + public boolean nextRaw(List outResults, ScannerContext scannerContext) throws IOException { if (storeHeap == null) { // scanner is closed throw new UnknownScannerException("Scanner was closed"); } - NextState state; + boolean moreValues; if (outResults.isEmpty()) { // Usually outResults is empty. This is true when next is called // to handle scan or get operation. - state = nextInternal(outResults, batchLimit, remainingResultSize); + moreValues = nextInternal(outResults, scannerContext); } else { List tmpList = new ArrayList(); - state = nextInternal(tmpList, batchLimit, remainingResultSize); + moreValues = nextInternal(tmpList, scannerContext); outResults.addAll(tmpList); } - // Invalid states should never be returned. Receiving an invalid state means that we have - // no clue how to proceed. Throw an exception. - if (!NextState.isValidState(state)) { - throw new IOException("Invalid state returned from nextInternal. state:" + state); - } // If the size limit was reached it means a partial Result is being returned. Returning a // partial Result means that we should not reset the filters; filters should only be reset in // between rows - if (!state.sizeLimitReached()) resetFilters(); + if (!scannerContext.partialResultFormed()) resetFilters(); if (isFilterDoneInternal()) { - state = NextState.makeState(NextState.State.NO_MORE_VALUES, state.getResultSize()); + moreValues = false; } - return state; + return moreValues; } /** - * @return the state the joinedHeap returned on the call to - * {@link KeyValueHeap#next(List, int, long)} + * @return true if more cells exist after this batch, false if scanner is done */ - private NextState populateFromJoinedHeap(List results, int limit, long resultSize) + private boolean populateFromJoinedHeap(List results, ScannerContext scannerContext) throws IOException { assert joinedContinuationRow != null; - NextState state = - populateResult(results, this.joinedHeap, limit, resultSize, + boolean moreValues = + populateResult(results, this.joinedHeap, scannerContext, joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(), joinedContinuationRow.getRowLength()); - if (state != null && !state.batchLimitReached() && !state.sizeLimitReached()) { + + if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { // We are done with this row, reset the continuation. joinedContinuationRow = null; } // As the data is obtained from two independent heaps, we need to // ensure that result list is sorted, because Result relies on that. Collections.sort(results, comparator); - return state; + return moreValues; } /** * 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 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 scannerContext * @param currentRow Byte array with key we are fetching. * @param offset offset for currentRow * @param length length for currentRow * @return state of last call to {@link KeyValueHeap#next()} */ - private NextState populateResult(List results, KeyValueHeap heap, int batchLimit, - long remainingResultSize, byte[] currentRow, int offset, short length) throws IOException { + private boolean populateResult(List results, KeyValueHeap heap, + ScannerContext scannerContext, byte[] currentRow, int offset, short length) + throws IOException { Cell nextKv; boolean moreCellsInRow = false; - long accumulatedResultSize = 0; - List tmpResults = new ArrayList(); + boolean tmpKeepProgress = scannerContext.getKeepProgress(); + // Scanning between column families and thus the scope is between cells + LimitScope limitScope = LimitScope.BETWEEN_CELLS; do { - int remainingBatchLimit = batchLimit - results.size(); - NextState heapState = - heap.next(tmpResults, remainingBatchLimit, remainingResultSize - accumulatedResultSize); - results.addAll(tmpResults); - accumulatedResultSize += calculateResultSize(tmpResults, heapState); - tmpResults.clear(); - - if (batchLimit > 0 && results.size() == batchLimit) { - return NextState.makeState(NextState.State.BATCH_LIMIT_REACHED, accumulatedResultSize); - } + // We want to maintain any progress that is made towards the limits while scanning across + // different column families. To do this, we toggle the keep progress flag on during calls + // to the StoreScanner to ensure that any progress made thus far is not wiped away. + scannerContext.setKeepProgress(true); + heap.next(results, scannerContext); + scannerContext.setKeepProgress(tmpKeepProgress); nextKv = heap.peek(); moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length); - boolean sizeLimitReached = - remainingResultSize > 0 && accumulatedResultSize >= remainingResultSize; - if (moreCellsInRow && sizeLimitReached) { - return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, accumulatedResultSize); + + if (scannerContext.checkBatchLimit(limitScope)) { + return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues(); + } else if (scannerContext.checkSizeLimit(limitScope)) { + ScannerContext.NextState state = + moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED; + return scannerContext.setScannerState(state).hasMoreValues(); } } while (moreCellsInRow); - if (nextKv != null) { - return NextState.makeState(NextState.State.MORE_VALUES, accumulatedResultSize); - } else { - return NextState.makeState(NextState.State.NO_MORE_VALUES, accumulatedResultSize); - } + return nextKv != null; } /** @@ -5429,30 +5416,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length); } - /** - * Calculates the size of the results. If the state of the scanner that these results came from - * indicates that an estimate of the result size has already been generated, we can skip the - * calculation and use that instead. - * @param results List of cells we want to calculate size of - * @param state The state returned from the scanner that generated these results - * @return aggregate size of results - */ - private long calculateResultSize(List results, NextState state) { - if (results == null || results.isEmpty()) return 0; - - // In general, the state should contain the estimate because the result size used to - // determine when the scan has exceeded its size limit. If the estimate is contained in the - // state then we can avoid an unnecesasry calculation. - if (state != null && state.hasResultSizeEstimate()) return state.getResultSize(); - - long size = 0; - for (Cell c : results) { - size += CellUtil.estimatedHeapSizeOfWithoutTags(c); - } - - return size; - } - /* * @return True if a filter rules the scanner is over, done. */ @@ -5465,20 +5428,37 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return this.filter != null && this.filter.filterAllRemaining(); } - private NextState nextInternal(List results, int batchLimit, long remainingResultSize) + private boolean nextInternal(List results, ScannerContext scannerContext) throws IOException { if (!results.isEmpty()) { throw new IllegalArgumentException("First parameter should be an empty list"); } - // Estimate of the size (heap size) of the results returned from this method - long resultSize = 0; + if (scannerContext == null) { + throw new IllegalArgumentException("Scanner context cannot be null"); + } RpcCallContext rpcCall = RpcServer.getCurrentCall(); + + // Save the initial progress from the Scanner context in these local variables. The progress + // may need to be reset a few times if rows are being filtered out so we save the initial + // progress. + int initialBatchProgress = scannerContext.getBatchProgress(); + long initialSizeProgress = scannerContext.getSizeProgress(); + // The loop here is used only when at some point during the next we determine // that due to effects of filters or otherwise, we have an empty row in the result. // Then we loop and try again. Otherwise, we must get out on the first iteration via return, // "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row, // and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow). while (true) { + // Starting to scan a new row. Reset the scanner progress according to whether or not + // progress should be kept. + if (scannerContext.getKeepProgress()) { + // Progress should be kept. Reset to initial values seen at start of method invocation. + scannerContext.setProgress(initialBatchProgress, initialSizeProgress); + } else { + scannerContext.clearProgress(); + } + if (rpcCall != null) { // If a user specifies a too-restrictive or too-slow scanner, the // client might time out and disconnect while the server side @@ -5506,21 +5486,24 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } boolean stopRow = isStopRow(currentRow, offset, length); + // When has filter row is true it means that the all the cells for a particular row must be + // read before a filtering decision can be made. This means that filters where hasFilterRow + // run the risk of encountering out of memory errors in the case that they are applied to a + // table that has very large rows. 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; + // scope of any limits that could potentially create partial results to + // LimitScope.BETWEEN_ROWS so that those limits are not reached mid-row + if (hasFilterRow) { if (LOG.isTraceEnabled()) { - LOG.trace("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."); + LOG.trace("filter#hasFilterRow is true which prevents partial results from being " + + " formed. Changing scope of limits that may create partials"); } + scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS); } - NextState joinedHeapState; // 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) { @@ -5529,47 +5512,30 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (hasFilterRow) { filter.filterRowCells(results); } - return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } // 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 NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize); + if (!moreRows) { + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); + } results.clear(); continue; } - NextState storeHeapState = - populateResult(results, this.storeHeap, batchLimit, remainingResultSize, currentRow, - offset, length); - resultSize += calculateResultSize(results, storeHeapState); - // Invalid states should never be returned. If one is seen, throw exception - // since we have no way of telling how we should proceed - if (!NextState.isValidState(storeHeapState)) { - throw new IOException("NextState returned from call storeHeap was invalid"); - } - // Ok, we are good, let's try to get some results from the main heap. - if (storeHeapState.batchLimitReached()) { + populateResult(results, this.storeHeap, scannerContext, currentRow, offset, length); + + if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { if (hasFilterRow) { throw new IncompatibleFilterException( - "Filter whose hasFilterRow() returns true is incompatible with scan with limit!"); + "Filter whose hasFilterRow() returns true is incompatible with scans that must " + + " stop mid-row because of a limit. ScannerContext:" + scannerContext); } - // We hit the batch limit. - return NextState.makeState(NextState.State.BATCH_LIMIT_REACHED, resultSize); - } else if (storeHeapState.sizeLimitReached()) { - 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"); - } - // We hit the size limit. - return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, resultSize); + return true; } Cell nextKv = this.storeHeap.peek(); stopRow = nextKv == null || @@ -5582,17 +5548,31 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED; if (hasFilterRow) { ret = filter.filterRowCellsWithRet(results); + + // We don't know how the results have changed after being filtered. Must set progress + // according to contents of results now. + if (scannerContext.getKeepProgress()) { + scannerContext.setProgress(initialBatchProgress, initialSizeProgress); + } else { + scannerContext.clearProgress(); + } + scannerContext.incrementBatchProgress(results.size()); + for (Cell cell : results) { + scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell)); + } } if ((isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE) || filterRow()) { results.clear(); boolean moreRows = nextRow(currentRow, offset, length); - if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, 0); + if (!moreRows) { + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); + } // 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 NextState.makeState(NextState.State.NO_MORE_VALUES, 0); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } // Ok, we are done with storeHeap for this row. @@ -5610,31 +5590,24 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi currentRow, offset, length)); if (mayHaveData) { joinedContinuationRow = current; - joinedHeapState = - populateFromJoinedHeap(results, batchLimit, remainingResultSize - resultSize); - resultSize += - joinedHeapState != null && joinedHeapState.hasResultSizeEstimate() ? - joinedHeapState.getResultSize() : 0; - if (joinedHeapState != null && joinedHeapState.sizeLimitReached()) { - return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, resultSize); + populateFromJoinedHeap(results, scannerContext); + + if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { + return true; } } } } else { // Populating from the joined heap was stopped by limits, populate some more. - joinedHeapState = - populateFromJoinedHeap(results, batchLimit, remainingResultSize - resultSize); - resultSize += - joinedHeapState != null && joinedHeapState.hasResultSizeEstimate() ? - joinedHeapState.getResultSize() : 0; - if (joinedHeapState != null && joinedHeapState.sizeLimitReached()) { - return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, resultSize); + populateFromJoinedHeap(results, scannerContext); + if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { + return true; } } // 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 NextState.makeState(NextState.State.MORE_VALUES, resultSize); + return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); } // Finally, we are done with both joinedHeap and storeHeap. @@ -5642,15 +5615,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // the case when SingleColumnValueExcludeFilter is used. if (results.isEmpty()) { boolean moreRows = nextRow(currentRow, offset, length); - if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, 0); + if (!moreRows) { + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); + } if (!stopRow) continue; } // We are done. Return the result. if (stopRow) { - return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } else { - return NextState.makeState(NextState.State.MORE_VALUES, resultSize); + return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); } } } @@ -7269,7 +7244,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi boolean done; do { kvs.clear(); - done = NextState.hasMoreValues(scanner.next(kvs)); + done = scanner.next(kvs); if (kvs.size() > 0) LOG.info(kvs); } while (done); } finally { 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 ea5a75f7383..f73e363cee1 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 @@ -42,218 +42,21 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; @InterfaceAudience.Private public interface InternalScanner extends Closeable { /** - * This class encapsulates all the meaningful state information that we would like the know about - * after a call to {@link InternalScanner#next(List)}. While this is not an enum, a restriction on - * the possible states is implied through the exposed {@link #makeState(State)} method. + * Grab the next row's worth of values. + * @param results return output array + * @return true if more rows exist after this one, false if scanner is done + * @throws IOException e */ - public static class NextState { - /** - * The possible states we want to restrict ourselves to. This enum is not sufficient to - * encapsulate all of the state information since some of the fields of the state must be - * dynamic (e.g. resultSize). - */ - public enum State { - MORE_VALUES(true), - NO_MORE_VALUES(false), - SIZE_LIMIT_REACHED(true), - BATCH_LIMIT_REACHED(true); - - private boolean moreValues; - - private State(final boolean moreValues) { - this.moreValues = moreValues; - } - - /** - * @return true when the state indicates that more values may follow those that have been - * returned - */ - public boolean hasMoreValues() { - return this.moreValues; - } - } - - /** - * state variables - */ - private final State state; - private long resultSize; - - /** - * Value to use for resultSize when the size has not been calculated. Must be a negative number - * so that {@link NextState#hasResultSizeEstimate()} returns false. - */ - private static final long DEFAULT_RESULT_SIZE = -1; - - private NextState(State state, long resultSize) { - this.state = state; - this.resultSize = resultSize; - } - - /** - * @param state - * @return An instance of {@link NextState} where the size of the results returned from the call - * to {@link InternalScanner#next(List)} is unknown. It it the responsibility of the - * caller of {@link InternalScanner#next(List)} to calculate the result size if needed - */ - public static NextState makeState(final State state) { - return makeState(state, DEFAULT_RESULT_SIZE); - } - - /** - * @param state - * @param resultSize - * @return An instance of {@link NextState} where the size of the values returned from the call - * to {@link InternalScanner#next(List)} is known. The caller can avoid recalculating - * the result size by using the cached value retrievable via {@link #getResultSize()} - */ - public static NextState makeState(final State state, long resultSize) { - switch (state) { - case MORE_VALUES: - return createMoreValuesState(resultSize); - case NO_MORE_VALUES: - return createNoMoreValuesState(resultSize); - case BATCH_LIMIT_REACHED: - return createBatchLimitReachedState(resultSize); - case SIZE_LIMIT_REACHED: - return createSizeLimitReachedState(resultSize); - default: - // If the state is not recognized, default to no more value state - return createNoMoreValuesState(resultSize); - } - } - - /** - * Convenience method for creating a state that indicates that more values can be scanned - * @param resultSize estimate of the size (heap size) of the values returned from the call to - * {@link InternalScanner#next(List)} - */ - private static NextState createMoreValuesState(long resultSize) { - return new NextState(State.MORE_VALUES, resultSize); - } - - /** - * Convenience method for creating a state that indicates that no more values can be scanned. - * @param resultSize estimate of the size (heap size) of the values returned from the call to - * {@link InternalScanner#next(List)} - */ - private static NextState createNoMoreValuesState(long resultSize) { - return new NextState(State.NO_MORE_VALUES, resultSize); - } - - /** - * Convenience method for creating a state that indicates that the scan stopped because the - * batch limit was exceeded - * @param resultSize estimate of the size (heap size) of the values returned from the call to - * {@link InternalScanner#next(List)} - */ - private static NextState createBatchLimitReachedState(long resultSize) { - return new NextState(State.BATCH_LIMIT_REACHED, resultSize); - } - - /** - * Convenience method for creating a state that indicates that the scan stopped due to the size - * limit - * @param resultSize estimate of the size (heap size) of the values returned from the call to - * {@link InternalScanner#next(List)} - */ - private static NextState createSizeLimitReachedState(long resultSize) { - return new NextState(State.SIZE_LIMIT_REACHED, resultSize); - } - - /** - * @return true when the scanner has more values to be scanned following the values returned by - * the call to {@link InternalScanner#next(List)} - */ - public boolean hasMoreValues() { - return this.state.hasMoreValues(); - } - - /** - * @return true when the scanner had to stop scanning because it reached the batch limit - */ - public boolean batchLimitReached() { - return this.state == State.BATCH_LIMIT_REACHED; - } - - /** - * @return true when the scanner had to stop scanning because it reached the size limit - */ - public boolean sizeLimitReached() { - return this.state == State.SIZE_LIMIT_REACHED; - } - - /** - * @return The size (heap size) of the values that were returned from the call to - * {@link InternalScanner#next(List)}. This value should only be used if - * {@link #hasResultSizeEstimate()} returns true. - */ - public long getResultSize() { - return resultSize; - } - - /** - * @return true when an estimate for the size of the values returned by - * {@link InternalScanner#next(List)} was provided. If false, it is the responsibility - * of the caller to calculate the result size - */ - public boolean hasResultSizeEstimate() { - return resultSize >= 0; - } - - @Override - public String toString() { - return "State: " + state + " resultSize: " + resultSize; - } - - /** - * Helper method to centralize all checks as to whether or not the state is valid. - * @param state - * @return true when the state is valid - */ - public static boolean isValidState(NextState state) { - return state != null; - } - - /** - * @param state - * @return true when the state is non null and indicates that more values exist - */ - public static boolean hasMoreValues(NextState state) { - return state != null && state.hasMoreValues(); - } - } + boolean next(List results) throws IOException; /** * Grab the next row's worth of values. - * @param results return output array - * @return state where {@link NextState#hasMoreValues()} is true if more rows exist after this - * one, false if scanner is done - * @throws IOException e - */ - NextState next(List results) throws IOException; - - /** - * Grab the next row's worth of values with a limit on the number of values to return. * @param result return output array - * @param limit limit on row count to get - * @return state where {@link NextState#hasMoreValues()} is true if more rows exist after this - * one, false if scanner is done + * @param scannerContext + * @return true if more rows exist after this one, false if scanner is done * @throws IOException e */ - NextState 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 state where {@link NextState#hasMoreValues()} is true if more rows exist after this - * one, false if scanner is done - * @throws IOException e - */ - NextState next(List result, int limit, long remainingResultSize) throws IOException; + boolean next(List result, ScannerContext scannerContext) throws IOException; /** * Closes the scanner and releases any resources it has allocated 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 beb23cfe0ff..761267f65ee 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 @@ -27,6 +27,7 @@ import java.util.PriorityQueue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; /** * Implements a heap merge across any number of KeyValueScanners. @@ -128,26 +129,20 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner * This can ONLY be called when you are using Scanners that implement InternalScanner as well as * KeyValueScanner (a {@link StoreScanner}). * @param result - * @param limit - * @return state where NextState#hasMoreValues() is true if more keys exist after this - * one, false if scanner is done + * @return true if more rows exist after this one, false if scanner is done */ - public NextState next(List result, int limit) throws IOException { - return next(result, limit, -1); + @Override + public boolean next(List result) throws IOException { + return next(result, NoLimitScannerContext.getInstance()); } - public NextState next(List result, int limit, long remainingResultSize) throws IOException { + @Override + public boolean next(List result, ScannerContext scannerContext) throws IOException { if (this.current == null) { - return NextState.makeState(NextState.State.NO_MORE_VALUES); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } InternalScanner currentAsInternal = (InternalScanner)this.current; - NextState state = currentAsInternal.next(result, limit, remainingResultSize); - // Invalid states should never be returned. Receiving an invalid state means that we have - // no clue how to proceed. Throw an exception. - if (!NextState.isValidState(state)) { - throw new IOException("Invalid state returned from InternalScanner#next"); - } - boolean mayContainMoreRows = NextState.hasMoreValues(state); + boolean moreCells = currentAsInternal.next(result, scannerContext); Cell pee = this.current.peek(); /* * By definition, any InternalScanner must return false only when it has no @@ -156,31 +151,16 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner * more efficient to close scanners which are not needed than keep them in * the heap. This is also required for certain optimizations. */ - if (pee == null || !mayContainMoreRows) { + if (pee == null || !moreCells) { this.current.close(); } else { this.heap.add(this.current); } this.current = pollRealKV(); if (this.current == null) { - state = NextState.makeState(NextState.State.NO_MORE_VALUES, state.getResultSize()); + moreCells = scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } - return state; - } - - /** - * Gets the next row of keys from the top-most scanner. - *

- * This method takes care of updating the heap. - *

- * This can ONLY be called when you are using Scanners that implement InternalScanner as well as - * KeyValueScanner (a {@link StoreScanner}). - * @param result - * @return state where NextState#hasMoreValues() is true if more keys exist after this - * one, false if scanner is done - */ - public NextState next(List result) throws IOException { - return next(result, -1); + return moreCells; } protected static class KVScannerComparator implements Comparator { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoLimitScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoLimitScannerContext.java new file mode 100644 index 00000000000..1484e8072e9 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoLimitScannerContext.java @@ -0,0 +1,102 @@ +/** + * 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.regionserver; + +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * This is a special {@link ScannerContext} subclass that is designed to be used globally when + * limits should not be enforced during invocations of {@link InternalScanner#next(java.util.List)} + * or {@link RegionScanner#next(java.util.List)}. + *

+ * Instances of {@link NoLimitScannerContext} are immutable after construction. Any attempt to + * change the limits or progress of a {@link NoLimitScannerContext} will fail silently. The net + * effect is that all limit checks will return false, thus indicating that a limit has not been + * reached. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +public class NoLimitScannerContext extends ScannerContext { + + public NoLimitScannerContext() { + super(false, null); + } + + /** + * Use this instance whenever limits do not need to be enforced. + */ + private static final ScannerContext NO_LIMIT = new NoLimitScannerContext(); + + /** + * @return The static, immutable instance of {@link NoLimitScannerContext} to be used whenever + * limits should not be enforced + */ + public static final ScannerContext getInstance() { + return NO_LIMIT; + } + + @Override + void setKeepProgress(boolean keepProgress) { + // Do nothing. NoLimitScannerContext instances are immutable post-construction + } + + @Override + void setBatchProgress(int batchProgress) { + // Do nothing. NoLimitScannerContext instances are immutable post-construction + } + + @Override + void setSizeProgress(long sizeProgress) { + // Do nothing. NoLimitScannerContext instances are immutable post-construction + } + + @Override + void setProgress(int batchProgress, long sizeProgress) { + // Do nothing. NoLimitScannerContext instances are immutable post-construction + } + + @Override + void setSizeLimitScope(LimitScope scope) { + // Do nothing. NoLimitScannerContext instances are immutable post-construction + } + + @Override + NextState setScannerState(NextState state) { + // Do nothing. NoLimitScannerContext instances are immutable post-construction + return state; + } + + @Override + boolean checkBatchLimit(LimitScope checkerScope) { + // No limits can be specified, thus return false to indicate no limit has been reached. + return false; + } + + @Override + boolean checkSizeLimit(LimitScope checkerScope) { + // No limits can be specified, thus return false to indicate no limit has been reached. + return false; + } + + @Override + boolean checkAnyLimitReached(LimitScope checkerScope) { + return false; + } +} 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 1508a15226a..10e39a18b13 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 @@ -105,8 +105,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest; @@ -120,6 +118,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfiguratio import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath; @@ -151,10 +151,10 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.quotas.OperationQuota; import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; import org.apache.hadoop.hbase.regionserver.Region.FlushResult; import org.apache.hadoop.hbase.regionserver.Region.Operation; +import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -2236,61 +2236,53 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // correct ordering of partial results and so we prevent partial results from being // formed. boolean serverGuaranteesOrderOfPartials = currentScanResultSize == 0; - boolean enforceMaxResultSizeAtCellLevel = + boolean allowPartialResults = clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan; - NextState state = null; + boolean moreRows = false; + + final LimitScope sizeScope = + allowPartialResults ? LimitScope.BETWEEN_CELLS : LimitScope.BETWEEN_ROWS; + + // Configure with limits for this RPC. Set keep progress true since size progress + // towards size limit should be kept between calls to nextRaw + ScannerContext.Builder contextBuilder = ScannerContext.newBuilder(true); + contextBuilder.setSizeLimit(sizeScope, maxResultSize); + contextBuilder.setBatchLimit(scanner.getBatch()); + ScannerContext scannerContext = contextBuilder.build(); while (i < rows) { // Stop collecting results if we have exceeded maxResultSize - if (currentScanResultSize >= maxResultSize) { + if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_ROWS)) { builder.setMoreResultsInRegion(true); break; } - // A negative remainingResultSize communicates that there is no limit on the size - // of the results. - final long remainingResultSize = - enforceMaxResultSizeAtCellLevel ? maxResultSize - currentScanResultSize - : -1; + // Reset the batch progress to 0 before every call to RegionScanner#nextRaw. The + // batch limit is a limit on the number of cells per Result. Thus, if progress is + // being tracked (i.e. scannerContext.keepProgress() is true) then we need to + // reset the batch progress between nextRaw invocations since we don't want the + // batch progress from previous calls to affect future calls + scannerContext.setBatchProgress(0); // Collect values to be returned here - state = scanner.nextRaw(values, scanner.getBatch(), remainingResultSize); - // Invalid states should never be returned. If one is seen, throw exception - // to stop the scan -- We have no way of telling how we should proceed - if (!NextState.isValidState(state)) { - throw new IOException("NextState returned from call to nextRaw was invalid"); - } - if (!values.isEmpty()) { - // The state should always contain an estimate of the result size because that - // estimate must be used to decide when partial results are formed. - boolean skipResultSizeCalculation = state.hasResultSizeEstimate(); - if (skipResultSizeCalculation) currentScanResultSize += state.getResultSize(); + moreRows = scanner.nextRaw(values, scannerContext); + if (!values.isEmpty()) { for (Cell cell : values) { totalCellSize += CellUtil.estimatedSerializedSizeOf(cell); - - // If the calculation can't be skipped, then do it now. - if (!skipResultSizeCalculation) { - currentScanResultSize += CellUtil.estimatedHeapSizeOfWithoutTags(cell); - } } - // 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 != null && state.sizeLimitReached(); + final boolean partial = scannerContext.partialResultFormed(); results.add(Result.create(values, null, stale, partial)); i++; } - if (!NextState.hasMoreValues(state)) { + if (!moreRows) { break; } values.clear(); } - // currentScanResultSize >= maxResultSize should be functionally equivalent to - // state.sizeLimitReached() - if (null != state - && (currentScanResultSize >= maxResultSize || i >= rows || state - .hasMoreValues())) { + + if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_ROWS) || i >= rows || + moreRows) { // We stopped prematurely builder.setMoreResultsInRegion(true); } else { 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 26f9aef576f..66e087bfd58 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 @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; * RegionScanner describes iterators over rows in an HRegion. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) -@InterfaceStability.Stable +@InterfaceStability.Evolving public interface RegionScanner extends InternalScanner { /** * @return The RegionInfo for this scanner. @@ -74,35 +74,22 @@ public interface RegionScanner extends InternalScanner { int getBatch(); /** - * 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, long)} + * Grab the next row's worth of 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. Caller should maintain and + * update metrics. See {@link #nextRaw(List, ScannerContext)} * @param result return output array - * @return a state where NextState#hasMoreValues() is true when more rows exist, false when - * scanner is done. + * @return true if more rows exist after this one, false if scanner is done * @throws IOException e */ - NextState nextRaw(List result) throws IOException; - + boolean nextRaw(List result) throws IOException; + /** - * 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, long)} - * @param result return output array - * @param limit limit on row count to get - * @return a state where NextState#hasMoreValues() is true when more rows exist, false when - * scanner is done. - * @throws IOException e - */ - NextState 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 + * Grab the next row's worth of values. The {@link ScannerContext} is used to enforce and track + * any limits associated with this call. Any progress that exists in the {@link ScannerContext} + * prior to calling this method will be LOST if {@link ScannerContext#getKeepProgress()} is false. + * Upon returning from this method, the {@link ScannerContext} will contain information about the + * progress made towards the limits. 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 = ...;
@@ -120,13 +107,12 @@ public interface RegionScanner extends InternalScanner {
    * }
    * 
* @param result return output array - * @param limit limit on row count to get - * @param remainingResultSize the space remaining within the restriction on the result size. - * Negative values indicate no limit - * @return a state where NextState#hasMoreValues() is true when more rows exist, false when - * scanner is done. + * @param scannerContext The {@link ScannerContext} instance encapsulating all limits that should + * be tracked during calls to this method. The progress towards these limits can be + * tracked within this instance. + * @return true if more rows exist after this one, false if scanner is done * @throws IOException e */ - NextState nextRaw(List result, int limit, final long remainingResultSize) + boolean nextRaw(List result, ScannerContext scannerContext) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java new file mode 100644 index 00000000000..6e487ca7a0c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java @@ -0,0 +1,527 @@ +/** + * 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.regionserver; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * ScannerContext instances encapsulate limit tracking AND progress towards those limits during + * invocations of {@link InternalScanner#next(java.util.List)} and + * {@link RegionScanner#next(java.util.List)}. + *

+ * A ScannerContext instance should be updated periodically throughout execution whenever progress + * towards a limit has been made. Each limit can be checked via the appropriate checkLimit method. + *

+ * Once a limit has been reached, the scan will stop. The invoker of + * {@link InternalScanner#next(java.util.List)} or {@link RegionScanner#next(java.util.List)} can + * use the appropriate check*Limit methods to see exactly which limits have been reached. + * Alternatively, {@link #checkAnyLimitReached(LimitScope)} is provided to see if ANY limit was + * reached + *

+ * {@link NoLimitScannerContext#NO_LIMIT} is an immutable static definition that can be used + * whenever a {@link ScannerContext} is needed but limits do not need to be enforced. + *

+ * NOTE: It is important that this class only ever expose setter methods that can be safely skipped + * when limits should be NOT enforced. This is because of the necessary immutability of the class + * {@link NoLimitScannerContext}. If a setter cannot be safely skipped, the immutable nature of + * {@link NoLimitScannerContext} will lead to incorrect behavior. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +public class ScannerContext { + private final Log LOG = LogFactory.getLog(this.getClass()); + + /** + * Two sets of the same fields. One for the limits, another for the progress towards those limits + */ + LimitFields limits; + LimitFields progress; + + /** + * The state of the scanner after the invocation of {@link InternalScanner#next(java.util.List)} + * or {@link RegionScanner#next(java.util.List)}. + */ + NextState scannerState; + private static final NextState DEFAULT_STATE = NextState.MORE_VALUES; + + /** + * Used as an indication to invocations of {@link InternalScanner#next(java.util.List)} and + * {@link RegionScanner#next(java.util.List)} that, if true, the progress tracked within this + * {@link ScannerContext} instance should be considered while evaluating the limits. Useful for + * enforcing a set of limits across multiple calls (i.e. the limit may not be reached in a single + * invocation, but any progress made should be considered in future invocations) + *

+ * Defaulting this value to false means that, by default, any tracked progress will be wiped clean + * on invocations to {@link InternalScanner#next(java.util.List)} and + * {@link RegionScanner#next(java.util.List)} and the call will be treated as though no progress + * has been made towards the limits so far. + *

+ * This is an important mechanism. Users of Internal/Region scanners expect that they can define + * some limits and then repeatedly invoke {@link InternalScanner#next(List)} or + * {@link RegionScanner#next(List)} where each invocation respects these limits separately. + *

+ * For example:

+   * ScannerContext context = new ScannerContext.newBuilder().setBatchLimit(5).build();
+   * RegionScanner scanner = ...
+   * List results = new ArrayList();
+   * while(scanner.next(results, context)) {
+   *   // Do something with a batch of 5 cells
+   * }
+   * 
However, in the case of RPCs, the server wants to be able to define a set of + * limits for a particular RPC request and have those limits respected across multiple + * invocations. This means that the progress made towards the limits in earlier calls will be + * saved and considered in future invocations + */ + boolean keepProgress; + private static boolean DEFAULT_KEEP_PROGRESS = false; + + ScannerContext(boolean keepProgress, LimitFields limitsToCopy) { + this.limits = new LimitFields(); + if (limitsToCopy != null) this.limits.copy(limitsToCopy); + + // Progress fields are initialized to 0 + progress = new LimitFields(0, LimitFields.DEFAULT_SCOPE, 0); + + this.keepProgress = keepProgress; + this.scannerState = DEFAULT_STATE; + } + + /** + * @return true if the progress tracked so far in this instance will be considered during an + * invocation of {@link InternalScanner#next(java.util.List)} or + * {@link RegionScanner#next(java.util.List)}. false when the progress tracked so far + * should not be considered and should instead be wiped away via {@link #clearProgress()} + */ + boolean getKeepProgress() { + return keepProgress; + } + + void setKeepProgress(boolean keepProgress) { + this.keepProgress = keepProgress; + } + + /** + * Progress towards the batch limit has been made. Increment internal tracking of batch progress + */ + void incrementBatchProgress(int batch) { + int currentBatch = progress.getBatch(); + progress.setBatch(currentBatch + batch); + } + + /** + * Progress towards the size limit has been made. Increment internal tracking of size progress + */ + void incrementSizeProgress(long size) { + long currentSize = progress.getSize(); + progress.setSize(currentSize + size); + } + + int getBatchProgress() { + return progress.getBatch(); + } + + long getSizeProgress() { + return progress.getSize(); + } + + void setProgress(int batchProgress, long sizeProgress) { + setBatchProgress(batchProgress); + setSizeProgress(sizeProgress); + } + + void setSizeProgress(long sizeProgress) { + progress.setSize(sizeProgress); + } + + void setBatchProgress(int batchProgress) { + progress.setBatch(batchProgress); + } + + /** + * Clear away any progress that has been made so far. All progress fields are reset to initial + * values + */ + void clearProgress() { + progress.setFields(0, LimitFields.DEFAULT_SCOPE, 0); + } + + /** + * Note that this is not a typical setter. This setter returns the {@link NextState} that was + * passed in so that methods can be invoked against the new state. Furthermore, this pattern + * allows the {@link NoLimitScannerContext} to cleanly override this setter and simply return the + * new state, thus preserving the immutability of {@link NoLimitScannerContext} + * @param state + * @return The state that + */ + NextState setScannerState(NextState state) { + if (!NextState.isValidState(state)) { + throw new IllegalArgumentException("Cannot set to invalid state: " + state); + } + + this.scannerState = state; + return state; + } + + /** + * @return true when a partial result is formed. A partial result is formed when a limit is + * reached in the middle of a row. + */ + boolean partialResultFormed() { + return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW; + } + + /** + * @param checkerScope + * @return true if the batch limit can be enforced in the checker's scope + */ + boolean hasBatchLimit(LimitScope checkerScope) { + return limits.canEnforceBatchLimitFromScope(checkerScope) && limits.getBatch() > 0; + } + + /** + * @param checkerScope + * @return true if the size limit can be enforced in the checker's scope + */ + boolean hasSizeLimit(LimitScope checkerScope) { + return limits.canEnforceSizeLimitFromScope(checkerScope) && limits.getSize() > 0; + } + + /** + * @param checkerScope + * @return true if any limit can be enforced within the checker's scope + */ + boolean hasAnyLimit(LimitScope checkerScope) { + return hasBatchLimit(checkerScope) || hasSizeLimit(checkerScope); + } + + /** + * @param scope The scope in which the size limit will be enforced + */ + void setSizeLimitScope(LimitScope scope) { + limits.setSizeScope(scope); + } + + int getBatchLimit() { + return limits.getBatch(); + } + + long getSizeLimit() { + return limits.getSize(); + } + + /** + * @param checkerScope The scope that the limit is being checked from + * @return true when the limit is enforceable from the checker's scope and it has been reached + */ + boolean checkBatchLimit(LimitScope checkerScope) { + return hasBatchLimit(checkerScope) && progress.getBatch() >= limits.getBatch(); + } + + /** + * @param checkerScope The scope that the limit is being checked from + * @return true when the limit is enforceable from the checker's scope and it has been reached + */ + boolean checkSizeLimit(LimitScope checkerScope) { + return hasSizeLimit(checkerScope) && progress.getSize() >= limits.getSize(); + } + + /** + * @param checkerScope The scope that the limits are being checked from + * @return true when some limit is enforceable from the checker's scope and it has been reached + */ + boolean checkAnyLimitReached(LimitScope checkerScope) { + return checkSizeLimit(checkerScope) || checkBatchLimit(checkerScope); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("{"); + + sb.append("limits:"); + sb.append(limits); + + sb.append(", progress:"); + sb.append(progress); + + sb.append(", keepProgress:"); + sb.append(keepProgress); + + sb.append(", state:"); + sb.append(scannerState); + + sb.append("}"); + return sb.toString(); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static Builder newBuilder(boolean keepProgress) { + return new Builder(keepProgress); + } + + public static final class Builder { + boolean keepProgress = DEFAULT_KEEP_PROGRESS; + LimitFields limits = new LimitFields(); + + private Builder() { + } + + private Builder(boolean keepProgress) { + this.keepProgress = keepProgress; + } + + public Builder setKeepProgress(boolean keepProgress) { + this.keepProgress = keepProgress; + return this; + } + + public Builder setSizeLimit(LimitScope sizeScope, long sizeLimit) { + limits.setSize(sizeLimit); + limits.setSizeScope(sizeScope); + return this; + } + + public Builder setBatchLimit(int batchLimit) { + limits.setBatch(batchLimit); + return this; + } + + public ScannerContext build() { + return new ScannerContext(keepProgress, limits); + } + } + + /** + * The possible states a scanner may be in following a call to {@link InternalScanner#next(List)} + */ + public enum NextState { + MORE_VALUES(true, false), + NO_MORE_VALUES(false, false), + SIZE_LIMIT_REACHED(true, true), + + /** + * Special case of size limit reached to indicate that the size limit was reached in the middle + * of a row and thus a partial results was formed + */ + SIZE_LIMIT_REACHED_MID_ROW(true, true), + BATCH_LIMIT_REACHED(true, true); + + private boolean moreValues; + private boolean limitReached; + + private NextState(boolean moreValues, boolean limitReached) { + this.moreValues = moreValues; + this.limitReached = limitReached; + } + + /** + * @return true when the state indicates that more values may follow those that have been + * returned + */ + public boolean hasMoreValues() { + return this.moreValues; + } + + /** + * @return true when the state indicates that a limit has been reached and scan should stop + */ + public boolean limitReached() { + return this.limitReached; + } + + public static boolean isValidState(NextState state) { + return state != null; + } + + public static boolean hasMoreValues(NextState state) { + return isValidState(state) && state.hasMoreValues(); + } + } + + /** + * The various scopes where a limit can be enforced. Used to differentiate when a limit should be + * enforced or not. + */ + public enum LimitScope { + /** + * Enforcing a limit between rows means that the limit will not be considered until all the + * cells for a particular row have been retrieved + */ + BETWEEN_ROWS(0), + + /** + * Enforcing a limit between cells means that the limit will be considered after each full cell + * has been retrieved + */ + BETWEEN_CELLS(1); + + /** + * When enforcing a limit, we must check that the scope is appropriate for enforcement. + *

+ * To communicate this concept, each scope has a depth. A limit will be enforced if the depth of + * the checker's scope is less than or equal to the limit's scope. This means that when checking + * limits, the checker must know their own scope (i.e. are they checking the limits between + * rows, between cells, etc...) + */ + int depth; + + LimitScope(int depth) { + this.depth = depth; + } + + int depth() { + return depth; + } + + /** + * @param checkerScope The scope in which the limit is being checked + * @return true when the checker is in a scope that indicates the limit can be enforced. Limits + * can be enforced from "higher or equal" scopes (i.e. the checker's scope is at a + * lesser depth than the limit) + */ + boolean canEnforceLimitFromScope(LimitScope checkerScope) { + return checkerScope != null && checkerScope.depth() <= depth; + } + } + + /** + * The different fields that can be used as limits in calls to + * {@link InternalScanner#next(java.util.List)} and {@link RegionScanner#next(java.util.List)} + */ + private static class LimitFields { + /** + * Default values of the limit fields. Defined such that if a field does NOT change from its + * default, it will not be enforced + */ + private static int DEFAULT_BATCH = -1; + private static long DEFAULT_SIZE = -1L; + + /** + * Default scope that is assigned to a limit if a scope is not specified. + */ + private static final LimitScope DEFAULT_SCOPE = LimitScope.BETWEEN_ROWS; + + // The batch limit will always be enforced between cells, thus, there isn't a field to hold the + // batch scope + int batch = DEFAULT_BATCH; + + LimitScope sizeScope = DEFAULT_SCOPE; + long size = DEFAULT_SIZE; + + /** + * Fields keep their default values. + */ + LimitFields() { + } + + LimitFields(int batch, LimitScope sizeScope, long size) { + setFields(batch, sizeScope, size); + } + + void copy(LimitFields limitsToCopy) { + if (limitsToCopy != null) { + setFields(limitsToCopy.getBatch(), limitsToCopy.getSizeScope(), limitsToCopy.getSize()); + } + } + + /** + * Set all fields together. + * @param batch + * @param sizeScope + * @param size + */ + void setFields(int batch, LimitScope sizeScope, long size) { + setBatch(batch); + setSizeScope(sizeScope); + setSize(size); + } + + int getBatch() { + return this.batch; + } + + void setBatch(int batch) { + this.batch = batch; + } + + /** + * @param checkerScope + * @return true when the limit can be enforced from the scope of the checker + */ + boolean canEnforceBatchLimitFromScope(LimitScope checkerScope) { + return LimitScope.BETWEEN_CELLS.canEnforceLimitFromScope(checkerScope); + } + + long getSize() { + return this.size; + } + + void setSize(long size) { + this.size = size; + } + + /** + * @return {@link LimitScope} indicating scope in which the size limit is enforced + */ + LimitScope getSizeScope() { + return this.sizeScope; + } + + /** + * Change the scope in which the size limit is enforced + */ + void setSizeScope(LimitScope scope) { + this.sizeScope = scope; + } + + /** + * @param checkerScope + * @return true when the limit can be enforced from the scope of the checker + */ + boolean canEnforceSizeLimitFromScope(LimitScope checkerScope) { + return this.sizeScope.canEnforceLimitFromScope(checkerScope); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("{"); + + sb.append("batch:"); + sb.append(batch); + + sb.append(", size:"); + sb.append(size); + + sb.append(", sizeScope:"); + sb.append(sizeScope); + + sb.append("}"); + return sb.toString(); + } + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java index 831673d3081..bcc0a904d3b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.monitoring.MonitoredTask; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.compactions.Compactor; /** @@ -110,10 +109,14 @@ abstract class StoreFlusher { Compactor.CellSink sink, long smallestReadPoint) throws IOException { int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT); + + ScannerContext scannerContext = + ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); + List kvs = new ArrayList(); boolean hasMore; do { - hasMore = NextState.hasMoreValues(scanner.next(kvs, compactionKVMax)); + hasMore = scanner.next(kvs, scannerContext); if (!kvs.isEmpty()) { for (Cell c : kvs) { // If we know that this KV is going to be included always, then let us 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 298d5bccca8..665ed461dae 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 @@ -43,6 +43,8 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode; +import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; +import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; import org.apache.hadoop.hbase.regionserver.handler.ParallelSeekHandler; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -442,45 +444,39 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } } - /** - * Get the next row of values from this Store. - * @param outResult - * @param limit - * @return true if there are more rows, false if scanner is done - */ @Override - public NextState next(List outResult, int limit) throws IOException { - // -1 means no limit - return next(outResult, limit, -1); + public boolean next(List outResult) throws IOException { + return next(outResult, NoLimitScannerContext.getInstance()); } /** * Get the next row of values from this Store. * @param outResult - * @param limit - * @param remainingResultSize + * @param scannerContext * @return true if there are more rows, false if scanner is done */ @Override - public NextState next(List outResult, int limit, long remainingResultSize) - throws IOException { + public boolean next(List outResult, ScannerContext scannerContext) throws IOException { lock.lock(); try { + if (scannerContext == null) { + throw new IllegalArgumentException("Scanner context cannot be null"); + } if (checkReseek()) { - return NextState.makeState(NextState.State.MORE_VALUES, 0); + return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); } // if the heap was left null, then the scanners had previously run out anyways, close and // return. if (this.heap == null) { close(); - return NextState.makeState(NextState.State.NO_MORE_VALUES, 0); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } Cell peeked = this.heap.peek(); if (peeked == null) { close(); - return NextState.makeState(NextState.State.NO_MORE_VALUES, 0); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } // only call setRow if the row changes; avoids confusing the query matcher @@ -489,16 +485,18 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner int offset = peeked.getRowOffset(); short length = peeked.getRowLength(); - // 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); + // If no limits exists in the scope LimitScope.Between_Cells then we are sure we are changing + // rows. Else it is possible we are still traversing the same row so we must perform the row + // comparison. + if (!scannerContext.hasAnyLimit(LimitScope.BETWEEN_CELLS) || matcher.row == null || + !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, matcher.rowLength)) { + this.countPerRow = 0; + matcher.setRow(row, offset, length); } + // Clear progress away unless invoker has indicated it should be kept. + if (!scannerContext.getKeepProgress()) scannerContext.clearProgress(); + Cell cell; // Only do a sanity-check if store and comparator are available. @@ -507,7 +505,6 @@ 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. @@ -532,7 +529,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner this.countPerRow > (storeLimit + storeOffset)) { // do what SEEK_NEXT_ROW does. if (!matcher.moreRowsMayExistAfter(cell)) { - return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } seekToNextRow(cell); break LOOP; @@ -542,9 +539,15 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // also update metric accordingly if (this.countPerRow > storeOffset) { outResult.add(cell); + + // Update local tracking information count++; totalBytesRead += CellUtil.estimatedSerializedSizeOf(cell); - totalHeapSize += CellUtil.estimatedHeapSizeOfWithoutTags(cell); + + // Update the progress of the scanner context + scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell)); + scannerContext.incrementBatchProgress(1); + if (totalBytesRead > maxRowSize) { throw new RowTooBigException("Max row size allowed: " + maxRowSize + ", but the row is bigger than that."); @@ -553,7 +556,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) { if (!matcher.moreRowsMayExistAfter(cell)) { - return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } seekToNextRow(cell); } else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) { @@ -562,26 +565,26 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner this.heap.next(); } - if (limit > 0 && (count == limit)) { + if (scannerContext.checkBatchLimit(LimitScope.BETWEEN_CELLS)) { break LOOP; } - if (remainingResultSize > 0 && (totalHeapSize >= remainingResultSize)) { + if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_CELLS)) { break LOOP; } continue; case DONE: - return NextState.makeState(NextState.State.MORE_VALUES, totalHeapSize); + return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); case DONE_SCAN: close(); - return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); case SEEK_NEXT_ROW: // This is just a relatively simple end of scan fix, to short-cut end // us if there is an endKey in the scan. if (!matcher.moreRowsMayExistAfter(cell)) { - return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } seekToNextRow(cell); @@ -611,12 +614,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } if (count > 0) { - return NextState.makeState(NextState.State.MORE_VALUES, totalHeapSize); + return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); } // No more keys close(); - return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize); + return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); } finally { lock.unlock(); } @@ -655,11 +658,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner return qcode; } - @Override - public NextState next(List outResult) throws IOException { - return next(outResult, -1); - } - // Implementation of ChangedReadersObserver @Override public void updateReaders() throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index ae820b55d82..d1bb65774e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; @@ -246,10 +246,13 @@ public abstract class Compactor { store.getRegionInfo().getRegionNameAsString() + "#" + store.getFamily().getNameAsString(); long now = 0; boolean hasMore; + ScannerContext scannerContext = + ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); + throughputController.start(compactionName); try { do { - hasMore = NextState.hasMoreValues(scanner.next(cells, compactionKVMax)); + hasMore = scanner.next(cells, scannerContext); if (LOG.isDebugEnabled()) { now = EnvironmentEdgeManager.currentTime(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java index a01f8a1a420..fafc5a5b0d6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java @@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -369,7 +368,7 @@ public class AccessControlLists { while (true) { List row = new ArrayList(); - boolean hasNext = NextState.hasMoreValues(iScanner.next(row)); + boolean hasNext = iScanner.next(row); ListMultimap perms = ArrayListMultimap.create(); byte[] entry = null; for (Cell kv : row) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 2bab7e8d9f3..cd8f5fff499 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -87,11 +87,11 @@ import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; @@ -807,10 +807,12 @@ public class AccessController extends BaseMasterAndRegionObserver boolean foundColumn = false; try { boolean more = false; + ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(1).build(); + do { cells.clear(); // scan with limit as 1 to hold down memory use on wide rows - more = NextState.hasMoreValues(scanner.next(cells, 1)); + more = scanner.next(cells, scannerContext); for (Cell cell: cells) { if (LOG.isTraceEnabled()) { LOG.trace("Found cell " + cell); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java index 7d1ff0d31d8..0d5b27ef76a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; @@ -568,7 +567,7 @@ public abstract class HBaseTestCase extends TestCase { @Override public boolean next(List results) throws IOException { - return NextState.hasMoreValues(scanner.next(results)); + return scanner.next(results); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java index e7c3813c0be..eef955ecf0d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java @@ -409,6 +409,7 @@ public class TestPartialResultsFromClientSide { scan.setBatch(batch); ResultScanner scanner = TABLE.getScanner(scan); Result result = scanner.next(); + int repCount = 0; while ((result = scanner.next()) != null) { assertTrue(result.rawCells() != null); @@ -416,11 +417,12 @@ public class TestPartialResultsFromClientSide { if (result.isPartial()) { final String error = "Cells:" + result.rawCells().length + " Batch size:" + batch - + " cellsPerPartialResult:" + cellsPerPartialResult; + + " cellsPerPartialResult:" + cellsPerPartialResult + " rep:" + repCount; assertTrue(error, result.rawCells().length <= Math.min(batch, cellsPerPartialResult)); } else { assertTrue(result.rawCells().length <= batch); } + repCount++; } scanner.close(); @@ -458,7 +460,7 @@ public class TestPartialResultsFromClientSide { do { partialResult = partialScanner.next(); partials.add(partialResult); - } while (partialResult.isPartial()); + } while (partialResult != null && partialResult.isPartial()); completeResult = Result.createCompleteResult(partials); oneShotResult = oneShotScanner.next(); @@ -696,7 +698,7 @@ public class TestPartialResultsFromClientSide { LOG.info("r2: " + r2); } - final String failureMessage = "Results r1:" + r1 + " r2:" + r2 + " are not equivalent"; + final String failureMessage = "Results r1:" + r1 + " \nr2:" + r2 + " are not equivalent"; if (r1 == null && r2 == null) fail(failureMessage); else if (r1 == null || r2 == null) fail(failureMessage); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java index cdfb774cb52..1f6dc98be67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -94,7 +93,7 @@ public class TestIntraRowPagination { RegionScanner scanner = region.getScanner(scan); List kvListScan = new ArrayList(); List results = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(results)) || !results.isEmpty()) { + while (scanner.next(results) || !results.isEmpty()) { kvListScan.addAll(results); results.clear(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index efc8db21da4..bfc1230d87f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableNotFoundException; @@ -659,25 +660,34 @@ public class TestReplicasClient { private void runMultipleScansOfOneType(boolean reversed, boolean small) throws Exception { openRegion(hriSecondary); int NUMROWS = 100; + int NUMCOLS = 10; try { for (int i = 0; i < NUMROWS; i++) { byte[] b1 = Bytes.toBytes("testUseRegionWithReplica" + i); - Put p = new Put(b1); - p.add(f, b1, b1); - table.put(p); + for (int col = 0; col < NUMCOLS; col++) { + Put p = new Put(b1); + String qualifier = "qualifer" + col; + KeyValue kv = new KeyValue(b1, f, qualifier.getBytes()); + p.add(kv); + table.put(p); + } } LOG.debug("PUT done"); int caching = 20; + long maxResultSize = Long.MAX_VALUE; + byte[] start; if (reversed) start = Bytes.toBytes("testUseRegionWithReplica" + (NUMROWS - 1)); else start = Bytes.toBytes("testUseRegionWithReplica" + 0); - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, start, NUMROWS, false, false); + scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, + start, NUMROWS, NUMCOLS, false, false); - //Even if we were to slow the server down, unless we ask for stale - //we won't get it + // Even if we were to slow the server down, unless we ask for stale + // we won't get it SlowMeCopro.sleepTime.set(5000); - scanWithReplicas(reversed, small, Consistency.STRONG, caching, start, NUMROWS, false, false); + scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, start, NUMROWS, + NUMCOLS, false, false); SlowMeCopro.sleepTime.set(0); flushRegion(hriPrimary); @@ -686,13 +696,32 @@ public class TestReplicasClient { //Now set the flag to get a response even if stale SlowMeCopro.sleepTime.set(5000); - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, start, NUMROWS, true, false); + scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, + start, NUMROWS, NUMCOLS, true, false); SlowMeCopro.sleepTime.set(0); // now make some 'next' calls slow SlowMeCopro.slowDownNext.set(true); SlowMeCopro.countOfNext.set(0); - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, start, NUMROWS, true, true); + scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start, + NUMROWS, NUMCOLS, true, true); + SlowMeCopro.slowDownNext.set(false); + SlowMeCopro.countOfNext.set(0); + + // Make sure we do not get stale data.. + SlowMeCopro.sleepTime.set(5000); + scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, + start, NUMROWS, NUMCOLS, false, false); + SlowMeCopro.sleepTime.set(0); + + // While the next calls are slow, set maxResultSize to 1 so that some partial results will be + // returned from the server before the replica switch occurs. + maxResultSize = 1; + SlowMeCopro.slowDownNext.set(true); + SlowMeCopro.countOfNext.set(0); + scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start, + NUMROWS, NUMCOLS, true, true); + maxResultSize = Long.MAX_VALUE; SlowMeCopro.slowDownNext.set(false); SlowMeCopro.countOfNext.set(0); } finally { @@ -710,33 +739,60 @@ public class TestReplicasClient { } private void scanWithReplicas(boolean reversed, boolean small, Consistency consistency, - int caching, byte[] startRow, int numRows, boolean staleExpected, boolean slowNext) + int caching, long maxResultSize, byte[] startRow, int numRows, int numCols, + boolean staleExpected, boolean slowNext) throws Exception { Scan scan = new Scan(startRow); scan.setCaching(caching); + scan.setMaxResultSize(maxResultSize); scan.setReversed(reversed); scan.setSmall(small); scan.setConsistency(consistency); ResultScanner scanner = table.getScanner(scan); Iterator iter = scanner.iterator(); + + // Maps of row keys that we have seen so far HashMap map = new HashMap(); - int count = 0; + + // Tracked metrics + int rowCount = 0; + int cellCount = 0; int countOfStale = 0; + while (iter.hasNext()) { - count++; + rowCount++; Result r = iter.next(); - if (map.containsKey(new String(r.getRow()))) { + String row = new String(r.getRow()); + + if (map.containsKey(row)) { throw new Exception("Unexpected scan result. Repeated row " + Bytes.toString(r.getRow())); } - map.put(new String(r.getRow()), true); + + map.put(row, true); + + for (Cell cell : r.rawCells()) { + cellCount++; + } + if (!slowNext) Assert.assertTrue(r.isStale() == staleExpected); if (r.isStale()) countOfStale++; } - LOG.debug("Count of rows " + count + " num rows expected " + numRows); - Assert.assertTrue(count == numRows); + Assert.assertTrue("Count of rows " + rowCount + " num rows expected " + numRows, + rowCount == numRows); + Assert.assertTrue("Count of cells: " + cellCount + " cells expected: " + numRows * numCols, + cellCount == (numRows * numCols)); + if (slowNext) { LOG.debug("Count of Stale " + countOfStale); - Assert.assertTrue(countOfStale > 1 && countOfStale < numRows); + Assert.assertTrue(countOfStale > 1); + + // If the scan was configured in such a way that a full row was NOT retrieved before the + // replica switch occurred, then it is possible that all rows were stale + if (maxResultSize != Long.MAX_VALUE) { + Assert.assertTrue(countOfStale <= numRows); + } else { + Assert.assertTrue(countOfStale < numRows); + } } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java index 68053c0dca4..8aa8da1d817 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationP import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.RpcCallback; @@ -89,7 +88,7 @@ implements Coprocessor, CoprocessorService { boolean hasMore = false; do { curVals.clear(); - hasMore = NextState.hasMoreValues(scanner.next(curVals)); + hasMore = scanner.next(curVals); for (Cell kv : curVals) { if (CellUtil.matchingQualifier(kv, qualifier)) { sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java index c9a628a0207..43159460f09 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; @@ -98,7 +97,7 @@ implements Coprocessor, CoprocessorService { boolean hasMore = false; do { curVals.clear(); - hasMore = NextState.hasMoreValues(scanner.next(curVals)); + hasMore = scanner.next(curVals); for (Cell kv : curVals) { if (CellUtil.matchingQualifier(kv, qualifier)) { sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java index 0c4d07605d9..54289ef1613 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; @@ -98,7 +97,7 @@ implements Coprocessor, CoprocessorService { boolean hasMore = false; do { curVals.clear(); - hasMore = NextState.hasMoreValues(scanner.next(curVals)); + hasMore = scanner.next(curVals); for (Cell kv : curVals) { if (CellUtil.matchingQualifier(kv, qualifier)) { sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset()); 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 10ecae334f5..a8b545647a7 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 @@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.SplitTransaction; import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory; import org.apache.hadoop.hbase.regionserver.Store; @@ -87,36 +88,26 @@ public class TestCoprocessorInterface { } @Override - public NextState next(List results) throws IOException { + public boolean next(List results) throws IOException { return delegate.next(results); } @Override - public NextState next(List result, int limit) throws IOException { - return delegate.next(result, limit); - } - - @Override - public NextState next(List result, int limit, long remainingResultSize) + public boolean next(List result, ScannerContext scannerContext) throws IOException { - return delegate.next(result, limit, remainingResultSize); + return delegate.next(result, scannerContext); } @Override - public NextState nextRaw(List result) + public boolean nextRaw(List result) throws IOException { return delegate.nextRaw(result); } @Override - public NextState nextRaw(List result, int limit) throws IOException { - return delegate.nextRaw(result, limit); - } - - @Override - public NextState nextRaw(List result, int limit, long remainingResultSize) + public boolean nextRaw(List result, ScannerContext context) throws IOException { - return delegate.nextRaw(result, limit, remainingResultSize); + return delegate.nextRaw(result, context); } @Override 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 a4963aec5a3..454a61d65ca 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 @@ -66,8 +66,10 @@ import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.NoLimitScannerContext; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; @@ -433,24 +435,17 @@ public class TestRegionObserverInterface { Store store, final InternalScanner scanner, final ScanType scanType) { return new InternalScanner() { @Override - public NextState next(List results) throws IOException { - return next(results, -1); + public boolean next(List results) throws IOException { + return next(results, NoLimitScannerContext.getInstance()); } @Override - public NextState next(List results, int limit) throws IOException { - return next(results, limit, -1); - } - - @Override - public NextState next(List results, int limit, long remainingResultSize) + public boolean next(List results, ScannerContext scannerContext) throws IOException { List internalResults = new ArrayList(); boolean hasMore; - NextState state; do { - state = scanner.next(internalResults, limit, remainingResultSize); - hasMore = state != null && state.hasMoreValues(); + hasMore = scanner.next(internalResults, scannerContext); if (!internalResults.isEmpty()) { long row = Bytes.toLong(CellUtil.cloneValue(internalResults.get(0))); if (row % 2 == 0) { @@ -465,7 +460,7 @@ public class TestRegionObserverInterface { if (!internalResults.isEmpty()) { results.addAll(internalResults); } - return state; + return hasMore; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java index abd99213b36..828842d7ae8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -105,7 +104,8 @@ public class TestColumnPrefixFilter { InternalScanner scanner = region.getScanner(scan); List results = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(results))); + while (scanner.next(results)) + ; assertEquals(prefixMap.get(s).size(), results.size()); } } finally { @@ -170,7 +170,8 @@ public class TestColumnPrefixFilter { InternalScanner scanner = region.getScanner(scan); List results = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(results))); + while (scanner.next(results)) + ; assertEquals(prefixMap.get(s).size(), results.size()); } } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java index 97f0874e98a..add549a9141 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -151,7 +150,7 @@ public class TestDependentColumnFilter { int i = 0; int cells = 0; for (boolean done = true; done; i++) { - done = NextState.hasMoreValues(scanner.next(results)); + done = scanner.next(results); Arrays.sort(results.toArray(new KeyValue[results.size()]), KeyValue.COMPARATOR); LOG.info("counter=" + i + ", " + results); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index 82ea5d463fc..5fcf64eed7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.FilterTests; @@ -503,7 +502,7 @@ public class TestFilter { InternalScanner scanner = this.region.getScanner(s); int scannerCounter = 0; while (true) { - boolean isMoreResults = NextState.hasMoreValues(scanner.next(new ArrayList())); + boolean isMoreResults = scanner.next(new ArrayList()); scannerCounter++; if (scannerCounter >= pageSize) { @@ -532,7 +531,7 @@ public class TestFilter { InternalScanner scanner = this.region.getScanner(s); while (true) { ArrayList values = new ArrayList(); - boolean isMoreResults = NextState.hasMoreValues(scanner.next(values)); + boolean isMoreResults = scanner.next(values); if (!isMoreResults || !Bytes.toString(values.get(0).getRow()).startsWith(prefix)) { Assert.assertTrue( @@ -566,7 +565,7 @@ public class TestFilter { InternalScanner scanner = this.region.getScanner(s); int scannerCounter = 0; while (true) { - boolean isMoreResults = NextState.hasMoreValues(scanner.next(new ArrayList())); + boolean isMoreResults = scanner.next(new ArrayList()); scannerCounter++; if (scannerCounter >= pageSize) { @@ -644,7 +643,7 @@ public class TestFilter { InternalScanner scanner = this.region.getScanner(s); while (true) { ArrayList values = new ArrayList(); - boolean isMoreResults = NextState.hasMoreValues(scanner.next(values)); + boolean isMoreResults = scanner.next(values); if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) { assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining()); } @@ -673,7 +672,7 @@ public class TestFilter { InternalScanner scanner = this.region.getScanner(s); while (true) { ArrayList values = new ArrayList(); - boolean isMoreResults = NextState.hasMoreValues(scanner.next(values)); + boolean isMoreResults = scanner.next(values); assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining()); if (!isMoreResults) { break; @@ -1476,7 +1475,7 @@ public class TestFilter { InternalScanner scanner = testRegion.getScanner(s1); List results = new ArrayList(); int resultCount = 0; - while (NextState.hasMoreValues(scanner.next(results))) { + while (scanner.next(results)) { resultCount++; byte[] row = CellUtil.cloneRow(results.get(0)); LOG.debug("Found row: " + Bytes.toStringBinary(row)); @@ -1618,7 +1617,7 @@ public class TestFilter { List results = new ArrayList(); int i = 0; for (boolean done = true; done; i++) { - done = NextState.hasMoreValues(scanner.next(results)); + done = scanner.next(results); Arrays.sort(results.toArray(new KeyValue[results.size()]), KeyValue.COMPARATOR); LOG.info("counter=" + i + ", " + results); @@ -1640,7 +1639,7 @@ public class TestFilter { List results = new ArrayList(); int i = 0; for (boolean done = true; done; i++) { - done = NextState.hasMoreValues(scanner.next(results)); + done = scanner.next(results); Arrays.sort(results.toArray(new KeyValue[results.size()]), KeyValue.COMPARATOR); LOG.info("counter=" + i + ", " + results); @@ -1662,7 +1661,7 @@ public class TestFilter { int row = 0; int idx = 0; for (boolean done = true; done; row++) { - done = NextState.hasMoreValues(scanner.next(results)); + done = scanner.next(results); Arrays.sort(results.toArray(new KeyValue[results.size()]), KeyValue.COMPARATOR); if(results.isEmpty()) break; @@ -1693,7 +1692,7 @@ public class TestFilter { int row = 0; int idx = 0; for (boolean more = true; more; row++) { - more = NextState.hasMoreValues(scanner.next(results)); + more = scanner.next(results); Arrays.sort(results.toArray(new KeyValue[results.size()]), KeyValue.COMPARATOR); if(results.isEmpty()) break; @@ -2029,7 +2028,7 @@ public class TestFilter { List results = new ArrayList(); int i = 5; for (boolean done = true; done; i++) { - done = NextState.hasMoreValues(scanner.next(results)); + done = scanner.next(results); assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i))); assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2); results.clear(); @@ -2052,7 +2051,7 @@ public class TestFilter { assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2); results.clear(); } - assertFalse(NextState.hasMoreValues(scanner.next(results))); + assertFalse(scanner.next(results)); // 3. let's begin to verify nested filter list // 3.1 add rowFilter, then add subFilterList FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); @@ -2074,7 +2073,7 @@ public class TestFilter { assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2); results.clear(); } - assertFalse(NextState.hasMoreValues(scanner.next(results))); + assertFalse(scanner.next(results)); // 3.2 MAGIC here! add subFilterList first, then add rowFilter filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); filterList.addFilter(subFilterList); @@ -2095,7 +2094,7 @@ public class TestFilter { assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2); results.clear(); } - assertFalse(NextState.hasMoreValues(scanner.next(results))); + assertFalse(scanner.next(results)); WAL wal = ((HRegion)testRegion).getWAL(); ((HRegion)testRegion).close(); wal.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index b88bbbfaf4e..a8651d83dbc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -34,12 +34,11 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -140,7 +139,7 @@ public class TestInvocationRecordFilter { List actualValues = new ArrayList(); List temp = new ArrayList(); InternalScanner scanner = this.region.getScanner(scan); - while (NextState.hasMoreValues(scanner.next(temp))) { + while (scanner.next(temp)) { actualValues.addAll(temp); temp.clear(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java index 25f2e88c322..7b700b705af 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -110,7 +109,7 @@ public class TestMultipleColumnPrefixFilter { scan.setFilter(filter); List results = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - while (NextState.hasMoreValues(scanner.next(results))) + while (scanner.next(results)) ; assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size()); @@ -183,7 +182,7 @@ public class TestMultipleColumnPrefixFilter { scan.setFilter(filter); List results = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - while (NextState.hasMoreValues(scanner.next(results))) + while (scanner.next(results)) ; assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size()); @@ -228,7 +227,7 @@ public class TestMultipleColumnPrefixFilter { scan1.setFilter(multiplePrefixFilter); List results1 = new ArrayList(); InternalScanner scanner1 = region.getScanner(scan1); - while (NextState.hasMoreValues(scanner1.next(results1))) + while (scanner1.next(results1)) ; ColumnPrefixFilter singlePrefixFilter; @@ -239,7 +238,7 @@ public class TestMultipleColumnPrefixFilter { scan2.setFilter(singlePrefixFilter); List results2 = new ArrayList(); InternalScanner scanner2 = region.getScanner(scan1); - while (NextState.hasMoreValues(scanner2.next(results2))) + while (scanner2.next(results2)) ; assertEquals(results1.size(), results2.size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java index 1eda567f923..e31a73bb4ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.IOTests; @@ -117,7 +116,7 @@ public class TestPrefixTree { RegionScanner scanner = region.getScanner(scan); List cells = new ArrayList(); for (int i = 0; i < 3; i++) { - assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells))); + assertEquals(i < 2, scanner.next(cells)); CellScanner cellScanner = Result.create(cells).cellScanner(); while (cellScanner.advance()) { assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner @@ -136,7 +135,7 @@ public class TestPrefixTree { scan.setStopRow(Bytes.toBytes("a-b-A-1:")); scanner = region.getScanner(scan); for (int i = 1; i < 3; i++) { - assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells))); + assertEquals(i < 2, scanner.next(cells)); CellScanner cellScanner = Result.create(cells).cellScanner(); while (cellScanner.advance()) { assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner @@ -152,7 +151,7 @@ public class TestPrefixTree { scan.setStopRow(Bytes.toBytes("a-b-A-1:")); scanner = region.getScanner(scan); for (int i = 1; i < 3; i++) { - assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells))); + assertEquals(i < 2, scanner.next(cells)); CellScanner cellScanner = Result.create(cells).cellScanner(); while (cellScanner.advance()) { assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner @@ -167,7 +166,7 @@ public class TestPrefixTree { scan.setStartRow(Bytes.toBytes("a-b-A-1-140239")); scan.setStopRow(Bytes.toBytes("a-b-A-1:")); scanner = region.getScanner(scan); - assertFalse(NextState.hasMoreValues(scanner.next(cells))); + assertFalse(scanner.next(cells)); assertFalse(cells.isEmpty()); scanner.close(); } @@ -186,7 +185,7 @@ public class TestPrefixTree { Scan scan = new Scan(Bytes.toBytes("obj29995")); RegionScanner scanner = region.getScanner(scan); List cells = new ArrayList(); - assertFalse(NextState.hasMoreValues(scanner.next(cells))); + assertFalse(scanner.next(cells)); assertArrayEquals(Bytes.toBytes("obj3"), Result.create(cells).getRow()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java index 6baadbb32bb..7584cf23cb4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java @@ -37,10 +37,9 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Test; @@ -122,7 +121,7 @@ public class TestScannerSelectionUsingKeyRange { cache.clearCache(); InternalScanner scanner = region.getScanner(scan); List results = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(results))) { + while (scanner.next(results)) { } scanner.close(); assertEquals(0, results.size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java index 4e0743dbb65..d5f4bcd202c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -140,7 +139,7 @@ public class TestScannerSelectionUsingTTL { final int expectedKVsPerRow = numFreshFiles * NUM_COLS_PER_ROW; int numReturnedRows = 0; LOG.info("Scanning the entire table"); - while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) { + while (scanner.next(results) || results.size() > 0) { assertEquals(expectedKVsPerRow, results.size()); ++numReturnedRows; results.clear(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index 478e239e1ee..66e19523de6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -60,11 +60,10 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.io.HeapSize; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -467,7 +466,8 @@ public class TestAtomicOperation { Scan s = new Scan(row); RegionScanner rs = region.getScanner(s); List r = new ArrayList(); - while(NextState.hasMoreValues(rs.next(r))); + while (rs.next(r)) + ; rs.close(); if (r.size() != 1) { LOG.debug(r); @@ -561,7 +561,8 @@ public class TestAtomicOperation { Scan s = new Scan(); RegionScanner scanner = region.getScanner(s); List results = new ArrayList(); - scanner.next(results, 2); + ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(2).build(); + scanner.next(results, scannerContext); for (Cell keyValue : results) { assertEquals("50",Bytes.toString(CellUtil.cloneValue(keyValue))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java index 446c64ccafc..b2ba97c9176 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java @@ -25,15 +25,14 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; -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.client.Scan; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheStats; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; import org.junit.Before; @@ -103,7 +102,8 @@ public class TestBlocksScanned extends HBaseTestCase { InternalScanner s = r.getScanner(scan); List results = new ArrayList(); - while (NextState.hasMoreValues(s.next(results))); + while (s.next(results)) + ; s.close(); int expectResultSize = 'z' - 'a'; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java index c09b32d8d17..1d5c61b387a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -161,7 +160,7 @@ public class TestColumnSeeking { } InternalScanner scanner = region.getScanner(scan); List results = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(results))) + while (scanner.next(results)) ; assertEquals(kvSet.size(), results.size()); assertTrue(KeyValueTestUtil.containsIgnoreMvccVersion(results, kvSet)); @@ -273,7 +272,7 @@ public class TestColumnSeeking { } InternalScanner scanner = region.getScanner(scan); List results = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(results))) + while (scanner.next(results)) ; assertEquals(kvSet.size(), results.size()); assertTrue(KeyValueTestUtil.containsIgnoreMvccVersion(results, kvSet)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index e1e5b895f78..622c14550eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -107,7 +106,7 @@ public class TestDefaultMemStore extends TestCase { StoreScanner s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners); int count = 0; try { - while (NextState.hasMoreValues(s.next(result))) { + while (s.next(result)) { LOG.info(result); count++; // Row count is same as column count. @@ -127,7 +126,7 @@ public class TestDefaultMemStore extends TestCase { s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners); count = 0; try { - while (NextState.hasMoreValues(s.next(result))) { + while (s.next(result)) { LOG.info(result); // Assert the stuff is coming out in right order. assertTrue(CellUtil.matchingRow(result.get(0), Bytes.toBytes(count))); @@ -154,7 +153,7 @@ public class TestDefaultMemStore extends TestCase { count = 0; int snapshotIndex = 5; try { - while (NextState.hasMoreValues(s.next(result))) { + while (s.next(result)) { LOG.info(result); // Assert the stuff is coming out in right order. assertTrue(CellUtil.matchingRow(result.get(0), Bytes.toBytes(count))); @@ -528,7 +527,7 @@ public class TestDefaultMemStore extends TestCase { Bytes.toBytes(startRowId)), scanInfo, scanType, null, memstore.getScanners(0)); List results = new ArrayList(); - for (int i = 0; NextState.hasMoreValues(scanner.next(results)); i++) { + for (int i = 0; scanner.next(results); i++) { int rowId = startRowId + i; Cell left = results.get(0); byte[] row1 = Bytes.toBytes(rowId); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index 416ee28137f..110cd365de4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -97,7 +96,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { InternalScanner s = mr.getScanner(new Scan()); try { List keys = new ArrayList(); - while (NextState.hasMoreValues(s.next(keys))) { + while (s.next(keys)) { LOG.info(keys); keys.clear(); } @@ -121,7 +120,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { s = mr.getScanner(scan); try { List keys = new ArrayList(); - while (NextState.hasMoreValues(s.next(keys))) { + while (s.next(keys)) { mr.delete(new Delete(CellUtil.cloneRow(keys.get(0)))); keys.clear(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 6a5e844f3b5..6abe0760604 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -128,11 +128,10 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.Stor import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Region.RowLock; import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL; import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; @@ -1265,7 +1264,7 @@ public class TestHRegion { boolean more = false; List results = new ArrayList(); do { - more = NextState.hasMoreValues(scanner.next(results)); + more = scanner.next(results); if (results != null && !results.isEmpty()) count++; else @@ -1284,7 +1283,7 @@ public class TestHRegion { List results = new ArrayList(); boolean more = false; do { - more = NextState.hasMoreValues(resultScanner.next(results)); + more = resultScanner.next(results); if (results != null && !results.isEmpty()) numberOfResults++; else @@ -2274,7 +2273,7 @@ public class TestHRegion { InternalScanner s = region.getScanner(scan); List results = new ArrayList(); - assertEquals(false, NextState.hasMoreValues(s.next(results))); + assertEquals(false, s.next(results)); assertEquals(1, results.size()); Cell kv = results.get(0); @@ -2766,7 +2765,7 @@ public class TestHRegion { List actual = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(actual)); + boolean hasNext = scanner.next(actual); assertEquals(false, hasNext); // Verify result @@ -2829,7 +2828,7 @@ public class TestHRegion { List actual = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(actual)); + boolean hasNext = scanner.next(actual); assertEquals(false, hasNext); // Verify result @@ -2911,7 +2910,7 @@ public class TestHRegion { List actual = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(actual)); + boolean hasNext = scanner.next(actual); assertEquals(false, hasNext); // Verify result @@ -2972,7 +2971,7 @@ public class TestHRegion { List actual = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(actual)); + boolean hasNext = scanner.next(actual); assertEquals(false, hasNext); // Verify result @@ -3033,7 +3032,7 @@ public class TestHRegion { List actual = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(actual)); + boolean hasNext = scanner.next(actual); assertEquals(false, hasNext); // Verify result @@ -3086,7 +3085,7 @@ public class TestHRegion { InternalScanner s = region.getScanner(scan); List results = new ArrayList(); - assertEquals(false, NextState.hasMoreValues(s.next(results))); + assertEquals(false, s.next(results)); assertEquals(0, results.size()); } finally { HBaseTestingUtility.closeRegionAndWAL(this.region); @@ -3160,7 +3159,7 @@ public class TestHRegion { List actual = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(actual)); + boolean hasNext = scanner.next(actual); assertEquals(false, hasNext); // Verify result @@ -3223,18 +3222,18 @@ public class TestHRegion { InternalScanner s = region.getScanner(scan); List results = new ArrayList(); - assertTrue(NextState.hasMoreValues(s.next(results))); + assertTrue(s.next(results)); assertEquals(results.size(), 1); results.clear(); - assertTrue(NextState.hasMoreValues(s.next(results))); + assertTrue(s.next(results)); assertEquals(results.size(), 3); assertTrue("orderCheck", CellUtil.matchingFamily(results.get(0), cf_alpha)); assertTrue("orderCheck", CellUtil.matchingFamily(results.get(1), cf_essential)); assertTrue("orderCheck", CellUtil.matchingFamily(results.get(2), cf_joined)); results.clear(); - assertFalse(NextState.hasMoreValues(s.next(results))); + assertFalse(s.next(results)); assertEquals(results.size(), 0); } finally { HBaseTestingUtility.closeRegionAndWAL(this.region); @@ -3317,8 +3316,9 @@ public class TestHRegion { List results = new ArrayList(); int index = 0; + ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(3).build(); while (true) { - boolean more = NextState.hasMoreValues(s.next(results, 3)); + boolean more = s.next(results, scannerContext); if ((index >> 1) < 5) { if (index % 2 == 0) assertEquals(results.size(), 3); @@ -3592,7 +3592,7 @@ public class TestHRegion { if (toggle) { flushThread.flush(); } - while (NextState.hasMoreValues(scanner.next(res))) + while (scanner.next(res)) ; if (!toggle) { flushThread.flush(); @@ -3717,7 +3717,7 @@ public class TestHRegion { boolean previousEmpty = res.isEmpty(); res.clear(); InternalScanner scanner = region.getScanner(scan); - while (NextState.hasMoreValues(scanner.next(res))) + while (scanner.next(res)) ; if (!res.isEmpty() || !previousEmpty || i > compactInterval) { assertEquals("i=" + i, expectedCount, res.size()); @@ -3999,7 +3999,7 @@ public class TestHRegion { InternalScanner scanner = region.getScanner(idxScan); List res = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(res))) + while (scanner.next(res)) ; assertEquals(1L, res.size()); } finally { @@ -4895,7 +4895,7 @@ public class TestHRegion { try { List curVals = new ArrayList(); boolean first = true; - OUTER_LOOP: while (NextState.hasMoreValues(s.next(curVals))) { + OUTER_LOOP: while (s.next(curVals)) { for (Cell kv : curVals) { byte[] val = CellUtil.cloneValue(kv); byte[] curval = val; @@ -5091,17 +5091,17 @@ public class TestHRegion { scan.setReversed(true); InternalScanner scanner = region.getScanner(scan); List currRow = new ArrayList(); - boolean hasNext = NextState.hasMoreValues(scanner.next(currRow)); + boolean hasNext = scanner.next(currRow); assertEquals(2, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA)); assertFalse(hasNext); @@ -5148,17 +5148,17 @@ public class TestHRegion { scan.setReversed(true); scan.setMaxVersions(5); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(currRow)); + boolean hasNext = scanner.next(currRow); assertEquals(2, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA)); assertFalse(hasNext); @@ -5202,17 +5202,17 @@ public class TestHRegion { List currRow = new ArrayList(); scan.setReversed(true); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(currRow)); + boolean hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA)); assertFalse(hasNext); @@ -5270,17 +5270,17 @@ public class TestHRegion { scan.setReversed(true); List currRow = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(currRow)); + boolean hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB)); assertFalse(hasNext); @@ -5291,7 +5291,7 @@ public class TestHRegion { scan.setReversed(true); currRow.clear(); scanner = region.getScanner(scan); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD)); scanner.close(); @@ -5350,17 +5350,17 @@ public class TestHRegion { scan.setReversed(true); List currRow = new ArrayList(); InternalScanner scanner = region.getScanner(scan); - boolean hasNext = NextState.hasMoreValues(scanner.next(currRow)); + boolean hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB)); assertFalse(hasNext); @@ -5371,7 +5371,7 @@ public class TestHRegion { scan.setReversed(true); currRow.clear(); scanner = region.getScanner(scan); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD)); scanner.close(); @@ -5495,42 +5495,42 @@ public class TestHRegion { // 1. scan out "row4" (5 kvs), "row5" can't be scanned out since not // included in scan range // "row4" takes 2 next() calls since batch=3 - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(3, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row4)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(2, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row4)); assertTrue(hasNext); // 2. scan out "row3" (2 kv) currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(2, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row3)); assertTrue(hasNext); // 3. scan out "row2" (4 kvs) // "row2" takes 2 next() calls since batch=3 currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(3, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row2)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row2)); assertTrue(hasNext); // 4. scan out "row1" (2 kv) currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(2, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row1)); assertTrue(hasNext); // 5. scan out "row0" (1 kv) currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row0)); assertFalse(hasNext); @@ -5591,22 +5591,22 @@ public class TestHRegion { scan.setBatch(10); InternalScanner scanner = region.getScanner(scan); List currRow = new ArrayList(); - boolean hasNext = NextState.hasMoreValues(scanner.next(currRow)); + boolean hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row4)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row3)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row2)); assertTrue(hasNext); currRow.clear(); - hasNext = NextState.hasMoreValues(scanner.next(currRow)); + hasNext = scanner.next(currRow); assertEquals(1, currRow.size()); assertTrue(Bytes.equals(currRow.get(0).getRow(), row1)); assertFalse(hasNext); @@ -5659,7 +5659,7 @@ public class TestHRegion { boolean more = false; int verify = startRow + 2 * numRows - 1; do { - more = NextState.hasMoreValues(scanner.next(currRow)); + more = scanner.next(currRow); assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + ""); verify--; currRow.clear(); @@ -5672,7 +5672,7 @@ public class TestHRegion { scanner = regions[1].getScanner(scan); verify = startRow + 2 * numRows - 1; do { - more = NextState.hasMoreValues(scanner.next(currRow)); + more = scanner.next(currRow); assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + ""); verify--; currRow.clear(); @@ -5685,7 +5685,7 @@ public class TestHRegion { scanner = regions[0].getScanner(scan); verify = startRow + numRows - 1; do { - more = NextState.hasMoreValues(scanner.next(currRow)); + more = scanner.next(currRow); assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + ""); verify--; currRow.clear(); @@ -5698,7 +5698,7 @@ public class TestHRegion { scanner = regions[0].getScanner(scan); verify = startRow + numRows - 1; do { - more = NextState.hasMoreValues(scanner.next(currRow)); + more = scanner.next(currRow); assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + ""); verify--; currRow.clear(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java index c480cd8f2af..9286e0d5dda 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -264,7 +263,8 @@ public class TestKeepDeletes { s.setTimeRange(0L, ts+1); InternalScanner scanner = region.getScanner(s); List kvs = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(kvs))); + while (scanner.next(kvs)) + ; assertTrue(kvs.isEmpty()); // flushing and minor compaction keep delete markers @@ -946,7 +946,7 @@ public class TestKeepDeletes { int res = 0; boolean hasMore; do { - hasMore = NextState.hasMoreValues(scan.next(kvs)); + hasMore = scan.next(kvs); for (Cell kv : kvs) { if(CellUtil.isDelete(kv)) res++; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 12ab2ad7778..cc168049b18 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java @@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; @@ -132,7 +131,7 @@ public class TestMajorCompaction { InternalScanner s = r.getScanner(new Scan()); do { List results = new ArrayList(); - boolean result = NextState.hasMoreValues(s.next(results)); + boolean result = s.next(results); r.delete(new Delete(CellUtil.cloneRow(results.get(0)))); if (!result) break; } while(true); @@ -145,7 +144,7 @@ public class TestMajorCompaction { int counter = 0; do { List results = new ArrayList(); - boolean result = NextState.hasMoreValues(s.next(results)); + boolean result = s.next(results); if (!result) break; counter++; } while(true); @@ -456,7 +455,7 @@ public class TestMajorCompaction { InternalScanner s = r.getScanner(scan); do { List results = new ArrayList(); - boolean result = NextState.hasMoreValues(s.next(results)); + boolean result = s.next(results); assertTrue(!results.isEmpty()); r.delete(new Delete(results.get(0).getRow())); if (!result) break; @@ -472,7 +471,7 @@ public class TestMajorCompaction { int counter = 0; do { List results = new ArrayList(); - boolean result = NextState.hasMoreValues(s.next(results)); + boolean result = s.next(results); if (!result) break; counter++; } while (true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java index 28d3ab9329f..dfdc4e431e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java @@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -250,7 +249,7 @@ public class TestMultiColumnScanner { String queryInfo = "columns queried: " + qualSet + " (columnBitMask=" + columnBitMask + "), maxVersions=" + maxVersions; - while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) { + while (scanner.next(results) || results.size() > 0) { for (Cell kv : results) { while (kvPos < kvs.size() && !matchesQuery(kvs.get(kvPos), qualSet, maxVersions, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java index 313a6ba2113..b2115b34ea4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -427,7 +426,7 @@ public class TestRegionMergeTransaction { List kvs = new ArrayList(); boolean hasNext = true; while (hasNext) { - hasNext = NextState.hasMoreValues(scanner.next(kvs)); + hasNext = scanner.next(kvs); if (!kvs.isEmpty()) rowcount++; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java index 92915e6da2d..8052d9c8582 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java @@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -434,7 +433,7 @@ public class TestReversibleScanners { int rowCount = 0; int kvCount = 0; try { - while (NextState.hasMoreValues(scanner.next(kvList))) { + while (scanner.next(kvList)) { if (kvList.isEmpty()) continue; rowCount++; kvCount += kvList.size(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java index ccae4bc5f7f..eada3fb7da3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java @@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -165,7 +164,7 @@ public class TestScanWithBloomError { { // Limit the scope of results. List results = new ArrayList(); - while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) { + while (scanner.next(results) || results.size() > 0) { allResults.addAll(results); results.clear(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java index ee72ce46268..84cb355f81d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java @@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter; import org.apache.hadoop.hbase.filter.PrefixFilter; import org.apache.hadoop.hbase.filter.WhileMatchFilter; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -137,7 +136,7 @@ public class TestScanner { InternalScanner s = r.getScanner(scan); int count = 0; - while (NextState.hasMoreValues(s.next(results))) { + while (s.next(results)) { count++; } s.close(); @@ -150,7 +149,7 @@ public class TestScanner { count = 0; Cell kv = null; results = new ArrayList(); - for (boolean first = true; NextState.hasMoreValues(s.next(results));) { + for (boolean first = true; s.next(results);) { kv = results.get(0); if (first) { assertTrue(CellUtil.matchingRow(kv, startrow)); @@ -173,7 +172,7 @@ public class TestScanner { InternalScanner s = r.getScanner(scan); boolean hasMore = true; while (hasMore) { - hasMore = NextState.hasMoreValues(s.next(results)); + hasMore = s.next(results); for (Cell kv : results) { assertEquals((byte)'a', CellUtil.cloneRow(kv)[0]); assertEquals((byte)'b', CellUtil.cloneRow(kv)[1]); @@ -189,7 +188,7 @@ public class TestScanner { InternalScanner s = r.getScanner(scan); boolean hasMore = true; while (hasMore) { - hasMore = NextState.hasMoreValues(s.next(results)); + hasMore = s.next(results); for (Cell kv : results) { assertTrue(Bytes.compareTo(CellUtil.cloneRow(kv), stopRow) <= 0); } @@ -389,7 +388,7 @@ public class TestScanner { scan.addColumn(COLS[0], EXPLICIT_COLS[ii]); } scanner = r.getScanner(scan); - while (NextState.hasMoreValues(scanner.next(results))) { + while (scanner.next(results)) { assertTrue(hasColumn(results, HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER)); byte [] val = CellUtil.cloneValue(getColumn(results, HConstants.CATALOG_FAMILY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java index dfced3b8fc3..1e09c406737 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.compress.Compression; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -225,7 +224,7 @@ public class TestSeekOptimizations { // result, not to the one already returned in results. boolean hasNext; do { - hasNext = NextState.hasMoreValues(scanner.next(results)); + hasNext = scanner.next(results); actualKVs.addAll(results); results.clear(); } while (hasNext); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java index da4f811702c..4f371bdb32a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java @@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.LruBlockCache; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -361,7 +360,7 @@ public class TestSplitTransaction { List kvs = new ArrayList(); boolean hasNext = true; while (hasNext) { - hasNext = NextState.hasMoreValues(scanner.next(kvs)); + hasNext = scanner.next(kvs); if (!kvs.isEmpty()) rowcount++; } } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java index c723c2e4631..ee392123497 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -91,7 +90,7 @@ public class TestStoreScanner extends TestCase { StoreScanner scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"), scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(5, results.size()); assertEquals(kvs[kvs.length - 1], results.get(0)); // Scan limited TimeRange @@ -101,7 +100,7 @@ public class TestStoreScanner extends TestCase { scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"), scanners); results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(2, results.size()); // Another range. scanSpec = new Scan(Bytes.toBytes(r1)); @@ -110,7 +109,7 @@ public class TestStoreScanner extends TestCase { scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"), scanners); results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(1, results.size()); // See how TimeRange and Versions interact. // Another range. @@ -120,7 +119,7 @@ public class TestStoreScanner extends TestCase { scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"), scanners); results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(3, results.size()); } @@ -141,7 +140,7 @@ public class TestStoreScanner extends TestCase { getCols("a"), scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(1, results.size()); assertEquals(kvs[0], results.get(0)); } @@ -196,7 +195,7 @@ public class TestStoreScanner extends TestCase { getCols("a"), scanners); List results = new ArrayList(); - assertFalse(NextState.hasMoreValues(scan.next(results))); + assertFalse(scan.next(results)); assertEquals(0, results.size()); } @@ -216,14 +215,14 @@ public class TestStoreScanner extends TestCase { getCols("a"), scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(0, results.size()); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(1, results.size()); assertEquals(kvs[2], results.get(0)); - assertEquals(false, NextState.hasMoreValues(scan.next(results))); + assertEquals(false, scan.next(results)); } public void testDeleteVersionMaskingMultiplePuts() throws IOException { @@ -245,7 +244,7 @@ public class TestStoreScanner extends TestCase { // the two put at ts=now will be masked by the 1 delete, and // since the scan default returns 1 version we'll return the newest // key, which is kvs[2], now-100. - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(1, results.size()); assertEquals(kvs2[1], results.get(0)); } @@ -267,7 +266,7 @@ public class TestStoreScanner extends TestCase { StoreScanner scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"), scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(2, results.size()); assertEquals(kvs2[1], results.get(0)); assertEquals(kvs2[0], results.get(1)); @@ -283,7 +282,7 @@ public class TestStoreScanner extends TestCase { StoreScanner scan = new StoreScanner(new Scan(Bytes.toBytes("R1")), scanInfo, scanType, null, scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(2, results.size()); assertEquals(kvs[0], results.get(0)); assertEquals(kvs[1], results.get(1)); @@ -312,7 +311,7 @@ public class TestStoreScanner extends TestCase { StoreScanner scan = new StoreScanner(new Scan().setMaxVersions(2), scanInfo, scanType, null, scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(5, results.size()); assertEquals(kvs[0], results.get(0)); assertEquals(kvs[2], results.get(1)); @@ -341,13 +340,13 @@ public class TestStoreScanner extends TestCase { new Scan().setMaxVersions(Integer.MAX_VALUE), scanInfo, scanType, null, scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(0, results.size()); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(1, results.size()); assertEquals(kvs[kvs.length-1], results.get(0)); - assertEquals(false, NextState.hasMoreValues(scan.next(results))); + assertEquals(false, scan.next(results)); } public void testDeleteColumn() throws IOException { @@ -361,7 +360,7 @@ public class TestStoreScanner extends TestCase { StoreScanner scan = new StoreScanner(new Scan(), scanInfo, scanType, null, scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(1, results.size()); assertEquals(kvs[3], results.get(0)); } @@ -385,18 +384,18 @@ public class TestStoreScanner extends TestCase { getCols("a", "d"), scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(2, results.size()); assertEquals(kvs[0], results.get(0)); assertEquals(kvs[3], results.get(1)); results.clear(); - assertEquals(true, NextState.hasMoreValues(scan.next(results))); + assertEquals(true, scan.next(results)); assertEquals(1, results.size()); assertEquals(kvs[kvs.length-1], results.get(0)); results.clear(); - assertEquals(false, NextState.hasMoreValues(scan.next(results))); + assertEquals(false, scan.next(results)); } /* @@ -426,20 +425,20 @@ public class TestStoreScanner extends TestCase { null, scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scanner.next(results))); + assertEquals(true, scanner.next(results)); assertEquals(2, results.size()); assertEquals(kvs[1], results.get(0)); assertEquals(kvs[2], results.get(1)); results.clear(); - assertEquals(true, NextState.hasMoreValues(scanner.next(results))); + assertEquals(true, scanner.next(results)); assertEquals(3, results.size()); assertEquals(kvs[4], results.get(0)); assertEquals(kvs[5], results.get(1)); assertEquals(kvs[6], results.get(2)); results.clear(); - assertEquals(false, NextState.hasMoreValues(scanner.next(results))); + assertEquals(false, scanner.next(results)); } public void testScannerReseekDoesntNPE() throws Exception { @@ -496,12 +495,12 @@ public class TestStoreScanner extends TestCase { new StoreScanner(scan, scanInfo, scanType, null, scanners); List results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scanner.next(results))); + assertEquals(true, scanner.next(results)); assertEquals(1, results.size()); assertEquals(kvs[1], results.get(0)); results.clear(); - assertEquals(false, NextState.hasMoreValues(scanner.next(results))); + assertEquals(false, scanner.next(results)); } public void testDeleteMarkerLongevity() throws Exception { @@ -561,7 +560,7 @@ public class TestStoreScanner extends TestCase { HConstants.OLDEST_TIMESTAMP); List results = new ArrayList(); results = new ArrayList(); - assertEquals(true, NextState.hasMoreValues(scanner.next(results))); + assertEquals(true, scanner.next(results)); assertEquals(kvs[0], results.get(0)); assertEquals(kvs[2], results.get(1)); assertEquals(kvs[3], results.get(2)); 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 06bbd5498e1..b7435504af3 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 @@ -237,22 +237,14 @@ public class TestStripeCompactor { } @Override - public NextState next(List results) throws IOException { - if (kvs.isEmpty()) return NextState.makeState(NextState.State.NO_MORE_VALUES); + public boolean next(List results) throws IOException { + if (kvs.isEmpty()) return false; results.add(kvs.remove(0)); - if (!kvs.isEmpty()) { - return NextState.makeState(NextState.State.MORE_VALUES); - } else { - return NextState.makeState(NextState.State.NO_MORE_VALUES); - } - } - @Override - public NextState next(List result, int limit) throws IOException { - return next(result); + return !kvs.isEmpty(); } @Override - public NextState next(List result, int limit, long remainingResultSize) + public boolean next(List result, ScannerContext scannerContext) throws IOException { return next(result); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java index 303ee365fe4..fb4561b5df9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -106,7 +105,7 @@ public class TestWideScanner extends HBaseTestCase { int i = 0; boolean more; do { - more = NextState.hasMoreValues(s.next(results)); + more = s.next(results); i++; LOG.info("iteration #" + i + ", results.size=" + results.size()); 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 3294f6de8b0..81f811528ad 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 @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -774,24 +775,15 @@ public class TestStripeCompactionPolicy { } @Override - public NextState next(List results) throws IOException { - if (kvs.isEmpty()) return NextState.makeState(NextState.State.NO_MORE_VALUES); + public boolean next(List results) throws IOException { + if (kvs.isEmpty()) return false; results.add(kvs.remove(0)); - if (!kvs.isEmpty()) { - return NextState.makeState(NextState.State.MORE_VALUES); - } else { - return NextState.makeState(NextState.State.NO_MORE_VALUES); - } + return !kvs.isEmpty(); } @Override - public NextState next(List result, int limit) throws IOException { - return next(result); - } - - @Override - public NextState next(List result, int limit, long remainingResultSize) + public boolean next(List result, ScannerContext scannerContext) throws IOException { return next(result); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index afdcdc75761..5310a2e6b47 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -71,7 +71,6 @@ import org.apache.hadoop.hbase.regionserver.FlushRequestListener; import org.apache.hadoop.hbase.regionserver.FlushRequester; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; @@ -751,7 +750,7 @@ public class TestWALReplay { int scannedCount = 0; List results = new ArrayList(); while (true) { - boolean existMore = NextState.hasMoreValues(scanner.next(results)); + boolean existMore = scanner.next(results); if (!results.isEmpty()) scannedCount++; if (!existMore) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java index c071a333fd1..056eeb2fee7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.wal.WAL; @@ -232,7 +231,7 @@ public class TestMergeTool extends HBaseTestCase { List testRes = null; while (true) { testRes = new ArrayList(); - boolean hasNext = NextState.hasMoreValues(scanner.next(testRes)); + boolean hasNext = scanner.next(testRes); if (!hasNext) { break; }