001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.client; 019 020import static org.awaitility.Awaitility.await; 021import static org.hamcrest.MatcherAssert.assertThat; 022import static org.hamcrest.Matchers.hasItem; 023import static org.hamcrest.Matchers.instanceOf; 024import static org.junit.jupiter.api.Assertions.assertArrayEquals; 025import static org.junit.jupiter.api.Assertions.assertEquals; 026import static org.junit.jupiter.api.Assertions.assertFalse; 027import static org.junit.jupiter.api.Assertions.assertNotNull; 028import static org.junit.jupiter.api.Assertions.assertNull; 029import static org.junit.jupiter.api.Assertions.assertThrows; 030import static org.junit.jupiter.api.Assertions.assertTrue; 031import static org.junit.jupiter.api.Assertions.fail; 032 033import java.io.IOException; 034import java.time.Duration; 035import java.util.ArrayList; 036import java.util.Arrays; 037import java.util.LinkedList; 038import java.util.List; 039import java.util.Map; 040import java.util.NavigableMap; 041import java.util.concurrent.Callable; 042import java.util.concurrent.ExecutorService; 043import java.util.concurrent.Executors; 044import java.util.concurrent.atomic.AtomicReference; 045import org.apache.commons.lang3.ArrayUtils; 046import org.apache.hadoop.conf.Configuration; 047import org.apache.hadoop.fs.Path; 048import org.apache.hadoop.hbase.Cell; 049import org.apache.hadoop.hbase.CellScanner; 050import org.apache.hadoop.hbase.CellUtil; 051import org.apache.hadoop.hbase.CompareOperator; 052import org.apache.hadoop.hbase.DoNotRetryIOException; 053import org.apache.hadoop.hbase.HConstants; 054import org.apache.hadoop.hbase.HRegionLocation; 055import org.apache.hadoop.hbase.PrivateCellUtil; 056import org.apache.hadoop.hbase.ServerName; 057import org.apache.hadoop.hbase.TableName; 058import org.apache.hadoop.hbase.Waiter; 059import org.apache.hadoop.hbase.client.Scan.ReadType; 060import org.apache.hadoop.hbase.client.metrics.ScanMetrics; 061import org.apache.hadoop.hbase.filter.BinaryComparator; 062import org.apache.hadoop.hbase.filter.Filter; 063import org.apache.hadoop.hbase.filter.FilterList; 064import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; 065import org.apache.hadoop.hbase.filter.InclusiveStopFilter; 066import org.apache.hadoop.hbase.filter.KeyOnlyFilter; 067import org.apache.hadoop.hbase.filter.QualifierFilter; 068import org.apache.hadoop.hbase.filter.RegexStringComparator; 069import org.apache.hadoop.hbase.filter.RowFilter; 070import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; 071import org.apache.hadoop.hbase.filter.SubstringComparator; 072import org.apache.hadoop.hbase.filter.ValueFilter; 073import org.apache.hadoop.hbase.io.TimeRange; 074import org.apache.hadoop.hbase.io.hfile.BlockCache; 075import org.apache.hadoop.hbase.io.hfile.CacheConfig; 076import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; 077import org.apache.hadoop.hbase.regionserver.HRegion; 078import org.apache.hadoop.hbase.regionserver.HRegionServer; 079import org.apache.hadoop.hbase.regionserver.HStore; 080import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; 081import org.apache.hadoop.hbase.util.Bytes; 082import org.apache.hadoop.hbase.util.CommonFSUtils; 083import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 084import org.apache.hadoop.hbase.util.FSUtils; 085import org.junit.jupiter.api.Disabled; 086import org.junit.jupiter.api.TestTemplate; 087import org.slf4j.Logger; 088import org.slf4j.LoggerFactory; 089 090import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 091import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto; 092import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; 093import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; 094import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; 095import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse; 096 097/** 098 * Run tests that use the HBase clients; {@link Table}. Sets up the HBase mini cluster once at start 099 * and runs through all client tests. Each creates a table named for the method and does its stuff 100 * against that. Parameterized to run with different registry implementations. 101 */ 102public class FromClientSideTest5 extends FromClientSideTestBase { 103 104 protected FromClientSideTest5(Class<? extends ConnectionRegistry> registryImpl, 105 int numHedgedReqs) { 106 super(registryImpl, numHedgedReqs); 107 } 108 109 private static final Logger LOG = LoggerFactory.getLogger(FromClientSideTest5.class); 110 111 @TestTemplate 112 public void testGetClosestRowBefore() throws IOException, InterruptedException { 113 114 final byte[] firstRow = Bytes.toBytes("row111"); 115 final byte[] secondRow = Bytes.toBytes("row222"); 116 final byte[] thirdRow = Bytes.toBytes("row333"); 117 final byte[] forthRow = Bytes.toBytes("row444"); 118 final byte[] beforeFirstRow = Bytes.toBytes("row"); 119 final byte[] beforeSecondRow = Bytes.toBytes("row22"); 120 final byte[] beforeThirdRow = Bytes.toBytes("row33"); 121 final byte[] beforeForthRow = Bytes.toBytes("row44"); 122 123 try ( 124 Table table = TEST_UTIL.createTable(tableName, 125 new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024); 126 RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 127 128 // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow 129 // in Store.rowAtOrBeforeFromStoreFile 130 String regionName = locator.getAllRegionLocations().get(0).getRegion().getEncodedName(); 131 HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); 132 Put put1 = new Put(firstRow); 133 Put put2 = new Put(secondRow); 134 Put put3 = new Put(thirdRow); 135 Put put4 = new Put(forthRow); 136 byte[] one = new byte[] { 1 }; 137 byte[] two = new byte[] { 2 }; 138 byte[] three = new byte[] { 3 }; 139 byte[] four = new byte[] { 4 }; 140 141 put1.addColumn(HConstants.CATALOG_FAMILY, null, one); 142 put2.addColumn(HConstants.CATALOG_FAMILY, null, two); 143 put3.addColumn(HConstants.CATALOG_FAMILY, null, three); 144 put4.addColumn(HConstants.CATALOG_FAMILY, null, four); 145 table.put(put1); 146 table.put(put2); 147 table.put(put3); 148 table.put(put4); 149 region.flush(true); 150 151 Result result; 152 153 // Test before first that null is returned 154 result = getReverseScanResult(table, beforeFirstRow); 155 assertNull(result); 156 157 // Test at first that first is returned 158 result = getReverseScanResult(table, firstRow); 159 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 160 assertTrue(Bytes.equals(result.getRow(), firstRow)); 161 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one)); 162 163 // Test in between first and second that first is returned 164 result = getReverseScanResult(table, beforeSecondRow); 165 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 166 assertTrue(Bytes.equals(result.getRow(), firstRow)); 167 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one)); 168 169 // Test at second make sure second is returned 170 result = getReverseScanResult(table, secondRow); 171 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 172 assertTrue(Bytes.equals(result.getRow(), secondRow)); 173 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two)); 174 175 // Test in second and third, make sure second is returned 176 result = getReverseScanResult(table, beforeThirdRow); 177 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 178 assertTrue(Bytes.equals(result.getRow(), secondRow)); 179 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two)); 180 181 // Test at third make sure third is returned 182 result = getReverseScanResult(table, thirdRow); 183 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 184 assertTrue(Bytes.equals(result.getRow(), thirdRow)); 185 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three)); 186 187 // Test in third and forth, make sure third is returned 188 result = getReverseScanResult(table, beforeForthRow); 189 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 190 assertTrue(Bytes.equals(result.getRow(), thirdRow)); 191 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three)); 192 193 // Test at forth make sure forth is returned 194 result = getReverseScanResult(table, forthRow); 195 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 196 assertTrue(Bytes.equals(result.getRow(), forthRow)); 197 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four)); 198 199 // Test after forth make sure forth is returned 200 result = getReverseScanResult(table, Bytes.add(forthRow, one)); 201 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 202 assertTrue(Bytes.equals(result.getRow(), forthRow)); 203 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four)); 204 } 205 } 206 207 private Result getReverseScanResult(Table table, byte[] row) throws IOException { 208 Scan scan = new Scan().withStartRow(row); 209 scan.setReadType(ReadType.PREAD); 210 scan.setReversed(true); 211 scan.setCaching(1); 212 scan.addFamily(HConstants.CATALOG_FAMILY); 213 try (ResultScanner scanner = table.getScanner(scan)) { 214 return scanner.next(); 215 } 216 } 217 218 @TestTemplate 219 public void testMultiRowMutation() throws Exception { 220 LOG.info("Starting testMultiRowMutation"); 221 final byte[] ROW1 = Bytes.toBytes("testRow1"); 222 final byte[] ROW2 = Bytes.toBytes("testRow2"); 223 final byte[] ROW3 = Bytes.toBytes("testRow3"); 224 TEST_UTIL.createTable(tableName, FAMILY); 225 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 226 // Add initial data 227 t.batch(Arrays.asList(new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE), 228 new Put(ROW2).addColumn(FAMILY, QUALIFIER, Bytes.toBytes(1L)), 229 new Put(ROW3).addColumn(FAMILY, QUALIFIER, VALUE)), new Object[3]); 230 231 // Execute MultiRowMutation 232 Put put = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); 233 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put); 234 235 Delete delete = new Delete(ROW1); 236 MutationProto m2 = ProtobufUtil.toMutation(MutationType.DELETE, delete); 237 238 Increment increment = new Increment(ROW2).addColumn(FAMILY, QUALIFIER, 1L); 239 MutationProto m3 = ProtobufUtil.toMutation(MutationType.INCREMENT, increment); 240 241 Append append = new Append(ROW3).addColumn(FAMILY, QUALIFIER, VALUE); 242 MutationProto m4 = ProtobufUtil.toMutation(MutationType.APPEND, append); 243 244 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 245 mrmBuilder.addMutationRequest(m1); 246 mrmBuilder.addMutationRequest(m2); 247 mrmBuilder.addMutationRequest(m3); 248 mrmBuilder.addMutationRequest(m4); 249 250 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 251 MultiRowMutationService.BlockingInterface service = 252 MultiRowMutationService.newBlockingStub(channel); 253 MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); 254 255 // Assert 256 assertTrue(response.getProcessed()); 257 258 Result r = t.get(new Get(ROW)); 259 assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 260 261 r = t.get(new Get(ROW1)); 262 assertTrue(r.isEmpty()); 263 264 r = t.get(new Get(ROW2)); 265 assertEquals(2L, Bytes.toLong(r.getValue(FAMILY, QUALIFIER))); 266 267 r = t.get(new Get(ROW3)); 268 assertEquals(Bytes.toString(VALUE) + Bytes.toString(VALUE), 269 Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 270 } 271 } 272 273 @TestTemplate 274 public void testMultiRowMutationWithSingleConditionWhenConditionMatches() throws Exception { 275 final byte[] ROW1 = Bytes.toBytes("testRow1"); 276 final byte[] ROW2 = Bytes.toBytes("testRow2"); 277 final byte[] VALUE1 = Bytes.toBytes("testValue1"); 278 final byte[] VALUE2 = Bytes.toBytes("testValue2"); 279 TEST_UTIL.createTable(tableName, FAMILY); 280 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 281 // Add initial data 282 t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); 283 284 // Execute MultiRowMutation with conditions 285 Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); 286 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); 287 Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); 288 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); 289 Delete delete = new Delete(ROW2); 290 MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); 291 292 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 293 mrmBuilder.addMutationRequest(m1); 294 mrmBuilder.addMutationRequest(m2); 295 mrmBuilder.addMutationRequest(m3); 296 mrmBuilder.addCondition( 297 ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2, null)); 298 299 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 300 MultiRowMutationService.BlockingInterface service = 301 MultiRowMutationService.newBlockingStub(channel); 302 MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); 303 304 // Assert 305 assertTrue(response.getProcessed()); 306 307 Result r = t.get(new Get(ROW)); 308 assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 309 310 r = t.get(new Get(ROW1)); 311 assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 312 313 r = t.get(new Get(ROW2)); 314 assertTrue(r.isEmpty()); 315 } 316 } 317 318 @TestTemplate 319 public void testMultiRowMutationWithSingleConditionWhenConditionNotMatch() throws Exception { 320 final byte[] ROW1 = Bytes.toBytes("testRow1"); 321 final byte[] ROW2 = Bytes.toBytes("testRow2"); 322 final byte[] VALUE1 = Bytes.toBytes("testValue1"); 323 final byte[] VALUE2 = Bytes.toBytes("testValue2"); 324 TEST_UTIL.createTable(tableName, FAMILY); 325 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 326 // Add initial data 327 t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); 328 329 // Execute MultiRowMutation with conditions 330 Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); 331 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); 332 Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); 333 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); 334 Delete delete = new Delete(ROW2); 335 MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); 336 337 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 338 mrmBuilder.addMutationRequest(m1); 339 mrmBuilder.addMutationRequest(m2); 340 mrmBuilder.addMutationRequest(m3); 341 mrmBuilder.addCondition( 342 ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE1, null)); 343 344 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 345 MultiRowMutationService.BlockingInterface service = 346 MultiRowMutationService.newBlockingStub(channel); 347 MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); 348 349 // Assert 350 assertFalse(response.getProcessed()); 351 352 Result r = t.get(new Get(ROW)); 353 assertTrue(r.isEmpty()); 354 355 r = t.get(new Get(ROW1)); 356 assertTrue(r.isEmpty()); 357 358 r = t.get(new Get(ROW2)); 359 assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 360 } 361 } 362 363 @TestTemplate 364 public void testMultiRowMutationWithMultipleConditionsWhenConditionsMatch() throws Exception { 365 final byte[] ROW1 = Bytes.toBytes("testRow1"); 366 final byte[] ROW2 = Bytes.toBytes("testRow2"); 367 final byte[] VALUE1 = Bytes.toBytes("testValue1"); 368 final byte[] VALUE2 = Bytes.toBytes("testValue2"); 369 TEST_UTIL.createTable(tableName, FAMILY); 370 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 371 // Add initial data 372 t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); 373 374 // Execute MultiRowMutation with conditions 375 Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); 376 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); 377 Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); 378 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); 379 Delete delete = new Delete(ROW2); 380 MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); 381 382 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 383 mrmBuilder.addMutationRequest(m1); 384 mrmBuilder.addMutationRequest(m2); 385 mrmBuilder.addMutationRequest(m3); 386 mrmBuilder.addCondition( 387 ProtobufUtil.toCondition(ROW, FAMILY, QUALIFIER, CompareOperator.EQUAL, null, null)); 388 mrmBuilder.addCondition( 389 ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2, null)); 390 391 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 392 MultiRowMutationService.BlockingInterface service = 393 MultiRowMutationService.newBlockingStub(channel); 394 MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); 395 396 // Assert 397 assertTrue(response.getProcessed()); 398 399 Result r = t.get(new Get(ROW)); 400 assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 401 402 r = t.get(new Get(ROW1)); 403 assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 404 405 r = t.get(new Get(ROW2)); 406 assertTrue(r.isEmpty()); 407 } 408 } 409 410 @TestTemplate 411 public void testMultiRowMutationWithMultipleConditionsWhenConditionsNotMatch() throws Exception { 412 final byte[] ROW1 = Bytes.toBytes("testRow1"); 413 final byte[] ROW2 = Bytes.toBytes("testRow2"); 414 final byte[] VALUE1 = Bytes.toBytes("testValue1"); 415 final byte[] VALUE2 = Bytes.toBytes("testValue2"); 416 TEST_UTIL.createTable(tableName, FAMILY); 417 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 418 // Add initial data 419 t.put(new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2)); 420 421 // Execute MultiRowMutation with conditions 422 Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); 423 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); 424 Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); 425 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); 426 Delete delete = new Delete(ROW2); 427 MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); 428 429 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 430 mrmBuilder.addMutationRequest(m1); 431 mrmBuilder.addMutationRequest(m2); 432 mrmBuilder.addMutationRequest(m3); 433 mrmBuilder.addCondition( 434 ProtobufUtil.toCondition(ROW1, FAMILY, QUALIFIER, CompareOperator.EQUAL, null, null)); 435 mrmBuilder.addCondition( 436 ProtobufUtil.toCondition(ROW2, FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE1, null)); 437 438 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 439 MultiRowMutationService.BlockingInterface service = 440 MultiRowMutationService.newBlockingStub(channel); 441 MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); 442 443 // Assert 444 assertFalse(response.getProcessed()); 445 446 Result r = t.get(new Get(ROW)); 447 assertTrue(r.isEmpty()); 448 449 r = t.get(new Get(ROW1)); 450 assertTrue(r.isEmpty()); 451 452 r = t.get(new Get(ROW2)); 453 assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 454 } 455 } 456 457 @TestTemplate 458 public void testMultiRowMutationWithFilterConditionWhenConditionMatches() throws Exception { 459 final byte[] ROW1 = Bytes.toBytes("testRow1"); 460 final byte[] ROW2 = Bytes.toBytes("testRow2"); 461 final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2"); 462 final byte[] VALUE1 = Bytes.toBytes("testValue1"); 463 final byte[] VALUE2 = Bytes.toBytes("testValue2"); 464 final byte[] VALUE3 = Bytes.toBytes("testValue3"); 465 TEST_UTIL.createTable(tableName, FAMILY); 466 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 467 // Add initial data 468 t.put( 469 new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2).addColumn(FAMILY, QUALIFIER2, VALUE3)); 470 471 // Execute MultiRowMutation with conditions 472 Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); 473 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); 474 Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); 475 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); 476 Delete delete = new Delete(ROW2); 477 MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); 478 479 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 480 mrmBuilder.addMutationRequest(m1); 481 mrmBuilder.addMutationRequest(m2); 482 mrmBuilder.addMutationRequest(m3); 483 mrmBuilder.addCondition(ProtobufUtil.toCondition(ROW2, 484 new FilterList( 485 new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2), 486 new SingleColumnValueFilter(FAMILY, QUALIFIER2, CompareOperator.EQUAL, VALUE3)), 487 null)); 488 489 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 490 MultiRowMutationService.BlockingInterface service = 491 MultiRowMutationService.newBlockingStub(channel); 492 MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); 493 494 // Assert 495 assertTrue(response.getProcessed()); 496 497 Result r = t.get(new Get(ROW)); 498 assertEquals(Bytes.toString(VALUE), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 499 500 r = t.get(new Get(ROW1)); 501 assertEquals(Bytes.toString(VALUE1), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 502 503 r = t.get(new Get(ROW2)); 504 assertTrue(r.isEmpty()); 505 } 506 } 507 508 @TestTemplate 509 public void testMultiRowMutationWithFilterConditionWhenConditionNotMatch() throws Exception { 510 final byte[] ROW1 = Bytes.toBytes("testRow1"); 511 final byte[] ROW2 = Bytes.toBytes("testRow2"); 512 final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2"); 513 final byte[] VALUE1 = Bytes.toBytes("testValue1"); 514 final byte[] VALUE2 = Bytes.toBytes("testValue2"); 515 final byte[] VALUE3 = Bytes.toBytes("testValue3"); 516 TEST_UTIL.createTable(tableName, FAMILY); 517 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 518 // Add initial data 519 t.put( 520 new Put(ROW2).addColumn(FAMILY, QUALIFIER, VALUE2).addColumn(FAMILY, QUALIFIER2, VALUE3)); 521 522 // Execute MultiRowMutation with conditions 523 Put put1 = new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE); 524 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put1); 525 Put put2 = new Put(ROW1).addColumn(FAMILY, QUALIFIER, VALUE1); 526 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, put2); 527 Delete delete = new Delete(ROW2); 528 MutationProto m3 = ProtobufUtil.toMutation(MutationType.DELETE, delete); 529 530 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 531 mrmBuilder.addMutationRequest(m1); 532 mrmBuilder.addMutationRequest(m2); 533 mrmBuilder.addMutationRequest(m3); 534 mrmBuilder.addCondition(ProtobufUtil.toCondition(ROW2, 535 new FilterList( 536 new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOperator.EQUAL, VALUE2), 537 new SingleColumnValueFilter(FAMILY, QUALIFIER2, CompareOperator.EQUAL, VALUE2)), 538 null)); 539 540 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 541 MultiRowMutationService.BlockingInterface service = 542 MultiRowMutationService.newBlockingStub(channel); 543 MutateRowsResponse response = service.mutateRows(null, mrmBuilder.build()); 544 545 // Assert 546 assertFalse(response.getProcessed()); 547 548 Result r = t.get(new Get(ROW)); 549 assertTrue(r.isEmpty()); 550 551 r = t.get(new Get(ROW1)); 552 assertTrue(r.isEmpty()); 553 554 r = t.get(new Get(ROW2)); 555 assertEquals(Bytes.toString(VALUE2), Bytes.toString(r.getValue(FAMILY, QUALIFIER))); 556 } 557 } 558 559 @TestTemplate 560 public void testRowMutations() throws Exception { 561 LOG.info("Starting testRowMutations"); 562 TEST_UTIL.createTable(tableName, FAMILY); 563 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 564 byte[][] QUALIFIERS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"), 565 Bytes.toBytes("c"), Bytes.toBytes("d") }; 566 567 // Test for Put operations 568 RowMutations arm = new RowMutations(ROW); 569 Put p = new Put(ROW); 570 p.addColumn(FAMILY, QUALIFIERS[0], VALUE); 571 arm.add(p); 572 Result r = t.mutateRow(arm); 573 assertTrue(r.getExists()); 574 assertTrue(r.isEmpty()); 575 576 Get g = new Get(ROW); 577 r = t.get(g); 578 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0]))); 579 580 // Test for Put and Delete operations 581 arm = new RowMutations(ROW); 582 p = new Put(ROW); 583 p.addColumn(FAMILY, QUALIFIERS[1], VALUE); 584 arm.add(p); 585 Delete d = new Delete(ROW); 586 d.addColumns(FAMILY, QUALIFIERS[0]); 587 arm.add(d); 588 // TODO: Trying mutateRow again. The batch was failing with a one try only. 589 r = t.mutateRow(arm); 590 assertTrue(r.getExists()); 591 assertTrue(r.isEmpty()); 592 593 r = t.get(g); 594 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1]))); 595 assertNull(r.getValue(FAMILY, QUALIFIERS[0])); 596 597 // Test for Increment and Append operations 598 arm = new RowMutations(ROW); 599 arm.add(Arrays.asList(new Put(ROW).addColumn(FAMILY, QUALIFIERS[0], VALUE), 600 new Delete(ROW).addColumns(FAMILY, QUALIFIERS[1]), 601 new Increment(ROW).addColumn(FAMILY, QUALIFIERS[2], 5L), 602 new Append(ROW).addColumn(FAMILY, QUALIFIERS[3], Bytes.toBytes("abc")))); 603 r = t.mutateRow(arm); 604 assertTrue(r.getExists()); 605 assertEquals(5L, Bytes.toLong(r.getValue(FAMILY, QUALIFIERS[2]))); 606 assertEquals("abc", Bytes.toString(r.getValue(FAMILY, QUALIFIERS[3]))); 607 608 g = new Get(ROW); 609 r = t.get(g); 610 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0]))); 611 assertNull(r.getValue(FAMILY, QUALIFIERS[1])); 612 assertEquals(5L, Bytes.toLong(r.getValue(FAMILY, QUALIFIERS[2]))); 613 assertEquals("abc", Bytes.toString(r.getValue(FAMILY, QUALIFIERS[3]))); 614 615 // Test that we get a region level exception 616 RowMutations nceRm = new RowMutations(ROW); 617 p = new Put(ROW); 618 p.addColumn(new byte[] { 'b', 'o', 'g', 'u', 's' }, QUALIFIERS[0], VALUE); 619 nceRm.add(p); 620 Exception e = assertThrows(Exception.class, () -> t.mutateRow(nceRm), 621 "Expected NoSuchColumnFamilyException"); 622 if (!(e instanceof NoSuchColumnFamilyException)) { 623 assertThat(e, instanceOf(RetriesExhaustedWithDetailsException.class)); 624 List<Throwable> causes = ((RetriesExhaustedWithDetailsException) e).getCauses(); 625 assertThat(causes, hasItem(instanceOf(NoSuchColumnFamilyException.class))); 626 } 627 } 628 } 629 630 @TestTemplate 631 public void testBatchAppendWithReturnResultFalse() throws Exception { 632 LOG.info("Starting testBatchAppendWithReturnResultFalse"); 633 TEST_UTIL.createTable(tableName, FAMILY); 634 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 635 Append append1 = new Append(Bytes.toBytes("row1")); 636 append1.setReturnResults(false); 637 append1.addColumn(FAMILY, Bytes.toBytes("f1"), Bytes.toBytes("value1")); 638 Append append2 = new Append(Bytes.toBytes("row1")); 639 append2.setReturnResults(false); 640 append2.addColumn(FAMILY, Bytes.toBytes("f1"), Bytes.toBytes("value2")); 641 List<Append> appends = new ArrayList<>(); 642 appends.add(append1); 643 appends.add(append2); 644 Object[] results = new Object[2]; 645 table.batch(appends, results); 646 assertEquals(2, results.length); 647 for (Object r : results) { 648 Result result = (Result) r; 649 assertTrue(result.isEmpty()); 650 } 651 } 652 } 653 654 @TestTemplate 655 public void testAppend() throws Exception { 656 LOG.info("Starting testAppend"); 657 TEST_UTIL.createTable(tableName, FAMILY); 658 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 659 byte[] v1 = Bytes.toBytes("42"); 660 byte[] v2 = Bytes.toBytes("23"); 661 byte[][] QUALIFIERS = 662 new byte[][] { Bytes.toBytes("b"), Bytes.toBytes("a"), Bytes.toBytes("c") }; 663 Append a = new Append(ROW); 664 a.addColumn(FAMILY, QUALIFIERS[0], v1); 665 a.addColumn(FAMILY, QUALIFIERS[1], v2); 666 a.setReturnResults(false); 667 assertEmptyResult(t.append(a)); 668 669 a = new Append(ROW); 670 a.addColumn(FAMILY, QUALIFIERS[0], v2); 671 a.addColumn(FAMILY, QUALIFIERS[1], v1); 672 a.addColumn(FAMILY, QUALIFIERS[2], v2); 673 Result r = t.append(a); 674 assertEquals(0, Bytes.compareTo(Bytes.add(v1, v2), r.getValue(FAMILY, QUALIFIERS[0]))); 675 assertEquals(0, Bytes.compareTo(Bytes.add(v2, v1), r.getValue(FAMILY, QUALIFIERS[1]))); 676 // QUALIFIERS[2] previously not exist, verify both value and timestamp are correct 677 assertEquals(0, Bytes.compareTo(v2, r.getValue(FAMILY, QUALIFIERS[2]))); 678 assertEquals(r.getColumnLatestCell(FAMILY, QUALIFIERS[0]).getTimestamp(), 679 r.getColumnLatestCell(FAMILY, QUALIFIERS[2]).getTimestamp()); 680 } 681 } 682 683 private List<Result> doAppend(final boolean walUsed) throws IOException { 684 LOG.info("Starting testAppend, walUsed is " + walUsed); 685 TableName tableName = TableName.valueOf( 686 this.tableName.getNameAsString() + (walUsed ? "_testAppendWithWAL" : "testAppendWithoutWAL")); 687 TEST_UTIL.createTable(tableName, FAMILY); 688 try (Connection conn = getConnection(); Table t = conn.getTable(tableName)) { 689 final byte[] row1 = Bytes.toBytes("c"); 690 final byte[] row2 = Bytes.toBytes("b"); 691 final byte[] row3 = Bytes.toBytes("a"); 692 final byte[] qual = Bytes.toBytes("qual"); 693 Put put_0 = new Put(row2); 694 put_0.addColumn(FAMILY, qual, Bytes.toBytes("put")); 695 Put put_1 = new Put(row3); 696 put_1.addColumn(FAMILY, qual, Bytes.toBytes("put")); 697 Append append_0 = new Append(row1); 698 append_0.addColumn(FAMILY, qual, Bytes.toBytes("i")); 699 Append append_1 = new Append(row1); 700 append_1.addColumn(FAMILY, qual, Bytes.toBytes("k")); 701 Append append_2 = new Append(row1); 702 append_2.addColumn(FAMILY, qual, Bytes.toBytes("e")); 703 if (!walUsed) { 704 append_2.setDurability(Durability.SKIP_WAL); 705 } 706 Append append_3 = new Append(row1); 707 append_3.addColumn(FAMILY, qual, Bytes.toBytes("a")); 708 Scan s = new Scan(); 709 s.setCaching(1); 710 t.append(append_0); 711 t.put(put_0); 712 t.put(put_1); 713 List<Result> results = new LinkedList<>(); 714 try (ResultScanner scanner = t.getScanner(s)) { 715 // get one row(should be row3) from the scanner to make sure that we have send a request to 716 // region server, which means we have already set the read point, so later we should not see 717 // the new appended values. 718 Result r = scanner.next(); 719 assertNotNull(r); 720 results.add(r); 721 t.append(append_1); 722 t.append(append_2); 723 t.append(append_3); 724 for (;;) { 725 r = scanner.next(); 726 if (r == null) { 727 break; 728 } 729 results.add(r); 730 } 731 } 732 return results; 733 } finally { 734 TEST_UTIL.deleteTable(tableName); 735 } 736 } 737 738 @TestTemplate 739 public void testAppendWithoutWAL() throws Exception { 740 List<Result> resultsWithWal = doAppend(true); 741 List<Result> resultsWithoutWal = doAppend(false); 742 assertEquals(resultsWithWal.size(), resultsWithoutWal.size()); 743 for (int i = 0; i < resultsWithWal.size(); ++i) { 744 Result resultWithWal = resultsWithWal.get(i); 745 Result resultWithoutWal = resultsWithoutWal.get(i); 746 assertEquals(resultWithWal.rawCells().length, resultWithoutWal.rawCells().length); 747 for (int j = 0; j < resultWithWal.rawCells().length; ++j) { 748 Cell cellWithWal = resultWithWal.rawCells()[j]; 749 Cell cellWithoutWal = resultWithoutWal.rawCells()[j]; 750 assertArrayEquals(CellUtil.cloneRow(cellWithWal), CellUtil.cloneRow(cellWithoutWal)); 751 assertArrayEquals(CellUtil.cloneFamily(cellWithWal), CellUtil.cloneFamily(cellWithoutWal)); 752 assertArrayEquals(CellUtil.cloneQualifier(cellWithWal), 753 CellUtil.cloneQualifier(cellWithoutWal)); 754 assertArrayEquals(CellUtil.cloneValue(cellWithWal), CellUtil.cloneValue(cellWithoutWal)); 755 } 756 } 757 } 758 759 @TestTemplate 760 public void testClientPoolRoundRobin() throws IOException { 761 int poolSize = 3; 762 int numVersions = poolSize * 2; 763 Configuration conf = TEST_UTIL.getConfiguration(); 764 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin"); 765 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); 766 TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, Integer.MAX_VALUE); 767 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 768 final long ts = EnvironmentEdgeManager.currentTime(); 769 Get get = new Get(ROW); 770 get.addColumn(FAMILY, QUALIFIER); 771 get.readAllVersions(); 772 773 for (int versions = 1; versions <= numVersions; versions++) { 774 Put put = new Put(ROW); 775 put.addColumn(FAMILY, QUALIFIER, ts + versions, VALUE); 776 table.put(put); 777 778 Result result = table.get(get); 779 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY).get(QUALIFIER); 780 781 assertEquals(versions, navigableMap.size(), "The number of versions of '" 782 + Bytes.toString(FAMILY) + ":" + Bytes.toString(QUALIFIER) + " did not match"); 783 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) { 784 assertTrue(Bytes.equals(VALUE, entry.getValue()), 785 "The value at time " + entry.getKey() + " did not match what was put"); 786 } 787 } 788 } 789 } 790 791 @Disabled("Flakey: HBASE-8989") 792 @TestTemplate 793 public void testClientPoolThreadLocal() throws IOException { 794 int poolSize = Integer.MAX_VALUE; 795 int numVersions = 3; 796 Configuration conf = TEST_UTIL.getConfiguration(); 797 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local"); 798 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); 799 TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, 3); 800 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 801 final long ts = EnvironmentEdgeManager.currentTime(); 802 final Get get = new Get(ROW); 803 get.addColumn(FAMILY, QUALIFIER); 804 get.readAllVersions(); 805 806 for (int versions = 1; versions <= numVersions; versions++) { 807 Put put = new Put(ROW); 808 put.addColumn(FAMILY, QUALIFIER, ts + versions, VALUE); 809 table.put(put); 810 811 Result result = table.get(get); 812 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY).get(QUALIFIER); 813 814 assertEquals(versions, navigableMap.size(), "The number of versions of '" 815 + Bytes.toString(FAMILY) + ":" + Bytes.toString(QUALIFIER) + " did not match"); 816 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) { 817 assertTrue(Bytes.equals(VALUE, entry.getValue()), 818 "The value at time " + entry.getKey() + " did not match what was put"); 819 } 820 } 821 822 final Object waitLock = new Object(); 823 ExecutorService executorService = Executors.newFixedThreadPool(numVersions); 824 final AtomicReference<AssertionError> error = new AtomicReference<>(null); 825 for (int versions = numVersions; versions < numVersions * 2; versions++) { 826 final int versionsCopy = versions; 827 executorService.submit((Callable<Void>) () -> { 828 try { 829 Put put = new Put(ROW); 830 put.addColumn(FAMILY, QUALIFIER, ts + versionsCopy, VALUE); 831 table.put(put); 832 833 Result result = table.get(get); 834 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY).get(QUALIFIER); 835 836 assertEquals(versionsCopy, navigableMap.size(), 837 "The number of versions of '" + Bytes.toString(FAMILY) + ":" 838 + Bytes.toString(QUALIFIER) + " did not match " + versionsCopy); 839 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) { 840 assertTrue(Bytes.equals(VALUE, entry.getValue()), 841 "The value at time " + entry.getKey() + " did not match what was put"); 842 } 843 synchronized (waitLock) { 844 waitLock.wait(); 845 } 846 } catch (Exception ignored) { 847 } catch (AssertionError e) { 848 // the error happens in a thread, it won't fail the test, 849 // need to pass it to the caller for proper handling. 850 error.set(e); 851 LOG.error(e.toString(), e); 852 } 853 854 return null; 855 }); 856 } 857 synchronized (waitLock) { 858 waitLock.notifyAll(); 859 } 860 executorService.shutdownNow(); 861 assertNull(error.get()); 862 } 863 } 864 865 @TestTemplate 866 public void testCheckAndPut() throws IOException { 867 final byte[] anotherrow = Bytes.toBytes("anotherrow"); 868 final byte[] value2 = Bytes.toBytes("abcd"); 869 TEST_UTIL.createTable(tableName, FAMILY); 870 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 871 Put put1 = new Put(ROW); 872 put1.addColumn(FAMILY, QUALIFIER, VALUE); 873 874 // row doesn't exist, so using non-null value should be considered "not match". 875 boolean ok = 876 table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(VALUE).thenPut(put1); 877 assertFalse(ok); 878 879 // row doesn't exist, so using "ifNotExists" should be considered "match". 880 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); 881 assertTrue(ok); 882 883 // row now exists, so using "ifNotExists" should be considered "not match". 884 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); 885 assertFalse(ok); 886 887 Put put2 = new Put(ROW); 888 put2.addColumn(FAMILY, QUALIFIER, value2); 889 890 // row now exists, use the matching value to check 891 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(VALUE).thenPut(put2); 892 assertTrue(ok); 893 894 Put put3 = new Put(anotherrow); 895 put3.addColumn(FAMILY, QUALIFIER, VALUE); 896 897 // try to do CheckAndPut on different rows 898 try { 899 table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(value2).thenPut(put3); 900 fail("trying to check and modify different rows should have failed."); 901 } catch (Exception ignored) { 902 } 903 } 904 } 905 906 @TestTemplate 907 public void testCheckAndMutateWithTimeRange() throws IOException { 908 TEST_UTIL.createTable(tableName, FAMILY); 909 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 910 final long ts = EnvironmentEdgeManager.currentTime() / 2; 911 Put put = new Put(ROW); 912 put.addColumn(FAMILY, QUALIFIER, ts, VALUE); 913 914 boolean ok = 915 table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put); 916 assertTrue(ok); 917 918 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 919 .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenPut(put); 920 assertFalse(ok); 921 922 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 923 .timeRange(TimeRange.from(ts + 10000)).ifEquals(VALUE).thenPut(put); 924 assertFalse(ok); 925 926 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 927 .timeRange(TimeRange.between(ts + 10000, ts + 20000)).ifEquals(VALUE).thenPut(put); 928 assertFalse(ok); 929 930 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.until(ts)) 931 .ifEquals(VALUE).thenPut(put); 932 assertFalse(ok); 933 934 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) 935 .ifEquals(VALUE).thenPut(put); 936 assertTrue(ok); 937 938 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.from(ts)) 939 .ifEquals(VALUE).thenPut(put); 940 assertTrue(ok); 941 942 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 943 .timeRange(TimeRange.between(ts, ts + 20000)).ifEquals(VALUE).thenPut(put); 944 assertTrue(ok); 945 946 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 947 .timeRange(TimeRange.until(ts + 10000)).ifEquals(VALUE).thenPut(put); 948 assertTrue(ok); 949 950 RowMutations rm = new RowMutations(ROW).add((Mutation) put); 951 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 952 .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenMutate(rm); 953 assertFalse(ok); 954 955 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) 956 .ifEquals(VALUE).thenMutate(rm); 957 assertTrue(ok); 958 959 Delete delete = new Delete(ROW).addColumn(FAMILY, QUALIFIER); 960 961 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 962 .timeRange(TimeRange.at(ts + 10000)).ifEquals(VALUE).thenDelete(delete); 963 assertFalse(ok); 964 965 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).timeRange(TimeRange.at(ts)) 966 .ifEquals(VALUE).thenDelete(delete); 967 assertTrue(ok); 968 } 969 } 970 971 @TestTemplate 972 public void testCheckAndPutWithCompareOp() throws IOException { 973 final byte[] value1 = Bytes.toBytes("aaaa"); 974 final byte[] value2 = Bytes.toBytes("bbbb"); 975 final byte[] value3 = Bytes.toBytes("cccc"); 976 final byte[] value4 = Bytes.toBytes("dddd"); 977 TEST_UTIL.createTable(tableName, FAMILY); 978 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 979 Put put2 = new Put(ROW); 980 put2.addColumn(FAMILY, QUALIFIER, value2); 981 982 Put put3 = new Put(ROW); 983 put3.addColumn(FAMILY, QUALIFIER, value3); 984 985 // row doesn't exist, so using "ifNotExists" should be considered "match". 986 boolean ok = 987 table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put2); 988 assertTrue(ok); 989 990 // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL 991 // turns out "match" 992 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 993 .ifMatches(CompareOperator.GREATER, value1).thenPut(put2); 994 assertFalse(ok); 995 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 996 .ifMatches(CompareOperator.EQUAL, value1).thenPut(put2); 997 assertFalse(ok); 998 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 999 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenPut(put2); 1000 assertFalse(ok); 1001 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1002 .ifMatches(CompareOperator.LESS, value1).thenPut(put2); 1003 assertTrue(ok); 1004 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1005 .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenPut(put2); 1006 assertTrue(ok); 1007 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1008 .ifMatches(CompareOperator.NOT_EQUAL, value1).thenPut(put3); 1009 assertTrue(ok); 1010 1011 // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL 1012 // turns out "match" 1013 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1014 .ifMatches(CompareOperator.LESS, value4).thenPut(put3); 1015 assertFalse(ok); 1016 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1017 .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenPut(put3); 1018 assertFalse(ok); 1019 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1020 .ifMatches(CompareOperator.EQUAL, value4).thenPut(put3); 1021 assertFalse(ok); 1022 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1023 .ifMatches(CompareOperator.GREATER, value4).thenPut(put3); 1024 assertTrue(ok); 1025 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1026 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenPut(put3); 1027 assertTrue(ok); 1028 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1029 .ifMatches(CompareOperator.NOT_EQUAL, value4).thenPut(put2); 1030 assertTrue(ok); 1031 1032 // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL 1033 // turns out "match" 1034 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1035 .ifMatches(CompareOperator.GREATER, value2).thenPut(put2); 1036 assertFalse(ok); 1037 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1038 .ifMatches(CompareOperator.NOT_EQUAL, value2).thenPut(put2); 1039 assertFalse(ok); 1040 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1041 .ifMatches(CompareOperator.LESS, value2).thenPut(put2); 1042 assertFalse(ok); 1043 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1044 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenPut(put2); 1045 assertTrue(ok); 1046 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1047 .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenPut(put2); 1048 assertTrue(ok); 1049 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1050 .ifMatches(CompareOperator.EQUAL, value2).thenPut(put3); 1051 assertTrue(ok); 1052 } 1053 } 1054 1055 @TestTemplate 1056 public void testCheckAndDelete() throws IOException { 1057 final byte[] value1 = Bytes.toBytes("aaaa"); 1058 TEST_UTIL.createTable(tableName, FAMILY); 1059 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 1060 Put put = new Put(ROW); 1061 put.addColumn(FAMILY, QUALIFIER, value1); 1062 table.put(put); 1063 1064 Delete delete = new Delete(ROW); 1065 delete.addColumns(FAMILY, QUALIFIER); 1066 1067 boolean ok = 1068 table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(value1).thenDelete(delete); 1069 assertTrue(ok); 1070 } 1071 } 1072 1073 @TestTemplate 1074 public void testCheckAndDeleteWithCompareOp() throws IOException { 1075 final byte[] value1 = Bytes.toBytes("aaaa"); 1076 final byte[] value2 = Bytes.toBytes("bbbb"); 1077 final byte[] value3 = Bytes.toBytes("cccc"); 1078 final byte[] value4 = Bytes.toBytes("dddd"); 1079 TEST_UTIL.createTable(tableName, FAMILY); 1080 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 1081 Put put2 = new Put(ROW); 1082 put2.addColumn(FAMILY, QUALIFIER, value2); 1083 table.put(put2); 1084 1085 Put put3 = new Put(ROW); 1086 put3.addColumn(FAMILY, QUALIFIER, value3); 1087 1088 Delete delete = new Delete(ROW); 1089 delete.addColumns(FAMILY, QUALIFIER); 1090 1091 // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL 1092 // turns out "match" 1093 boolean ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1094 .ifMatches(CompareOperator.GREATER, value1).thenDelete(delete); 1095 assertFalse(ok); 1096 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1097 .ifMatches(CompareOperator.EQUAL, value1).thenDelete(delete); 1098 assertFalse(ok); 1099 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1100 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenDelete(delete); 1101 assertFalse(ok); 1102 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1103 .ifMatches(CompareOperator.LESS, value1).thenDelete(delete); 1104 assertTrue(ok); 1105 table.put(put2); 1106 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1107 .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenDelete(delete); 1108 assertTrue(ok); 1109 table.put(put2); 1110 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1111 .ifMatches(CompareOperator.NOT_EQUAL, value1).thenDelete(delete); 1112 assertTrue(ok); 1113 1114 // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL 1115 // turns out "match" 1116 table.put(put3); 1117 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1118 .ifMatches(CompareOperator.LESS, value4).thenDelete(delete); 1119 assertFalse(ok); 1120 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1121 .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenDelete(delete); 1122 assertFalse(ok); 1123 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1124 .ifMatches(CompareOperator.EQUAL, value4).thenDelete(delete); 1125 assertFalse(ok); 1126 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1127 .ifMatches(CompareOperator.GREATER, value4).thenDelete(delete); 1128 assertTrue(ok); 1129 table.put(put3); 1130 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1131 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenDelete(delete); 1132 assertTrue(ok); 1133 table.put(put3); 1134 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1135 .ifMatches(CompareOperator.NOT_EQUAL, value4).thenDelete(delete); 1136 assertTrue(ok); 1137 1138 // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL 1139 // turns out "match" 1140 table.put(put2); 1141 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1142 .ifMatches(CompareOperator.GREATER, value2).thenDelete(delete); 1143 assertFalse(ok); 1144 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1145 .ifMatches(CompareOperator.NOT_EQUAL, value2).thenDelete(delete); 1146 assertFalse(ok); 1147 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1148 .ifMatches(CompareOperator.LESS, value2).thenDelete(delete); 1149 assertFalse(ok); 1150 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1151 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenDelete(delete); 1152 assertTrue(ok); 1153 table.put(put2); 1154 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1155 .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenDelete(delete); 1156 assertTrue(ok); 1157 table.put(put2); 1158 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 1159 .ifMatches(CompareOperator.EQUAL, value2).thenDelete(delete); 1160 assertTrue(ok); 1161 } 1162 } 1163 1164 /** 1165 * Test ScanMetrics 1166 */ 1167 @TestTemplate 1168 @SuppressWarnings({ "unused", "checkstyle:EmptyBlock" }) 1169 public void testScanMetrics() throws Exception { 1170 // Set up test table: 1171 // Create table: 1172 TEST_UTIL.createTable(tableName, FAMILY); 1173 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 1174 int numOfRegions; 1175 try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 1176 numOfRegions = r.getStartKeys().length; 1177 } 1178 // Create 3 rows in the table, with rowkeys starting with "zzz*" so that 1179 // scan are forced to hit all the regions. 1180 Put put1 = new Put(Bytes.toBytes("zzz1")); 1181 put1.addColumn(FAMILY, QUALIFIER, VALUE); 1182 Put put2 = new Put(Bytes.toBytes("zzz2")); 1183 put2.addColumn(FAMILY, QUALIFIER, VALUE); 1184 Put put3 = new Put(Bytes.toBytes("zzz3")); 1185 put3.addColumn(FAMILY, QUALIFIER, VALUE); 1186 ht.put(Arrays.asList(put1, put2, put3)); 1187 1188 Scan scan1 = new Scan(); 1189 int numRecords = 0; 1190 try (ResultScanner scanner = ht.getScanner(scan1)) { 1191 for (Result result : scanner) { 1192 numRecords++; 1193 } 1194 1195 LOG.info("test data has {} records.", numRecords); 1196 1197 // by default, scan metrics collection is turned off 1198 assertNull(scanner.getScanMetrics()); 1199 } 1200 1201 // turn on scan metrics 1202 Scan scan2 = new Scan(); 1203 scan2.setScanMetricsEnabled(true); 1204 scan2.setCaching(numRecords + 1); 1205 try (ResultScanner scanner = ht.getScanner(scan2)) { 1206 for (Result result : scanner.next(numRecords - 1)) { 1207 } 1208 assertNotNull(scanner.getScanMetrics()); 1209 } 1210 1211 // set caching to 1, because metrics are collected in each roundtrip only 1212 scan2 = new Scan(); 1213 scan2.setScanMetricsEnabled(true); 1214 scan2.setCaching(1); 1215 try (ResultScanner scanner = ht.getScanner(scan2)) { 1216 // per HBASE-5717, this should still collect even if you don't run all the way to 1217 // the end of the scanner. So this is asking for 2 of the 3 rows we inserted. 1218 for (Result result : scanner.next(numRecords - 1)) { 1219 } 1220 ScanMetrics scanMetrics = scanner.getScanMetrics(); 1221 assertEquals(numOfRegions, scanMetrics.countOfRegions.get(), 1222 "Did not access all the regions in the table"); 1223 } 1224 1225 // check byte counters 1226 scan2 = new Scan(); 1227 scan2.setScanMetricsEnabled(true); 1228 scan2.setCaching(1); 1229 try (ResultScanner scanner = ht.getScanner(scan2)) { 1230 int numBytes = 0; 1231 for (Result result : scanner) { 1232 for (Cell cell : result.listCells()) { 1233 numBytes += PrivateCellUtil.estimatedSerializedSizeOf(cell); 1234 } 1235 } 1236 ScanMetrics scanMetrics = scanner.getScanMetrics(); 1237 assertEquals(numBytes, scanMetrics.countOfBytesInResults.get(), 1238 "Did not count the result bytes"); 1239 } 1240 1241 // check byte counters on a small scan 1242 scan2 = new Scan(); 1243 scan2.setScanMetricsEnabled(true); 1244 scan2.setCaching(1); 1245 scan2.setReadType(ReadType.PREAD); 1246 try (ResultScanner scanner = ht.getScanner(scan2)) { 1247 int numBytes = 0; 1248 for (Result result : scanner) { 1249 for (Cell cell : result.listCells()) { 1250 numBytes += PrivateCellUtil.estimatedSerializedSizeOf(cell); 1251 } 1252 } 1253 ScanMetrics scanMetrics = scanner.getScanMetrics(); 1254 assertEquals(numBytes, scanMetrics.countOfBytesInResults.get(), 1255 "Did not count the result bytes"); 1256 } 1257 1258 // now, test that the metrics are still collected even if you don't call close, but do 1259 // run past the end of all the records 1260 /** 1261 * There seems to be a timing issue here. Comment out for now. Fix when time. Scan 1262 * scanWithoutClose = new Scan(); scanWithoutClose.setCaching(1); 1263 * scanWithoutClose.setScanMetricsEnabled(true); ResultScanner scannerWithoutClose = 1264 * ht.getScanner(scanWithoutClose); for (Result result : scannerWithoutClose.next(numRecords + 1265 * 1)) { } ScanMetrics scanMetricsWithoutClose = getScanMetrics(scanWithoutClose); 1266 * assertEquals("Did not access all the regions in the table", numOfRegions, 1267 * scanMetricsWithoutClose.countOfRegions.get()); 1268 */ 1269 1270 // finally, 1271 // test that the metrics are collected correctly if you both run past all the records, 1272 // AND close the scanner 1273 Scan scanWithClose = new Scan(); 1274 // make sure we can set caching up to the number of a scanned values 1275 scanWithClose.setCaching(numRecords); 1276 scanWithClose.setScanMetricsEnabled(true); 1277 try (ResultScanner scannerWithClose = ht.getScanner(scanWithClose)) { 1278 for (Result result : scannerWithClose.next(numRecords + 1)) { 1279 } 1280 scannerWithClose.close(); 1281 ScanMetrics scanMetricsWithClose = scannerWithClose.getScanMetrics(); 1282 assertEquals(numOfRegions, scanMetricsWithClose.countOfRegions.get(), 1283 "Did not access all the regions in the table"); 1284 } 1285 } finally { 1286 TEST_UTIL.deleteTable(tableName); 1287 } 1288 } 1289 1290 /** 1291 * Tests that cache on write works all the way up from the client-side. Performs inserts, flushes, 1292 * and compactions, verifying changes in the block cache along the way. 1293 */ 1294 @TestTemplate 1295 public void testCacheOnWriteEvictOnClose() throws Exception { 1296 byte[] data = Bytes.toBytes("data"); 1297 TEST_UTIL.createTable(tableName, FAMILY); 1298 try (Connection conn = getConnection(); Table table = conn.getTable(tableName); 1299 RegionLocator locator = conn.getRegionLocator(tableName)) { 1300 // get the block cache and region 1301 String regionName = locator.getAllRegionLocations().get(0).getRegion().getEncodedName(); 1302 1303 HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); 1304 HStore store = region.getStores().iterator().next(); 1305 CacheConfig cacheConf = store.getCacheConfig(); 1306 cacheConf.setCacheDataOnWrite(true); 1307 cacheConf.setEvictOnClose(true); 1308 BlockCache cache = cacheConf.getBlockCache().get(); 1309 1310 // establish baseline stats 1311 long startBlockCount = cache.getBlockCount(); 1312 long startBlockHits = cache.getStats().getHitCount(); 1313 long startBlockMiss = cache.getStats().getMissCount(); 1314 1315 // wait till baseline is stable, (minimal 500 ms) 1316 for (int i = 0; i < 5; i++) { 1317 Thread.sleep(100); 1318 if ( 1319 startBlockCount != cache.getBlockCount() 1320 || startBlockHits != cache.getStats().getHitCount() 1321 || startBlockMiss != cache.getStats().getMissCount() 1322 ) { 1323 startBlockCount = cache.getBlockCount(); 1324 startBlockHits = cache.getStats().getHitCount(); 1325 startBlockMiss = cache.getStats().getMissCount(); 1326 i = -1; 1327 } 1328 } 1329 1330 // insert data 1331 Put put = new Put(ROW); 1332 put.addColumn(FAMILY, QUALIFIER, data); 1333 table.put(put); 1334 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); 1335 1336 // data was in memstore so don't expect any changes 1337 assertEquals(startBlockCount, cache.getBlockCount()); 1338 assertEquals(startBlockHits, cache.getStats().getHitCount()); 1339 assertEquals(startBlockMiss, cache.getStats().getMissCount()); 1340 1341 // flush the data 1342 LOG.debug("Flushing cache"); 1343 region.flush(true); 1344 1345 // expect two more blocks in cache - DATA and ROOT_INDEX 1346 // , no change in hits/misses 1347 long expectedBlockCount = startBlockCount + 2; 1348 long expectedBlockHits = startBlockHits; 1349 long expectedBlockMiss = startBlockMiss; 1350 assertEquals(expectedBlockCount, cache.getBlockCount()); 1351 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 1352 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 1353 // read the data and expect same blocks, one new hit, no misses 1354 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); 1355 assertEquals(expectedBlockCount, cache.getBlockCount()); 1356 assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); 1357 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 1358 // insert a second column, read the row, no new blocks, one new hit 1359 byte[] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER); 1360 byte[] data2 = Bytes.add(data, data); 1361 put = new Put(ROW); 1362 put.addColumn(FAMILY, QUALIFIER2, data2); 1363 table.put(put); 1364 Result r = table.get(new Get(ROW)); 1365 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); 1366 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); 1367 assertEquals(expectedBlockCount, cache.getBlockCount()); 1368 assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); 1369 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 1370 // flush, one new block 1371 LOG.info("Flushing cache"); 1372 region.flush(true); 1373 1374 // + 1 for Index Block, +1 for data block 1375 expectedBlockCount += 2; 1376 assertEquals(expectedBlockCount, cache.getBlockCount()); 1377 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 1378 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 1379 // compact, net minus two blocks, two hits, no misses 1380 LOG.info("Compacting"); 1381 assertEquals(2, store.getStorefilesCount()); 1382 region.compact(true); 1383 store.closeAndArchiveCompactedFiles(); 1384 waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max 1385 assertEquals(1, store.getStorefilesCount()); 1386 // evicted two data blocks and two index blocks and compaction does not cache new blocks 1387 expectedBlockCount = startBlockCount; 1388 assertEquals(expectedBlockCount, cache.getBlockCount()); 1389 expectedBlockHits += 2; 1390 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 1391 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 1392 // read the row, this should be a cache miss because we don't cache data 1393 // blocks on compaction 1394 r = table.get(new Get(ROW)); 1395 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); 1396 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); 1397 expectedBlockCount += 1; // cached one data block 1398 assertEquals(expectedBlockCount, cache.getBlockCount()); 1399 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 1400 assertEquals(++expectedBlockMiss, cache.getStats().getMissCount()); 1401 } 1402 } 1403 1404 private void waitForStoreFileCount(HStore store, int count, int timeout) 1405 throws InterruptedException { 1406 await().atMost(Duration.ofMillis(timeout)) 1407 .untilAsserted(() -> assertEquals(count, store.getStorefilesCount())); 1408 } 1409 1410 /** 1411 * Tests the non cached version of getRegionLocator by moving a region. 1412 */ 1413 @TestTemplate 1414 public void testNonCachedGetRegionLocation() throws Exception { 1415 // Test Initialization. 1416 byte[] family1 = Bytes.toBytes("f1"); 1417 byte[] family2 = Bytes.toBytes("f2"); 1418 TEST_UTIL.createTable(tableName, new byte[][] { family1, family2 }, 10); 1419 try (Connection conn = getConnection(); Table ignored = conn.getTable(tableName); 1420 Admin admin = conn.getAdmin(); 1421 RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 1422 List<HRegionLocation> allRegionLocations = locator.getAllRegionLocations(); 1423 assertEquals(1, allRegionLocations.size()); 1424 RegionInfo regionInfo = allRegionLocations.get(0).getRegion(); 1425 ServerName addrBefore = allRegionLocations.get(0).getServerName(); 1426 // Verify region location before move. 1427 HRegionLocation addrCache = locator.getRegionLocation(regionInfo.getStartKey(), false); 1428 HRegionLocation addrNoCache = locator.getRegionLocation(regionInfo.getStartKey(), true); 1429 1430 assertEquals(addrBefore.getPort(), addrCache.getPort()); 1431 assertEquals(addrBefore.getPort(), addrNoCache.getPort()); 1432 1433 // Make sure more than one server. 1434 if (TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size() <= 1) { 1435 TEST_UTIL.getMiniHBaseCluster().startRegionServer(); 1436 Waiter.waitFor(TEST_UTIL.getConfiguration(), 30000, new Waiter.Predicate<Exception>() { 1437 @Override 1438 public boolean evaluate() throws Exception { 1439 return TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size() > 1; 1440 } 1441 }); 1442 } 1443 1444 ServerName addrAfter = null; 1445 // Now move the region to a different server. 1446 for (int i = 0; i 1447 < TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size(); i++) { 1448 HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(i); 1449 ServerName addr = regionServer.getServerName(); 1450 if (addr.getPort() != addrBefore.getPort()) { 1451 admin.move(regionInfo.getEncodedNameAsBytes(), addr); 1452 // Wait for the region to move. 1453 Thread.sleep(5000); 1454 addrAfter = addr; 1455 break; 1456 } 1457 } 1458 1459 // Verify the region was moved. 1460 addrCache = locator.getRegionLocation(regionInfo.getStartKey(), false); 1461 addrNoCache = locator.getRegionLocation(regionInfo.getStartKey(), true); 1462 assertNotNull(addrAfter); 1463 assertTrue(addrAfter.getPort() != addrCache.getPort()); 1464 assertEquals(addrAfter.getPort(), addrNoCache.getPort()); 1465 } 1466 } 1467 1468 /** 1469 * Tests getRegionsInRange by creating some regions over which a range of keys spans; then 1470 * changing the key range. 1471 */ 1472 @TestTemplate 1473 public void testGetRegionsInRange() throws Exception { 1474 // Test Initialization. 1475 byte[] startKey = Bytes.toBytes("ddc"); 1476 byte[] endKey = Bytes.toBytes("mmm"); 1477 TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10); 1478 1479 int numOfRegions; 1480 try (Connection conn = getConnection(); RegionLocator r = conn.getRegionLocator(tableName)) { 1481 numOfRegions = r.getStartKeys().length; 1482 } 1483 assertEquals(26, numOfRegions); 1484 1485 // Get the regions in this range 1486 List<HRegionLocation> regionsList = getRegionsInRange(tableName, startKey, endKey); 1487 assertEquals(10, regionsList.size()); 1488 1489 // Change the start key 1490 startKey = Bytes.toBytes("fff"); 1491 regionsList = getRegionsInRange(tableName, startKey, endKey); 1492 assertEquals(7, regionsList.size()); 1493 1494 // Change the end key 1495 endKey = Bytes.toBytes("nnn"); 1496 regionsList = getRegionsInRange(tableName, startKey, endKey); 1497 assertEquals(8, regionsList.size()); 1498 1499 // Empty start key 1500 regionsList = getRegionsInRange(tableName, HConstants.EMPTY_START_ROW, endKey); 1501 assertEquals(13, regionsList.size()); 1502 1503 // Empty end key 1504 regionsList = getRegionsInRange(tableName, startKey, HConstants.EMPTY_END_ROW); 1505 assertEquals(21, regionsList.size()); 1506 1507 // Both start and end keys empty 1508 regionsList = 1509 getRegionsInRange(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); 1510 assertEquals(26, regionsList.size()); 1511 1512 // Change the end key to somewhere in the last block 1513 endKey = Bytes.toBytes("zzz1"); 1514 regionsList = getRegionsInRange(tableName, startKey, endKey); 1515 assertEquals(21, regionsList.size()); 1516 1517 // Change the start key to somewhere in the first block 1518 startKey = Bytes.toBytes("aac"); 1519 regionsList = getRegionsInRange(tableName, startKey, endKey); 1520 assertEquals(26, regionsList.size()); 1521 1522 // Make start and end key the same 1523 startKey = Bytes.toBytes("ccc"); 1524 endKey = Bytes.toBytes("ccc"); 1525 regionsList = getRegionsInRange(tableName, startKey, endKey); 1526 assertEquals(1, regionsList.size()); 1527 } 1528 1529 private List<HRegionLocation> getRegionsInRange(TableName tableName, byte[] startKey, 1530 byte[] endKey) throws IOException { 1531 List<HRegionLocation> regionsInRange = new ArrayList<>(); 1532 byte[] currentKey = startKey; 1533 final boolean endKeyIsEndOfTable = Bytes.equals(endKey, HConstants.EMPTY_END_ROW); 1534 try (Connection conn = getConnection(); RegionLocator r = conn.getRegionLocator(tableName)) { 1535 do { 1536 HRegionLocation regionLocation = r.getRegionLocation(currentKey); 1537 regionsInRange.add(regionLocation); 1538 currentKey = regionLocation.getRegion().getEndKey(); 1539 } while ( 1540 !Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) 1541 && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0) 1542 ); 1543 return regionsInRange; 1544 } 1545 } 1546 1547 @TestTemplate 1548 public void testJira6912() throws Exception { 1549 TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, 10); 1550 try (Connection conn = getConnection(); Table foo = conn.getTable(tableName)) { 1551 List<Put> puts = new ArrayList<>(); 1552 for (int i = 0; i != 100; i++) { 1553 Put put = new Put(Bytes.toBytes(i)); 1554 put.addColumn(FAMILY, FAMILY, Bytes.toBytes(i)); 1555 puts.add(put); 1556 } 1557 foo.put(puts); 1558 // If i comment this out it works 1559 TEST_UTIL.flush(); 1560 1561 Scan scan = new Scan(); 1562 scan.withStartRow(Bytes.toBytes(1)); 1563 scan.withStopRow(Bytes.toBytes(3)); 1564 scan.addColumn(FAMILY, FAMILY); 1565 scan.setFilter( 1566 new RowFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(Bytes.toBytes(1)))); 1567 1568 try (ResultScanner scanner = foo.getScanner(scan)) { 1569 Result[] bar = scanner.next(100); 1570 assertEquals(1, bar.length); 1571 } 1572 } 1573 } 1574 1575 @TestTemplate 1576 public void testScanNullQualifier() throws IOException { 1577 TEST_UTIL.createTable(tableName, FAMILY); 1578 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 1579 Put put = new Put(ROW); 1580 put.addColumn(FAMILY, QUALIFIER, VALUE); 1581 table.put(put); 1582 1583 put = new Put(ROW); 1584 put.addColumn(FAMILY, null, VALUE); 1585 table.put(put); 1586 LOG.info("Row put"); 1587 1588 Scan scan = new Scan(); 1589 scan.addColumn(FAMILY, null); 1590 1591 ResultScanner scanner = table.getScanner(scan); 1592 Result[] bar = scanner.next(100); 1593 assertEquals(1, bar.length); 1594 assertEquals(1, bar[0].size()); 1595 1596 scan = new Scan(); 1597 scan.addFamily(FAMILY); 1598 1599 scanner = table.getScanner(scan); 1600 bar = scanner.next(100); 1601 assertEquals(1, bar.length); 1602 assertEquals(2, bar[0].size()); 1603 } 1604 } 1605 1606 @TestTemplate 1607 public void testRawScanRespectsVersions() throws Exception { 1608 TEST_UTIL.createTable(tableName, FAMILY); 1609 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 1610 byte[] row = Bytes.toBytes("row"); 1611 1612 // put the same row 4 times, with different values 1613 Put p = new Put(row); 1614 p.addColumn(FAMILY, QUALIFIER, 10, VALUE); 1615 table.put(p); 1616 p = new Put(row); 1617 p.addColumn(FAMILY, QUALIFIER, 11, ArrayUtils.add(VALUE, (byte) 2)); 1618 table.put(p); 1619 1620 p = new Put(row); 1621 p.addColumn(FAMILY, QUALIFIER, 12, ArrayUtils.add(VALUE, (byte) 3)); 1622 table.put(p); 1623 1624 p = new Put(row); 1625 p.addColumn(FAMILY, QUALIFIER, 13, ArrayUtils.add(VALUE, (byte) 4)); 1626 table.put(p); 1627 1628 int versions = 4; 1629 Scan s = new Scan().withStartRow(row); 1630 // get all the possible versions 1631 s.readAllVersions(); 1632 s.setRaw(true); 1633 1634 try (ResultScanner scanner = table.getScanner(s)) { 1635 int count = 0; 1636 for (Result r : scanner) { 1637 assertEquals(versions, r.listCells().size(), 1638 "Found an unexpected number of results for the row!"); 1639 count++; 1640 } 1641 assertEquals(1, count, 1642 "Found more than a single row when raw scanning the table with a single row!"); 1643 } 1644 1645 // then if we decrease the number of versions, but keep the scan raw, we should see exactly 1646 // that number of versions 1647 versions = 2; 1648 s.readVersions(versions); 1649 try (ResultScanner scanner = table.getScanner(s)) { 1650 int count = 0; 1651 for (Result r : scanner) { 1652 assertEquals(versions, r.listCells().size(), 1653 "Found an unexpected number of results for the row!"); 1654 count++; 1655 } 1656 assertEquals(1, count, 1657 "Found more than a single row when raw scanning the table with a single row!"); 1658 } 1659 1660 // finally, if we turn off raw scanning, but max out the number of versions, we should go back 1661 // to seeing just three 1662 versions = 3; 1663 s.readVersions(versions); 1664 try (ResultScanner scanner = table.getScanner(s)) { 1665 int count = 0; 1666 for (Result r : scanner) { 1667 assertEquals(versions, r.listCells().size(), 1668 "Found an unexpected number of results for the row!"); 1669 count++; 1670 } 1671 assertEquals(1, count, 1672 "Found more than a single row when raw scanning the table with a single row!"); 1673 } 1674 1675 } 1676 TEST_UTIL.deleteTable(tableName); 1677 } 1678 1679 @TestTemplate 1680 public void testEmptyFilterList() throws Exception { 1681 // Test Initialization. 1682 TEST_UTIL.createTable(tableName, FAMILY); 1683 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 1684 // Insert one row each region 1685 Put put = new Put(Bytes.toBytes("row")); 1686 put.addColumn(FAMILY, QUALIFIER, VALUE); 1687 table.put(put); 1688 1689 List<Result> scanResults = new LinkedList<>(); 1690 Scan scan = new Scan(); 1691 scan.setFilter(new FilterList()); 1692 try (ResultScanner scanner = table.getScanner(scan)) { 1693 for (Result r : scanner) { 1694 scanResults.add(r); 1695 } 1696 } 1697 assertEquals(1, scanResults.size()); 1698 Get g = new Get(Bytes.toBytes("row")); 1699 g.setFilter(new FilterList()); 1700 Result getResult = table.get(g); 1701 Result scanResult = scanResults.get(0); 1702 assertEquals(scanResult.rawCells().length, getResult.rawCells().length); 1703 for (int i = 0; i != scanResult.rawCells().length; ++i) { 1704 Cell scanCell = scanResult.rawCells()[i]; 1705 Cell getCell = getResult.rawCells()[i]; 1706 assertEquals(0, Bytes.compareTo(CellUtil.cloneRow(scanCell), CellUtil.cloneRow(getCell))); 1707 assertEquals(0, 1708 Bytes.compareTo(CellUtil.cloneFamily(scanCell), CellUtil.cloneFamily(getCell))); 1709 assertEquals(0, 1710 Bytes.compareTo(CellUtil.cloneQualifier(scanCell), CellUtil.cloneQualifier(getCell))); 1711 assertEquals(0, 1712 Bytes.compareTo(CellUtil.cloneValue(scanCell), CellUtil.cloneValue(getCell))); 1713 } 1714 } 1715 } 1716 1717 @TestTemplate 1718 public void testSmallScan() throws Exception { 1719 // Test Initialization. 1720 TEST_UTIL.createTable(tableName, FAMILY); 1721 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 1722 // Insert one row each region 1723 int insertNum = 10; 1724 for (int i = 0; i < 10; i++) { 1725 Put put = new Put(Bytes.toBytes("row" + String.format("%03d", i))); 1726 put.addColumn(FAMILY, QUALIFIER, VALUE); 1727 table.put(put); 1728 } 1729 1730 // normal scan 1731 try (ResultScanner scanner = table.getScanner(new Scan())) { 1732 int count = 0; 1733 for (Result r : scanner) { 1734 assertFalse(r.isEmpty()); 1735 count++; 1736 } 1737 assertEquals(insertNum, count); 1738 } 1739 1740 // small scan 1741 Scan scan = new Scan().withStartRow(HConstants.EMPTY_START_ROW) 1742 .withStopRow(HConstants.EMPTY_END_ROW, true); 1743 scan.setReadType(ReadType.PREAD); 1744 scan.setCaching(2); 1745 try (ResultScanner scanner = table.getScanner(scan)) { 1746 int count = 0; 1747 for (Result r : scanner) { 1748 assertFalse(r.isEmpty()); 1749 count++; 1750 } 1751 assertEquals(insertNum, count); 1752 } 1753 } 1754 } 1755 1756 @TestTemplate 1757 public void testSuperSimpleWithReverseScan() throws Exception { 1758 TEST_UTIL.createTable(tableName, FAMILY); 1759 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 1760 Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); 1761 put.addColumn(FAMILY, QUALIFIER, VALUE); 1762 ht.put(put); 1763 put = new Put(Bytes.toBytes("0-b11111-0000000000000000002")); 1764 put.addColumn(FAMILY, QUALIFIER, VALUE); 1765 ht.put(put); 1766 put = new Put(Bytes.toBytes("0-b11111-0000000000000000004")); 1767 put.addColumn(FAMILY, QUALIFIER, VALUE); 1768 ht.put(put); 1769 put = new Put(Bytes.toBytes("0-b11111-0000000000000000006")); 1770 put.addColumn(FAMILY, QUALIFIER, VALUE); 1771 ht.put(put); 1772 put = new Put(Bytes.toBytes("0-b11111-0000000000000000008")); 1773 put.addColumn(FAMILY, QUALIFIER, VALUE); 1774 ht.put(put); 1775 put = new Put(Bytes.toBytes("0-b22222-0000000000000000001")); 1776 put.addColumn(FAMILY, QUALIFIER, VALUE); 1777 ht.put(put); 1778 put = new Put(Bytes.toBytes("0-b22222-0000000000000000003")); 1779 put.addColumn(FAMILY, QUALIFIER, VALUE); 1780 ht.put(put); 1781 put = new Put(Bytes.toBytes("0-b22222-0000000000000000005")); 1782 put.addColumn(FAMILY, QUALIFIER, VALUE); 1783 ht.put(put); 1784 put = new Put(Bytes.toBytes("0-b22222-0000000000000000007")); 1785 put.addColumn(FAMILY, QUALIFIER, VALUE); 1786 ht.put(put); 1787 put = new Put(Bytes.toBytes("0-b22222-0000000000000000009")); 1788 put.addColumn(FAMILY, QUALIFIER, VALUE); 1789 ht.put(put); 1790 Scan scan = new Scan().withStartRow(Bytes.toBytes("0-b11111-9223372036854775807")) 1791 .withStopRow(Bytes.toBytes("0-b11111-0000000000000000000"), true); 1792 scan.setReversed(true); 1793 try (ResultScanner scanner = ht.getScanner(scan)) { 1794 Result result = scanner.next(); 1795 assertTrue(Bytes.equals(result.getRow(), Bytes.toBytes("0-b11111-0000000000000000008"))); 1796 } 1797 } 1798 } 1799 1800 @TestTemplate 1801 public void testFiltersWithReverseScan() throws Exception { 1802 TEST_UTIL.createTable(tableName, FAMILY); 1803 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 1804 byte[][] ROWS = makeN(ROW, 10); 1805 byte[][] QUALIFIERS = 1806 { Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"), 1807 Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"), 1808 Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"), 1809 Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"), 1810 Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>") }; 1811 for (int i = 0; i < 10; i++) { 1812 Put put = new Put(ROWS[i]); 1813 put.addColumn(FAMILY, QUALIFIERS[i], VALUE); 1814 ht.put(put); 1815 } 1816 Scan scan = new Scan(); 1817 scan.setReversed(true); 1818 scan.addFamily(FAMILY); 1819 Filter filter = 1820 new QualifierFilter(CompareOperator.EQUAL, new RegexStringComparator("col[1-5]")); 1821 scan.setFilter(filter); 1822 try (ResultScanner scanner = ht.getScanner(scan)) { 1823 int expectedIndex = 5; 1824 for (Result result : scanner) { 1825 assertEquals(1, result.size()); 1826 Cell c = result.rawCells()[0]; 1827 assertTrue(Bytes.equals(c.getRowArray(), c.getRowOffset(), c.getRowLength(), 1828 ROWS[expectedIndex], 0, ROWS[expectedIndex].length)); 1829 assertTrue( 1830 Bytes.equals(c.getQualifierArray(), c.getQualifierOffset(), c.getQualifierLength(), 1831 QUALIFIERS[expectedIndex], 0, QUALIFIERS[expectedIndex].length)); 1832 expectedIndex--; 1833 } 1834 assertEquals(0, expectedIndex); 1835 } 1836 } 1837 } 1838 1839 @TestTemplate 1840 public void testKeyOnlyFilterWithReverseScan() throws Exception { 1841 TEST_UTIL.createTable(tableName, FAMILY); 1842 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 1843 byte[][] ROWS = makeN(ROW, 10); 1844 byte[][] QUALIFIERS = 1845 { Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"), 1846 Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"), 1847 Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"), 1848 Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"), 1849 Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>") }; 1850 for (int i = 0; i < 10; i++) { 1851 Put put = new Put(ROWS[i]); 1852 put.addColumn(FAMILY, QUALIFIERS[i], VALUE); 1853 ht.put(put); 1854 } 1855 Scan scan = new Scan(); 1856 scan.setReversed(true); 1857 scan.addFamily(FAMILY); 1858 Filter filter = new KeyOnlyFilter(true); 1859 scan.setFilter(filter); 1860 try (ResultScanner ignored = ht.getScanner(scan)) { 1861 int count = 0; 1862 for (Result result : ht.getScanner(scan)) { 1863 assertEquals(1, result.size()); 1864 assertEquals(Bytes.SIZEOF_INT, result.rawCells()[0].getValueLength()); 1865 assertEquals(VALUE.length, Bytes.toInt(CellUtil.cloneValue(result.rawCells()[0]))); 1866 count++; 1867 } 1868 assertEquals(10, count); 1869 } 1870 } 1871 } 1872 1873 /** 1874 * Test simple table and non-existent row cases. 1875 */ 1876 @TestTemplate 1877 public void testSimpleMissingWithReverseScan() throws Exception { 1878 TEST_UTIL.createTable(tableName, FAMILY); 1879 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 1880 byte[][] ROWS = makeN(ROW, 4); 1881 1882 // Try to get a row on an empty table 1883 Scan scan = new Scan(); 1884 scan.setReversed(true); 1885 Result result = getSingleScanResult(ht, scan); 1886 assertNullResult(result); 1887 1888 scan = new Scan().withStartRow(ROWS[0]); 1889 scan.setReversed(true); 1890 result = getSingleScanResult(ht, scan); 1891 assertNullResult(result); 1892 1893 scan = new Scan().withStartRow(ROWS[0]).withStopRow(ROWS[1], true); 1894 scan.setReversed(true); 1895 result = getSingleScanResult(ht, scan); 1896 assertNullResult(result); 1897 1898 scan = new Scan(); 1899 scan.setReversed(true); 1900 scan.addFamily(FAMILY); 1901 result = getSingleScanResult(ht, scan); 1902 assertNullResult(result); 1903 1904 scan = new Scan(); 1905 scan.setReversed(true); 1906 scan.addColumn(FAMILY, QUALIFIER); 1907 result = getSingleScanResult(ht, scan); 1908 assertNullResult(result); 1909 1910 // Insert a row 1911 Put put = new Put(ROWS[2]); 1912 put.addColumn(FAMILY, QUALIFIER, VALUE); 1913 ht.put(put); 1914 1915 // Make sure we can scan the row 1916 scan = new Scan(); 1917 scan.setReversed(true); 1918 result = getSingleScanResult(ht, scan); 1919 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 1920 1921 scan = new Scan().withStartRow(ROWS[3]).withStopRow(ROWS[0], true); 1922 scan.setReversed(true); 1923 result = getSingleScanResult(ht, scan); 1924 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 1925 1926 scan = new Scan().withStartRow(ROWS[2]).withStopRow(ROWS[1], true); 1927 scan.setReversed(true); 1928 result = getSingleScanResult(ht, scan); 1929 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 1930 1931 // Try to scan empty rows around it 1932 // Introduced MemStore#shouldSeekForReverseScan to fix the following 1933 scan = new Scan().withStartRow(ROWS[1]); 1934 scan.setReversed(true); 1935 result = getSingleScanResult(ht, scan); 1936 assertNullResult(result); 1937 } 1938 } 1939 1940 @TestTemplate 1941 public void testNullWithReverseScan() throws Exception { 1942 TEST_UTIL.createTable(tableName, FAMILY); 1943 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 1944 // Null qualifier (should work) 1945 Put put = new Put(ROW); 1946 put.addColumn(FAMILY, null, VALUE); 1947 ht.put(put); 1948 scanTestNull(ht, ROW, FAMILY, VALUE, true); 1949 Delete delete = new Delete(ROW); 1950 delete.addColumns(FAMILY, null); 1951 ht.delete(delete); 1952 } 1953 1954 // Use a new table 1955 TableName newTableName = TableName.valueOf(tableName.toString() + "2"); 1956 TEST_UTIL.createTable(newTableName, FAMILY); 1957 try (Connection conn = getConnection(); Table ht = conn.getTable(newTableName)) { 1958 // Empty qualifier, byte[0] instead of null (should work) 1959 Put put = new Put(ROW); 1960 put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); 1961 ht.put(put); 1962 scanTestNull(ht, ROW, FAMILY, VALUE, true); 1963 TEST_UTIL.flush(); 1964 scanTestNull(ht, ROW, FAMILY, VALUE, true); 1965 Delete delete = new Delete(ROW); 1966 delete.addColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY); 1967 ht.delete(delete); 1968 // Null value 1969 put = new Put(ROW); 1970 put.addColumn(FAMILY, QUALIFIER, null); 1971 ht.put(put); 1972 Scan scan = new Scan(); 1973 scan.setReversed(true); 1974 scan.addColumn(FAMILY, QUALIFIER); 1975 Result result = getSingleScanResult(ht, scan); 1976 assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); 1977 } 1978 } 1979 1980 @TestTemplate 1981 @SuppressWarnings("checkstyle:MethodLength") 1982 public void testDeletesWithReverseScan() throws Exception { 1983 byte[][] ROWS = makeNAscii(ROW, 6); 1984 byte[][] FAMILIES = makeNAscii(FAMILY, 3); 1985 byte[][] VALUES = makeN(VALUE, 5); 1986 long[] ts = { 1000, 2000, 3000, 4000, 5000 }; 1987 TEST_UTIL.createTable(tableName, FAMILIES, 3); 1988 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 1989 Put put = new Put(ROW); 1990 put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); 1991 put.addColumn(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]); 1992 ht.put(put); 1993 1994 Delete delete = new Delete(ROW); 1995 delete.addFamily(FAMILIES[0], ts[0]); 1996 ht.delete(delete); 1997 1998 Scan scan = new Scan().withStartRow(ROW); 1999 scan.setReversed(true); 2000 scan.addFamily(FAMILIES[0]); 2001 scan.readVersions(Integer.MAX_VALUE); 2002 Result result = getSingleScanResult(ht, scan); 2003 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1] }, 2004 new byte[][] { VALUES[1] }, 0, 0); 2005 2006 // Test delete latest version 2007 put = new Put(ROW); 2008 put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); 2009 put.addColumn(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]); 2010 put.addColumn(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]); 2011 put.addColumn(FAMILIES[0], null, ts[4], VALUES[4]); 2012 put.addColumn(FAMILIES[0], null, ts[2], VALUES[2]); 2013 put.addColumn(FAMILIES[0], null, ts[3], VALUES[3]); 2014 ht.put(put); 2015 2016 delete = new Delete(ROW); 2017 delete.addColumn(FAMILIES[0], QUALIFIER); // ts[4] 2018 ht.delete(delete); 2019 2020 scan = new Scan().withStartRow(ROW); 2021 scan.setReversed(true); 2022 scan.addColumn(FAMILIES[0], QUALIFIER); 2023 scan.readVersions(Integer.MAX_VALUE); 2024 result = getSingleScanResult(ht, scan); 2025 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1], ts[2], ts[3] }, 2026 new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2); 2027 2028 // Test for HBASE-1847 2029 delete = new Delete(ROW); 2030 delete.addColumn(FAMILIES[0], null); 2031 ht.delete(delete); 2032 2033 // Cleanup null qualifier 2034 delete = new Delete(ROW); 2035 delete.addColumns(FAMILIES[0], null); 2036 ht.delete(delete); 2037 2038 // Expected client behavior might be that you can re-put deleted values 2039 // But alas, this is not to be. We can't put them back in either case. 2040 2041 put = new Put(ROW); 2042 put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); 2043 put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); 2044 ht.put(put); 2045 2046 // The Scanner returns the previous values, the expected-naive-unexpected 2047 // behavior 2048 2049 scan = new Scan().withStartRow(ROW); 2050 scan.setReversed(true); 2051 scan.addFamily(FAMILIES[0]); 2052 scan.readVersions(Integer.MAX_VALUE); 2053 result = getSingleScanResult(ht, scan); 2054 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1], ts[2], ts[3] }, 2055 new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2); 2056 2057 // Test deleting an entire family from one row but not the other various 2058 // ways 2059 2060 put = new Put(ROWS[0]); 2061 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 2062 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 2063 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 2064 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 2065 ht.put(put); 2066 2067 put = new Put(ROWS[1]); 2068 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 2069 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 2070 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 2071 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 2072 ht.put(put); 2073 2074 put = new Put(ROWS[2]); 2075 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 2076 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 2077 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 2078 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 2079 ht.put(put); 2080 2081 delete = new Delete(ROWS[0]); 2082 delete.addFamily(FAMILIES[2]); 2083 ht.delete(delete); 2084 2085 delete = new Delete(ROWS[1]); 2086 delete.addColumns(FAMILIES[1], QUALIFIER); 2087 ht.delete(delete); 2088 2089 delete = new Delete(ROWS[2]); 2090 delete.addColumn(FAMILIES[1], QUALIFIER); 2091 delete.addColumn(FAMILIES[1], QUALIFIER); 2092 delete.addColumn(FAMILIES[2], QUALIFIER); 2093 ht.delete(delete); 2094 2095 scan = new Scan().withStartRow(ROWS[0]); 2096 scan.setReversed(true); 2097 scan.addFamily(FAMILIES[1]); 2098 scan.addFamily(FAMILIES[2]); 2099 scan.readVersions(Integer.MAX_VALUE); 2100 result = getSingleScanResult(ht, scan); 2101 assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); 2102 assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, new long[] { ts[0], ts[1] }, 2103 new byte[][] { VALUES[0], VALUES[1] }, 0, 1); 2104 2105 scan = new Scan().withStartRow(ROWS[1]); 2106 scan.setReversed(true); 2107 scan.addFamily(FAMILIES[1]); 2108 scan.addFamily(FAMILIES[2]); 2109 scan.readVersions(Integer.MAX_VALUE); 2110 result = getSingleScanResult(ht, scan); 2111 assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); 2112 2113 scan = new Scan().withStartRow(ROWS[2]); 2114 scan.setReversed(true); 2115 scan.addFamily(FAMILIES[1]); 2116 scan.addFamily(FAMILIES[2]); 2117 scan.readVersions(Integer.MAX_VALUE); 2118 result = getSingleScanResult(ht, scan); 2119 assertEquals(1, result.size()); 2120 assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER, new long[] { ts[2] }, 2121 new byte[][] { VALUES[2] }, 0, 0); 2122 2123 // Test if we delete the family first in one row (HBASE-1541) 2124 2125 delete = new Delete(ROWS[3]); 2126 delete.addFamily(FAMILIES[1]); 2127 ht.delete(delete); 2128 2129 put = new Put(ROWS[3]); 2130 put.addColumn(FAMILIES[2], QUALIFIER, VALUES[0]); 2131 ht.put(put); 2132 2133 put = new Put(ROWS[4]); 2134 put.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]); 2135 put.addColumn(FAMILIES[2], QUALIFIER, VALUES[2]); 2136 ht.put(put); 2137 2138 scan = new Scan().withStartRow(ROWS[4]); 2139 scan.setReversed(true); 2140 scan.addFamily(FAMILIES[1]); 2141 scan.addFamily(FAMILIES[2]); 2142 scan.readVersions(Integer.MAX_VALUE); 2143 try (ResultScanner scanner = ht.getScanner(scan)) { 2144 result = scanner.next(); 2145 assertEquals(2, result.size(), "Expected 2 keys but received " + result.size()); 2146 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[4])); 2147 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[1]), ROWS[4])); 2148 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[1])); 2149 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[1]), VALUES[2])); 2150 result = scanner.next(); 2151 assertEquals(1, result.size(), "Expected 1 key but received " + result.size()); 2152 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[3])); 2153 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[0])); 2154 } 2155 } 2156 } 2157 2158 /** 2159 * Tests reversed scan under multi regions 2160 */ 2161 @TestTemplate 2162 public void testReversedScanUnderMultiRegions() throws Exception { 2163 // Test Initialization. 2164 byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY; 2165 byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), 2166 Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), Bytes.toBytes("006"), 2167 Bytes.add(Bytes.toBytes("006"), Bytes.multiple(maxByteArray, 8)), Bytes.toBytes("007"), 2168 Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)), Bytes.toBytes("008"), 2169 Bytes.multiple(maxByteArray, 2) }; 2170 TEST_UTIL.createTable(tableName, FAMILY, splitRows); 2171 TEST_UTIL.waitUntilAllRegionsAssigned(tableName); 2172 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 2173 try (RegionLocator l = conn.getRegionLocator(tableName)) { 2174 assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); 2175 } 2176 // Insert one row each region 2177 int insertNum = splitRows.length; 2178 for (byte[] splitRow : splitRows) { 2179 Put put = new Put(splitRow); 2180 put.addColumn(FAMILY, QUALIFIER, VALUE); 2181 table.put(put); 2182 } 2183 2184 // scan forward 2185 try (ResultScanner scanner = table.getScanner(new Scan())) { 2186 int count = 0; 2187 for (Result r : scanner) { 2188 assertFalse(r.isEmpty()); 2189 count++; 2190 } 2191 assertEquals(insertNum, count); 2192 } 2193 2194 // scan backward 2195 Scan scan = new Scan(); 2196 scan.setReversed(true); 2197 try (ResultScanner scanner = table.getScanner(scan)) { 2198 int count = 0; 2199 byte[] lastRow = null; 2200 for (Result r : scanner) { 2201 assertFalse(r.isEmpty()); 2202 count++; 2203 byte[] thisRow = r.getRow(); 2204 if (lastRow != null) { 2205 assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " 2206 + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); 2207 } 2208 lastRow = thisRow; 2209 } 2210 assertEquals(insertNum, count); 2211 } 2212 } 2213 } 2214 2215 /** 2216 * Tests reversed scan under multi regions 2217 */ 2218 @TestTemplate 2219 public void testSmallReversedScanUnderMultiRegions() throws Exception { 2220 // Test Initialization. 2221 byte[][] splitRows = new byte[][] { Bytes.toBytes("000"), Bytes.toBytes("002"), 2222 Bytes.toBytes("004"), Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010") }; 2223 TEST_UTIL.createTable(tableName, FAMILY, splitRows); 2224 TEST_UTIL.waitUntilAllRegionsAssigned(tableName); 2225 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 2226 try (RegionLocator l = conn.getRegionLocator(tableName)) { 2227 assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); 2228 } 2229 for (byte[] splitRow : splitRows) { 2230 Put put = new Put(splitRow); 2231 put.addColumn(FAMILY, QUALIFIER, VALUE); 2232 table.put(put); 2233 2234 byte[] nextRow = Bytes.copy(splitRow); 2235 nextRow[nextRow.length - 1]++; 2236 2237 put = new Put(nextRow); 2238 put.addColumn(FAMILY, QUALIFIER, VALUE); 2239 table.put(put); 2240 } 2241 2242 // scan forward 2243 try (ResultScanner scanner = table.getScanner(new Scan())) { 2244 int count = 0; 2245 for (Result r : scanner) { 2246 assertTrue(!r.isEmpty()); 2247 count++; 2248 } 2249 assertEquals(12, count); 2250 } 2251 2252 reverseScanTest(table, ReadType.STREAM); 2253 reverseScanTest(table, ReadType.PREAD); 2254 reverseScanTest(table, ReadType.DEFAULT); 2255 } 2256 } 2257 2258 private void reverseScanTest(Table table, ReadType readType) throws IOException { 2259 // scan backward 2260 Scan scan = new Scan(); 2261 scan.setReversed(true); 2262 try (ResultScanner scanner = table.getScanner(scan)) { 2263 int count = 0; 2264 byte[] lastRow = null; 2265 for (Result r : scanner) { 2266 assertTrue(!r.isEmpty()); 2267 count++; 2268 byte[] thisRow = r.getRow(); 2269 if (lastRow != null) { 2270 assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " 2271 + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); 2272 } 2273 lastRow = thisRow; 2274 } 2275 assertEquals(12, count); 2276 } 2277 2278 scan = new Scan(); 2279 scan.setReadType(readType); 2280 scan.setReversed(true); 2281 scan.withStartRow(Bytes.toBytes("002")); 2282 try (ResultScanner scanner = table.getScanner(scan)) { 2283 int count = 0; 2284 byte[] lastRow = null; 2285 for (Result r : scanner) { 2286 assertTrue(!r.isEmpty()); 2287 count++; 2288 byte[] thisRow = r.getRow(); 2289 if (lastRow != null) { 2290 assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " 2291 + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); 2292 } 2293 lastRow = thisRow; 2294 } 2295 assertEquals(3, count); // 000 001 002 2296 } 2297 2298 scan = new Scan(); 2299 scan.setReadType(readType); 2300 scan.setReversed(true); 2301 scan.withStartRow(Bytes.toBytes("002")); 2302 scan.withStopRow(Bytes.toBytes("000")); 2303 try (ResultScanner scanner = table.getScanner(scan)) { 2304 int count = 0; 2305 byte[] lastRow = null; 2306 for (Result r : scanner) { 2307 assertFalse(r.isEmpty()); 2308 count++; 2309 byte[] thisRow = r.getRow(); 2310 if (lastRow != null) { 2311 assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " 2312 + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); 2313 } 2314 lastRow = thisRow; 2315 } 2316 assertEquals(2, count); // 001 002 2317 } 2318 2319 scan = new Scan(); 2320 scan.setReadType(readType); 2321 scan.setReversed(true); 2322 scan.withStartRow(Bytes.toBytes("001")); 2323 try (ResultScanner scanner = table.getScanner(scan)) { 2324 int count = 0; 2325 byte[] lastRow = null; 2326 for (Result r : scanner) { 2327 assertFalse(r.isEmpty()); 2328 count++; 2329 byte[] thisRow = r.getRow(); 2330 if (lastRow != null) { 2331 assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " 2332 + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); 2333 } 2334 lastRow = thisRow; 2335 } 2336 assertEquals(2, count); // 000 001 2337 } 2338 2339 scan = new Scan(); 2340 scan.setReadType(readType); 2341 scan.setReversed(true); 2342 scan.withStartRow(Bytes.toBytes("000")); 2343 try (ResultScanner scanner = table.getScanner(scan)) { 2344 int count = 0; 2345 byte[] lastRow = null; 2346 for (Result r : scanner) { 2347 assertFalse(r.isEmpty()); 2348 count++; 2349 byte[] thisRow = r.getRow(); 2350 if (lastRow != null) { 2351 assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " 2352 + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); 2353 } 2354 lastRow = thisRow; 2355 } 2356 assertEquals(1, count); // 000 2357 } 2358 2359 scan = new Scan(); 2360 scan.setReadType(readType); 2361 scan.setReversed(true); 2362 scan.withStartRow(Bytes.toBytes("006")); 2363 scan.withStopRow(Bytes.toBytes("002")); 2364 try (ResultScanner scanner = table.getScanner(scan)) { 2365 int count = 0; 2366 byte[] lastRow = null; 2367 for (Result r : scanner) { 2368 assertFalse(r.isEmpty()); 2369 count++; 2370 byte[] thisRow = r.getRow(); 2371 if (lastRow != null) { 2372 assertTrue(Bytes.compareTo(thisRow, lastRow) < 0, "Error scan order, last row= " 2373 + Bytes.toString(lastRow) + ",this row=" + Bytes.toString(thisRow)); 2374 } 2375 lastRow = thisRow; 2376 } 2377 assertEquals(4, count); // 003 004 005 006 2378 } 2379 } 2380 2381 @TestTemplate 2382 public void testFilterAllRecords() throws IOException { 2383 Scan scan = new Scan(); 2384 scan.setBatch(1); 2385 scan.setCaching(1); 2386 // Filter out any records 2387 scan.setFilter(new FilterList(new FirstKeyOnlyFilter(), new InclusiveStopFilter(new byte[0]))); 2388 try (Connection conn = getConnection(); 2389 Table table = conn.getTable(TableName.META_TABLE_NAME)) { 2390 try (ResultScanner s = table.getScanner(scan)) { 2391 assertNull(s.next()); 2392 } 2393 } 2394 } 2395 2396 @TestTemplate 2397 public void testCellSizeLimit() throws IOException { 2398 TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) 2399 .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024)) 2400 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); 2401 try (Connection conn = getConnection()) { 2402 try (Admin admin = conn.getAdmin()) { 2403 admin.createTable(tableDescriptor); 2404 } 2405 try (Table t = conn.getTable(tableName)) { 2406 // Will succeed 2407 t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, Bytes.toBytes(0L))); 2408 t.increment(new Increment(ROW).addColumn(FAMILY, QUALIFIER, 1L)); 2409 2410 // Will succeed 2411 t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, new byte[9 * 1024])); 2412 2413 // Will fail 2414 assertThrows(IOException.class, 2415 () -> t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, new byte[10 * 1024])), 2416 "Oversize cell failed to trigger exception"); 2417 assertThrows(IOException.class, 2418 () -> t.append(new Append(ROW).addColumn(FAMILY, QUALIFIER, new byte[2 * 1024])), 2419 "Oversize cell failed to trigger exception"); 2420 } 2421 } 2422 } 2423 2424 @TestTemplate 2425 public void testCellSizeNoLimit() throws IOException { 2426 2427 TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) 2428 .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(0)) 2429 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); 2430 try (Connection conn = getConnection()) { 2431 try (Admin admin = conn.getAdmin()) { 2432 admin.createTable(tableDescriptor); 2433 } 2434 // Will succeed 2435 try (Table ht = conn.getTable(tableName)) { 2436 ht.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, 2437 new byte[HRegion.DEFAULT_MAX_CELL_SIZE - 1024])); 2438 ht.append(new Append(ROW).addColumn(FAMILY, QUALIFIER, new byte[1024 + 1])); 2439 } 2440 } 2441 } 2442 2443 @TestTemplate 2444 public void testDeleteSpecifiedVersionOfSpecifiedColumn() throws Exception { 2445 TEST_UTIL.createTable(tableName, FAMILY, 5); 2446 byte[][] VALUES = makeN(VALUE, 5); 2447 long[] ts = { 1000, 2000, 3000, 4000, 5000 }; 2448 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 2449 Put put = new Put(ROW); 2450 // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER 2451 for (int t = 0; t < 4; t++) { 2452 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUES[t]); 2453 } 2454 ht.put(put); 2455 2456 Delete delete = new Delete(ROW); 2457 // Delete version 3000 of column FAMILY:QUALIFIER 2458 delete.addColumn(FAMILY, QUALIFIER, ts[2]); 2459 ht.delete(delete); 2460 2461 Get get = new Get(ROW); 2462 get.addColumn(FAMILY, QUALIFIER); 2463 get.readVersions(Integer.MAX_VALUE); 2464 Result result = ht.get(get); 2465 // verify version 1000,2000,4000 remains for column FAMILY:QUALIFIER 2466 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[1], ts[3] }, 2467 new byte[][] { VALUES[0], VALUES[1], VALUES[3] }, 0, 2); 2468 2469 delete = new Delete(ROW); 2470 // Delete a version 5000 of column FAMILY:QUALIFIER which didn't exist 2471 delete.addColumn(FAMILY, QUALIFIER, ts[4]); 2472 ht.delete(delete); 2473 2474 get = new Get(ROW); 2475 get.addColumn(FAMILY, QUALIFIER); 2476 get.readVersions(Integer.MAX_VALUE); 2477 result = ht.get(get); 2478 // verify version 1000,2000,4000 remains for column FAMILY:QUALIFIER 2479 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[1], ts[3] }, 2480 new byte[][] { VALUES[0], VALUES[1], VALUES[3] }, 0, 2); 2481 } 2482 } 2483 2484 @TestTemplate 2485 public void testDeleteLatestVersionOfSpecifiedColumn() throws Exception { 2486 TEST_UTIL.createTable(tableName, FAMILY, 5); 2487 byte[][] VALUES = makeN(VALUE, 5); 2488 long[] ts = { 1000, 2000, 3000, 4000, 5000 }; 2489 try (Connection conn = getConnection(); Table ht = conn.getTable(tableName)) { 2490 Put put = new Put(ROW); 2491 // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER 2492 for (int t = 0; t < 4; t++) { 2493 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUES[t]); 2494 } 2495 ht.put(put); 2496 2497 Delete delete = new Delete(ROW); 2498 // Delete latest version of column FAMILY:QUALIFIER 2499 delete.addColumn(FAMILY, QUALIFIER); 2500 ht.delete(delete); 2501 2502 Get get = new Get(ROW); 2503 get.addColumn(FAMILY, QUALIFIER); 2504 get.readVersions(Integer.MAX_VALUE); 2505 Result result = ht.get(get); 2506 // verify version 1000,2000,3000 remains for column FAMILY:QUALIFIER 2507 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[1], ts[2] }, 2508 new byte[][] { VALUES[0], VALUES[1], VALUES[2] }, 0, 2); 2509 2510 delete = new Delete(ROW); 2511 // Delete two latest version of column FAMILY:QUALIFIER 2512 delete.addColumn(FAMILY, QUALIFIER); 2513 delete.addColumn(FAMILY, QUALIFIER); 2514 ht.delete(delete); 2515 2516 get = new Get(ROW); 2517 get.addColumn(FAMILY, QUALIFIER); 2518 get.readVersions(Integer.MAX_VALUE); 2519 result = ht.get(get); 2520 // verify version 1000 remains for column FAMILY:QUALIFIER 2521 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0] }, 2522 new byte[][] { VALUES[0] }, 0, 0); 2523 2524 put = new Put(ROW); 2525 // Put a version 5000 of column FAMILY:QUALIFIER 2526 put.addColumn(FAMILY, QUALIFIER, ts[4], VALUES[4]); 2527 ht.put(put); 2528 2529 get = new Get(ROW); 2530 get.addColumn(FAMILY, QUALIFIER); 2531 get.readVersions(Integer.MAX_VALUE); 2532 result = ht.get(get); 2533 // verify version 1000,5000 remains for column FAMILY:QUALIFIER 2534 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[4] }, 2535 new byte[][] { VALUES[0], VALUES[4] }, 0, 1); 2536 } 2537 } 2538 2539 /** 2540 * Test for HBASE-17125 2541 */ 2542 @TestTemplate 2543 public void testReadWithFilter() throws Exception { 2544 TEST_UTIL.createTable(tableName, FAMILY, 3); 2545 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 2546 byte[] VALUEA = Bytes.toBytes("value-a"); 2547 byte[] VALUEB = Bytes.toBytes("value-b"); 2548 long[] ts = { 1000, 2000, 3000, 4000 }; 2549 2550 Put put = new Put(ROW); 2551 // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER 2552 for (int t = 0; t <= 3; t++) { 2553 if (t <= 1) { 2554 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUEA); 2555 } else { 2556 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUEB); 2557 } 2558 } 2559 table.put(put); 2560 2561 Scan scan = new Scan() 2562 .setFilter(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("value-a"))) 2563 .readVersions(3); 2564 Result result = getSingleScanResult(table, scan); 2565 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 2566 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 2567 0, 0); 2568 2569 Get get = new Get(ROW) 2570 .setFilter(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("value-a"))) 2571 .readVersions(3); 2572 result = table.get(get); 2573 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 2574 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 2575 0, 0); 2576 2577 // Test with max versions 1, it should still read ts[1] 2578 scan = new Scan() 2579 .setFilter(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("value-a"))) 2580 .readVersions(1); 2581 result = getSingleScanResult(table, scan); 2582 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 2583 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 2584 0, 0); 2585 2586 // Test with max versions 1, it should still read ts[1] 2587 get = new Get(ROW) 2588 .setFilter(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("value-a"))) 2589 .readVersions(1); 2590 result = table.get(get); 2591 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 2592 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 2593 0, 0); 2594 2595 // Test with max versions 5, it should still read ts[1] 2596 scan = new Scan() 2597 .setFilter(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("value-a"))) 2598 .readVersions(5); 2599 result = getSingleScanResult(table, scan); 2600 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 2601 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 2602 0, 0); 2603 2604 // Test with max versions 5, it should still read ts[1] 2605 get = new Get(ROW) 2606 .setFilter(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("value-a"))) 2607 .readVersions(5); 2608 result = table.get(get); 2609 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 2610 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 2611 0, 0); 2612 } 2613 } 2614 2615 @TestTemplate 2616 public void testCellUtilTypeMethods() throws IOException { 2617 TEST_UTIL.createTable(tableName, FAMILY); 2618 try (Connection conn = getConnection(); Table table = conn.getTable(tableName)) { 2619 final byte[] row = Bytes.toBytes("p"); 2620 Put p = new Put(row); 2621 p.addColumn(FAMILY, QUALIFIER, VALUE); 2622 table.put(p); 2623 2624 try (ResultScanner scanner = table.getScanner(new Scan())) { 2625 Result result = scanner.next(); 2626 assertNotNull(result); 2627 CellScanner cs = result.cellScanner(); 2628 assertTrue(cs.advance()); 2629 Cell c = cs.current(); 2630 assertTrue(CellUtil.isPut(c)); 2631 assertFalse(CellUtil.isDelete(c)); 2632 assertFalse(cs.advance()); 2633 assertNull(scanner.next()); 2634 } 2635 2636 Delete d = new Delete(row); 2637 d.addColumn(FAMILY, QUALIFIER); 2638 table.delete(d); 2639 2640 Scan scan = new Scan(); 2641 scan.setRaw(true); 2642 try (ResultScanner scanner = table.getScanner(scan)) { 2643 Result result = scanner.next(); 2644 assertNotNull(result); 2645 CellScanner cs = result.cellScanner(); 2646 assertTrue(cs.advance()); 2647 2648 // First cell should be the delete (masking the Put) 2649 Cell c = cs.current(); 2650 assertTrue(CellUtil.isDelete(c), "Cell should be a Delete: " + c); 2651 assertFalse(CellUtil.isPut(c), "Cell should not be a Put: " + c); 2652 2653 // Second cell should be the original Put 2654 assertTrue(cs.advance()); 2655 c = cs.current(); 2656 assertFalse(CellUtil.isDelete(c), "Cell should not be a Delete: " + c); 2657 assertTrue(CellUtil.isPut(c), "Cell should be a Put: " + c); 2658 2659 // No more cells in this row 2660 assertFalse(cs.advance()); 2661 2662 // No more results in this scan 2663 assertNull(scanner.next()); 2664 } 2665 } 2666 } 2667 2668 @TestTemplate 2669 public void testCreateTableWithZeroRegionReplicas() throws Exception { 2670 TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) 2671 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))) 2672 .setRegionReplication(0).build(); 2673 2674 try (Connection conn = getConnection(); Admin admin = conn.getAdmin()) { 2675 assertThrows(DoNotRetryIOException.class, () -> admin.createTable(desc)); 2676 } 2677 } 2678 2679 @TestTemplate 2680 public void testModifyTableWithZeroRegionReplicas() throws Exception { 2681 TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) 2682 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build(); 2683 TableDescriptor newDesc = 2684 TableDescriptorBuilder.newBuilder(desc).setRegionReplication(0).build(); 2685 try (Connection conn = getConnection(); Admin admin = conn.getAdmin()) { 2686 admin.createTable(desc); 2687 assertThrows(DoNotRetryIOException.class, () -> admin.modifyTable(newDesc)); 2688 } 2689 } 2690 2691 @TestTemplate 2692 public void testModifyTableWithMemstoreData() throws Exception { 2693 createTableAndValidateTableSchemaModification(tableName, true); 2694 } 2695 2696 @TestTemplate 2697 public void testDeleteCFWithMemstoreData() throws Exception { 2698 createTableAndValidateTableSchemaModification(tableName, false); 2699 } 2700 2701 /** 2702 * Create table and validate online schema modification 2703 * @param tableName Table name 2704 * @param modifyTable Modify table if true otherwise delete column family 2705 * @throws IOException in case of failures 2706 */ 2707 private void createTableAndValidateTableSchemaModification(TableName tableName, 2708 boolean modifyTable) throws Exception { 2709 try (Connection conn = getConnection(); Admin admin = conn.getAdmin()) { 2710 // Create table with two Cfs 2711 byte[] cf1 = Bytes.toBytes("cf1"); 2712 byte[] cf2 = Bytes.toBytes("cf2"); 2713 TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(tableName) 2714 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf1)) 2715 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf2)).build(); 2716 admin.createTable(tableDesc); 2717 2718 Table t = TEST_UTIL.getConnection().getTable(tableName); 2719 // Insert few records and flush the table 2720 t.put(new Put(ROW).addColumn(cf1, QUALIFIER, Bytes.toBytes("val1"))); 2721 t.put(new Put(ROW).addColumn(cf2, QUALIFIER, Bytes.toBytes("val2"))); 2722 admin.flush(tableName); 2723 Path tableDir = CommonFSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), tableName); 2724 List<Path> regionDirs = FSUtils.getRegionDirs(TEST_UTIL.getTestFileSystem(), tableDir); 2725 assertEquals(1, regionDirs.size()); 2726 List<Path> familyDirs = 2727 FSUtils.getFamilyDirs(TEST_UTIL.getTestFileSystem(), regionDirs.get(0)); 2728 assertEquals(2, familyDirs.size()); 2729 2730 // Insert record but dont flush the table 2731 t.put(new Put(ROW).addColumn(cf1, QUALIFIER, Bytes.toBytes("val2"))); 2732 t.put(new Put(ROW).addColumn(cf2, QUALIFIER, Bytes.toBytes("val2"))); 2733 2734 if (modifyTable) { 2735 tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).removeColumnFamily(cf2).build(); 2736 admin.modifyTable(tableDesc); 2737 } else { 2738 admin.deleteColumnFamily(tableName, cf2); 2739 } 2740 // After table modification or delete family there should be only one CF in FS 2741 familyDirs = FSUtils.getFamilyDirs(TEST_UTIL.getTestFileSystem(), regionDirs.get(0)); 2742 assertEquals(1, familyDirs.size(), "CF dir count should be 1, but was " + familyDirs.size()); 2743 } 2744 } 2745}