Jaehui-Lee commented on code in PR #7469:
URL: https://github.com/apache/hbase/pull/7469#discussion_r2540756623
##########
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:
You're right. While it's not a common case, it's definitely possible. I've
updated the code to include size validation for increment as well.
https://github.com/apache/hbase/pull/7469/commits/59beea6cdbe8c797ffb54d3992493cf2299e8b80
--
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]