HBASE-17599 Use mayHaveMoreCellsInRow instead of isPartial
This commit is contained in:
parent
b23890157c
commit
712fe69e4d
|
@ -325,7 +325,7 @@ public final class ConnectionUtils {
|
|||
return null;
|
||||
}
|
||||
return Result.create(Arrays.copyOfRange(rawCells, index, rawCells.length), null,
|
||||
result.isStale(), true);
|
||||
result.isStale(), result.mayHaveMoreCellsInRow());
|
||||
}
|
||||
|
||||
// Add a delta to avoid timeout immediately after a retry sleeping.
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -85,15 +84,10 @@ public class Result implements CellScannable, CellScanner {
|
|||
private boolean stale = false;
|
||||
|
||||
/**
|
||||
* Partial results do not contain the full row's worth of cells. The result had to be returned in
|
||||
* parts because the size of the cells in the row exceeded the RPC result size on the server.
|
||||
* Partial results must be combined client side with results representing the remainder of the
|
||||
* row's cells to form the complete result. Partial results and RPC result size allow us to avoid
|
||||
* OOME on the server when servicing requests for large rows. The Scan configuration used to
|
||||
* control the result size on the server is {@link Scan#setMaxResultSize(long)} and the default
|
||||
* value can be seen here: {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE}
|
||||
* See {@link #mayHaveMoreCellsInRow()}. And please notice that, The client side implementation
|
||||
* should also check for row key change to determine if a Result is the last one for a row.
|
||||
*/
|
||||
private boolean partial = false;
|
||||
private boolean mayHaveMoreCellsInRow = false;
|
||||
// We're not using java serialization. Transient here is just a marker to say
|
||||
// that this is where we cache row if we're ever asked for it.
|
||||
private transient byte [] row = null;
|
||||
|
@ -171,19 +165,20 @@ public class Result implements CellScannable, CellScanner {
|
|||
return create(cells, exists, stale, false);
|
||||
}
|
||||
|
||||
public static Result create(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
|
||||
if (exists != null){
|
||||
return new Result(null, exists, stale, partial);
|
||||
public static Result create(Cell[] cells, Boolean exists, boolean stale,
|
||||
boolean mayHaveMoreCellsInRow) {
|
||||
if (exists != null) {
|
||||
return new Result(null, exists, stale, mayHaveMoreCellsInRow);
|
||||
}
|
||||
return new Result(cells, null, stale, partial);
|
||||
return new Result(cells, null, stale, mayHaveMoreCellsInRow);
|
||||
}
|
||||
|
||||
/** Private ctor. Use {@link #create(Cell[])}. */
|
||||
private Result(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
|
||||
private Result(Cell[] cells, Boolean exists, boolean stale, boolean mayHaveMoreCellsInRow) {
|
||||
this.cells = cells;
|
||||
this.exists = exists;
|
||||
this.stale = stale;
|
||||
this.partial = partial;
|
||||
this.mayHaveMoreCellsInRow = mayHaveMoreCellsInRow;
|
||||
this.readonly = false;
|
||||
}
|
||||
|
||||
|
@ -828,7 +823,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
// Result1: -1- -2- (2 cells, size limit reached, mark as partial)
|
||||
// Result2: -3- -4- (2 cells, size limit reached, mark as partial)
|
||||
// Result3: -5- (1 cell, size limit NOT reached, NOT marked as partial)
|
||||
if (i != (partialResults.size() - 1) && !r.isPartial()) {
|
||||
if (i != (partialResults.size() - 1) && !r.mayHaveMoreCellsInRow()) {
|
||||
throw new IOException(
|
||||
"Cannot form complete result. Result is missing partial flag. " +
|
||||
"Partial Results: " + partialResults);
|
||||
|
@ -915,9 +910,28 @@ public class Result implements CellScannable, CellScanner {
|
|||
* for a row and should be combined with a result representing the remaining cells in that row to
|
||||
* form a complete (non-partial) result.
|
||||
* @return Whether or not the result is a partial result
|
||||
* @deprecated the word 'partial' ambiguous, use {@link #mayHaveMoreCellsInRow()} instead.
|
||||
* Deprecated since 1.4.0.
|
||||
* @see #mayHaveMoreCellsInRow()
|
||||
*/
|
||||
@Deprecated
|
||||
public boolean isPartial() {
|
||||
return partial;
|
||||
return mayHaveMoreCellsInRow;
|
||||
}
|
||||
|
||||
/**
|
||||
* For scanning large rows, the RS may choose to return the cells chunk by chunk to prevent OOM.
|
||||
* This flag is used to tell you if the current Result is the last one of the current row. False
|
||||
* means this Result is the last one. True means there may still be more cells for the current
|
||||
* row. Notice that, 'may' have, not must have. This is because we may reach the size or time
|
||||
* limit just at the last cell of row at RS, so we do not know if it is the last one.
|
||||
* <p>
|
||||
* The Scan configuration used to control the result size on the server is
|
||||
* {@link Scan#setMaxResultSize(long)} and the default value can be seen here:
|
||||
* {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE}
|
||||
*/
|
||||
public boolean mayHaveMoreCellsInRow() {
|
||||
return mayHaveMoreCellsInRow;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1329,7 +1329,7 @@ public final class ProtobufUtil {
|
|||
}
|
||||
|
||||
builder.setStale(result.isStale());
|
||||
builder.setPartial(result.isPartial());
|
||||
builder.setPartial(result.mayHaveMoreCellsInRow());
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
|
|
@ -1443,7 +1443,7 @@ public final class ProtobufUtil {
|
|||
}
|
||||
|
||||
builder.setStale(result.isStale());
|
||||
builder.setPartial(result.isPartial());
|
||||
builder.setPartial(result.mayHaveMoreCellsInRow());
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
|
|
@ -4732,7 +4732,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
@ -4743,7 +4744,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
@ -5002,7 +5004,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
@ -5015,7 +5018,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
@ -5946,7 +5950,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
@ -5959,7 +5964,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
@ -5972,7 +5978,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
@ -5988,7 +5995,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*
|
||||
* <code>optional bool partial = 5 [default = false];</code>
|
||||
|
|
|
@ -108,7 +108,8 @@ message Result {
|
|||
// Whether or not the entire result could be returned. Results will be split when
|
||||
// the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
// cells for a row and must be combined with a result containing the remaining cells
|
||||
// to form a complete result
|
||||
// to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
// mayHaveMoreCellsInRow.
|
||||
optional bool partial = 5 [default = false];
|
||||
}
|
||||
|
||||
|
|
|
@ -4626,7 +4626,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
boolean hasPartial();
|
||||
|
@ -4637,7 +4638,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
boolean getPartial();
|
||||
|
@ -4919,7 +4921,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasPartial() {
|
||||
|
@ -4932,7 +4935,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
public boolean getPartial() {
|
||||
|
@ -5848,7 +5852,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasPartial() {
|
||||
|
@ -5861,7 +5866,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
public boolean getPartial() {
|
||||
|
@ -5874,7 +5880,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
public Builder setPartial(boolean value) {
|
||||
|
@ -5890,7 +5897,8 @@ public final class ClientProtos {
|
|||
* Whether or not the entire result could be returned. Results will be split when
|
||||
* the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
* cells for a row and must be combined with a result containing the remaining cells
|
||||
* to form a complete result
|
||||
* to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
* mayHaveMoreCellsInRow.
|
||||
* </pre>
|
||||
*/
|
||||
public Builder clearPartial() {
|
||||
|
|
|
@ -108,7 +108,8 @@ message Result {
|
|||
// Whether or not the entire result could be returned. Results will be split when
|
||||
// the RPC chunk size limit is reached. Partial results contain only a subset of the
|
||||
// cells for a row and must be combined with a result containing the remaining cells
|
||||
// to form a complete result
|
||||
// to form a complete result. The equivalent flag in o.a.h.h.client.Result is
|
||||
// mayHaveMoreCellsInRow.
|
||||
optional bool partial = 5 [default = false];
|
||||
}
|
||||
|
||||
|
|
|
@ -72,7 +72,6 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import java.util.concurrent.atomic.LongAdder;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -5945,13 +5944,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
outResults.addAll(tmpList);
|
||||
}
|
||||
|
||||
// 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
|
||||
// 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 (!scannerContext.midRowResultFormed())
|
||||
if (!scannerContext.mayHaveMoreCellsInRow()) {
|
||||
resetFilters();
|
||||
}
|
||||
|
||||
if (isFilterDoneInternal()) {
|
||||
moreValues = false;
|
||||
|
|
|
@ -482,11 +482,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
private void addResults(ScanResponse.Builder builder, List<Result> results,
|
||||
HBaseRpcController controller, boolean isDefaultRegion, boolean clientCellBlockSupported) {
|
||||
builder.setStale(!isDefaultRegion);
|
||||
if (results.isEmpty()) return;
|
||||
if (results.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
if (clientCellBlockSupported) {
|
||||
for (Result res : results) {
|
||||
builder.addCellsPerResult(res.size());
|
||||
builder.addPartialFlagPerResult(res.isPartial());
|
||||
builder.addPartialFlagPerResult(res.mayHaveMoreCellsInRow());
|
||||
}
|
||||
controller.setCellScanner(CellUtil.createCellScanner(results));
|
||||
} else {
|
||||
|
@ -2851,8 +2853,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
moreRows = scanner.nextRaw(values, scannerContext);
|
||||
|
||||
if (!values.isEmpty()) {
|
||||
final boolean partial = scannerContext.partialResultFormed();
|
||||
Result r = Result.create(values, null, stale, partial);
|
||||
Result r = Result.create(values, null, stale, scannerContext.mayHaveMoreCellsInRow());
|
||||
lastBlock.setValue(addSize(context, r, lastBlock.getValue()));
|
||||
results.add(r);
|
||||
i++;
|
||||
|
@ -3071,7 +3072,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
// is false. Can remove the isEmpty check after we get rid of the old implementation.
|
||||
moreResults = false;
|
||||
} else if (limitOfRows > 0 && results.size() >= limitOfRows
|
||||
&& !results.get(results.size() - 1).isPartial()) {
|
||||
&& !results.get(results.size() - 1).mayHaveMoreCellsInRow()) {
|
||||
// if we have reached the limit of rows
|
||||
moreResults = false;
|
||||
}
|
||||
|
|
|
@ -19,8 +19,6 @@ 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;
|
||||
|
@ -51,7 +49,6 @@ import org.apache.hadoop.hbase.client.metrics.ServerSideScanMetrics;
|
|||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public class ScannerContext {
|
||||
private static final Log LOG = LogFactory.getLog(ScannerContext.class);
|
||||
|
||||
/**
|
||||
* Two sets of the same fields. One for the limits, another for the progress towards those limits
|
||||
|
@ -221,21 +218,13 @@ public class ScannerContext {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return true when a partial result is formed. A partial result is formed when a limit is
|
||||
* reached in the middle of a row.
|
||||
* @return true when we may have more cells for the current row. This usually because we have
|
||||
* reached a limit in the middle of a row
|
||||
*/
|
||||
boolean partialResultFormed() {
|
||||
return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW
|
||||
|| scannerState == NextState.TIME_LIMIT_REACHED_MID_ROW;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when a mid-row result is formed.
|
||||
*/
|
||||
boolean midRowResultFormed() {
|
||||
return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW
|
||||
|| scannerState == NextState.TIME_LIMIT_REACHED_MID_ROW
|
||||
|| scannerState == NextState.BATCH_LIMIT_REACHED;
|
||||
boolean mayHaveMoreCellsInRow() {
|
||||
return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW ||
|
||||
scannerState == NextState.TIME_LIMIT_REACHED_MID_ROW ||
|
||||
scannerState == NextState.BATCH_LIMIT_REACHED;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue