Repository: hbase Updated Branches: refs/heads/branch-1 f61b840a3 -> e9abe0762
HBASE-17599 Use mayHaveMoreCellsInRow instead of isPartial Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e9abe076 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e9abe076 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e9abe076 Branch: refs/heads/branch-1 Commit: e9abe07629bd698b8c727903a0826cde2bf6d1d5 Parents: f61b840 Author: zhangduo <zhang...@apache.org> Authored: Thu Feb 9 10:51:16 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Feb 9 15:38:23 2017 +0800 ---------------------------------------------------------------------- .../org/apache/hadoop/hbase/client/Result.java | 47 +++++++++++++------- .../hadoop/hbase/protobuf/ProtobufUtil.java | 2 +- .../hbase/protobuf/generated/ClientProtos.java | 24 ++++++---- hbase-protocol/src/main/protobuf/Client.proto | 3 +- .../hadoop/hbase/regionserver/HRegion.java | 34 +++++++------- .../hbase/regionserver/RSRpcServices.java | 11 ++--- .../hbase/regionserver/ScannerContext.java | 23 +++------- 7 files changed, 81 insertions(+), 63 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/e9abe076/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java index 3074d3e..d27518e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java @@ -83,15 +83,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; @@ -185,19 +180,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; } @@ -890,7 +886,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); @@ -977,9 +973,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; } /** http://git-wip-us.apache.org/repos/asf/hbase/blob/e9abe076/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 896185f..b198d53 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -1423,7 +1423,7 @@ public final class ProtobufUtil { } builder.setStale(result.isStale()); - builder.setPartial(result.isPartial()); + builder.setPartial(result.mayHaveMoreCellsInRow()); return builder.build(); } http://git-wip-us.apache.org/repos/asf/hbase/blob/e9abe076/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java ---------------------------------------------------------------------- diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index cefdd38..9133c19 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -4636,7 +4636,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(); @@ -4647,7 +4648,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(); @@ -4929,7 +4931,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() { @@ -4942,7 +4945,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() { @@ -5858,7 +5862,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() { @@ -5871,7 +5876,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() { @@ -5884,7 +5890,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) { @@ -5900,7 +5907,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() { http://git-wip-us.apache.org/repos/asf/hbase/blob/e9abe076/hbase-protocol/src/main/protobuf/Client.proto ---------------------------------------------------------------------- diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index 4fb64bd..d55b97a 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -112,7 +112,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]; } http://git-wip-us.apache.org/repos/asf/hbase/blob/e9abe076/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ---------------------------------------------------------------------- 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 a32fe65..1f648f7 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 @@ -18,6 +18,20 @@ */ package org.apache.hadoop.hbase.regionserver; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.io.Closeables; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import com.google.protobuf.TextFormat; + import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; @@ -65,6 +79,8 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.jws.soap.SOAPBinding.Use; + import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -189,20 +205,6 @@ import org.apache.hadoop.util.StringUtils; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.io.Closeables; -import com.google.protobuf.ByteString; -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; -import com.google.protobuf.TextFormat; - @InterfaceAudience.Private public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region { private static final Log LOG = LogFactory.getLog(HRegion.class); @@ -6054,7 +6056,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // 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()) resetFilters(); + if (!scannerContext.mayHaveMoreCellsInRow()) { + resetFilters(); + } if (isFilterDoneInternal()) { moreValues = false; http://git-wip-us.apache.org/repos/asf/hbase/blob/e9abe076/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java ---------------------------------------------------------------------- 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 f13d8b7..e14356a 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 @@ -408,11 +408,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, private void addResults(final ScanResponse.Builder builder, final List<Result> results, final PayloadCarryingRpcController controller, boolean isDefaultRegion) { builder.setStale(!isDefaultRegion); - if (results == null || results.isEmpty()) return; + if (results.isEmpty()) { + return; + } if (isClientCellBlockSupport()) { for (Result res : results) { builder.addCellsPerResult(res.size()); - builder.addPartialFlagPerResult(res.isPartial()); + builder.addPartialFlagPerResult(res.mayHaveMoreCellsInRow()); } controller.setCellScanner(CellUtil.createCellScanner(results)); } else { @@ -2636,8 +2638,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++; @@ -2856,7 +2857,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; } http://git-wip-us.apache.org/repos/asf/hbase/blob/e9abe076/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java index bffcdf6..4e61f1b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java @@ -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; } /**