diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTest5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTest5.java index 1fff2b6910bc..2bd8bce55ca3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTest5.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTest5.java @@ -28,7 +28,6 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.time.Duration; @@ -59,21 +58,14 @@ import org.apache.hadoop.hbase.client.Scan.ReadType; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.filter.BinaryComparator; -import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter; -import org.apache.hadoop.hbase.filter.KeyOnlyFilter; -import org.apache.hadoop.hbase.filter.QualifierFilter; -import org.apache.hadoop.hbase.filter.RegexStringComparator; import org.apache.hadoop.hbase.filter.RowFilter; -import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; import org.apache.hadoop.hbase.filter.SubstringComparator; import org.apache.hadoop.hbase.filter.ValueFilter; -import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; @@ -87,13 +79,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse; - /** * Run tests that use the HBase clients; {@link Table}. Sets up the HBase mini cluster once at start * and runs through all client tests. Each creates a table named for the method and does its stuff @@ -215,347 +200,6 @@ private Result getReverseScanResult(Table table, byte[] row) throws IOException } } - @TestTemplate - public void testMultiRowMutation() throws Exception { - LOG.info("Starting testMultiRowMutation"); - final byte[] ROW1 = Bytes.toBytes("testRow1"); - final byte[] ROW2 = Bytes.toBytes("testRow2"); - final byte[] ROW3 = Bytes.toBytes("testRow3"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { - // Add initial data - t.batch(Arrays.asList(new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE), - new Put(ROW2).addColumn(FAMILY, QUALIFIER, Bytes.toBytes(1L)), - new Put(ROW3).addColumn(FAMILY, QUALIFIER, VALUE)), new Object[3]); - - // Execute MultiRowMutation - Put put = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put); - - Delete delete = new Delete(ROW1); - MutationProto m2 = ProtobufUtil.toMutation(MutationType.DELETE, delete); - - Increment increment = new Increment(ROW2).addColumn(FAMILY, QUALIFIER, 1L); - MutationProto m3 = ProtobufUtil.toMutation(MutationType.INCREMENT, increment); - - Append append = new Append(ROW3).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m4 = ProtobufUtil.toMutation(MutationType.APPEND, append); - - MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); - mrmBuilder.addMutationRequest(m1); - mrmBuilder.addMutationRequest(m2); - mrmBuilder.addMutationRequest(m3); - mrmBuilder.addMutationRequest(m4); - - CoprocessorRpcChannel channel = t.coprocessorService(ROW); - MultiRowMutationService.BlockingInterface service = - MultiRowMutationService.newBlockingStub(channel); - MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); - - // Assert - assertTrue(response.getProcessed()); - - Result r = t.get(new Get(ROW)); - assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW1)); - assertTrue(r.isEmpty()); - - r = t.get(new Get(ROW2)); - assertEquals(2L, Bytes.toLong(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW3)); - assertEquals(Bytes.toString(VALUE) + Bytes.toString(VALUE), - Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - } - } - - @TestTemplate - public void testMultiRowMutationWithSingleConditionWhenConditionMatches() throws Exception { - final byte[] ROW1 = Bytes.toBytes("testRow1"); - final byte[] ROW2 = Bytes.toBytes("testRow2"); - final byte[] VALUE1 = Bytes.toBytes("testValue1"); - final byte[] VALUE2 = Bytes.toBytes("testValue2"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { - // Add initial data - t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); - - // Execute MultiRowMutation with conditions - Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); - Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); - MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); - Delete delete = new Delete(ROW2); - MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); - - MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); - mrmBuilder.addMutationRequest(m1); - mrmBuilder.addMutationRequest(m2); - mrmBuilder.addMutationRequest(m3); - mrmBuilder.addCondition( - ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2, null)); - - CoprocessorRpcChannel channel = t.coprocessorService(ROW); - MultiRowMutationService.BlockingInterface service = - MultiRowMutationService.newBlockingStub(channel); - MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); - - // Assert - assertTrue(response.getProcessed()); - - Result r = t.get(new Get(ROW)); - assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW1)); - assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW2)); - assertTrue(r.isEmpty()); - } - } - - @TestTemplate - public void testMultiRowMutationWithSingleConditionWhenConditionNotMatch() throws Exception { - final byte[] ROW1 = Bytes.toBytes("testRow1"); - final byte[] ROW2 = Bytes.toBytes("testRow2"); - final byte[] VALUE1 = Bytes.toBytes("testValue1"); - final byte[] VALUE2 = Bytes.toBytes("testValue2"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { - // Add initial data - t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); - - // Execute MultiRowMutation with conditions - Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); - Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); - MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); - Delete delete = new Delete(ROW2); - MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); - - MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); - mrmBuilder.addMutationRequest(m1); - mrmBuilder.addMutationRequest(m2); - mrmBuilder.addMutationRequest(m3); - mrmBuilder.addCondition( - ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE1, null)); - - CoprocessorRpcChannel channel = t.coprocessorService(ROW); - MultiRowMutationService.BlockingInterface service = - MultiRowMutationService.newBlockingStub(channel); - MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); - - // Assert - assertFalse(response.getProcessed()); - - Result r = t.get(new Get(ROW)); - assertTrue(r.isEmpty()); - - r = t.get(new Get(ROW1)); - assertTrue(r.isEmpty()); - - r = t.get(new Get(ROW2)); - assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - } - } - - @TestTemplate - public void testMultiRowMutationWithMultipleConditionsWhenConditionsMatch() throws Exception { - final byte[] ROW1 = Bytes.toBytes("testRow1"); - final byte[] ROW2 = Bytes.toBytes("testRow2"); - final byte[] VALUE1 = Bytes.toBytes("testValue1"); - final byte[] VALUE2 = Bytes.toBytes("testValue2"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { - // Add initial data - t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); - - // Execute MultiRowMutation with conditions - Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); - Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); - MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); - Delete delete = new Delete(ROW2); - MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); - - MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); - mrmBuilder.addMutationRequest(m1); - mrmBuilder.addMutationRequest(m2); - mrmBuilder.addMutationRequest(m3); - mrmBuilder.addCondition( - ProtobufUtil.toCondition(ROW, FAMILY, QUALIFIER, CompareOperator.EQUAL, null, null)); - mrmBuilder.addCondition( - ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2, null)); - - CoprocessorRpcChannel channel = t.coprocessorService(ROW); - MultiRowMutationService.BlockingInterface service = - MultiRowMutationService.newBlockingStub(channel); - MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); - - // Assert - assertTrue(response.getProcessed()); - - Result r = t.get(new Get(ROW)); - assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW1)); - assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW2)); - assertTrue(r.isEmpty()); - } - } - - @TestTemplate - public void testMultiRowMutationWithMultipleConditionsWhenConditionsNotMatch() throws Exception { - final byte[] ROW1 = Bytes.toBytes("testRow1"); - final byte[] ROW2 = Bytes.toBytes("testRow2"); - final byte[] VALUE1 = Bytes.toBytes("testValue1"); - final byte[] VALUE2 = Bytes.toBytes("testValue2"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { - // Add initial data - t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); - - // Execute MultiRowMutation with conditions - Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); - Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); - MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); - Delete delete = new Delete(ROW2); - MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); - - MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); - mrmBuilder.addMutationRequest(m1); - mrmBuilder.addMutationRequest(m2); - mrmBuilder.addMutationRequest(m3); - mrmBuilder.addCondition( - ProtobufUtil.toCondition(ROW1, FAMILY, QUALIFIER, CompareOperator.EQUAL, null, null)); - mrmBuilder.addCondition( - ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE1, null)); - - CoprocessorRpcChannel channel = t.coprocessorService(ROW); - MultiRowMutationService.BlockingInterface service = - MultiRowMutationService.newBlockingStub(channel); - MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); - - // Assert - assertFalse(response.getProcessed()); - - Result r = t.get(new Get(ROW)); - assertTrue(r.isEmpty()); - - r = t.get(new Get(ROW1)); - assertTrue(r.isEmpty()); - - r = t.get(new Get(ROW2)); - assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - } - } - - @TestTemplate - public void testMultiRowMutationWithFilterConditionWhenConditionMatches() throws Exception { - final byte[] ROW1 = Bytes.toBytes("testRow1"); - final byte[] ROW2 = Bytes.toBytes("testRow2"); - final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2"); - final byte[] VALUE1 = Bytes.toBytes("testValue1"); - final byte[] VALUE2 = Bytes.toBytes("testValue2"); - final byte[] VALUE3 = Bytes.toBytes("testValue3"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { - // Add initial data - t.put( - new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2).addColumn(FAMILY, QUALIFIER2, VALUE3)); - - // Execute MultiRowMutation with conditions - Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); - Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); - MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); - Delete delete = new Delete(ROW2); - MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); - - MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); - mrmBuilder.addMutationRequest(m1); - mrmBuilder.addMutationRequest(m2); - mrmBuilder.addMutationRequest(m3); - mrmBuilder.addCondition(ProtobufUtil.toCondition(ROW2, - new FilterList( - new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2), - new SingleColumnValueFilter(FAMILY, QUALIFIER2, CompareOperator.EQUAL, VALUE3)), - null)); - - CoprocessorRpcChannel channel = t.coprocessorService(ROW); - MultiRowMutationService.BlockingInterface service = - MultiRowMutationService.newBlockingStub(channel); - MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); - - // Assert - assertTrue(response.getProcessed()); - - Result r = t.get(new Get(ROW)); - assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW1)); - assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - - r = t.get(new Get(ROW2)); - assertTrue(r.isEmpty()); - } - } - - @TestTemplate - public void testMultiRowMutationWithFilterConditionWhenConditionNotMatch() throws Exception { - final byte[] ROW1 = Bytes.toBytes("testRow1"); - final byte[] ROW2 = Bytes.toBytes("testRow2"); - final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2"); - final byte[] VALUE1 = Bytes.toBytes("testValue1"); - final byte[] VALUE2 = Bytes.toBytes("testValue2"); - final byte[] VALUE3 = Bytes.toBytes("testValue3"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { - // Add initial data - t.put( - new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2).addColumn(FAMILY, QUALIFIER2, VALUE3)); - - // Execute MultiRowMutation with conditions - Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); - MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); - Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); - MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); - Delete delete = new Delete(ROW2); - MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); - - MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); - mrmBuilder.addMutationRequest(m1); - mrmBuilder.addMutationRequest(m2); - mrmBuilder.addMutationRequest(m3); - mrmBuilder.addCondition(ProtobufUtil.toCondition(ROW2, - new FilterList( - new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2), - new SingleColumnValueFilter(FAMILY, QUALIFIER2, CompareOperator.EQUAL, VALUE2)), - null)); - - CoprocessorRpcChannel channel = t.coprocessorService(ROW); - MultiRowMutationService.BlockingInterface service = - MultiRowMutationService.newBlockingStub(channel); - MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); - - // Assert - assertFalse(response.getProcessed()); - - Result r = t.get(new Get(ROW)); - assertTrue(r.isEmpty()); - - r = t.get(new Get(ROW1)); - assertTrue(r.isEmpty()); - - r = t.get(new Get(ROW2)); - assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); - } - } - @TestTemplate public void testRowMutations() throws Exception { LOG.info("Starting testRowMutations"); @@ -862,305 +506,6 @@ public void testClientPoolThreadLocal() throws IOException { } } - @TestTemplate - public void testCheckAndPut() throws IOException { - final byte[] anotherrow = Bytes.toBytes("anotherrow"); - final byte[] value2 = Bytes.toBytes("abcd"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { - Put put1 = new Put(ROW); - put1.addColumn(FAMILY, QUALIFIER, VALUE); - - // row doesn't exist, so using non-null value should be considered "not match". - boolean ok = - table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(VALUE).thenPut(put1); - assertFalse(ok); - - // row doesn't exist, so using "ifNotExists" should be considered "match". - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); - assertTrue(ok); - - // row now exists, so using "ifNotExists" should be considered "not match". - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); - assertFalse(ok); - - Put put2 = new Put(ROW); - put2.addColumn(FAMILY, QUALIFIER, value2); - - // row now exists, use the matching value to check - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(VALUE).thenPut(put2); - assertTrue(ok); - - Put put3 = new Put(anotherrow); - put3.addColumn(FAMILY, QUALIFIER, VALUE); - - // try to do CheckAndPut on different rows - try { - table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(value2).thenPut(put3); - fail("trying to check and modify different rows should have failed."); - } catch (Exception ignored) { - } - } - } - - @TestTemplate - public void testCheckAndMutateWithTimeRange() throws IOException { - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { - final long ts = EnvironmentEdgeManager.currentTime() / 2; - Put put = new Put(ROW); - put.addColumn(FAMILY, QUALIFIER, ts, VALUE); - - boolean ok = - table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put); - assertTrue(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenPut(put); - assertFalse(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .timeRange(TimeRange.from(ts + 10000)).ifEquals(VALUE).thenPut(put); - assertFalse(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .timeRange(TimeRange.between(ts + 10000, ts + 20000)).ifEquals(VALUE).thenPut(put); - assertFalse(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.until(ts)) - .ifEquals(VALUE).thenPut(put); - assertFalse(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) - .ifEquals(VALUE).thenPut(put); - assertTrue(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.from(ts)) - .ifEquals(VALUE).thenPut(put); - assertTrue(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .timeRange(TimeRange.between(ts, ts + 20000)).ifEquals(VALUE).thenPut(put); - assertTrue(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .timeRange(TimeRange.until(ts + 10000)).ifEquals(VALUE).thenPut(put); - assertTrue(ok); - - RowMutations rm = new RowMutations(ROW).add((Mutation) put); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenMutate(rm); - assertFalse(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) - .ifEquals(VALUE).thenMutate(rm); - assertTrue(ok); - - Delete delete = new Delete(ROW).addColumn(FAMILY, QUALIFIER); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenDelete(delete); - assertFalse(ok); - - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) - .ifEquals(VALUE).thenDelete(delete); - assertTrue(ok); - } - } - - @TestTemplate - public void testCheckAndPutWithCompareOp() throws IOException { - final byte[] value1 = Bytes.toBytes("aaaa"); - final byte[] value2 = Bytes.toBytes("bbbb"); - final byte[] value3 = Bytes.toBytes("cccc"); - final byte[] value4 = Bytes.toBytes("dddd"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { - Put put2 = new Put(ROW); - put2.addColumn(FAMILY, QUALIFIER, value2); - - Put put3 = new Put(ROW); - put3.addColumn(FAMILY, QUALIFIER, value3); - - // row doesn't exist, so using "ifNotExists" should be considered "match". - boolean ok = - table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put2); - assertTrue(ok); - - // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL - // turns out "match" - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER, value1).thenPut(put2); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.EQUAL, value1).thenPut(put2); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenPut(put2); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS, value1).thenPut(put2); - assertTrue(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenPut(put2); - assertTrue(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.NOT_EQUAL, value1).thenPut(put3); - assertTrue(ok); - - // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL - // turns out "match" - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS, value4).thenPut(put3); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenPut(put3); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.EQUAL, value4).thenPut(put3); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER, value4).thenPut(put3); - assertTrue(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenPut(put3); - assertTrue(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.NOT_EQUAL, value4).thenPut(put2); - assertTrue(ok); - - // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL - // turns out "match" - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER, value2).thenPut(put2); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.NOT_EQUAL, value2).thenPut(put2); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS, value2).thenPut(put2); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenPut(put2); - assertTrue(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenPut(put2); - assertTrue(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.EQUAL, value2).thenPut(put3); - assertTrue(ok); - } - } - - @TestTemplate - public void testCheckAndDelete() throws IOException { - final byte[] value1 = Bytes.toBytes("aaaa"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { - Put put = new Put(ROW); - put.addColumn(FAMILY, QUALIFIER, value1); - table.put(put); - - Delete delete = new Delete(ROW); - delete.addColumns(FAMILY, QUALIFIER); - - boolean ok = - table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(value1).thenDelete(delete); - assertTrue(ok); - } - } - - @TestTemplate - public void testCheckAndDeleteWithCompareOp() throws IOException { - final byte[] value1 = Bytes.toBytes("aaaa"); - final byte[] value2 = Bytes.toBytes("bbbb"); - final byte[] value3 = Bytes.toBytes("cccc"); - final byte[] value4 = Bytes.toBytes("dddd"); - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { - Put put2 = new Put(ROW); - put2.addColumn(FAMILY, QUALIFIER, value2); - table.put(put2); - - Put put3 = new Put(ROW); - put3.addColumn(FAMILY, QUALIFIER, value3); - - Delete delete = new Delete(ROW); - delete.addColumns(FAMILY, QUALIFIER); - - // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL - // turns out "match" - boolean ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER, value1).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.EQUAL, value1).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS, value1).thenDelete(delete); - assertTrue(ok); - table.put(put2); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenDelete(delete); - assertTrue(ok); - table.put(put2); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.NOT_EQUAL, value1).thenDelete(delete); - assertTrue(ok); - - // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL - // turns out "match" - table.put(put3); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS, value4).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.EQUAL, value4).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER, value4).thenDelete(delete); - assertTrue(ok); - table.put(put3); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenDelete(delete); - assertTrue(ok); - table.put(put3); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.NOT_EQUAL, value4).thenDelete(delete); - assertTrue(ok); - - // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL - // turns out "match" - table.put(put2); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER, value2).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.NOT_EQUAL, value2).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS, value2).thenDelete(delete); - assertFalse(ok); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenDelete(delete); - assertTrue(ok); - table.put(put2); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenDelete(delete); - assertTrue(ok); - table.put(put2); - ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) - .ifMatches(CompareOperator.EQUAL, value2).thenDelete(delete); - assertTrue(ok); - } - } - /** * Test ScanMetrics */ @@ -1753,631 +1098,6 @@ public void testSmallScan() throws Exception { } } - @TestTemplate - public void testSuperSimpleWithReverseScan() throws Exception { - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { - Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b11111-0000000000000000002")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b11111-0000000000000000004")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b11111-0000000000000000006")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b11111-0000000000000000008")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b22222-0000000000000000001")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b22222-0000000000000000003")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b22222-0000000000000000005")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b22222-0000000000000000007")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - put = new Put(Bytes.toBytes("0-b22222-0000000000000000009")); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - Scan scan = new Scan().withStartRow(Bytes.toBytes("0-b11111-9223372036854775807")) - .withStopRow(Bytes.toBytes("0-b11111-0000000000000000000"), true); - scan.setReversed(true); - try (ResultScanner scanner = ht.getScanner(scan)) { - Result result = scanner.next(); - assertTrue(Bytes.equals(result.getRow(), Bytes.toBytes("0-b11111-0000000000000000008"))); - } - } - } - - @TestTemplate - public void testFiltersWithReverseScan() throws Exception { - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { - byte[][] ROWS = makeN(ROW, 10); - byte[][] QUALIFIERS = - { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), - Bytes.toBytes("col2--"), Bytes.toBytes("col3--"), - Bytes.toBytes("col4--"), Bytes.toBytes("col5--"), - Bytes.toBytes("col6--"), Bytes.toBytes("col7--"), - Bytes.toBytes("col8--"), Bytes.toBytes("col9--") }; - for (int i = 0; i < 10; i++) { - Put put = new Put(ROWS[i]); - put.addColumn(FAMILY, QUALIFIERS[i], VALUE); - ht.put(put); - } - Scan scan = new Scan(); - scan.setReversed(true); - scan.addFamily(FAMILY); - Filter filter = - new QualifierFilter(CompareOperator.EQUAL, new RegexStringComparator("col[1-5]")); - scan.setFilter(filter); - try (ResultScanner scanner = ht.getScanner(scan)) { - int expectedIndex = 5; - for (Result result : scanner) { - assertEquals(1, result.size()); - Cell c = result.rawCells()[0]; - assertTrue(Bytes.equals(c.getRowArray(), c.getRowOffset(), c.getRowLength(), - ROWS[expectedIndex], 0, ROWS[expectedIndex].length)); - assertTrue( - Bytes.equals(c.getQualifierArray(), c.getQualifierOffset(), c.getQualifierLength(), - QUALIFIERS[expectedIndex], 0, QUALIFIERS[expectedIndex].length)); - expectedIndex--; - } - assertEquals(0, expectedIndex); - } - } - } - - @TestTemplate - public void testKeyOnlyFilterWithReverseScan() throws Exception { - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { - byte[][] ROWS = makeN(ROW, 10); - byte[][] QUALIFIERS = - { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), - Bytes.toBytes("col2--"), Bytes.toBytes("col3--"), - Bytes.toBytes("col4--"), Bytes.toBytes("col5--"), - Bytes.toBytes("col6--"), Bytes.toBytes("col7--"), - Bytes.toBytes("col8--"), Bytes.toBytes("col9--") }; - for (int i = 0; i < 10; i++) { - Put put = new Put(ROWS[i]); - put.addColumn(FAMILY, QUALIFIERS[i], VALUE); - ht.put(put); - } - Scan scan = new Scan(); - scan.setReversed(true); - scan.addFamily(FAMILY); - Filter filter = new KeyOnlyFilter(true); - scan.setFilter(filter); - try (ResultScanner ignored = ht.getScanner(scan)) { - int count = 0; - for (Result result : ht.getScanner(scan)) { - assertEquals(1, result.size()); - assertEquals(Bytes.SIZEOF_INT, result.rawCells()[0].getValueLength()); - assertEquals(VALUE.length, Bytes.toInt(CellUtil.cloneValue(result.rawCells()[0]))); - count++; - } - assertEquals(10, count); - } - } - } - - /** - * Test simple table and non-existent row cases. - */ - @TestTemplate - public void testSimpleMissingWithReverseScan() throws Exception { - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { - byte[][] ROWS = makeN(ROW, 4); - - // Try to get a row on an empty table - Scan scan = new Scan(); - scan.setReversed(true); - Result result = getSingleScanResult(ht, scan); - assertNullResult(result); - - scan = new Scan().withStartRow(ROWS[0]); - scan.setReversed(true); - result = getSingleScanResult(ht, scan); - assertNullResult(result); - - scan = new Scan().withStartRow(ROWS[0]).withStopRow(ROWS[1], true); - scan.setReversed(true); - result = getSingleScanResult(ht, scan); - assertNullResult(result); - - scan = new Scan(); - scan.setReversed(true); - scan.addFamily(FAMILY); - result = getSingleScanResult(ht, scan); - assertNullResult(result); - - scan = new Scan(); - scan.setReversed(true); - scan.addColumn(FAMILY, QUALIFIER); - result = getSingleScanResult(ht, scan); - assertNullResult(result); - - // Insert a row - Put put = new Put(ROWS[2]); - put.addColumn(FAMILY, QUALIFIER, VALUE); - ht.put(put); - - // Make sure we can scan the row - scan = new Scan(); - scan.setReversed(true); - result = getSingleScanResult(ht, scan); - assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); - - scan = new Scan().withStartRow(ROWS[3]).withStopRow(ROWS[0], true); - scan.setReversed(true); - result = getSingleScanResult(ht, scan); - assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); - - scan = new Scan().withStartRow(ROWS[2]).withStopRow(ROWS[1], true); - scan.setReversed(true); - result = getSingleScanResult(ht, scan); - assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); - - // Try to scan empty rows around it - // Introduced MemStore#shouldSeekForReverseScan to fix the following - scan = new Scan().withStartRow(ROWS[1]); - scan.setReversed(true); - result = getSingleScanResult(ht, scan); - assertNullResult(result); - } - } - - @TestTemplate - public void testNullWithReverseScan() throws Exception { - TEST_UTIL.createTable(tableName, FAMILY); - try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { - // Null qualifier (should work) - Put put = new Put(ROW); - put.addColumn(FAMILY, null, VALUE); - ht.put(put); - scanTestNull(ht, ROW, FAMILY, VALUE, true); - Delete delete = new Delete(ROW); - delete.addColumns(FAMILY, null); - ht.delete(delete); - } - - // Use a new table - TableName newTableName = TableName.valueOf(tableName.toString() + "2"); - TEST_UTIL.createTable(newTableName, FAMILY); - try (Connection conn = getConnection(); Table ht = conn.getTable(newTableName)) { - // Empty qualifier, byte[0] instead of null (should work) - Put put = new Put(ROW); - put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); - ht.put(put); - scanTestNull(ht, ROW, FAMILY, VALUE, true); - TEST_UTIL.flush(); - scanTestNull(ht, ROW, FAMILY, VALUE, true); - Delete delete = new Delete(ROW); - delete.addColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY); - ht.delete(delete); - // Null value - put = new Put(ROW); - put.addColumn(FAMILY, QUALIFIER, null); - ht.put(put); - Scan scan = new Scan(); - scan.setReversed(true); - scan.addColumn(FAMILY, QUALIFIER); - Result result = getSingleScanResult(ht, scan); - assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); - } - } - - @TestTemplate - @SuppressWarnings("checkstyle:MethodLength") - public void testDeletesWithReverseScan() throws Exception { - byte[][] ROWS = makeNAscii(ROW, 6); - byte[][] FAMILIES = makeNAscii(FAMILY, 3); - byte[][] VALUES = makeN(VALUE, 5); - long[] ts = { 1000, 2000, 3000, 4000, 5000 }; - TEST_UTIL.createTable(tableName, FAMILIES, 3); - try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { - Put put = new Put(ROW); - put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); - put.addColumn(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]); - ht.put(put); - - Delete delete = new Delete(ROW); - delete.addFamily(FAMILIES[0], ts[0]); - ht.delete(delete); - - Scan scan = new Scan().withStartRow(ROW); - scan.setReversed(true); - scan.addFamily(FAMILIES[0]); - scan.readVersions(Integer.MAX_VALUE); - Result result = getSingleScanResult(ht, scan); - assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1] }, - new byte[][] { VALUES[1] }, 0, 0); - - // Test delete latest version - put = new Put(ROW); - put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); - put.addColumn(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]); - put.addColumn(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]); - put.addColumn(FAMILIES[0], null, ts[4], VALUES[4]); - put.addColumn(FAMILIES[0], null, ts[2], VALUES[2]); - put.addColumn(FAMILIES[0], null, ts[3], VALUES[3]); - ht.put(put); - - delete = new Delete(ROW); - delete.addColumn(FAMILIES[0], QUALIFIER); // ts[4] - ht.delete(delete); - - scan = new Scan().withStartRow(ROW); - scan.setReversed(true); - scan.addColumn(FAMILIES[0], QUALIFIER); - scan.readVersions(Integer.MAX_VALUE); - result = getSingleScanResult(ht, scan); - assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1], ts[2], ts[3] }, - new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2); - - // Test for HBASE-1847 - delete = new Delete(ROW); - delete.addColumn(FAMILIES[0], null); - ht.delete(delete); - - // Cleanup null qualifier - delete = new Delete(ROW); - delete.addColumns(FAMILIES[0], null); - ht.delete(delete); - - // Expected client behavior might be that you can re-put deleted values - // But alas, this is not to be. We can't put them back in either case. - - put = new Put(ROW); - put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); - put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); - ht.put(put); - - // The Scanner returns the previous values, the expected-naive-unexpected - // behavior - - scan = new Scan().withStartRow(ROW); - scan.setReversed(true); - scan.addFamily(FAMILIES[0]); - scan.readVersions(Integer.MAX_VALUE); - result = getSingleScanResult(ht, scan); - assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1], ts[2], ts[3] }, - new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2); - - // Test deleting an entire family from one row but not the other various - // ways - - put = new Put(ROWS[0]); - put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); - put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); - put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); - put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); - ht.put(put); - - put = new Put(ROWS[1]); - put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); - put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); - put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); - put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); - ht.put(put); - - put = new Put(ROWS[2]); - put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); - put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); - put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); - put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); - ht.put(put); - - delete = new Delete(ROWS[0]); - delete.addFamily(FAMILIES[2]); - ht.delete(delete); - - delete = new Delete(ROWS[1]); - delete.addColumns(FAMILIES[1], QUALIFIER); - ht.delete(delete); - - delete = new Delete(ROWS[2]); - delete.addColumn(FAMILIES[1], QUALIFIER); - delete.addColumn(FAMILIES[1], QUALIFIER); - delete.addColumn(FAMILIES[2], QUALIFIER); - ht.delete(delete); - - scan = new Scan().withStartRow(ROWS[0]); - scan.setReversed(true); - scan.addFamily(FAMILIES[1]); - scan.addFamily(FAMILIES[2]); - scan.readVersions(Integer.MAX_VALUE); - result = getSingleScanResult(ht, scan); - assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); - assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, new long[] { ts[0], ts[1] }, - new byte[][] { VALUES[0], VALUES[1] }, 0, 1); - - scan = new Scan().withStartRow(ROWS[1]); - scan.setReversed(true); - scan.addFamily(FAMILIES[1]); - scan.addFamily(FAMILIES[2]); - scan.readVersions(Integer.MAX_VALUE); - result = getSingleScanResult(ht, scan); - assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); - - scan = new Scan().withStartRow(ROWS[2]); - scan.setReversed(true); - scan.addFamily(FAMILIES[1]); - scan.addFamily(FAMILIES[2]); - scan.readVersions(Integer.MAX_VALUE); - result = getSingleScanResult(ht, scan); - assertEquals(1, result.size()); - assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER, new long[] { ts[2] }, - new byte[][] { VALUES[2] }, 0, 0); - - // Test if we delete the family first in one row (HBASE-1541) - - delete = new Delete(ROWS[3]); - delete.addFamily(FAMILIES[1]); - ht.delete(delete); - - put = new Put(ROWS[3]); - put.addColumn(FAMILIES[2], QUALIFIER, VALUES[0]); - ht.put(put); - - put = new Put(ROWS[4]); - put.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]); - put.addColumn(FAMILIES[2], QUALIFIER, VALUES[2]); - ht.put(put); - - scan = new Scan().withStartRow(ROWS[4]); - scan.setReversed(true); - scan.addFamily(FAMILIES[1]); - scan.addFamily(FAMILIES[2]); - scan.readVersions(Integer.MAX_VALUE); - try (ResultScanner scanner = ht.getScanner(scan)) { - result = scanner.next(); - assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); - assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[4])); - assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[1]), ROWS[4])); - assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[1])); - assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[1]), VALUES[2])); - result = scanner.next(); - assertEquals(1, result.size(), "Expected 1 key but received " + result.size()); - assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[3])); - assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[0])); - } - } - } - - /** - * Tests reversed scan under multi regions - */ - @TestTemplate - public void testReversedScanUnderMultiRegions() throws Exception { - // Test Initialization. - byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY; - byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), - Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), Bytes.toBytes("006"), - Bytes.add(Bytes.toBytes("006"), Bytes.multiple(maxByteArray, 8)), Bytes.toBytes("007"), - Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)), Bytes.toBytes("008"), - Bytes.multiple(maxByteArray, 2) }; - TEST_UTIL.createTable(tableName, FAMILY, splitRows); - TEST_UTIL.waitUntilAllRegionsAssigned(tableName); - try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { - try (RegionLocator l = conn.getRegionLocator(tableName)) { - assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); - } - // Insert one row each region - int insertNum = splitRows.length; - for (byte[] splitRow : splitRows) { - Put put = new Put(splitRow); - put.addColumn(FAMILY, QUALIFIER, VALUE); - table.put(put); - } - - // scan forward - try (ResultScanner scanner = table.getScanner(new Scan())) { - int count = 0; - for (Result r : scanner) { - assertFalse(r.isEmpty()); - count++; - } - assertEquals(insertNum, count); - } - - // scan backward - Scan scan = new Scan(); - scan.setReversed(true); - try (ResultScanner scanner = table.getScanner(scan)) { - int count = 0; - byte[] lastRow = null; - for (Result r : scanner) { - assertFalse(r.isEmpty()); - count++; - byte[] thisRow = r.getRow(); - if (lastRow != null) { - assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " - + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); - } - lastRow = thisRow; - } - assertEquals(insertNum, count); - } - } - } - - /** - * Tests reversed scan under multi regions - */ - @TestTemplate - public void testSmallReversedScanUnderMultiRegions() throws Exception { - // Test Initialization. - byte[][] splitRows = new byte[][] { Bytes.toBytes("000"), Bytes.toBytes("002"), - Bytes.toBytes("004"), Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010") }; - TEST_UTIL.createTable(tableName, FAMILY, splitRows); - TEST_UTIL.waitUntilAllRegionsAssigned(tableName); - try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { - try (RegionLocator l = conn.getRegionLocator(tableName)) { - assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); - } - for (byte[] splitRow : splitRows) { - Put put = new Put(splitRow); - put.addColumn(FAMILY, QUALIFIER, VALUE); - table.put(put); - - byte[] nextRow = Bytes.copy(splitRow); - nextRow[nextRow.length - 1]++; - - put = new Put(nextRow); - put.addColumn(FAMILY, QUALIFIER, VALUE); - table.put(put); - } - - // scan forward - try (ResultScanner scanner = table.getScanner(new Scan())) { - int count = 0; - for (Result r : scanner) { - assertTrue(!r.isEmpty()); - count++; - } - assertEquals(12, count); - } - - reverseScanTest(table, ReadType.STREAM); - reverseScanTest(table, ReadType.PREAD); - reverseScanTest(table, ReadType.DEFAULT); - } - } - - private void reverseScanTest(Table table, ReadType readType) throws IOException { - // scan backward - Scan scan = new Scan(); - scan.setReversed(true); - try (ResultScanner scanner = table.getScanner(scan)) { - int count = 0; - byte[] lastRow = null; - for (Result r : scanner) { - assertTrue(!r.isEmpty()); - count++; - byte[] thisRow = r.getRow(); - if (lastRow != null) { - assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " - + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); - } - lastRow = thisRow; - } - assertEquals(12, count); - } - - scan = new Scan(); - scan.setReadType(readType); - scan.setReversed(true); - scan.withStartRow(Bytes.toBytes("002")); - try (ResultScanner scanner = table.getScanner(scan)) { - int count = 0; - byte[] lastRow = null; - for (Result r : scanner) { - assertTrue(!r.isEmpty()); - count++; - byte[] thisRow = r.getRow(); - if (lastRow != null) { - assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " - + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); - } - lastRow = thisRow; - } - assertEquals(3, count); // 000 001 002 - } - - scan = new Scan(); - scan.setReadType(readType); - scan.setReversed(true); - scan.withStartRow(Bytes.toBytes("002")); - scan.withStopRow(Bytes.toBytes("000")); - try (ResultScanner scanner = table.getScanner(scan)) { - int count = 0; - byte[] lastRow = null; - for (Result r : scanner) { - assertFalse(r.isEmpty()); - count++; - byte[] thisRow = r.getRow(); - if (lastRow != null) { - assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " - + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); - } - lastRow = thisRow; - } - assertEquals(2, count); // 001 002 - } - - scan = new Scan(); - scan.setReadType(readType); - scan.setReversed(true); - scan.withStartRow(Bytes.toBytes("001")); - try (ResultScanner scanner = table.getScanner(scan)) { - int count = 0; - byte[] lastRow = null; - for (Result r : scanner) { - assertFalse(r.isEmpty()); - count++; - byte[] thisRow = r.getRow(); - if (lastRow != null) { - assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " - + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); - } - lastRow = thisRow; - } - assertEquals(2, count); // 000 001 - } - - scan = new Scan(); - scan.setReadType(readType); - scan.setReversed(true); - scan.withStartRow(Bytes.toBytes("000")); - try (ResultScanner scanner = table.getScanner(scan)) { - int count = 0; - byte[] lastRow = null; - for (Result r : scanner) { - assertFalse(r.isEmpty()); - count++; - byte[] thisRow = r.getRow(); - if (lastRow != null) { - assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " - + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); - } - lastRow = thisRow; - } - assertEquals(1, count); // 000 - } - - scan = new Scan(); - scan.setReadType(readType); - scan.setReversed(true); - scan.withStartRow(Bytes.toBytes("006")); - scan.withStopRow(Bytes.toBytes("002")); - try (ResultScanner scanner = table.getScanner(scan)) { - int count = 0; - byte[] lastRow = null; - for (Result r : scanner) { - assertFalse(r.isEmpty()); - count++; - byte[] thisRow = r.getRow(); - if (lastRow != null) { - assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " - + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); - } - lastRow = thisRow; - } - assertEquals(4, count); // 003 004 005 006 - } - } - @TestTemplate public void testFilterAllRecords() throws IOException { Scan scan = new Scan(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestBase.java index a7f9ce7e6657..a347a045e0fe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestBase.java @@ -18,9 +18,9 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.HBaseTestingUtil.countRows; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.Arrays; @@ -146,7 +146,7 @@ protected void deleteColumns(Table ht, String value, String keyPrefix) throws IO ht.delete(delete); count++; } - assertEquals("Did not perform correct number of deletes", 3, count); + assertEquals(3, count, "Did not perform correct number of deletes"); } protected int getNumberOfRows(String keyPrefix, String value, Table ht) throws Exception { @@ -332,47 +332,42 @@ protected static boolean equals(byte[] left, byte[] right) { } protected void assertKey(Cell key, byte[] row, byte[] family, byte[] qualifier, byte[] value) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(CellUtil.cloneRow(key)) + "]", equals(row, CellUtil.cloneRow(key))); - assertTrue( + assertTrue(equals(row, CellUtil.cloneRow(key)), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) + "]"); + assertTrue(equals(family, CellUtil.cloneFamily(key)), "Expected family [" + Bytes.toString(family) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(key)) + "]", - equals(family, CellUtil.cloneFamily(key))); - assertTrue( + + Bytes.toString(CellUtil.cloneFamily(key)) + "]"); + assertTrue(equals(qualifier, CellUtil.cloneQualifier(key)), "Expected qualifier [" + Bytes.toString(qualifier) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(key)) + "]", - equals(qualifier, CellUtil.cloneQualifier(key))); - assertTrue("Expected value [" + Bytes.toString(value) + "] " + "Got value [" - + Bytes.toString(CellUtil.cloneValue(key)) + "]", equals(value, CellUtil.cloneValue(key))); + + Bytes.toString(CellUtil.cloneQualifier(key)) + "]"); + assertTrue(equals(value, CellUtil.cloneValue(key)), "Expected value [" + Bytes.toString(value) + + "] " + "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]"); } static void assertIncrementKey(Cell key, byte[] row, byte[] family, byte[] qualifier, long value) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(CellUtil.cloneRow(key)) + "]", equals(row, CellUtil.cloneRow(key))); - assertTrue( + assertTrue(equals(row, CellUtil.cloneRow(key)), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) + "]"); + assertTrue(equals(family, CellUtil.cloneFamily(key)), "Expected family [" + Bytes.toString(family) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(key)) + "]", - equals(family, CellUtil.cloneFamily(key))); - assertTrue( + + Bytes.toString(CellUtil.cloneFamily(key)) + "]"); + assertTrue(equals(qualifier, CellUtil.cloneQualifier(key)), "Expected qualifier [" + Bytes.toString(qualifier) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(key)) + "]", - equals(qualifier, CellUtil.cloneQualifier(key))); - assertEquals("Expected value [" + value + "] " + "Got value [" - + Bytes.toLong(CellUtil.cloneValue(key)) + "]", Bytes.toLong(CellUtil.cloneValue(key)), - value); + + Bytes.toString(CellUtil.cloneQualifier(key)) + "]"); + assertEquals(Bytes.toLong(CellUtil.cloneValue(key)), value, "Expected value [" + value + "] " + + "Got value [" + Bytes.toLong(CellUtil.cloneValue(key)) + "]"); } protected void assertNumKeys(Result result, int n) throws Exception { - assertEquals("Expected " + n + " keys but got " + result.size(), result.size(), n); + assertEquals(result.size(), n, "Expected " + n + " keys but got " + result.size()); } protected void assertNResult(Result result, byte[] row, byte[][] families, byte[][] qualifiers, byte[][] values, int[][] idxs) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(result.getRow()) + "]", equals(row, result.getRow())); - assertEquals("Expected " + idxs.length + " keys but result contains " + result.size(), - result.size(), idxs.length); + assertTrue(equals(row, result.getRow()), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(result.getRow()) + "]"); + assertEquals(result.size(), idxs.length, + "Expected " + idxs.length + " keys but result contains " + result.size()); Cell[] keys = result.rawCells(); @@ -385,19 +380,19 @@ protected void assertNResult(Result result, byte[] row, byte[][] families, byte[ byte[] famb = CellUtil.cloneFamily(key); byte[] qualb = CellUtil.cloneQualifier(key); byte[] valb = CellUtil.cloneValue(key); - assertTrue("(" + i + ") Expected family [" + Bytes.toString(family) + "] " + "Got family [" - + Bytes.toString(famb) + "]", equals(family, famb)); - assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier) + "] " - + "Got qualifier [" + Bytes.toString(qualb) + "]", equals(qualifier, qualb)); - assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] " + "Got value [" - + Bytes.toString(valb) + "]", equals(value, valb)); + assertTrue(equals(family, famb), "(" + i + ") Expected family [" + Bytes.toString(family) + + "] " + "Got family [" + Bytes.toString(famb) + "]"); + assertTrue(equals(qualifier, qualb), "(" + i + ") Expected qualifier [" + + Bytes.toString(qualifier) + "] " + "Got qualifier [" + Bytes.toString(qualb) + "]"); + assertTrue(equals(value, valb), "(" + i + ") Expected value [" + Bytes.toString(value) + "] " + + "Got value [" + Bytes.toString(valb) + "]"); } } protected void assertNResult(Result result, byte[] row, byte[] family, byte[] qualifier, long[] stamps, byte[][] values, int start, int end) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(result.getRow()) + "]", equals(row, result.getRow())); + assertTrue(equals(row, result.getRow()), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(result.getRow()) + "]"); int expectedResults = end - start + 1; assertEquals(expectedResults, result.size()); @@ -408,16 +403,17 @@ protected void assertNResult(Result result, byte[] row, byte[] family, byte[] qu long ts = stamps[end - i]; Cell key = keys[i]; - assertTrue("(" + i + ") Expected family [" + Bytes.toString(family) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(key)) + "]", CellUtil.matchingFamily(key, family)); - assertTrue( + assertTrue(CellUtil.matchingFamily(key, family), + "(" + i + ") Expected family [" + Bytes.toString(family) + "] " + "Got family [" + + Bytes.toString(CellUtil.cloneFamily(key)) + "]"); + assertTrue(CellUtil.matchingQualifier(key, qualifier), "(" + i + ") Expected qualifier [" + Bytes.toString(qualifier) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(key)) + "]", - CellUtil.matchingQualifier(key, qualifier)); - assertEquals("Expected ts [" + ts + "] " + "Got ts [" + key.getTimestamp() + "]", ts, - key.getTimestamp()); - assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] " + "Got value [" - + Bytes.toString(CellUtil.cloneValue(key)) + "]", CellUtil.matchingValue(key, value)); + + Bytes.toString(CellUtil.cloneQualifier(key)) + "]"); + assertEquals(ts, key.getTimestamp(), + "Expected ts [" + ts + "] " + "Got ts [" + key.getTimestamp() + "]"); + assertTrue(CellUtil.matchingValue(key, value), + "(" + i + ") Expected value [" + Bytes.toString(value) + "] " + "Got value [" + + Bytes.toString(CellUtil.cloneValue(key)) + "]"); } } @@ -427,91 +423,89 @@ protected void assertNResult(Result result, byte[] row, byte[] family, byte[] qu */ protected void assertDoubleResult(Result result, byte[] row, byte[] familyA, byte[] qualifierA, byte[] valueA, byte[] familyB, byte[] qualifierB, byte[] valueB) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(result.getRow()) + "]", equals(row, result.getRow())); - assertEquals("Expected two keys but result contains " + result.size(), 2, result.size()); + assertTrue(equals(row, result.getRow()), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(result.getRow()) + "]"); + assertEquals(2, result.size(), "Expected two keys but result contains " + result.size()); Cell[] kv = result.rawCells(); Cell kvA = kv[0]; - assertTrue( + assertTrue(equals(familyA, CellUtil.cloneFamily(kvA)), "(A) Expected family [" + Bytes.toString(familyA) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(kvA)) + "]", - equals(familyA, CellUtil.cloneFamily(kvA))); - assertTrue( + + Bytes.toString(CellUtil.cloneFamily(kvA)) + "]"); + assertTrue(equals(qualifierA, CellUtil.cloneQualifier(kvA)), "(A) Expected qualifier [" + Bytes.toString(qualifierA) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(kvA)) + "]", - equals(qualifierA, CellUtil.cloneQualifier(kvA))); - assertTrue("(A) Expected value [" + Bytes.toString(valueA) + "] " + "Got value [" - + Bytes.toString(CellUtil.cloneValue(kvA)) + "]", equals(valueA, CellUtil.cloneValue(kvA))); + + Bytes.toString(CellUtil.cloneQualifier(kvA)) + "]"); + assertTrue(equals(valueA, CellUtil.cloneValue(kvA)), + "(A) Expected value [" + Bytes.toString(valueA) + "] " + "Got value [" + + Bytes.toString(CellUtil.cloneValue(kvA)) + "]"); Cell kvB = kv[1]; - assertTrue( + assertTrue(equals(familyB, CellUtil.cloneFamily(kvB)), "(B) Expected family [" + Bytes.toString(familyB) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(kvB)) + "]", - equals(familyB, CellUtil.cloneFamily(kvB))); - assertTrue( + + Bytes.toString(CellUtil.cloneFamily(kvB)) + "]"); + assertTrue(equals(qualifierB, CellUtil.cloneQualifier(kvB)), "(B) Expected qualifier [" + Bytes.toString(qualifierB) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(kvB)) + "]", - equals(qualifierB, CellUtil.cloneQualifier(kvB))); - assertTrue("(B) Expected value [" + Bytes.toString(valueB) + "] " + "Got value [" - + Bytes.toString(CellUtil.cloneValue(kvB)) + "]", equals(valueB, CellUtil.cloneValue(kvB))); + + Bytes.toString(CellUtil.cloneQualifier(kvB)) + "]"); + assertTrue(equals(valueB, CellUtil.cloneValue(kvB)), + "(B) Expected value [" + Bytes.toString(valueB) + "] " + "Got value [" + + Bytes.toString(CellUtil.cloneValue(kvB)) + "]"); } protected void assertSingleResult(Result result, byte[] row, byte[] family, byte[] qualifier, byte[] value) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(result.getRow()) + "]", equals(row, result.getRow())); - assertEquals("Expected a single key but result contains " + result.size(), 1, result.size()); + assertTrue(equals(row, result.getRow()), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(result.getRow()) + "]"); + assertEquals(1, result.size(), "Expected a single key but result contains " + result.size()); Cell kv = result.rawCells()[0]; - assertTrue("Expected family [" + Bytes.toString(family) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(kv)) + "]", equals(family, CellUtil.cloneFamily(kv))); - assertTrue( + assertTrue(equals(family, CellUtil.cloneFamily(kv)), + "Expected family [" + Bytes.toString(family) + "] " + "Got family [" + + Bytes.toString(CellUtil.cloneFamily(kv)) + "]"); + assertTrue(equals(qualifier, CellUtil.cloneQualifier(kv)), "Expected qualifier [" + Bytes.toString(qualifier) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]", - equals(qualifier, CellUtil.cloneQualifier(kv))); - assertTrue("Expected value [" + Bytes.toString(value) + "] " + "Got value [" - + Bytes.toString(CellUtil.cloneValue(kv)) + "]", equals(value, CellUtil.cloneValue(kv))); + + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]"); + assertTrue(equals(value, CellUtil.cloneValue(kv)), "Expected value [" + Bytes.toString(value) + + "] " + "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]"); } protected void assertSingleResult(Result result, byte[] row, byte[] family, byte[] qualifier, long value) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(result.getRow()) + "]", equals(row, result.getRow())); - assertEquals("Expected a single key but result contains " + result.size(), 1, result.size()); + assertTrue(equals(row, result.getRow()), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(result.getRow()) + "]"); + assertEquals(1, result.size(), "Expected a single key but result contains " + result.size()); Cell kv = result.rawCells()[0]; - assertTrue("Expected family [" + Bytes.toString(family) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(kv)) + "]", equals(family, CellUtil.cloneFamily(kv))); - assertTrue( + assertTrue(equals(family, CellUtil.cloneFamily(kv)), + "Expected family [" + Bytes.toString(family) + "] " + "Got family [" + + Bytes.toString(CellUtil.cloneFamily(kv)) + "]"); + assertTrue(equals(qualifier, CellUtil.cloneQualifier(kv)), "Expected qualifier [" + Bytes.toString(qualifier) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]", - equals(qualifier, CellUtil.cloneQualifier(kv))); - assertEquals("Expected value [" + value + "] " + "Got value [" - + Bytes.toLong(CellUtil.cloneValue(kv)) + "]", value, Bytes.toLong(CellUtil.cloneValue(kv))); + + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]"); + assertEquals(value, Bytes.toLong(CellUtil.cloneValue(kv)), "Expected value [" + value + "] " + + "Got value [" + Bytes.toLong(CellUtil.cloneValue(kv)) + "]"); } protected void assertSingleResult(Result result, byte[] row, byte[] family, byte[] qualifier, long ts, byte[] value) { - assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" - + Bytes.toString(result.getRow()) + "]", equals(row, result.getRow())); - assertEquals("Expected a single key but result contains " + result.size(), 1, result.size()); + assertTrue(equals(row, result.getRow()), "Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(result.getRow()) + "]"); + assertEquals(1, result.size(), "Expected a single key but result contains " + result.size()); Cell kv = result.rawCells()[0]; - assertTrue("Expected family [" + Bytes.toString(family) + "] " + "Got family [" - + Bytes.toString(CellUtil.cloneFamily(kv)) + "]", equals(family, CellUtil.cloneFamily(kv))); - assertTrue( + assertTrue(equals(family, CellUtil.cloneFamily(kv)), + "Expected family [" + Bytes.toString(family) + "] " + "Got family [" + + Bytes.toString(CellUtil.cloneFamily(kv)) + "]"); + assertTrue(equals(qualifier, CellUtil.cloneQualifier(kv)), "Expected qualifier [" + Bytes.toString(qualifier) + "] " + "Got qualifier [" - + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]", - equals(qualifier, CellUtil.cloneQualifier(kv))); - assertEquals("Expected ts [" + ts + "] " + "Got ts [" + kv.getTimestamp() + "]", ts, - kv.getTimestamp()); - assertTrue("Expected value [" + Bytes.toString(value) + "] " + "Got value [" - + Bytes.toString(CellUtil.cloneValue(kv)) + "]", equals(value, CellUtil.cloneValue(kv))); + + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]"); + assertEquals(ts, kv.getTimestamp(), + "Expected ts [" + ts + "] " + "Got ts [" + kv.getTimestamp() + "]"); + assertTrue(equals(value, CellUtil.cloneValue(kv)), "Expected value [" + Bytes.toString(value) + + "] " + "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]"); } protected void assertEmptyResult(Result result) throws Exception { - assertTrue("expected an empty result but result contains " + result.size() + " keys", - result.isEmpty()); + assertTrue(result.isEmpty(), + "expected an empty result but result contains " + result.size() + " keys"); } protected void assertNullResult(Result result) throws Exception { - assertNull("expected null result but received a non-null result", result); + assertNull(result, "expected null result but received a non-null result"); } protected void getVersionRangeAndVerifyGreaterThan(Table ht, byte[] row, byte[] family, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestCheckAndMutate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestCheckAndMutate.java new file mode 100644 index 000000000000..8e0e0e3df306 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestCheckAndMutate.java @@ -0,0 +1,337 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +import java.io.IOException; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.jupiter.api.TestTemplate; + +@SuppressWarnings("deprecation") +public class FromClientSideTestCheckAndMutate extends FromClientSideTestBase { + + protected FromClientSideTestCheckAndMutate(Class registryImpl, + int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @TestTemplate + public void testCheckAndPut() throws IOException { + final byte[] anotherrow = Bytes.toBytes("anotherrow"); + final byte[] value2 = Bytes.toBytes("abcd"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { + Put put1 = new Put(ROW); + put1.addColumn(FAMILY, QUALIFIER, VALUE); + + // row doesn't exist, so using non-null value should be considered "not match". + boolean ok = + table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(VALUE).thenPut(put1); + assertFalse(ok); + + // row doesn't exist, so using "ifNotExists" should be considered "match". + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); + assertTrue(ok); + + // row now exists, so using "ifNotExists" should be considered "not match". + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); + assertFalse(ok); + + Put put2 = new Put(ROW); + put2.addColumn(FAMILY, QUALIFIER, value2); + + // row now exists, use the matching value to check + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(VALUE).thenPut(put2); + assertTrue(ok); + + Put put3 = new Put(anotherrow); + put3.addColumn(FAMILY, QUALIFIER, VALUE); + + // try to do CheckAndPut on different rows + try { + table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(value2).thenPut(put3); + fail("trying to check and modify different rows should have failed."); + } catch (Exception ignored) { + } + } + } + + @TestTemplate + public void testCheckAndMutateWithTimeRange() throws IOException { + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { + final long ts = EnvironmentEdgeManager.currentTime() / 2; + Put put = new Put(ROW); + put.addColumn(FAMILY, QUALIFIER, ts, VALUE); + + boolean ok = + table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put); + assertTrue(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenPut(put); + assertFalse(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .timeRange(TimeRange.from(ts + 10000)).ifEquals(VALUE).thenPut(put); + assertFalse(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .timeRange(TimeRange.between(ts + 10000, ts + 20000)).ifEquals(VALUE).thenPut(put); + assertFalse(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.until(ts)) + .ifEquals(VALUE).thenPut(put); + assertFalse(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) + .ifEquals(VALUE).thenPut(put); + assertTrue(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.from(ts)) + .ifEquals(VALUE).thenPut(put); + assertTrue(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .timeRange(TimeRange.between(ts, ts + 20000)).ifEquals(VALUE).thenPut(put); + assertTrue(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .timeRange(TimeRange.until(ts + 10000)).ifEquals(VALUE).thenPut(put); + assertTrue(ok); + + RowMutations rm = new RowMutations(ROW).add((Mutation) put); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenMutate(rm); + assertFalse(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) + .ifEquals(VALUE).thenMutate(rm); + assertTrue(ok); + + Delete delete = new Delete(ROW).addColumn(FAMILY, QUALIFIER); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenDelete(delete); + assertFalse(ok); + + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) + .ifEquals(VALUE).thenDelete(delete); + assertTrue(ok); + } + } + + @TestTemplate + public void testCheckAndPutWithCompareOp() throws IOException { + final byte[] value1 = Bytes.toBytes("aaaa"); + final byte[] value2 = Bytes.toBytes("bbbb"); + final byte[] value3 = Bytes.toBytes("cccc"); + final byte[] value4 = Bytes.toBytes("dddd"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { + Put put2 = new Put(ROW); + put2.addColumn(FAMILY, QUALIFIER, value2); + + Put put3 = new Put(ROW); + put3.addColumn(FAMILY, QUALIFIER, value3); + + // row doesn't exist, so using "ifNotExists" should be considered "match". + boolean ok = + table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put2); + assertTrue(ok); + + // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL + // turns out "match" + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER, value1).thenPut(put2); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.EQUAL, value1).thenPut(put2); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenPut(put2); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS, value1).thenPut(put2); + assertTrue(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenPut(put2); + assertTrue(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.NOT_EQUAL, value1).thenPut(put3); + assertTrue(ok); + + // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL + // turns out "match" + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS, value4).thenPut(put3); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenPut(put3); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.EQUAL, value4).thenPut(put3); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER, value4).thenPut(put3); + assertTrue(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenPut(put3); + assertTrue(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.NOT_EQUAL, value4).thenPut(put2); + assertTrue(ok); + + // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL + // turns out "match" + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER, value2).thenPut(put2); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.NOT_EQUAL, value2).thenPut(put2); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS, value2).thenPut(put2); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenPut(put2); + assertTrue(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenPut(put2); + assertTrue(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.EQUAL, value2).thenPut(put3); + assertTrue(ok); + } + } + + @TestTemplate + public void testCheckAndDelete() throws IOException { + final byte[] value1 = Bytes.toBytes("aaaa"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { + Put put = new Put(ROW); + put.addColumn(FAMILY, QUALIFIER, value1); + table.put(put); + + Delete delete = new Delete(ROW); + delete.addColumns(FAMILY, QUALIFIER); + + boolean ok = + table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(value1).thenDelete(delete); + assertTrue(ok); + } + } + + @TestTemplate + public void testCheckAndDeleteWithCompareOp() throws IOException { + final byte[] value1 = Bytes.toBytes("aaaa"); + final byte[] value2 = Bytes.toBytes("bbbb"); + final byte[] value3 = Bytes.toBytes("cccc"); + final byte[] value4 = Bytes.toBytes("dddd"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { + Put put2 = new Put(ROW); + put2.addColumn(FAMILY, QUALIFIER, value2); + table.put(put2); + + Put put3 = new Put(ROW); + put3.addColumn(FAMILY, QUALIFIER, value3); + + Delete delete = new Delete(ROW); + delete.addColumns(FAMILY, QUALIFIER); + + // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL + // turns out "match" + boolean ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER, value1).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.EQUAL, value1).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS, value1).thenDelete(delete); + assertTrue(ok); + table.put(put2); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenDelete(delete); + assertTrue(ok); + table.put(put2); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.NOT_EQUAL, value1).thenDelete(delete); + assertTrue(ok); + + // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL + // turns out "match" + table.put(put3); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS, value4).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.EQUAL, value4).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER, value4).thenDelete(delete); + assertTrue(ok); + table.put(put3); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenDelete(delete); + assertTrue(ok); + table.put(put3); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.NOT_EQUAL, value4).thenDelete(delete); + assertTrue(ok); + + // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL + // turns out "match" + table.put(put2); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER, value2).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.NOT_EQUAL, value2).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS, value2).thenDelete(delete); + assertFalse(ok); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenDelete(delete); + assertTrue(ok); + table.put(put2); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenDelete(delete); + assertTrue(ok); + table.put(put2); + ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) + .ifMatches(CompareOperator.EQUAL, value2).thenDelete(delete); + assertTrue(ok); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestMultiRowMutation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestMultiRowMutation.java new file mode 100644 index 000000000000..e05e7e40bfb9 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestMultiRowMutation.java @@ -0,0 +1,385 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.jupiter.api.TestTemplate; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse; + +public class FromClientSideTestMultiRowMutation extends FromClientSideTestBase { + + protected FromClientSideTestMultiRowMutation(Class registryImpl, + int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @TestTemplate + public void testMultiRowMutation() throws Exception { + final byte[] ROW1 = Bytes.toBytes("testRow1"); + final byte[] ROW2 = Bytes.toBytes("testRow2"); + final byte[] ROW3 = Bytes.toBytes("testRow3"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { + // Add initial data + t.batch(Arrays.asList(new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE), + new Put(ROW2).addColumn(FAMILY, QUALIFIER, Bytes.toBytes(1L)), + new Put(ROW3).addColumn(FAMILY, QUALIFIER, VALUE)), new Object[3]); + + // Execute MultiRowMutation + Put put = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put); + + Delete delete = new Delete(ROW1); + MutationProto m2 = ProtobufUtil.toMutation(MutationType.DELETE, delete); + + Increment increment = new Increment(ROW2).addColumn(FAMILY, QUALIFIER, 1L); + MutationProto m3 = ProtobufUtil.toMutation(MutationType.INCREMENT, increment); + + Append append = new Append(ROW3).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m4 = ProtobufUtil.toMutation(MutationType.APPEND, append); + + MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); + mrmBuilder.addMutationRequest(m1); + mrmBuilder.addMutationRequest(m2); + mrmBuilder.addMutationRequest(m3); + mrmBuilder.addMutationRequest(m4); + + CoprocessorRpcChannel channel = t.coprocessorService(ROW); + MultiRowMutationService.BlockingInterface service = + MultiRowMutationService.newBlockingStub(channel); + MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); + + // Assert + assertTrue(response.getProcessed()); + + Result r = t.get(new Get(ROW)); + assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW1)); + assertTrue(r.isEmpty()); + + r = t.get(new Get(ROW2)); + assertEquals(2L, Bytes.toLong(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW3)); + assertEquals(Bytes.toString(VALUE) + Bytes.toString(VALUE), + Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + } + } + + @TestTemplate + public void testMultiRowMutationWithSingleConditionWhenConditionMatches() throws Exception { + final byte[] ROW1 = Bytes.toBytes("testRow1"); + final byte[] ROW2 = Bytes.toBytes("testRow2"); + final byte[] VALUE1 = Bytes.toBytes("testValue1"); + final byte[] VALUE2 = Bytes.toBytes("testValue2"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { + // Add initial data + t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); + + // Execute MultiRowMutation with conditions + Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); + Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); + MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); + Delete delete = new Delete(ROW2); + MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); + + MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); + mrmBuilder.addMutationRequest(m1); + mrmBuilder.addMutationRequest(m2); + mrmBuilder.addMutationRequest(m3); + mrmBuilder.addCondition( + ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2, null)); + + CoprocessorRpcChannel channel = t.coprocessorService(ROW); + MultiRowMutationService.BlockingInterface service = + MultiRowMutationService.newBlockingStub(channel); + MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); + + // Assert + assertTrue(response.getProcessed()); + + Result r = t.get(new Get(ROW)); + assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW1)); + assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW2)); + assertTrue(r.isEmpty()); + } + } + + @TestTemplate + public void testMultiRowMutationWithSingleConditionWhenConditionNotMatch() throws Exception { + final byte[] ROW1 = Bytes.toBytes("testRow1"); + final byte[] ROW2 = Bytes.toBytes("testRow2"); + final byte[] VALUE1 = Bytes.toBytes("testValue1"); + final byte[] VALUE2 = Bytes.toBytes("testValue2"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { + // Add initial data + t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); + + // Execute MultiRowMutation with conditions + Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); + Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); + MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); + Delete delete = new Delete(ROW2); + MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); + + MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); + mrmBuilder.addMutationRequest(m1); + mrmBuilder.addMutationRequest(m2); + mrmBuilder.addMutationRequest(m3); + mrmBuilder.addCondition( + ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE1, null)); + + CoprocessorRpcChannel channel = t.coprocessorService(ROW); + MultiRowMutationService.BlockingInterface service = + MultiRowMutationService.newBlockingStub(channel); + MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); + + // Assert + assertFalse(response.getProcessed()); + + Result r = t.get(new Get(ROW)); + assertTrue(r.isEmpty()); + + r = t.get(new Get(ROW1)); + assertTrue(r.isEmpty()); + + r = t.get(new Get(ROW2)); + assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + } + } + + @TestTemplate + public void testMultiRowMutationWithMultipleConditionsWhenConditionsMatch() throws Exception { + final byte[] ROW1 = Bytes.toBytes("testRow1"); + final byte[] ROW2 = Bytes.toBytes("testRow2"); + final byte[] VALUE1 = Bytes.toBytes("testValue1"); + final byte[] VALUE2 = Bytes.toBytes("testValue2"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { + // Add initial data + t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); + + // Execute MultiRowMutation with conditions + Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); + Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); + MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); + Delete delete = new Delete(ROW2); + MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); + + MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); + mrmBuilder.addMutationRequest(m1); + mrmBuilder.addMutationRequest(m2); + mrmBuilder.addMutationRequest(m3); + mrmBuilder.addCondition( + ProtobufUtil.toCondition(ROW, FAMILY, QUALIFIER, CompareOperator.EQUAL, null, null)); + mrmBuilder.addCondition( + ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2, null)); + + CoprocessorRpcChannel channel = t.coprocessorService(ROW); + MultiRowMutationService.BlockingInterface service = + MultiRowMutationService.newBlockingStub(channel); + MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); + + // Assert + assertTrue(response.getProcessed()); + + Result r = t.get(new Get(ROW)); + assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW1)); + assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW2)); + assertTrue(r.isEmpty()); + } + } + + @TestTemplate + public void testMultiRowMutationWithMultipleConditionsWhenConditionsNotMatch() throws Exception { + final byte[] ROW1 = Bytes.toBytes("testRow1"); + final byte[] ROW2 = Bytes.toBytes("testRow2"); + final byte[] VALUE1 = Bytes.toBytes("testValue1"); + final byte[] VALUE2 = Bytes.toBytes("testValue2"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { + // Add initial data + t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); + + // Execute MultiRowMutation with conditions + Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); + Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); + MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); + Delete delete = new Delete(ROW2); + MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); + + MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); + mrmBuilder.addMutationRequest(m1); + mrmBuilder.addMutationRequest(m2); + mrmBuilder.addMutationRequest(m3); + mrmBuilder.addCondition( + ProtobufUtil.toCondition(ROW1, FAMILY, QUALIFIER, CompareOperator.EQUAL, null, null)); + mrmBuilder.addCondition( + ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE1, null)); + + CoprocessorRpcChannel channel = t.coprocessorService(ROW); + MultiRowMutationService.BlockingInterface service = + MultiRowMutationService.newBlockingStub(channel); + MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); + + // Assert + assertFalse(response.getProcessed()); + + Result r = t.get(new Get(ROW)); + assertTrue(r.isEmpty()); + + r = t.get(new Get(ROW1)); + assertTrue(r.isEmpty()); + + r = t.get(new Get(ROW2)); + assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + } + } + + @TestTemplate + public void testMultiRowMutationWithFilterConditionWhenConditionMatches() throws Exception { + final byte[] ROW1 = Bytes.toBytes("testRow1"); + final byte[] ROW2 = Bytes.toBytes("testRow2"); + final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2"); + final byte[] VALUE1 = Bytes.toBytes("testValue1"); + final byte[] VALUE2 = Bytes.toBytes("testValue2"); + final byte[] VALUE3 = Bytes.toBytes("testValue3"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { + // Add initial data + t.put( + new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2).addColumn(FAMILY, QUALIFIER2, VALUE3)); + + // Execute MultiRowMutation with conditions + Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); + Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); + MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); + Delete delete = new Delete(ROW2); + MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); + + MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); + mrmBuilder.addMutationRequest(m1); + mrmBuilder.addMutationRequest(m2); + mrmBuilder.addMutationRequest(m3); + mrmBuilder.addCondition(ProtobufUtil.toCondition(ROW2, + new FilterList( + new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2), + new SingleColumnValueFilter(FAMILY, QUALIFIER2, CompareOperator.EQUAL, VALUE3)), + null)); + + CoprocessorRpcChannel channel = t.coprocessorService(ROW); + MultiRowMutationService.BlockingInterface service = + MultiRowMutationService.newBlockingStub(channel); + MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); + + // Assert + assertTrue(response.getProcessed()); + + Result r = t.get(new Get(ROW)); + assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW1)); + assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + + r = t.get(new Get(ROW2)); + assertTrue(r.isEmpty()); + } + } + + @TestTemplate + public void testMultiRowMutationWithFilterConditionWhenConditionNotMatch() throws Exception { + final byte[] ROW1 = Bytes.toBytes("testRow1"); + final byte[] ROW2 = Bytes.toBytes("testRow2"); + final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2"); + final byte[] VALUE1 = Bytes.toBytes("testValue1"); + final byte[] VALUE2 = Bytes.toBytes("testValue2"); + final byte[] VALUE3 = Bytes.toBytes("testValue3"); + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { + // Add initial data + t.put( + new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2).addColumn(FAMILY, QUALIFIER2, VALUE3)); + + // Execute MultiRowMutation with conditions + Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); + MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); + Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); + MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); + Delete delete = new Delete(ROW2); + MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); + + MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); + mrmBuilder.addMutationRequest(m1); + mrmBuilder.addMutationRequest(m2); + mrmBuilder.addMutationRequest(m3); + mrmBuilder.addCondition(ProtobufUtil.toCondition(ROW2, + new FilterList( + new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2), + new SingleColumnValueFilter(FAMILY, QUALIFIER2, CompareOperator.EQUAL, VALUE2)), + null)); + + CoprocessorRpcChannel channel = t.coprocessorService(ROW); + MultiRowMutationService.BlockingInterface service = + MultiRowMutationService.newBlockingStub(channel); + MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); + + // Assert + assertFalse(response.getProcessed()); + + Result r = t.get(new Get(ROW)); + assertTrue(r.isEmpty()); + + r = t.get(new Get(ROW1)); + assertTrue(r.isEmpty()); + + r = t.get(new Get(ROW2)); + assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestReverseScan.java new file mode 100644 index 000000000000..f0f479bc273c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/FromClientSideTestReverseScan.java @@ -0,0 +1,669 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Scan.ReadType; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.KeyOnlyFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.hbase.filter.RegexStringComparator; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.jupiter.api.TestTemplate; + +public class FromClientSideTestReverseScan extends FromClientSideTestBase { + + protected FromClientSideTestReverseScan(Class registryImpl, + int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @TestTemplate + public void testSuperSimpleWithReverseScan() throws Exception { + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { + Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b11111-0000000000000000002")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b11111-0000000000000000004")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b11111-0000000000000000006")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b11111-0000000000000000008")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b22222-0000000000000000001")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b22222-0000000000000000003")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b22222-0000000000000000005")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b22222-0000000000000000007")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + put = new Put(Bytes.toBytes("0-b22222-0000000000000000009")); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + Scan scan = new Scan().withStartRow(Bytes.toBytes("0-b11111-9223372036854775807")) + .withStopRow(Bytes.toBytes("0-b11111-0000000000000000000"), true); + scan.setReversed(true); + try (ResultScanner scanner = ht.getScanner(scan)) { + Result result = scanner.next(); + assertTrue(Bytes.equals(result.getRow(), Bytes.toBytes("0-b11111-0000000000000000008"))); + } + } + } + + @TestTemplate + public void testFiltersWithReverseScan() throws Exception { + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { + byte[][] ROWS = makeN(ROW, 10); + byte[][] QUALIFIERS = + { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), + Bytes.toBytes("col2--"), Bytes.toBytes("col3--"), + Bytes.toBytes("col4--"), Bytes.toBytes("col5--"), + Bytes.toBytes("col6--"), Bytes.toBytes("col7--"), + Bytes.toBytes("col8--"), Bytes.toBytes("col9--") }; + for (int i = 0; i < 10; i++) { + Put put = new Put(ROWS[i]); + put.addColumn(FAMILY, QUALIFIERS[i], VALUE); + ht.put(put); + } + Scan scan = new Scan(); + scan.setReversed(true); + scan.addFamily(FAMILY); + Filter filter = + new QualifierFilter(CompareOperator.EQUAL, new RegexStringComparator("col[1-5]")); + scan.setFilter(filter); + try (ResultScanner scanner = ht.getScanner(scan)) { + int expectedIndex = 5; + for (Result result : scanner) { + assertEquals(1, result.size()); + Cell c = result.rawCells()[0]; + assertTrue(Bytes.equals(c.getRowArray(), c.getRowOffset(), c.getRowLength(), + ROWS[expectedIndex], 0, ROWS[expectedIndex].length)); + assertTrue( + Bytes.equals(c.getQualifierArray(), c.getQualifierOffset(), c.getQualifierLength(), + QUALIFIERS[expectedIndex], 0, QUALIFIERS[expectedIndex].length)); + expectedIndex--; + } + assertEquals(0, expectedIndex); + } + } + } + + @TestTemplate + public void testKeyOnlyFilterWithReverseScan() throws Exception { + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { + byte[][] ROWS = makeN(ROW, 10); + byte[][] QUALIFIERS = + { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), + Bytes.toBytes("col2--"), Bytes.toBytes("col3--"), + Bytes.toBytes("col4--"), Bytes.toBytes("col5--"), + Bytes.toBytes("col6--"), Bytes.toBytes("col7--"), + Bytes.toBytes("col8--"), Bytes.toBytes("col9--") }; + for (int i = 0; i < 10; i++) { + Put put = new Put(ROWS[i]); + put.addColumn(FAMILY, QUALIFIERS[i], VALUE); + ht.put(put); + } + Scan scan = new Scan(); + scan.setReversed(true); + scan.addFamily(FAMILY); + Filter filter = new KeyOnlyFilter(true); + scan.setFilter(filter); + try (ResultScanner ignored = ht.getScanner(scan)) { + int count = 0; + for (Result result : ht.getScanner(scan)) { + assertEquals(1, result.size()); + assertEquals(Bytes.SIZEOF_INT, result.rawCells()[0].getValueLength()); + assertEquals(VALUE.length, Bytes.toInt(CellUtil.cloneValue(result.rawCells()[0]))); + count++; + } + assertEquals(10, count); + } + } + } + + /** + * Test simple table and non-existent row cases. + */ + @TestTemplate + public void testSimpleMissingWithReverseScan() throws Exception { + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { + byte[][] ROWS = makeN(ROW, 4); + + // Try to get a row on an empty table + Scan scan = new Scan(); + scan.setReversed(true); + Result result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan().withStartRow(ROWS[0]); + scan.setReversed(true); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan().withStartRow(ROWS[0]).withStopRow(ROWS[1], true); + scan.setReversed(true); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(); + scan.setReversed(true); + scan.addFamily(FAMILY); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(); + scan.setReversed(true); + scan.addColumn(FAMILY, QUALIFIER); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Insert a row + Put put = new Put(ROWS[2]); + put.addColumn(FAMILY, QUALIFIER, VALUE); + ht.put(put); + + // Make sure we can scan the row + scan = new Scan(); + scan.setReversed(true); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + scan = new Scan().withStartRow(ROWS[3]).withStopRow(ROWS[0], true); + scan.setReversed(true); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + scan = new Scan().withStartRow(ROWS[2]).withStopRow(ROWS[1], true); + scan.setReversed(true); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + // Try to scan empty rows around it + // Introduced MemStore#shouldSeekForReverseScan to fix the following + scan = new Scan().withStartRow(ROWS[1]); + scan.setReversed(true); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + } + } + + @TestTemplate + public void testNullWithReverseScan() throws Exception { + TEST_UTIL.createTable(tableName, FAMILY); + try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { + // Null qualifier (should work) + Put put = new Put(ROW); + put.addColumn(FAMILY, null, VALUE); + ht.put(put); + scanTestNull(ht, ROW, FAMILY, VALUE, true); + Delete delete = new Delete(ROW); + delete.addColumns(FAMILY, null); + ht.delete(delete); + } + + // Use a new table + TableName newTableName = TableName.valueOf(tableName.toString() + "2"); + TEST_UTIL.createTable(newTableName, FAMILY); + try (Connection conn = getConnection(); Table ht = conn.getTable(newTableName)) { + // Empty qualifier, byte[0] instead of null (should work) + Put put = new Put(ROW); + put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); + ht.put(put); + scanTestNull(ht, ROW, FAMILY, VALUE, true); + TEST_UTIL.flush(); + scanTestNull(ht, ROW, FAMILY, VALUE, true); + Delete delete = new Delete(ROW); + delete.addColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY); + ht.delete(delete); + // Null value + put = new Put(ROW); + put.addColumn(FAMILY, QUALIFIER, null); + ht.put(put); + Scan scan = new Scan(); + scan.setReversed(true); + scan.addColumn(FAMILY, QUALIFIER); + Result result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); + } + } + + @TestTemplate + @SuppressWarnings("checkstyle:MethodLength") + public void testDeletesWithReverseScan() throws Exception { + byte[][] ROWS = makeNAscii(ROW, 6); + byte[][] FAMILIES = makeNAscii(FAMILY, 3); + byte[][] VALUES = makeN(VALUE, 5); + long[] ts = { 1000, 2000, 3000, 4000, 5000 }; + TEST_UTIL.createTable(tableName, FAMILIES, 3); + try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { + Put put = new Put(ROW); + put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); + put.addColumn(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]); + ht.put(put); + + Delete delete = new Delete(ROW); + delete.addFamily(FAMILIES[0], ts[0]); + ht.delete(delete); + + Scan scan = new Scan().withStartRow(ROW); + scan.setReversed(true); + scan.addFamily(FAMILIES[0]); + scan.readVersions(Integer.MAX_VALUE); + Result result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1] }, + new byte[][] { VALUES[1] }, 0, 0); + + // Test delete latest version + put = new Put(ROW); + put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); + put.addColumn(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]); + put.addColumn(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]); + put.addColumn(FAMILIES[0], null, ts[4], VALUES[4]); + put.addColumn(FAMILIES[0], null, ts[2], VALUES[2]); + put.addColumn(FAMILIES[0], null, ts[3], VALUES[3]); + ht.put(put); + + delete = new Delete(ROW); + delete.addColumn(FAMILIES[0], QUALIFIER); // ts[4] + ht.delete(delete); + + scan = new Scan().withStartRow(ROW); + scan.setReversed(true); + scan.addColumn(FAMILIES[0], QUALIFIER); + scan.readVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1], ts[2], ts[3] }, + new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2); + + // Test for HBASE-1847 + delete = new Delete(ROW); + delete.addColumn(FAMILIES[0], null); + ht.delete(delete); + + // Cleanup null qualifier + delete = new Delete(ROW); + delete.addColumns(FAMILIES[0], null); + ht.delete(delete); + + // Expected client behavior might be that you can re-put deleted values + // But alas, this is not to be. We can't put them back in either case. + + put = new Put(ROW); + put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); + put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); + ht.put(put); + + // The Scanner returns the previous values, the expected-naive-unexpected + // behavior + + scan = new Scan().withStartRow(ROW); + scan.setReversed(true); + scan.addFamily(FAMILIES[0]); + scan.readVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1], ts[2], ts[3] }, + new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2); + + // Test deleting an entire family from one row but not the other various + // ways + + put = new Put(ROWS[0]); + put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); + put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); + put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); + put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); + ht.put(put); + + put = new Put(ROWS[1]); + put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); + put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); + put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); + put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); + ht.put(put); + + put = new Put(ROWS[2]); + put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); + put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); + put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); + put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); + ht.put(put); + + delete = new Delete(ROWS[0]); + delete.addFamily(FAMILIES[2]); + ht.delete(delete); + + delete = new Delete(ROWS[1]); + delete.addColumns(FAMILIES[1], QUALIFIER); + ht.delete(delete); + + delete = new Delete(ROWS[2]); + delete.addColumn(FAMILIES[1], QUALIFIER); + delete.addColumn(FAMILIES[1], QUALIFIER); + delete.addColumn(FAMILIES[2], QUALIFIER); + ht.delete(delete); + + scan = new Scan().withStartRow(ROWS[0]); + scan.setReversed(true); + scan.addFamily(FAMILIES[1]); + scan.addFamily(FAMILIES[2]); + scan.readVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); + assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, new long[] { ts[0], ts[1] }, + new byte[][] { VALUES[0], VALUES[1] }, 0, 1); + + scan = new Scan().withStartRow(ROWS[1]); + scan.setReversed(true); + scan.addFamily(FAMILIES[1]); + scan.addFamily(FAMILIES[2]); + scan.readVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); + + scan = new Scan().withStartRow(ROWS[2]); + scan.setReversed(true); + scan.addFamily(FAMILIES[1]); + scan.addFamily(FAMILIES[2]); + scan.readVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertEquals(1, result.size()); + assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER, new long[] { ts[2] }, + new byte[][] { VALUES[2] }, 0, 0); + + // Test if we delete the family first in one row (HBASE-1541) + + delete = new Delete(ROWS[3]); + delete.addFamily(FAMILIES[1]); + ht.delete(delete); + + put = new Put(ROWS[3]); + put.addColumn(FAMILIES[2], QUALIFIER, VALUES[0]); + ht.put(put); + + put = new Put(ROWS[4]); + put.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]); + put.addColumn(FAMILIES[2], QUALIFIER, VALUES[2]); + ht.put(put); + + scan = new Scan().withStartRow(ROWS[4]); + scan.setReversed(true); + scan.addFamily(FAMILIES[1]); + scan.addFamily(FAMILIES[2]); + scan.readVersions(Integer.MAX_VALUE); + try (ResultScanner scanner = ht.getScanner(scan)) { + result = scanner.next(); + assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); + assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[4])); + assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[1]), ROWS[4])); + assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[1])); + assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[1]), VALUES[2])); + result = scanner.next(); + assertEquals(1, result.size(), "Expected 1 key but received " + result.size()); + assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[3])); + assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[0])); + } + } + } + + /** + * Tests reversed scan under multi regions + */ + @TestTemplate + public void testReversedScanUnderMultiRegions() throws Exception { + // Test Initialization. + byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY; + byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), + Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), Bytes.toBytes("006"), + Bytes.add(Bytes.toBytes("006"), Bytes.multiple(maxByteArray, 8)), Bytes.toBytes("007"), + Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)), Bytes.toBytes("008"), + Bytes.multiple(maxByteArray, 2) }; + TEST_UTIL.createTable(tableName, FAMILY, splitRows); + TEST_UTIL.waitUntilAllRegionsAssigned(tableName); + try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { + try (RegionLocator l = conn.getRegionLocator(tableName)) { + assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); + } + // Insert one row each region + int insertNum = splitRows.length; + for (byte[] splitRow : splitRows) { + Put put = new Put(splitRow); + put.addColumn(FAMILY, QUALIFIER, VALUE); + table.put(put); + } + + // scan forward + try (ResultScanner scanner = table.getScanner(new Scan())) { + int count = 0; + for (Result r : scanner) { + assertFalse(r.isEmpty()); + count++; + } + assertEquals(insertNum, count); + } + + // scan backward + Scan scan = new Scan(); + scan.setReversed(true); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + byte[] lastRow = null; + for (Result r : scanner) { + assertFalse(r.isEmpty()); + count++; + byte[] thisRow = r.getRow(); + if (lastRow != null) { + assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " + + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); + } + lastRow = thisRow; + } + assertEquals(insertNum, count); + } + } + } + + /** + * Tests reversed scan under multi regions + */ + @TestTemplate + public void testSmallReversedScanUnderMultiRegions() throws Exception { + // Test Initialization. + byte[][] splitRows = new byte[][] { Bytes.toBytes("000"), Bytes.toBytes("002"), + Bytes.toBytes("004"), Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010") }; + TEST_UTIL.createTable(tableName, FAMILY, splitRows); + TEST_UTIL.waitUntilAllRegionsAssigned(tableName); + try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { + try (RegionLocator l = conn.getRegionLocator(tableName)) { + assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); + } + for (byte[] splitRow : splitRows) { + Put put = new Put(splitRow); + put.addColumn(FAMILY, QUALIFIER, VALUE); + table.put(put); + + byte[] nextRow = Bytes.copy(splitRow); + nextRow[nextRow.length - 1]++; + + put = new Put(nextRow); + put.addColumn(FAMILY, QUALIFIER, VALUE); + table.put(put); + } + + // scan forward + try (ResultScanner scanner = table.getScanner(new Scan())) { + int count = 0; + for (Result r : scanner) { + assertTrue(!r.isEmpty()); + count++; + } + assertEquals(12, count); + } + + reverseScanTest(table, ReadType.STREAM); + reverseScanTest(table, ReadType.PREAD); + reverseScanTest(table, ReadType.DEFAULT); + } + } + + private void reverseScanTest(Table table, ReadType readType) throws IOException { + // scan backward + Scan scan = new Scan(); + scan.setReversed(true); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + byte[] lastRow = null; + for (Result r : scanner) { + assertTrue(!r.isEmpty()); + count++; + byte[] thisRow = r.getRow(); + if (lastRow != null) { + assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " + + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); + } + lastRow = thisRow; + } + assertEquals(12, count); + } + + scan = new Scan(); + scan.setReadType(readType); + scan.setReversed(true); + scan.withStartRow(Bytes.toBytes("002")); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + byte[] lastRow = null; + for (Result r : scanner) { + assertTrue(!r.isEmpty()); + count++; + byte[] thisRow = r.getRow(); + if (lastRow != null) { + assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " + + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); + } + lastRow = thisRow; + } + assertEquals(3, count); // 000 001 002 + } + + scan = new Scan(); + scan.setReadType(readType); + scan.setReversed(true); + scan.withStartRow(Bytes.toBytes("002")); + scan.withStopRow(Bytes.toBytes("000")); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + byte[] lastRow = null; + for (Result r : scanner) { + assertFalse(r.isEmpty()); + count++; + byte[] thisRow = r.getRow(); + if (lastRow != null) { + assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " + + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); + } + lastRow = thisRow; + } + assertEquals(2, count); // 001 002 + } + + scan = new Scan(); + scan.setReadType(readType); + scan.setReversed(true); + scan.withStartRow(Bytes.toBytes("001")); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + byte[] lastRow = null; + for (Result r : scanner) { + assertFalse(r.isEmpty()); + count++; + byte[] thisRow = r.getRow(); + if (lastRow != null) { + assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " + + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); + } + lastRow = thisRow; + } + assertEquals(2, count); // 000 001 + } + + scan = new Scan(); + scan.setReadType(readType); + scan.setReversed(true); + scan.withStartRow(Bytes.toBytes("000")); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + byte[] lastRow = null; + for (Result r : scanner) { + assertFalse(r.isEmpty()); + count++; + byte[] thisRow = r.getRow(); + if (lastRow != null) { + assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " + + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); + } + lastRow = thisRow; + } + assertEquals(1, count); // 000 + } + + scan = new Scan(); + scan.setReadType(readType); + scan.setReversed(true); + scan.withStartRow(Bytes.toBytes("006")); + scan.withStopRow(Bytes.toBytes("002")); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + byte[] lastRow = null; + for (Result r : scanner) { + assertFalse(r.isEmpty()); + count++; + byte[] thisRow = r.getRow(); + if (lastRow != null) { + assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " + + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); + } + lastRow = thisRow; + } + assertEquals(4, count); // 003 004 005 006 + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java index 4efedafbf2a1..b4017a23803a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java @@ -19,7 +19,6 @@ import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; -import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.jupiter.api.BeforeAll; @@ -41,6 +40,6 @@ public TestFromClientSide5(Class registryImpl, int @BeforeAll public static void setUpBeforeClass() throws Exception { - initialize(NoOpScanPolicyObserver.class, MultiRowMutationEndpoint.class); + initialize(MultiRowMutationEndpoint.class); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5WithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5WithCoprocessor.java index b7203a0b5858..c1785665d899 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5WithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5WithCoprocessor.java @@ -19,6 +19,7 @@ import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; +import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.jupiter.api.BeforeAll; @@ -41,6 +42,6 @@ public TestFromClientSide5WithCoprocessor(Class re @BeforeAll public static void setUpBeforeClass() throws Exception { - initialize(MultiRowMutationEndpoint.class); + initialize(NoOpScanPolicyObserver.class, MultiRowMutationEndpoint.class); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideCheckAndMutate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideCheckAndMutate.java new file mode 100644 index 000000000000..be10e310ddb7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideCheckAndMutate.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; +import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; + +@Tag(LargeTests.TAG) +@Tag(ClientTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: registryImpl={0}, numHedgedReqs={1}") +public class TestFromClientSideCheckAndMutate extends FromClientSideTestCheckAndMutate { + + public TestFromClientSideCheckAndMutate(Class registryImpl, + int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @BeforeAll + public static void setUpBeforeClass() throws Exception { + initialize(MultiRowMutationEndpoint.class); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideCheckAndMutateWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideCheckAndMutateWithCoprocessor.java new file mode 100644 index 000000000000..965f84b4a662 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideCheckAndMutateWithCoprocessor.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; +import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; +import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; + +@Tag(LargeTests.TAG) +@Tag(ClientTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: registryImpl={0}, numHedgedReqs={1}") +public class TestFromClientSideCheckAndMutateWithCoprocessor + extends FromClientSideTestCheckAndMutate { + + public TestFromClientSideCheckAndMutateWithCoprocessor( + Class registryImpl, int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @BeforeAll + public static void setUpBeforeClass() throws Exception { + initialize(NoOpScanPolicyObserver.class, MultiRowMutationEndpoint.class); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideMultiRowMutation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideMultiRowMutation.java new file mode 100644 index 000000000000..b94d225ccfac --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideMultiRowMutation.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; +import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; + +@Tag(LargeTests.TAG) +@Tag(ClientTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: registryImpl={0}, numHedgedReqs={1}") +public class TestFromClientSideMultiRowMutation extends FromClientSideTestMultiRowMutation { + + public TestFromClientSideMultiRowMutation(Class registryImpl, + int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @BeforeAll + public static void setUpBeforeClass() throws Exception { + initialize(MultiRowMutationEndpoint.class); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideMultiRowMutationWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideMultiRowMutationWithCoprocessor.java new file mode 100644 index 000000000000..8a7685e05e3b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideMultiRowMutationWithCoprocessor.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; +import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; +import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; + +@Tag(LargeTests.TAG) +@Tag(ClientTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: registryImpl={0}, numHedgedReqs={1}") +public class TestFromClientSideMultiRowMutationWithCoprocessor + extends FromClientSideTestMultiRowMutation { + + public TestFromClientSideMultiRowMutationWithCoprocessor( + Class registryImpl, int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @BeforeAll + public static void setUpBeforeClass() throws Exception { + initialize(NoOpScanPolicyObserver.class, MultiRowMutationEndpoint.class); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideReverseScan.java new file mode 100644 index 000000000000..e4d3c27a1782 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideReverseScan.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; +import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; + +@Tag(LargeTests.TAG) +@Tag(ClientTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: registryImpl={0}, numHedgedReqs={1}") +public class TestFromClientSideReverseScan extends FromClientSideTestReverseScan { + + public TestFromClientSideReverseScan(Class registryImpl, + int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @BeforeAll + public static void setUpBeforeClass() throws Exception { + initialize(MultiRowMutationEndpoint.class); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideReverseScanWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideReverseScanWithCoprocessor.java new file mode 100644 index 000000000000..03c6693f1330 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideReverseScanWithCoprocessor.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; +import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; +import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; + +@Tag(LargeTests.TAG) +@Tag(ClientTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: registryImpl={0}, numHedgedReqs={1}") +public class TestFromClientSideReverseScanWithCoprocessor extends FromClientSideTestReverseScan { + + public TestFromClientSideReverseScanWithCoprocessor( + Class registryImpl, int numHedgedReqs) { + super(registryImpl, numHedgedReqs); + } + + @BeforeAll + public static void setUpBeforeClass() throws Exception { + initialize(NoOpScanPolicyObserver.class, MultiRowMutationEndpoint.class); + } +}