This is an automated email from the ASF dual-hosted git repository.
junegunn pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2 by this push:
new 5496539da0d HBASE-29721 Add validation for Mutations without column
families (#7768)
5496539da0d is described below
commit 5496539da0d6625878bf5afb5c66bcb429670c9d
Author: Jaehui Lee <[email protected]>
AuthorDate: Mon Feb 23 14:04:41 2026 +0900
HBASE-29721 Add validation for Mutations without column families (#7768)
Signed-off-by: Junegunn Choi <[email protected]>
---
.../hbase/client/AsyncBufferedMutatorImpl.java | 6 +-
.../hadoop/hbase/client/BufferedMutatorImpl.java | 4 +-
.../hadoop/hbase/client/ConnectionUtils.java | 24 ++--
.../org/apache/hadoop/hbase/client/HTable.java | 2 +-
.../hadoop/hbase/client/HTableMultiplexer.java | 2 +-
.../hadoop/hbase/client/RawAsyncTableImpl.java | 40 +++---
.../apache/hadoop/hbase/client/TestAsyncTable.java | 136 ++++++++++++++-------
.../hadoop/hbase/client/TestAsyncTableBatch.java | 134 ++++++++++++--------
8 files changed, 218 insertions(+), 130 deletions(-)
diff --git
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
index 452318c1ef5..6d41d28c852 100644
---
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
+++
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.client;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.validateMutation;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import java.io.IOException;
@@ -175,9 +175,7 @@ class AsyncBufferedMutatorImpl implements
AsyncBufferedMutator {
long heapSize = 0;
for (Mutation mutation : mutations) {
heapSize += mutation.heapSize();
- if (mutation instanceof Put) {
- validatePut((Put) mutation, maxKeyValueSize);
- }
+ validateMutation(mutation, maxKeyValueSize);
}
Batch batch = null;
lock.lock();
diff --git
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index 29fb2c43bc0..8737172deee 100644
---
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -193,9 +193,7 @@ public class BufferedMutatorImpl implements BufferedMutator
{
long toAddSize = 0;
int toAddCount = 0;
for (Mutation m : ms) {
- if (m instanceof Put) {
- ConnectionUtils.validatePut((Put) m, maxKeyValueSize);
- }
+ ConnectionUtils.validateMutation(m, maxKeyValueSize);
toAddSize += m.heapSize();
++toAddCount;
}
diff --git
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 8b8a1cc4034..acc183834da 100644
---
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -506,13 +506,20 @@ public final class ConnectionUtils {
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) {
+ // Skip Delete
+ if (mutation instanceof Delete) return;
+
+ // 1. Check if empty
+ if (mutation.isEmpty()) {
+ throw new IllegalArgumentException(
+ "No columns to " + mutation.getClass().getSimpleName().toLowerCase());
}
+
+ // 2. Check if size exceeds maxKeyValueSize
if (maxKeyValueSize > 0) {
- for (List<Cell> list : put.getFamilyCellMap().values()) {
+ for (List<Cell> list : mutation.getFamilyCellMap().values()) {
for (Cell cell : list) {
if (cell.getSerializedSize() > maxKeyValueSize) {
throw new IllegalArgumentException("KeyValue size too large");
@@ -522,11 +529,10 @@ public final class ConnectionUtils {
}
}
- static void validatePutsInRowMutations(RowMutations rowMutations, int
maxKeyValueSize) {
+ // Validate RowMutations
+ static void validateRowMutations(RowMutations rowMutations, int
maxKeyValueSize) {
for (Mutation mutation : rowMutations.getMutations()) {
- if (mutation instanceof Put) {
- validatePut((Put) mutation, maxKeyValueSize);
- }
+ validateMutation(mutation, maxKeyValueSize);
}
}
diff --git
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index bb916a32c31..31b38cf55dc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -1105,7 +1105,7 @@ public class HTable implements Table {
// validate for well-formedness
private void validatePut(final Put put) throws IllegalArgumentException {
- ConnectionUtils.validatePut(put, connConfiguration.getMaxKeyValueSize());
+ ConnectionUtils.validateMutation(put,
connConfiguration.getMaxKeyValueSize());
}
/**
diff --git
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index e81be5bcb25..562bc960f74 100644
---
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -180,7 +180,7 @@ public class HTableMultiplexer {
}
try {
- ConnectionUtils.validatePut(put, maxKeyValueSize);
+ ConnectionUtils.validateMutation(put, maxKeyValueSize);
// Allow mocking to get at the connection, but don't expose the
connection to users.
ClusterConnection conn = (ClusterConnection) getConnection();
// AsyncProcess in the FlushWorker should take care of refreshing the
location cache
diff --git
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index 05dd5ea6394..05cc3e5ed00 100644
---
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -18,11 +18,10 @@
package org.apache.hadoop.hbase.client;
import static java.util.stream.Collectors.toList;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
import static
org.apache.hadoop.hbase.client.ConnectionUtils.timelineConsistentRead;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;
-import static
org.apache.hadoop.hbase.client.ConnectionUtils.validatePutsInRowMutations;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.validateMutation;
+import static
org.apache.hadoop.hbase.client.ConnectionUtils.validateRowMutations;
import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;
import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFutures;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
@@ -287,7 +286,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Void> put(Put put) {
- validatePut(put, conn.connConf.getMaxKeyValueSize());
+ validateMutation(put, conn.connConf.getMaxKeyValueSize());
final Supplier<Span> supplier =
newTableOperationSpanBuilder().setOperation(put);
return tracedFuture(() -> this.<Void, Put> newCaller(put,
writeRpcTimeoutNs)
.action((controller, loc, stub) -> RawAsyncTableImpl.<Put>
voidMutate(controller, loc, stub,
@@ -306,7 +305,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Result> append(Append append) {
- checkHasFamilies(append);
+ validateMutation(append, conn.connConf.getMaxKeyValueSize());
final Supplier<Span> supplier =
newTableOperationSpanBuilder().setOperation(append);
return tracedFuture(() -> {
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
@@ -321,7 +320,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Result> increment(Increment increment) {
- checkHasFamilies(increment);
+ validateMutation(increment, conn.connConf.getMaxKeyValueSize());
final Supplier<Span> supplier =
newTableOperationSpanBuilder().setOperation(increment);
return tracedFuture(() -> {
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
@@ -387,7 +386,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Boolean> thenPut(Put put) {
- validatePut(put, conn.connConf.getMaxKeyValueSize());
+ validateMutation(put, conn.connConf.getMaxKeyValueSize());
preCheck();
final Supplier<Span> supplier = newTableOperationSpanBuilder()
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
@@ -421,7 +420,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {
preCheck();
- validatePutsInRowMutations(mutations,
conn.connConf.getMaxKeyValueSize());
+ validateRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
final Supplier<Span> supplier = newTableOperationSpanBuilder()
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
.setContainerOperations(mutations);
@@ -463,7 +462,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Boolean> thenPut(Put put) {
- validatePut(put, conn.connConf.getMaxKeyValueSize());
+ validateMutation(put, conn.connConf.getMaxKeyValueSize());
final Supplier<Span> supplier = newTableOperationSpanBuilder()
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
.setContainerOperations(put);
@@ -494,7 +493,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {
- validatePutsInRowMutations(mutations,
conn.connConf.getMaxKeyValueSize());
+ validateRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
final Supplier<Span> supplier = newTableOperationSpanBuilder()
.setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)
.setContainerOperations(mutations);
@@ -525,9 +524,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
|| checkAndMutate.getAction() instanceof Append
) {
Mutation mutation = (Mutation) checkAndMutate.getAction();
- if (mutation instanceof Put) {
- validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize());
- }
+ validateMutation(mutation, conn.connConf.getMaxKeyValueSize());
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
long nonce = conn.getNonceGenerator().newNonce();
return RawAsyncTableImpl.this
@@ -544,7 +541,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
.call();
} else if (checkAndMutate.getAction() instanceof RowMutations) {
RowMutations rowMutations = (RowMutations) checkAndMutate.getAction();
- validatePutsInRowMutations(rowMutations,
conn.connConf.getMaxKeyValueSize());
+ validateRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize());
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
long nonce = conn.getNonceGenerator().newNonce();
return RawAsyncTableImpl.this
@@ -623,7 +620,7 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
@Override
public CompletableFuture<Result> mutateRow(RowMutations mutations) {
- validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
+ validateRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
long nonce = conn.getNonceGenerator().newNonce();
final Supplier<Span> supplier =
@@ -732,18 +729,19 @@ class RawAsyncTableImpl implements
AsyncTable<AdvancedScanResultConsumer> {
private <T> List<CompletableFuture<T>> batch(List<? extends Row> actions,
long rpcTimeoutNs) {
for (Row action : actions) {
- if (action instanceof Put) {
- validatePut((Put) action, conn.connConf.getMaxKeyValueSize());
+ if (action instanceof Mutation) {
+ validateMutation((Mutation) action,
conn.connConf.getMaxKeyValueSize());
} else if (action instanceof CheckAndMutate) {
CheckAndMutate checkAndMutate = (CheckAndMutate) action;
- if (checkAndMutate.getAction() instanceof Put) {
- validatePut((Put) checkAndMutate.getAction(),
conn.connConf.getMaxKeyValueSize());
+ if (checkAndMutate.getAction() instanceof Mutation) {
+ validateMutation((Mutation) checkAndMutate.getAction(),
+ conn.connConf.getMaxKeyValueSize());
} else if (checkAndMutate.getAction() instanceof RowMutations) {
- validatePutsInRowMutations((RowMutations) checkAndMutate.getAction(),
+ validateRowMutations((RowMutations) checkAndMutate.getAction(),
conn.connConf.getMaxKeyValueSize());
}
} else if (action instanceof RowMutations) {
- validatePutsInRowMutations((RowMutations) action,
conn.connConf.getMaxKeyValueSize());
+ validateRowMutations((RowMutations) action,
conn.connConf.getMaxKeyValueSize());
}
}
return conn.callerFactory.batch().table(tableName).actions(actions)
diff --git
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java
index d150d3c43b3..bd47a67a2d4 100644
---
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java
+++
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java
@@ -39,6 +39,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.ForkJoinPool;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
import java.util.function.Supplier;
import java.util.stream.IntStream;
import org.apache.hadoop.hbase.CompareOperator;
@@ -1641,61 +1642,112 @@ public class TestAsyncTable {
}
@Test
- public void testInvalidPut() {
- try {
- getTable.get().put(new Put(Bytes.toBytes(0)));
- fail("Should fail since the put does not contain any cells");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("No columns to insert"));
- }
+ public void testInvalidMutation() throws Exception {
+ Consumer<Mutation> executeMutation = mutation -> {
+ if (mutation instanceof Put) {
+ getTable.get().put((Put) mutation);
+ } else if (mutation instanceof Increment) {
+ getTable.get().increment((Increment) mutation);
+ } else if (mutation instanceof Append) {
+ getTable.get().append((Append) mutation);
+ }
+ };
- try {
- getTable.get()
- .put(new Put(Bytes.toBytes(0)).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]));
- fail("Should fail since the put exceeds the max key value size");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ Mutation[] emptyMutations =
+ { new Put(Bytes.toBytes(0)), new Increment(Bytes.toBytes(0)), new
Append(Bytes.toBytes(0)) };
+
+ String[] emptyMessages =
+ { "No columns to put", "No columns to increment", "No columns to append"
};
+
+ Mutation[] oversizedMutations =
+ { new Put(Bytes.toBytes(0)).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]),
+ new Increment(Bytes.toBytes(0)).addColumn(FAMILY, new
byte[MAX_KEY_VALUE_SIZE], 1),
+ new Append(Bytes.toBytes(0)).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]) };
+
+ for (int i = 0; i < emptyMutations.length; i++) {
+ // Test empty mutation
+ try {
+ executeMutation.accept(emptyMutations[i]);
+ fail("Should fail since the mutation does not contain any cells");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString(emptyMessages[i]));
+ }
+
+ // Test oversized mutation
+ try {
+ executeMutation.accept(oversizedMutations[i]);
+ fail("Should fail since the mutation exceeds the max key value size");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ }
}
}
@Test
- public void testInvalidPutInRowMutations() throws IOException {
+ public void testInvalidMutationInRowMutations() throws IOException {
final byte[] row = Bytes.toBytes(0);
- try {
- getTable.get().mutateRow(new RowMutations(row).add((Mutation) new
Put(row)));
- fail("Should fail since the put does not contain any cells");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("No columns to insert"));
- }
- try {
- getTable.get().mutateRow(new RowMutations(row)
- .add((Mutation) new Put(row).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE])));
- fail("Should fail since the put exceeds the max key value size");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ Mutation[] emptyMutations = { new Put(row), new Increment(row), new
Append(row) };
+
+ String[] emptyMessages =
+ { "No columns to put", "No columns to increment", "No columns to append"
};
+
+ Mutation[] oversizedMutations =
+ { new Put(row).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]),
+ new Increment(row).addColumn(FAMILY, new byte[MAX_KEY_VALUE_SIZE], 1),
+ new Append(row).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]) };
+
+ for (int i = 0; i < emptyMutations.length; i++) {
+ // Test empty mutation
+ try {
+ getTable.get().mutateRow(new RowMutations(row).add(emptyMutations[i]));
+ fail("Should fail since the mutation does not contain any cells");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString(emptyMessages[i]));
+ }
+
+ // Test oversized mutation
+ try {
+ getTable.get().mutateRow(new
RowMutations(row).add(oversizedMutations[i]));
+ fail("Should fail since the mutation exceeds the max key value size");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ }
}
}
@Test
- public void testInvalidPutInRowMutationsInCheckAndMutate() throws
IOException {
+ public void testInvalidMutationInRowMutationsInCheckAndMutate() throws
IOException {
final byte[] row = Bytes.toBytes(0);
- try {
-
getTable.get().checkAndMutate(CheckAndMutate.newBuilder(row).ifNotExists(FAMILY,
QUALIFIER)
- .build(new RowMutations(row).add((Mutation) new Put(row))));
- fail("Should fail since the put does not contain any cells");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("No columns to insert"));
- }
- try {
- getTable.get()
- .checkAndMutate(CheckAndMutate.newBuilder(row).ifNotExists(FAMILY,
QUALIFIER)
- .build(new RowMutations(row).add(
- (Mutation) new Put(row).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]))));
- fail("Should fail since the put exceeds the max key value size");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ Mutation[] emptyMutations = { new Put(row), new Increment(row), new
Append(row) };
+
+ String[] emptyMessages =
+ { "No columns to put", "No columns to increment", "No columns to append"
};
+
+ Mutation[] oversizedMutations =
+ { new Put(row).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]),
+ new Increment(row).addColumn(FAMILY, new byte[MAX_KEY_VALUE_SIZE], 1),
+ new Append(row).addColumn(FAMILY, QUALIFIER, new
byte[MAX_KEY_VALUE_SIZE]) };
+
+ for (int i = 0; i < emptyMutations.length; i++) {
+ // Test empty mutation
+ try {
+
getTable.get().checkAndMutate(CheckAndMutate.newBuilder(row).ifNotExists(FAMILY,
QUALIFIER)
+ .build(new RowMutations(row).add(emptyMutations[i])));
+ fail("Should fail since the mutation does not contain any cells");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString(emptyMessages[i]));
+ }
+
+ // Test oversized mutation
+ try {
+
getTable.get().checkAndMutate(CheckAndMutate.newBuilder(row).ifNotExists(FAMILY,
QUALIFIER)
+ .build(new RowMutations(row).add(oversizedMutations[i])));
+ fail("Should fail since the mutation exceeds the max key value size");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ }
}
}
}
diff --git
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
index f01325bc685..15938b10277 100644
---
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
+++
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
@@ -317,71 +317,107 @@ public class TestAsyncTableBatch {
}
@Test
- public void testInvalidPut() {
+ public void testInvalidMutation() {
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
- try {
- table.batch(Arrays.asList(new Delete(Bytes.toBytes(0)), new
Put(Bytes.toBytes(0))));
- fail("Should fail since the put does not contain any cells");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("No columns to insert"));
- }
- try {
- table.batch(
- Arrays.asList(new Put(Bytes.toBytes(0)).addColumn(FAMILY, CQ, new
byte[MAX_KEY_VALUE_SIZE]),
- new Delete(Bytes.toBytes(0))));
- fail("Should fail since the put exceeds the max key value size");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ Mutation[] emptyMutations =
+ { new Put(Bytes.toBytes(0)), new Increment(Bytes.toBytes(0)), new
Append(Bytes.toBytes(0)) };
+
+ String[] emptyMessages =
+ { "No columns to put", "No columns to increment", "No columns to append"
};
+
+ Mutation[] oversizedMutations =
+ { new Put(Bytes.toBytes(0)).addColumn(FAMILY, CQ, new
byte[MAX_KEY_VALUE_SIZE]),
+ new Increment(Bytes.toBytes(0)).addColumn(FAMILY, new
byte[MAX_KEY_VALUE_SIZE], 1),
+ new Append(Bytes.toBytes(0)).addColumn(FAMILY, CQ, new
byte[MAX_KEY_VALUE_SIZE]) };
+
+ for (int i = 0; i < emptyMutations.length; i++) {
+ // Test empty mutation
+ try {
+ table.batch(Arrays.asList(new Delete(Bytes.toBytes(0)),
emptyMutations[i]));
+ fail("Should fail since the mutation does not contain any cells");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString(emptyMessages[i]));
+ }
+
+ // Test oversized mutation
+ try {
+ table.batch(Arrays.asList(oversizedMutations[i], new
Delete(Bytes.toBytes(0))));
+ fail("Should fail since the mutation exceeds the max key value size");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ }
}
}
@Test
- public void testInvalidPutInRowMutations() throws IOException {
+ public void testInvalidMutationInRowMutations() throws IOException {
final byte[] row = Bytes.toBytes(0);
-
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
- try {
- table
- .batch(Arrays.asList(new Delete(row), new
RowMutations(row).add((Mutation) new Put(row))));
- fail("Should fail since the put does not contain any cells");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("No columns to insert"));
- }
- try {
- table.batch(Arrays.asList(
- new RowMutations(row)
- .add((Mutation) new Put(row).addColumn(FAMILY, CQ, new
byte[MAX_KEY_VALUE_SIZE])),
- new Delete(row)));
- fail("Should fail since the put exceeds the max key value size");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ Mutation[] emptyMutations = { new Put(row), new Increment(row), new
Append(row) };
+
+ String[] emptyMessages =
+ { "No columns to put", "No columns to increment", "No columns to append"
};
+
+ Mutation[] oversizedMutations =
+ { new Put(row).addColumn(FAMILY, CQ, new byte[MAX_KEY_VALUE_SIZE]),
+ new Increment(row).addColumn(FAMILY, new byte[MAX_KEY_VALUE_SIZE], 1),
+ new Append(row).addColumn(FAMILY, CQ, new byte[MAX_KEY_VALUE_SIZE]) };
+
+ for (int i = 0; i < emptyMutations.length; i++) {
+ // Test empty mutation
+ try {
+ table.batch(Arrays.asList(new Delete(row), new
RowMutations(row).add(emptyMutations[i])));
+ fail("Should fail since the mutation does not contain any cells");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString(emptyMessages[i]));
+ }
+
+ // Test oversized mutation
+ try {
+ table
+ .batch(Arrays.asList(new
RowMutations(row).add(oversizedMutations[i]), new Delete(row)));
+ fail("Should fail since the mutation exceeds the max key value size");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ }
}
}
@Test
- public void testInvalidPutInRowMutationsInCheckAndMutate() throws
IOException {
+ public void testInvalidMutationInRowMutationsInCheckAndMutate() throws
IOException {
final byte[] row = Bytes.toBytes(0);
-
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
- try {
- table.batch(Arrays.asList(new Delete(row), CheckAndMutate.newBuilder(row)
- .ifNotExists(FAMILY, CQ).build(new RowMutations(row).add((Mutation)
new Put(row)))));
- fail("Should fail since the put does not contain any cells");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("No columns to insert"));
- }
- try {
- table.batch(Arrays.asList(
- CheckAndMutate.newBuilder(row).ifNotExists(FAMILY, CQ)
- .build(new RowMutations(row)
- .add((Mutation) new Put(row).addColumn(FAMILY, CQ, new
byte[MAX_KEY_VALUE_SIZE]))),
- new Delete(row)));
- fail("Should fail since the put exceeds the max key value size");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ Mutation[] emptyMutations = { new Put(row), new Increment(row), new
Append(row) };
+
+ String[] emptyMessages =
+ { "No columns to put", "No columns to increment", "No columns to append"
};
+
+ Mutation[] oversizedMutations =
+ { new Put(row).addColumn(FAMILY, CQ, new byte[MAX_KEY_VALUE_SIZE]),
+ new Increment(row).addColumn(FAMILY, new byte[MAX_KEY_VALUE_SIZE], 1),
+ new Append(row).addColumn(FAMILY, CQ, new byte[MAX_KEY_VALUE_SIZE]) };
+
+ for (int i = 0; i < emptyMutations.length; i++) {
+ // Test empty mutation
+ try {
+ table.batch(Arrays.asList(new Delete(row),
CheckAndMutate.newBuilder(row)
+ .ifNotExists(FAMILY, CQ).build(new
RowMutations(row).add(emptyMutations[i]))));
+ fail("Should fail since the mutation does not contain any cells");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString(emptyMessages[i]));
+ }
+
+ // Test oversized mutation
+ try {
+
table.batch(Arrays.asList(CheckAndMutate.newBuilder(row).ifNotExists(FAMILY, CQ)
+ .build(new RowMutations(row).add(oversizedMutations[i])), new
Delete(row)));
+ fail("Should fail since the mutation exceeds the max key value size");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("KeyValue size too large"));
+ }
}
}