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