junegunn commented on code in PR #7469:
URL: https://github.com/apache/hbase/pull/7469#discussion_r2540205958


##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java:
##########
@@ -480,13 +480,19 @@ static <T> CompletableFuture<T> 
timelineConsistentRead(AsyncRegionLocator locato
     return future;
   }
 
-  // validate for well-formedness
-  static void validatePut(Put put, int maxKeyValueSize) {
-    if (put.isEmpty()) {
-      throw new IllegalArgumentException("No columns to insert");
+  // Validate individual Mutation
+  static void validateMutation(Mutation mutation, int maxKeyValueSize) {
+    if (mutation instanceof Delete) return;
+
+    // 1. Check if empty (excluding Delete)
+    if (mutation.isEmpty()) {
+      throw new IllegalArgumentException(
+        "No columns to " + mutation.getClass().getSimpleName().toLowerCase());
     }
-    if (maxKeyValueSize > 0) {
-      for (List<Cell> list : put.getFamilyCellMap().values()) {
+
+    // 2. Validate size (excluding Increment and Delete)
+    if (maxKeyValueSize > 0 && (mutation instanceof Put) || (mutation 
instanceof Append)) {

Review Comment:
   Maybe we shouldn't skip for increments? It is indeed possible to build large 
increments.
   
   ```sh
   create 't', 'd'
   
   put 't', '1', "d:#{'x' * 1024 * 1024 * 10}", Bytes.toBytes(0)
     # ERROR: KeyValue size too large
   
   incr 't', '1', "d:#{'x' * 1024 * 1024 * 10}", 1
     # ... size 10485794 exceeds limit of 10485760 bytes
     # at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.checkCellSizeLimit(RSRpcServices.java:917)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to