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