This is an automated email from the ASF dual-hosted git repository.

xucang pushed a commit to branch branch-1.4
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1.4 by this push:
     new 14e8bd5  HBASE-22274 Cell size limit check on append considers cell's 
previous size
14e8bd5 is described below

commit 14e8bd538423a2f6cb1171da98aa62dea1462228
Author: Xu Cang <[email protected]>
AuthorDate: Thu May 9 17:01:58 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 b5d1c43..fc8d3aa 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
@@ -7839,7 +7839,15 @@ public class HRegion implements HeapSize, 
PropagatingConfigurationObserver, Regi
                 tags = carryForwardTTLTag(tags, mutate);
 
                 newCell = getNewCell(row, ts, cell, oldCell, 
Tag.fromList(tags));
-
+                int newCellSize = CellUtil.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);
+                }
                 idx++;
               } else {
                 // Append's KeyValue.Type==Put and 
ts==HConstants.LATEST_TIMESTAMP
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 b63b27b..254d8f2 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
@@ -6539,7 +6539,7 @@ public class TestFromClientSide {
         // expected
       }
       try {
-        t.append(new Append(ROW).add(FAMILY, QUALIFIER, new byte[10 * 1024]));
+        t.append(new Append(ROW).add(FAMILY, QUALIFIER, new byte[2 * 1024]));
         fail("Oversize cell failed to trigger exception");
       } catch (IOException e) {
         // expected

Reply via email to