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