Skip to content

Commit e62b4f8

Browse files
committed
HBASE-29721 Add validation for Mutations without column families
1 parent 4ba9b46 commit e62b4f8

File tree

4 files changed

+123
-40
lines changed

4 files changed

+123
-40
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
*/
1818
package org.apache.hadoop.hbase.client;
1919

20-
import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;
20+
import static org.apache.hadoop.hbase.client.ConnectionUtils.validateMutation;
2121
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
2222

2323
import java.io.IOException;
@@ -124,9 +124,7 @@ Stream.<CompletableFuture<Void>> generate(CompletableFuture::new).limit(mutation
124124
long heapSize = 0;
125125
for (Mutation mutation : mutations) {
126126
heapSize += mutation.heapSize();
127-
if (mutation instanceof Put) {
128-
validatePut((Put) mutation, maxKeyValueSize);
129-
}
127+
validateMutation(mutation, maxKeyValueSize);
130128
}
131129
synchronized (this) {
132130
if (closed) {

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java

Lines changed: 16 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -480,13 +480,20 @@ static <T> CompletableFuture<T> timelineConsistentRead(AsyncRegionLocator locato
480480
return future;
481481
}
482482

483-
// validate for well-formedness
484-
static void validatePut(Put put, int maxKeyValueSize) {
485-
if (put.isEmpty()) {
486-
throw new IllegalArgumentException("No columns to insert");
483+
// Validate individual Mutation
484+
static void validateMutation(Mutation mutation, int maxKeyValueSize) {
485+
if (mutation instanceof Delete) return;
486+
487+
// 1. Check if empty (excluding Delete)
488+
if (mutation.isEmpty()) {
489+
throw new IllegalArgumentException("No columns to " + mutation.getClass().getSimpleName().toLowerCase());
487490
}
488-
if (maxKeyValueSize > 0) {
489-
for (List<Cell> list : put.getFamilyCellMap().values()) {
491+
492+
// 2. Validate size (excluding Increment and Delete)
493+
if (maxKeyValueSize > 0 &&
494+
(mutation instanceof Put) ||
495+
(mutation instanceof Append)) {
496+
for (List<Cell> list : mutation.getFamilyCellMap().values()) {
490497
for (Cell cell : list) {
491498
if (cell.getSerializedSize() > maxKeyValueSize) {
492499
throw new IllegalArgumentException("KeyValue size too large");
@@ -496,11 +503,10 @@ static void validatePut(Put put, int maxKeyValueSize) {
496503
}
497504
}
498505

499-
static void validatePutsInRowMutations(RowMutations rowMutations, int maxKeyValueSize) {
506+
// Validate RowMutations
507+
static void validateRowMutations(RowMutations rowMutations, int maxKeyValueSize) {
500508
for (Mutation mutation : rowMutations.getMutations()) {
501-
if (mutation instanceof Put) {
502-
validatePut((Put) mutation, maxKeyValueSize);
503-
}
509+
validateMutation(mutation, maxKeyValueSize);
504510
}
505511
}
506512

hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java

Lines changed: 18 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -21,8 +21,8 @@
2121
import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
2222
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
2323
import static org.apache.hadoop.hbase.client.ConnectionUtils.timelineConsistentRead;
24-
import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;
25-
import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePutsInRowMutations;
24+
import static org.apache.hadoop.hbase.client.ConnectionUtils.validateMutation;
25+
import static org.apache.hadoop.hbase.client.ConnectionUtils.validateRowMutations;
2626
import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;
2727
import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFutures;
2828
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
@@ -249,7 +249,7 @@ public CompletableFuture<Result> get(Get get) {
249249

250250
@Override
251251
public CompletableFuture<Void> put(Put put) {
252-
validatePut(put, conn.connConf.getMaxKeyValueSize());
252+
validateMutation(put, conn.connConf.getMaxKeyValueSize());
253253
final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(put);
254254
return tracedFuture(() -> this.<Void, Put> newCaller(put, writeRpcTimeoutNs)
255255
.action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,
@@ -268,7 +268,7 @@ public CompletableFuture<Void> delete(Delete delete) {
268268

269269
@Override
270270
public CompletableFuture<Result> append(Append append) {
271-
checkHasFamilies(append);
271+
validateMutation(append, conn.connConf.getMaxKeyValueSize());
272272
final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(append);
273273
return tracedFuture(() -> {
274274
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
@@ -283,7 +283,7 @@ public CompletableFuture<Result> append(Append append) {
283283

284284
@Override
285285
public CompletableFuture<Result> increment(Increment increment) {
286-
checkHasFamilies(increment);
286+
validateMutation(increment, conn.connConf.getMaxKeyValueSize());
287287
final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(increment);
288288
return tracedFuture(() -> {
289289
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
@@ -349,7 +349,7 @@ private void preCheck() {
349349

350350
@Override
351351
public CompletableFuture<Boolean> thenPut(Put put) {
352-
validatePut(put, conn.connConf.getMaxKeyValueSize());
352+
validateMutation(put, conn.connConf.getMaxKeyValueSize());
353353
preCheck();
354354
final Supplier<Span> supplier = newTableOperationSpanBuilder()
355355
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
@@ -383,7 +383,7 @@ public CompletableFuture<Boolean> thenDelete(Delete delete) {
383383
@Override
384384
public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {
385385
preCheck();
386-
validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
386+
validateRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
387387
final Supplier<Span> supplier = newTableOperationSpanBuilder()
388388
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
389389
.setContainerOperations(mutations);
@@ -425,7 +425,7 @@ public CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange) {
425425

426426
@Override
427427
public CompletableFuture<Boolean> thenPut(Put put) {
428-
validatePut(put, conn.connConf.getMaxKeyValueSize());
428+
validateMutation(put, conn.connConf.getMaxKeyValueSize());
429429
final Supplier<Span> supplier = newTableOperationSpanBuilder()
430430
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
431431
.setContainerOperations(put);
@@ -456,7 +456,7 @@ public CompletableFuture<Boolean> thenDelete(Delete delete) {
456456

457457
@Override
458458
public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {
459-
validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
459+
validateRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
460460
final Supplier<Span> supplier = newTableOperationSpanBuilder()
461461
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
462462
.setContainerOperations(mutations);
@@ -487,9 +487,7 @@ public CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate che
487487
|| checkAndMutate.getAction() instanceof Append
488488
) {
489489
Mutation mutation = (Mutation) checkAndMutate.getAction();
490-
if (mutation instanceof Put) {
491-
validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize());
492-
}
490+
validateMutation(mutation, conn.connConf.getMaxKeyValueSize());
493491
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
494492
long nonce = conn.getNonceGenerator().newNonce();
495493
return RawAsyncTableImpl.this
@@ -506,7 +504,7 @@ public CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate che
506504
.call();
507505
} else if (checkAndMutate.getAction() instanceof RowMutations) {
508506
RowMutations rowMutations = (RowMutations) checkAndMutate.getAction();
509-
validatePutsInRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize());
507+
validateRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize());
510508
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
511509
long nonce = conn.getNonceGenerator().newNonce();
512510
return RawAsyncTableImpl.this
@@ -585,7 +583,7 @@ public void run(MultiResponse resp) {
585583

586584
@Override
587585
public CompletableFuture<Result> mutateRow(RowMutations mutations) {
588-
validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
586+
validateRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
589587
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
590588
long nonce = conn.getNonceGenerator().newNonce();
591589
final Supplier<Span> supplier =
@@ -694,18 +692,18 @@ private List<CompletableFuture<Void>> voidMutate(List<? extends Row> actions) {
694692

695693
private <T> List<CompletableFuture<T>> batch(List<? extends Row> actions, long rpcTimeoutNs) {
696694
for (Row action : actions) {
697-
if (action instanceof Put) {
698-
validatePut((Put) action, conn.connConf.getMaxKeyValueSize());
695+
if (action instanceof Mutation) {
696+
validateMutation((Mutation) action, conn.connConf.getMaxKeyValueSize());
699697
} else if (action instanceof CheckAndMutate) {
700698
CheckAndMutate checkAndMutate = (CheckAndMutate) action;
701-
if (checkAndMutate.getAction() instanceof Put) {
702-
validatePut((Put) checkAndMutate.getAction(), conn.connConf.getMaxKeyValueSize());
699+
if (checkAndMutate.getAction() instanceof Mutation) {
700+
validateMutation((Mutation) checkAndMutate.getAction(), conn.connConf.getMaxKeyValueSize());
703701
} else if (checkAndMutate.getAction() instanceof RowMutations) {
704-
validatePutsInRowMutations((RowMutations) checkAndMutate.getAction(),
702+
validateRowMutations((RowMutations) checkAndMutate.getAction(),
705703
conn.connConf.getMaxKeyValueSize());
706704
}
707705
} else if (action instanceof RowMutations) {
708-
validatePutsInRowMutations((RowMutations) action, conn.connConf.getMaxKeyValueSize());
706+
validateRowMutations((RowMutations) action, conn.connConf.getMaxKeyValueSize());
709707
}
710708
}
711709
return conn.callerFactory.batch().table(tableName).actions(actions)

hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java

Lines changed: 87 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1641,12 +1641,13 @@ public void testDisabled() throws InterruptedException, ExecutionException {
16411641
}
16421642

16431643
@Test
1644-
public void testInvalidPut() {
1644+
public void testInvalidMutation() {
1645+
// put
16451646
try {
16461647
getTable.get().put(new Put(Bytes.toBytes(0)));
16471648
fail("Should fail since the put does not contain any cells");
16481649
} catch (IllegalArgumentException e) {
1649-
assertThat(e.getMessage(), containsString("No columns to insert"));
1650+
assertThat(e.getMessage(), containsString("No columns to put"));
16501651
}
16511652

16521653
try {
@@ -1656,16 +1657,41 @@ public void testInvalidPut() {
16561657
} catch (IllegalArgumentException e) {
16571658
assertThat(e.getMessage(), containsString("KeyValue size too large"));
16581659
}
1660+
1661+
// increment
1662+
try {
1663+
getTable.get().increment(new Increment(Bytes.toBytes(0)));
1664+
fail("Should fail since the increment does not contain any columns");
1665+
} catch (IllegalArgumentException e) {
1666+
assertThat(e.getMessage(), containsString("No columns to increment"));
1667+
}
1668+
1669+
// append
1670+
try {
1671+
getTable.get().append(new Append(Bytes.toBytes(0)));
1672+
fail("Should fail since the append does not contain any columns");
1673+
} catch (IllegalArgumentException e) {
1674+
assertThat(e.getMessage(), containsString("No columns to append"));
1675+
}
1676+
1677+
try {
1678+
getTable.get()
1679+
.append(new Append(Bytes.toBytes(0)).addColumn(FAMILY, QUALIFIER, new byte[MAX_KEY_VALUE_SIZE]));
1680+
fail("Should fail since the append exceeds the max key value size");
1681+
} catch (IllegalArgumentException e) {
1682+
assertThat(e.getMessage(), containsString("KeyValue size too large"));
1683+
}
16591684
}
16601685

16611686
@Test
1662-
public void testInvalidPutInRowMutations() throws IOException {
1687+
public void testInvalidMutationInRowMutations() throws IOException {
16631688
final byte[] row = Bytes.toBytes(0);
1689+
// put
16641690
try {
16651691
getTable.get().mutateRow(new RowMutations(row).add(new Put(row)));
16661692
fail("Should fail since the put does not contain any cells");
16671693
} catch (IllegalArgumentException e) {
1668-
assertThat(e.getMessage(), containsString("No columns to insert"));
1694+
assertThat(e.getMessage(), containsString("No columns to put"));
16691695
}
16701696

16711697
try {
@@ -1675,17 +1701,42 @@ public void testInvalidPutInRowMutations() throws IOException {
16751701
} catch (IllegalArgumentException e) {
16761702
assertThat(e.getMessage(), containsString("KeyValue size too large"));
16771703
}
1704+
1705+
// increment
1706+
try {
1707+
getTable.get().mutateRow(new RowMutations(row).add(new Increment(row)));
1708+
fail("Should fail since the increment does not contain any columns");
1709+
} catch (IllegalArgumentException e) {
1710+
assertThat(e.getMessage(), containsString("No columns to increment"));
1711+
}
1712+
1713+
// append
1714+
try {
1715+
getTable.get().mutateRow(new RowMutations(row).add(new Append(row)));
1716+
fail("Should fail since the append does not contain any columns");
1717+
} catch (IllegalArgumentException e) {
1718+
assertThat(e.getMessage(), containsString("No columns to append"));
1719+
}
1720+
1721+
try {
1722+
getTable.get().mutateRow(new RowMutations(row)
1723+
.add(new Append(row).addColumn(FAMILY, QUALIFIER, new byte[MAX_KEY_VALUE_SIZE])));
1724+
fail("Should fail since the append exceeds the max key value size");
1725+
} catch (IllegalArgumentException e) {
1726+
assertThat(e.getMessage(), containsString("KeyValue size too large"));
1727+
}
16781728
}
16791729

16801730
@Test
1681-
public void testInvalidPutInRowMutationsInCheckAndMutate() throws IOException {
1731+
public void testInvalidMutationInRowMutationsInCheckAndMutate() throws IOException {
16821732
final byte[] row = Bytes.toBytes(0);
1733+
// put
16831734
try {
16841735
getTable.get().checkAndMutate(CheckAndMutate.newBuilder(row).ifNotExists(FAMILY, QUALIFIER)
16851736
.build(new RowMutations(row).add(new Put(row))));
16861737
fail("Should fail since the put does not contain any cells");
16871738
} catch (IllegalArgumentException e) {
1688-
assertThat(e.getMessage(), containsString("No columns to insert"));
1739+
assertThat(e.getMessage(), containsString("No columns to put"));
16891740
}
16901741

16911742
try {
@@ -1696,5 +1747,35 @@ public void testInvalidPutInRowMutationsInCheckAndMutate() throws IOException {
16961747
} catch (IllegalArgumentException e) {
16971748
assertThat(e.getMessage(), containsString("KeyValue size too large"));
16981749
}
1750+
1751+
// increment
1752+
try {
1753+
getTable.get().checkAndMutate(
1754+
CheckAndMutate.newBuilder(row).ifNotExists(FAMILY, QUALIFIER)
1755+
.build(new RowMutations(row).add(new Increment(row))));
1756+
fail("Should fail since the increment does not contain any columns");
1757+
} catch (IllegalArgumentException e) {
1758+
assertThat(e.getMessage(), containsString("No columns to increment"));
1759+
}
1760+
1761+
// append
1762+
try {
1763+
getTable.get().checkAndMutate(
1764+
CheckAndMutate.newBuilder(row).ifNotExists(FAMILY, QUALIFIER)
1765+
.build(new RowMutations(row).add(new Append(row))));
1766+
fail("Should fail since the append does not contain any columns");
1767+
} catch (IllegalArgumentException e) {
1768+
assertThat(e.getMessage(), containsString("No columns to append"));
1769+
}
1770+
1771+
try {
1772+
getTable.get().checkAndMutate(
1773+
CheckAndMutate.newBuilder(row).ifNotExists(FAMILY, QUALIFIER)
1774+
.build(new RowMutations(row)
1775+
.add(new Append(row).addColumn(FAMILY, QUALIFIER, new byte[MAX_KEY_VALUE_SIZE]))));
1776+
fail("Should fail since the append exceeds the max key value size");
1777+
} catch (IllegalArgumentException e) {
1778+
assertThat(e.getMessage(), containsString("KeyValue size too large"));
1779+
}
16991780
}
17001781
}

0 commit comments

Comments
 (0)