This is an automated email from the ASF dual-hosted git repository.
xucang pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.1 by this push:
new 0132407 HBASE-22274 Cell size limit check on append considers cell's
previous size
0132407 is described below
commit 0132407c3e9b5b0550e04cce38c4bee2817598d6
Author: Xu Cang <[email protected]>
AuthorDate: Thu May 9 23:24:23 2019 -0700
HBASE-22274 Cell size limit check on append considers cell's previous size
---
.../java/org/apache/hadoop/hbase/regionserver/HRegion.java | 10 +++++++++-
.../org/apache/hadoop/hbase/client/TestFromClientSide.java | 2 +-
2 files changed, 10 insertions(+), 2 deletions(-)
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 005a27a..abc4352 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
@@ -7989,7 +7989,15 @@ public class HRegion implements HeapSize,
PropagatingConfigurationObserver, Regi
break;
default: throw new UnsupportedOperationException(op.toString());
}
-
+ int newCellSize = PrivateCellUtil.estimatedSerializedSizeOf(newCell);
+ if (newCellSize > this.maxCellSize) {
+ String msg = "Cell with size " + newCellSize + " exceeds limit of " +
+ this.maxCellSize + " bytes";
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(msg);
+ }
+ throw new DoNotRetryIOException(msg);
+ }
// Give coprocessors a chance to update the new cell
if (coprocessorHost != null) {
newCell =
diff --git
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 4f29279..d5816b8 100644
---
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -6454,7 +6454,7 @@ public class TestFromClientSide {
// expected
}
try {
- t.append(new Append(ROW).addColumn(FAMILY, QUALIFIER, new byte[10 *
1024]));
+ t.append(new Append(ROW).addColumn(FAMILY, QUALIFIER, new byte[2 *
1024]));
fail("Oversize cell failed to trigger exception");
} catch (IOException e) {
// expected