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.assertSame; 026import static org.junit.Assert.assertTrue; 027import static org.junit.Assert.fail; 028 029import java.io.IOException; 030import java.util.ArrayList; 031import java.util.Arrays; 032import java.util.EnumSet; 033import java.util.HashMap; 034import java.util.HashSet; 035import java.util.Iterator; 036import java.util.LinkedList; 037import java.util.List; 038import java.util.Map; 039import java.util.NavigableMap; 040import java.util.concurrent.Callable; 041import java.util.concurrent.ExecutorService; 042import java.util.concurrent.Executors; 043import java.util.concurrent.atomic.AtomicReference; 044import org.apache.commons.lang3.ArrayUtils; 045import org.apache.hadoop.conf.Configuration; 046import org.apache.hadoop.fs.Path; 047import org.apache.hadoop.hbase.Cell; 048import org.apache.hadoop.hbase.CellScanner; 049import org.apache.hadoop.hbase.CellUtil; 050import org.apache.hadoop.hbase.ClusterMetrics.Option; 051import org.apache.hadoop.hbase.CompareOperator; 052import org.apache.hadoop.hbase.DoNotRetryIOException; 053import org.apache.hadoop.hbase.HBaseClassTestRule; 054import org.apache.hadoop.hbase.HBaseTestingUtility; 055import org.apache.hadoop.hbase.HColumnDescriptor; 056import org.apache.hadoop.hbase.HConstants; 057import org.apache.hadoop.hbase.HRegionInfo; 058import org.apache.hadoop.hbase.HRegionLocation; 059import org.apache.hadoop.hbase.HTableDescriptor; 060import org.apache.hadoop.hbase.KeepDeletedCells; 061import org.apache.hadoop.hbase.KeyValue; 062import org.apache.hadoop.hbase.MiniHBaseCluster; 063import org.apache.hadoop.hbase.PrivateCellUtil; 064import org.apache.hadoop.hbase.RegionLocations; 065import org.apache.hadoop.hbase.ServerName; 066import org.apache.hadoop.hbase.TableName; 067import org.apache.hadoop.hbase.Waiter; 068import org.apache.hadoop.hbase.client.metrics.ScanMetrics; 069import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; 070import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; 071import org.apache.hadoop.hbase.filter.BinaryComparator; 072import org.apache.hadoop.hbase.filter.Filter; 073import org.apache.hadoop.hbase.filter.FilterList; 074import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; 075import org.apache.hadoop.hbase.filter.InclusiveStopFilter; 076import org.apache.hadoop.hbase.filter.KeyOnlyFilter; 077import org.apache.hadoop.hbase.filter.LongComparator; 078import org.apache.hadoop.hbase.filter.PrefixFilter; 079import org.apache.hadoop.hbase.filter.QualifierFilter; 080import org.apache.hadoop.hbase.filter.RegexStringComparator; 081import org.apache.hadoop.hbase.filter.RowFilter; 082import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; 083import org.apache.hadoop.hbase.filter.SubstringComparator; 084import org.apache.hadoop.hbase.filter.ValueFilter; 085import org.apache.hadoop.hbase.filter.WhileMatchFilter; 086import org.apache.hadoop.hbase.io.TimeRange; 087import org.apache.hadoop.hbase.io.hfile.BlockCache; 088import org.apache.hadoop.hbase.io.hfile.CacheConfig; 089import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; 090import org.apache.hadoop.hbase.master.LoadBalancer; 091import org.apache.hadoop.hbase.protobuf.ProtobufUtil; 092import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; 093import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; 094import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; 095import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; 096import org.apache.hadoop.hbase.regionserver.HRegion; 097import org.apache.hadoop.hbase.regionserver.HRegionServer; 098import org.apache.hadoop.hbase.regionserver.HStore; 099import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; 100import org.apache.hadoop.hbase.testclassification.ClientTests; 101import org.apache.hadoop.hbase.testclassification.LargeTests; 102import org.apache.hadoop.hbase.util.Bytes; 103import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 104import org.apache.hadoop.hbase.util.FSUtils; 105import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge; 106import org.apache.hadoop.hbase.util.Pair; 107import org.apache.hadoop.hbase.util.TableDescriptorChecker; 108import org.junit.AfterClass; 109import org.junit.BeforeClass; 110import org.junit.ClassRule; 111import org.junit.Ignore; 112import org.junit.Rule; 113import org.junit.Test; 114import org.junit.experimental.categories.Category; 115import org.junit.rules.TestName; 116import org.slf4j.Logger; 117import org.slf4j.LoggerFactory; 118 119/** 120 * Run tests that use the HBase clients; {@link Table}. 121 * Sets up the HBase mini cluster once at start and runs through all client tests. 122 * Each creates a table named for the method and does its stuff against that. 123 */ 124@Category({LargeTests.class, ClientTests.class}) 125@SuppressWarnings ("deprecation") 126public class TestFromClientSide { 127 128 @ClassRule 129 public static final HBaseClassTestRule CLASS_RULE = 130 HBaseClassTestRule.forClass(TestFromClientSide.class); 131 132 // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide. 133 private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class); 134 protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); 135 private static byte [] ROW = Bytes.toBytes("testRow"); 136 private static byte [] FAMILY = Bytes.toBytes("testFamily"); 137 private static final byte[] INVALID_FAMILY = Bytes.toBytes("invalidTestFamily"); 138 private static byte [] QUALIFIER = Bytes.toBytes("testQualifier"); 139 private static byte [] VALUE = Bytes.toBytes("testValue"); 140 protected static int SLAVES = 3; 141 142 @Rule 143 public TestName name = new TestName(); 144 145 protected static final void initialize(Class<?>... cps) throws Exception { 146 // Uncomment the following lines if more verbosity is needed for 147 // debugging (see HBASE-12285 for details). 148 // ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); 149 // ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); 150 // ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); 151 // make sure that we do not get the same ts twice, see HBASE-19731 for more details. 152 EnvironmentEdgeManager.injectEdge(new NonRepeatedEnvironmentEdge()); 153 Configuration conf = TEST_UTIL.getConfiguration(); 154 conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, 155 Arrays.stream(cps).map(Class::getName).toArray(String[]::new)); 156 conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, true); // enable for below tests 157 // We need more than one region server in this test 158 TEST_UTIL.startMiniCluster(SLAVES); 159 } 160 161 @BeforeClass 162 public static void setUpBeforeClass() throws Exception { 163 initialize(MultiRowMutationEndpoint.class); 164 } 165 166 @AfterClass 167 public static void tearDownAfterClass() throws Exception { 168 TEST_UTIL.shutdownMiniCluster(); 169 } 170 171 /** 172 * Test append result when there are duplicate rpc request. 173 */ 174 @Test 175 public void testDuplicateAppend() throws Exception { 176 HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getMethodName()); 177 Map<String, String> kvs = new HashMap<>(); 178 kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000"); 179 hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs); 180 TEST_UTIL.createTable(hdt, new byte[][] { ROW }).close(); 181 182 Configuration c = new Configuration(TEST_UTIL.getConfiguration()); 183 c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50); 184 // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call 185 c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500); 186 187 try (Connection connection = ConnectionFactory.createConnection(c)) { 188 try (Table t = connection.getTable(TableName.valueOf(name.getMethodName()))) { 189 if (t instanceof HTable) { 190 HTable table = (HTable) t; 191 table.setOperationTimeout(3 * 1000); 192 193 Append append = new Append(ROW); 194 append.addColumn(HBaseTestingUtility.fam1, QUALIFIER, VALUE); 195 Result result = table.append(append); 196 197 // Verify expected result 198 Cell[] cells = result.rawCells(); 199 assertEquals(1, cells.length); 200 assertKey(cells[0], ROW, HBaseTestingUtility.fam1, QUALIFIER, VALUE); 201 202 // Verify expected result again 203 Result readResult = table.get(new Get(ROW)); 204 cells = readResult.rawCells(); 205 assertEquals(1, cells.length); 206 assertKey(cells[0], ROW, HBaseTestingUtility.fam1, QUALIFIER, VALUE); 207 } 208 } 209 } 210 } 211 212 /** 213 * Basic client side validation of HBASE-4536 214 */ 215 @Test 216 public void testKeepDeletedCells() throws Exception { 217 final TableName tableName = TableName.valueOf(name.getMethodName()); 218 final byte[] FAMILY = Bytes.toBytes("family"); 219 final byte[] C0 = Bytes.toBytes("c0"); 220 221 final byte[] T1 = Bytes.toBytes("T1"); 222 final byte[] T2 = Bytes.toBytes("T2"); 223 final byte[] T3 = Bytes.toBytes("T3"); 224 HColumnDescriptor hcd = 225 new HColumnDescriptor(FAMILY).setKeepDeletedCells(KeepDeletedCells.TRUE).setMaxVersions(3); 226 227 HTableDescriptor desc = new HTableDescriptor(tableName); 228 desc.addFamily(hcd); 229 TEST_UTIL.getAdmin().createTable(desc); 230 try (Table h = TEST_UTIL.getConnection().getTable(tableName)) { 231 long ts = System.currentTimeMillis(); 232 Put p = new Put(T1, ts); 233 p.addColumn(FAMILY, C0, T1); 234 h.put(p); 235 p = new Put(T1, ts + 2); 236 p.addColumn(FAMILY, C0, T2); 237 h.put(p); 238 p = new Put(T1, ts + 4); 239 p.addColumn(FAMILY, C0, T3); 240 h.put(p); 241 242 Delete d = new Delete(T1, ts + 3); 243 h.delete(d); 244 245 d = new Delete(T1, ts + 3); 246 d.addColumns(FAMILY, C0, ts + 3); 247 h.delete(d); 248 249 Get g = new Get(T1); 250 // does *not* include the delete 251 g.setTimeRange(0, ts + 3); 252 Result r = h.get(g); 253 assertArrayEquals(T2, r.getValue(FAMILY, C0)); 254 255 Scan s = new Scan(T1); 256 s.setTimeRange(0, ts + 3); 257 s.setMaxVersions(); 258 try (ResultScanner scanner = h.getScanner(s)) { 259 Cell[] kvs = scanner.next().rawCells(); 260 assertArrayEquals(T2, CellUtil.cloneValue(kvs[0])); 261 assertArrayEquals(T1, CellUtil.cloneValue(kvs[1])); 262 } 263 264 s = new Scan(T1); 265 s.setRaw(true); 266 s.setMaxVersions(); 267 try (ResultScanner scanner = h.getScanner(s)) { 268 Cell[] kvs = scanner.next().rawCells(); 269 assertTrue(PrivateCellUtil.isDeleteFamily(kvs[0])); 270 assertArrayEquals(T3, CellUtil.cloneValue(kvs[1])); 271 assertTrue(CellUtil.isDelete(kvs[2])); 272 assertArrayEquals(T2, CellUtil.cloneValue(kvs[3])); 273 assertArrayEquals(T1, CellUtil.cloneValue(kvs[4])); 274 } 275 } 276 } 277 278 /** 279 * Basic client side validation of HBASE-10118 280 */ 281 @Test 282 public void testPurgeFutureDeletes() throws Exception { 283 final TableName tableName = TableName.valueOf(name.getMethodName()); 284 final byte[] ROW = Bytes.toBytes("row"); 285 final byte[] FAMILY = Bytes.toBytes("family"); 286 final byte[] COLUMN = Bytes.toBytes("column"); 287 final byte[] VALUE = Bytes.toBytes("value"); 288 289 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 290 // future timestamp 291 long ts = System.currentTimeMillis() * 2; 292 Put put = new Put(ROW, ts); 293 put.addColumn(FAMILY, COLUMN, VALUE); 294 table.put(put); 295 296 Get get = new Get(ROW); 297 Result result = table.get(get); 298 assertArrayEquals(VALUE, result.getValue(FAMILY, COLUMN)); 299 300 Delete del = new Delete(ROW); 301 del.addColumn(FAMILY, COLUMN, ts); 302 table.delete(del); 303 304 get = new Get(ROW); 305 result = table.get(get); 306 assertNull(result.getValue(FAMILY, COLUMN)); 307 308 // major compaction, purged future deletes 309 TEST_UTIL.getAdmin().flush(tableName); 310 TEST_UTIL.getAdmin().majorCompact(tableName); 311 312 // waiting for the major compaction to complete 313 TEST_UTIL.waitFor(6000, new Waiter.Predicate<IOException>() { 314 @Override 315 public boolean evaluate() throws IOException { 316 return TEST_UTIL.getAdmin().getCompactionState(tableName) == CompactionState.NONE; 317 } 318 }); 319 320 put = new Put(ROW, ts); 321 put.addColumn(FAMILY, COLUMN, VALUE); 322 table.put(put); 323 324 get = new Get(ROW); 325 result = table.get(get); 326 assertArrayEquals(VALUE, result.getValue(FAMILY, COLUMN)); 327 } 328 } 329 330 /** 331 * Verifies that getConfiguration returns the same Configuration object used 332 * to create the HTable instance. 333 */ 334 @Test 335 public void testGetConfiguration() throws Exception { 336 final TableName tableName = TableName.valueOf(name.getMethodName()); 337 byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") }; 338 Configuration conf = TEST_UTIL.getConfiguration(); 339 try (Table table = TEST_UTIL.createTable(tableName, FAMILIES)) { 340 assertSame(conf, table.getConfiguration()); 341 } 342 } 343 344 /** 345 * Test from client side of an involved filter against a multi family that 346 * involves deletes. 347 */ 348 @Test 349 public void testWeirdCacheBehaviour() throws Exception { 350 final TableName tableName = TableName.valueOf(name.getMethodName()); 351 byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"), 352 Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"), 353 Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") }; 354 try (Table ht = TEST_UTIL.createTable(tableName, FAMILIES)) { 355 String value = "this is the value"; 356 String value2 = "this is some other value"; 357 String keyPrefix1 = TEST_UTIL.getRandomUUID().toString(); 358 String keyPrefix2 = TEST_UTIL.getRandomUUID().toString(); 359 String keyPrefix3 = TEST_UTIL.getRandomUUID().toString(); 360 putRows(ht, 3, value, keyPrefix1); 361 putRows(ht, 3, value, keyPrefix2); 362 putRows(ht, 3, value, keyPrefix3); 363 putRows(ht, 3, value2, keyPrefix1); 364 putRows(ht, 3, value2, keyPrefix2); 365 putRows(ht, 3, value2, keyPrefix3); 366 try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { 367 System.out.println("Checking values for key: " + keyPrefix1); 368 assertEquals("Got back incorrect number of rows from scan", 3, 369 getNumberOfRows(keyPrefix1, value2, table)); 370 System.out.println("Checking values for key: " + keyPrefix2); 371 assertEquals("Got back incorrect number of rows from scan", 3, 372 getNumberOfRows(keyPrefix2, value2, table)); 373 System.out.println("Checking values for key: " + keyPrefix3); 374 assertEquals("Got back incorrect number of rows from scan", 3, 375 getNumberOfRows(keyPrefix3, value2, table)); 376 deleteColumns(ht, value2, keyPrefix1); 377 deleteColumns(ht, value2, keyPrefix2); 378 deleteColumns(ht, value2, keyPrefix3); 379 System.out.println("Starting important checks....."); 380 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1, 381 0, getNumberOfRows(keyPrefix1, value2, table)); 382 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2, 383 0, getNumberOfRows(keyPrefix2, value2, table)); 384 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3, 385 0, getNumberOfRows(keyPrefix3, value2, table)); 386 } 387 } 388 } 389 390 private void deleteColumns(Table ht, String value, String keyPrefix) 391 throws IOException { 392 ResultScanner scanner = buildScanner(keyPrefix, value, ht); 393 Iterator<Result> it = scanner.iterator(); 394 int count = 0; 395 while (it.hasNext()) { 396 Result result = it.next(); 397 Delete delete = new Delete(result.getRow()); 398 delete.addColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2")); 399 ht.delete(delete); 400 count++; 401 } 402 assertEquals("Did not perform correct number of deletes", 3, count); 403 } 404 405 private int getNumberOfRows(String keyPrefix, String value, Table ht) 406 throws Exception { 407 ResultScanner resultScanner = buildScanner(keyPrefix, value, ht); 408 Iterator<Result> scanner = resultScanner.iterator(); 409 int numberOfResults = 0; 410 while (scanner.hasNext()) { 411 Result result = scanner.next(); 412 System.out.println("Got back key: " + Bytes.toString(result.getRow())); 413 for (Cell kv : result.rawCells()) { 414 System.out.println("kv=" + kv.toString() + ", " 415 + Bytes.toString(CellUtil.cloneValue(kv))); 416 } 417 numberOfResults++; 418 } 419 return numberOfResults; 420 } 421 422 private ResultScanner buildScanner(String keyPrefix, String value, Table ht) 423 throws IOException { 424 // OurFilterList allFilters = new OurFilterList(); 425 FilterList allFilters = new FilterList(/* FilterList.Operator.MUST_PASS_ALL */); 426 allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix))); 427 SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes 428 .toBytes("trans-tags"), Bytes.toBytes("qual2"), CompareOperator.EQUAL, Bytes 429 .toBytes(value)); 430 filter.setFilterIfMissing(true); 431 allFilters.addFilter(filter); 432 433 // allFilters.addFilter(new 434 // RowExcludingSingleColumnValueFilter(Bytes.toBytes("trans-tags"), 435 // Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value))); 436 437 Scan scan = new Scan(); 438 scan.addFamily(Bytes.toBytes("trans-blob")); 439 scan.addFamily(Bytes.toBytes("trans-type")); 440 scan.addFamily(Bytes.toBytes("trans-date")); 441 scan.addFamily(Bytes.toBytes("trans-tags")); 442 scan.addFamily(Bytes.toBytes("trans-group")); 443 scan.setFilter(allFilters); 444 445 return ht.getScanner(scan); 446 } 447 448 private void putRows(Table ht, int numRows, String value, String key) 449 throws IOException { 450 for (int i = 0; i < numRows; i++) { 451 String row = key + "_" + TEST_UTIL.getRandomUUID().toString(); 452 System.out.println(String.format("Saving row: %s, with value %s", row, 453 value)); 454 Put put = new Put(Bytes.toBytes(row)); 455 put.setDurability(Durability.SKIP_WAL); 456 put.addColumn(Bytes.toBytes("trans-blob"), null, Bytes 457 .toBytes("value for blob")); 458 put.addColumn(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement")); 459 put.addColumn(Bytes.toBytes("trans-date"), null, Bytes 460 .toBytes("20090921010101999")); 461 put.addColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"), Bytes 462 .toBytes(value)); 463 put.addColumn(Bytes.toBytes("trans-group"), null, Bytes 464 .toBytes("adhocTransactionGroupId")); 465 ht.put(put); 466 } 467 } 468 469 /** 470 * Test filters when multiple regions. It does counts. Needs eye-balling of 471 * logs to ensure that we're not scanning more regions that we're supposed to. 472 * Related to the TestFilterAcrossRegions over in the o.a.h.h.filter package. 473 */ 474 @Test 475 public void testFilterAcrossMultipleRegions() 476 throws IOException, InterruptedException { 477 final TableName tableName = TableName.valueOf(name.getMethodName()); 478 try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { 479 int rowCount = TEST_UTIL.loadTable(t, FAMILY, false); 480 assertRowCount(t, rowCount); 481 // Split the table. Should split on a reasonable key; 'lqj' 482 List<HRegionLocation> regions = splitTable(t); 483 assertRowCount(t, rowCount); 484 // Get end key of first region. 485 byte[] endKey = regions.get(0).getRegionInfo().getEndKey(); 486 // Count rows with a filter that stops us before passed 'endKey'. 487 // Should be count of rows in first region. 488 int endKeyCount = TEST_UTIL.countRows(t, createScanWithRowFilter(endKey)); 489 assertTrue(endKeyCount < rowCount); 490 491 // How do I know I did not got to second region? Thats tough. Can't really 492 // do that in client-side region test. I verified by tracing in debugger. 493 // I changed the messages that come out when set to DEBUG so should see 494 // when scanner is done. Says "Finished with scanning..." with region name. 495 // Check that its finished in right region. 496 497 // New test. Make it so scan goes into next region by one and then two. 498 // Make sure count comes out right. 499 byte[] key = new byte[]{endKey[0], endKey[1], (byte) (endKey[2] + 1)}; 500 int plusOneCount = TEST_UTIL.countRows(t, createScanWithRowFilter(key)); 501 assertEquals(endKeyCount + 1, plusOneCount); 502 key = new byte[]{endKey[0], endKey[1], (byte) (endKey[2] + 2)}; 503 int plusTwoCount = TEST_UTIL.countRows(t, createScanWithRowFilter(key)); 504 assertEquals(endKeyCount + 2, plusTwoCount); 505 506 // New test. Make it so I scan one less than endkey. 507 key = new byte[]{endKey[0], endKey[1], (byte) (endKey[2] - 1)}; 508 int minusOneCount = TEST_UTIL.countRows(t, createScanWithRowFilter(key)); 509 assertEquals(endKeyCount - 1, minusOneCount); 510 // For above test... study logs. Make sure we do "Finished with scanning.." 511 // in first region and that we do not fall into the next region. 512 513 key = new byte[]{'a', 'a', 'a'}; 514 int countBBB = TEST_UTIL.countRows(t, 515 createScanWithRowFilter(key, null, CompareOperator.EQUAL)); 516 assertEquals(1, countBBB); 517 518 int countGreater = TEST_UTIL.countRows(t, createScanWithRowFilter(endKey, null, 519 CompareOperator.GREATER_OR_EQUAL)); 520 // Because started at start of table. 521 assertEquals(0, countGreater); 522 countGreater = TEST_UTIL.countRows(t, createScanWithRowFilter(endKey, endKey, 523 CompareOperator.GREATER_OR_EQUAL)); 524 assertEquals(rowCount - endKeyCount, countGreater); 525 } 526 } 527 528 /* 529 * @param key 530 * @return Scan with RowFilter that does LESS than passed key. 531 */ 532 private Scan createScanWithRowFilter(final byte [] key) { 533 return createScanWithRowFilter(key, null, CompareOperator.LESS); 534 } 535 536 /* 537 * @param key 538 * @param op 539 * @param startRow 540 * @return Scan with RowFilter that does CompareOp op on passed key. 541 */ 542 private Scan createScanWithRowFilter(final byte [] key, 543 final byte [] startRow, CompareOperator op) { 544 // Make sure key is of some substance... non-null and > than first key. 545 assertTrue(key != null && key.length > 0 && 546 Bytes.BYTES_COMPARATOR.compare(key, new byte [] {'a', 'a', 'a'}) >= 0); 547 LOG.info("Key=" + Bytes.toString(key)); 548 Scan s = startRow == null? new Scan(): new Scan(startRow); 549 Filter f = new RowFilter(op, new BinaryComparator(key)); 550 f = new WhileMatchFilter(f); 551 s.setFilter(f); 552 return s; 553 } 554 555 private void assertRowCount(final Table t, final int expected) 556 throws IOException { 557 assertEquals(expected, TEST_UTIL.countRows(t, new Scan())); 558 } 559 560 /* 561 * Split table into multiple regions. 562 * @param t Table to split. 563 * @return Map of regions to servers. 564 * @throws IOException 565 */ 566 private List<HRegionLocation> splitTable(final Table t) 567 throws IOException, InterruptedException { 568 // Split this table in two. 569 try (Admin admin = TEST_UTIL.getAdmin()) { 570 admin.split(t.getName()); 571 } 572 List<HRegionLocation> regions = waitOnSplit(t); 573 assertTrue(regions.size() > 1); 574 return regions; 575 } 576 577 /* 578 * Wait on table split. May return because we waited long enough on the split 579 * and it didn't happen. Caller should check. 580 * @param t 581 * @return Map of table regions; caller needs to check table actually split. 582 */ 583 private List<HRegionLocation> waitOnSplit(final Table t) 584 throws IOException { 585 try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) { 586 List<HRegionLocation> regions = locator.getAllRegionLocations(); 587 int originalCount = regions.size(); 588 for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) { 589 Thread.currentThread(); 590 try { 591 Thread.sleep(1000); 592 } catch (InterruptedException e) { 593 e.printStackTrace(); 594 } 595 regions = locator.getAllRegionLocations(); 596 if (regions.size() > originalCount) 597 break; 598 } 599 return regions; 600 } 601 } 602 603 @Test 604 public void testSuperSimple() throws Exception { 605 final TableName tableName = TableName.valueOf(name.getMethodName()); 606 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 607 Put put = new Put(ROW); 608 put.addColumn(FAMILY, QUALIFIER, VALUE); 609 ht.put(put); 610 Scan scan = new Scan(); 611 scan.addColumn(FAMILY, tableName.toBytes()); 612 try (ResultScanner scanner = ht.getScanner(scan)) { 613 Result result = scanner.next(); 614 assertTrue("Expected null result", result == null); 615 } 616 } 617 } 618 619 @Test 620 public void testMaxKeyValueSize() throws Exception { 621 final TableName tableName = TableName.valueOf(name.getMethodName()); 622 Configuration conf = TEST_UTIL.getConfiguration(); 623 String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY); 624 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 625 byte[] value = new byte[4 * 1024 * 1024]; 626 Put put = new Put(ROW); 627 put.addColumn(FAMILY, QUALIFIER, value); 628 ht.put(put); 629 630 try { 631 TEST_UTIL.getConfiguration().setInt( 632 ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, 2 * 1024 * 1024); 633 // Create new table so we pick up the change in Configuration. 634 try (Connection connection = 635 ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { 636 try (Table t = connection.getTable(TableName.valueOf(FAMILY))) { 637 put = new Put(ROW); 638 put.addColumn(FAMILY, QUALIFIER, value); 639 t.put(put); 640 } 641 } 642 fail("Inserting a too large KeyValue worked, should throw exception"); 643 } catch (Exception e) { 644 } 645 } 646 conf.set(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, oldMaxSize); 647 } 648 649 @Test 650 public void testFilters() throws Exception { 651 final TableName tableName = TableName.valueOf(name.getMethodName()); 652 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 653 byte[][] ROWS = makeN(ROW, 10); 654 byte[][] QUALIFIERS = { 655 Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"), 656 Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"), 657 Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"), 658 Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"), 659 Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>") 660 }; 661 for (int i = 0; i < 10; i++) { 662 Put put = new Put(ROWS[i]); 663 put.setDurability(Durability.SKIP_WAL); 664 put.addColumn(FAMILY, QUALIFIERS[i], VALUE); 665 ht.put(put); 666 } 667 Scan scan = new Scan(); 668 scan.addFamily(FAMILY); 669 Filter filter = new QualifierFilter(CompareOperator.EQUAL, 670 new RegexStringComparator("col[1-5]")); 671 scan.setFilter(filter); 672 try (ResultScanner scanner = ht.getScanner(scan)) { 673 int expectedIndex = 1; 674 for (Result result : scanner) { 675 assertEquals(1, result.size()); 676 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[expectedIndex])); 677 assertTrue(Bytes.equals(CellUtil.cloneQualifier(result.rawCells()[0]), 678 QUALIFIERS[expectedIndex])); 679 expectedIndex++; 680 } 681 assertEquals(6, expectedIndex); 682 } 683 } 684 } 685 686 @Test 687 public void testFilterWithLongCompartor() throws Exception { 688 final TableName tableName = TableName.valueOf(name.getMethodName()); 689 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 690 byte[][] ROWS = makeN(ROW, 10); 691 byte[][] values = new byte[10][]; 692 for (int i = 0; i < 10; i++) { 693 values[i] = Bytes.toBytes(100L * i); 694 } 695 for (int i = 0; i < 10; i++) { 696 Put put = new Put(ROWS[i]); 697 put.setDurability(Durability.SKIP_WAL); 698 put.addColumn(FAMILY, QUALIFIER, values[i]); 699 ht.put(put); 700 } 701 Scan scan = new Scan(); 702 scan.addFamily(FAMILY); 703 Filter filter = new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOperator.GREATER, 704 new LongComparator(500)); 705 scan.setFilter(filter); 706 try (ResultScanner scanner = ht.getScanner(scan)) { 707 int expectedIndex = 0; 708 for (Result result : scanner) { 709 assertEquals(1, result.size()); 710 assertTrue(Bytes.toLong(result.getValue(FAMILY, QUALIFIER)) > 500); 711 expectedIndex++; 712 } 713 assertEquals(4, expectedIndex); 714 } 715 } 716 } 717 718 @Test 719 public void testKeyOnlyFilter() throws Exception { 720 final TableName tableName = TableName.valueOf(name.getMethodName()); 721 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 722 byte[][] ROWS = makeN(ROW, 10); 723 byte[][] QUALIFIERS = { 724 Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"), 725 Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"), 726 Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"), 727 Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"), 728 Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>") 729 }; 730 for (int i = 0; i < 10; i++) { 731 Put put = new Put(ROWS[i]); 732 put.setDurability(Durability.SKIP_WAL); 733 put.addColumn(FAMILY, QUALIFIERS[i], VALUE); 734 ht.put(put); 735 } 736 Scan scan = new Scan(); 737 scan.addFamily(FAMILY); 738 Filter filter = new KeyOnlyFilter(true); 739 scan.setFilter(filter); 740 try (ResultScanner scanner = ht.getScanner(scan)) { 741 int count = 0; 742 for (Result result : scanner) { 743 assertEquals(1, result.size()); 744 assertEquals(Bytes.SIZEOF_INT, result.rawCells()[0].getValueLength()); 745 assertEquals(VALUE.length, Bytes.toInt(CellUtil.cloneValue(result.rawCells()[0]))); 746 count++; 747 } 748 assertEquals(10, count); 749 } 750 } 751 } 752 753 /** 754 * Test simple table and non-existent row cases. 755 */ 756 @Test 757 public void testSimpleMissing() throws Exception { 758 final TableName tableName = TableName.valueOf(name.getMethodName()); 759 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 760 byte[][] ROWS = makeN(ROW, 4); 761 762 // Try to get a row on an empty table 763 Get get = new Get(ROWS[0]); 764 Result result = ht.get(get); 765 assertEmptyResult(result); 766 767 get = new Get(ROWS[0]); 768 get.addFamily(FAMILY); 769 result = ht.get(get); 770 assertEmptyResult(result); 771 772 get = new Get(ROWS[0]); 773 get.addColumn(FAMILY, QUALIFIER); 774 result = ht.get(get); 775 assertEmptyResult(result); 776 777 Scan scan = new Scan(); 778 result = getSingleScanResult(ht, scan); 779 assertNullResult(result); 780 781 scan = new Scan(ROWS[0]); 782 result = getSingleScanResult(ht, scan); 783 assertNullResult(result); 784 785 scan = new Scan(ROWS[0], ROWS[1]); 786 result = getSingleScanResult(ht, scan); 787 assertNullResult(result); 788 789 scan = new Scan(); 790 scan.addFamily(FAMILY); 791 result = getSingleScanResult(ht, scan); 792 assertNullResult(result); 793 794 scan = new Scan(); 795 scan.addColumn(FAMILY, QUALIFIER); 796 result = getSingleScanResult(ht, scan); 797 assertNullResult(result); 798 799 // Insert a row 800 801 Put put = new Put(ROWS[2]); 802 put.addColumn(FAMILY, QUALIFIER, VALUE); 803 ht.put(put); 804 805 // Try to get empty rows around it 806 807 get = new Get(ROWS[1]); 808 result = ht.get(get); 809 assertEmptyResult(result); 810 811 get = new Get(ROWS[0]); 812 get.addFamily(FAMILY); 813 result = ht.get(get); 814 assertEmptyResult(result); 815 816 get = new Get(ROWS[3]); 817 get.addColumn(FAMILY, QUALIFIER); 818 result = ht.get(get); 819 assertEmptyResult(result); 820 821 // Try to scan empty rows around it 822 823 scan = new Scan(ROWS[3]); 824 result = getSingleScanResult(ht, scan); 825 assertNullResult(result); 826 827 scan = new Scan(ROWS[0], ROWS[2]); 828 result = getSingleScanResult(ht, scan); 829 assertNullResult(result); 830 831 // Make sure we can actually get the row 832 833 get = new Get(ROWS[2]); 834 result = ht.get(get); 835 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 836 837 get = new Get(ROWS[2]); 838 get.addFamily(FAMILY); 839 result = ht.get(get); 840 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 841 842 get = new Get(ROWS[2]); 843 get.addColumn(FAMILY, QUALIFIER); 844 result = ht.get(get); 845 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 846 847 // Make sure we can scan the row 848 849 scan = new Scan(); 850 result = getSingleScanResult(ht, scan); 851 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 852 853 scan = new Scan(ROWS[0], ROWS[3]); 854 result = getSingleScanResult(ht, scan); 855 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 856 857 scan = new Scan(ROWS[2], ROWS[3]); 858 result = getSingleScanResult(ht, scan); 859 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 860 } 861 } 862 863 /** 864 * Test basic puts, gets, scans, and deletes for a single row 865 * in a multiple family table. 866 */ 867 @Test 868 public void testSingleRowMultipleFamily() throws Exception { 869 final TableName tableName = TableName.valueOf(name.getMethodName()); 870 byte [][] ROWS = makeN(ROW, 3); 871 byte [][] FAMILIES = makeNAscii(FAMILY, 10); 872 byte [][] QUALIFIERS = makeN(QUALIFIER, 10); 873 byte [][] VALUES = makeN(VALUE, 10); 874 875 try (Table ht = TEST_UTIL.createTable(tableName, FAMILIES)) { 876 877 Get get; 878 Scan scan; 879 Delete delete; 880 Put put; 881 Result result; 882 883 //////////////////////////////////////////////////////////////////////////// 884 // Insert one column to one family 885 //////////////////////////////////////////////////////////////////////////// 886 887 put = new Put(ROWS[0]); 888 put.addColumn(FAMILIES[4], QUALIFIERS[0], VALUES[0]); 889 ht.put(put); 890 891 // Get the single column 892 getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); 893 894 // Scan the single column 895 scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); 896 897 // Get empty results around inserted column 898 getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); 899 900 // Scan empty results around inserted column 901 scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); 902 903 //////////////////////////////////////////////////////////////////////////// 904 // Flush memstore and run same tests from storefiles 905 //////////////////////////////////////////////////////////////////////////// 906 907 TEST_UTIL.flush(); 908 909 // Redo get and scan tests from storefile 910 getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); 911 scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); 912 getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); 913 scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); 914 915 //////////////////////////////////////////////////////////////////////////// 916 // Now, Test reading from memstore and storefiles at once 917 //////////////////////////////////////////////////////////////////////////// 918 919 // Insert multiple columns to two other families 920 put = new Put(ROWS[0]); 921 put.addColumn(FAMILIES[2], QUALIFIERS[2], VALUES[2]); 922 put.addColumn(FAMILIES[2], QUALIFIERS[4], VALUES[4]); 923 put.addColumn(FAMILIES[4], QUALIFIERS[4], VALUES[4]); 924 put.addColumn(FAMILIES[6], QUALIFIERS[6], VALUES[6]); 925 put.addColumn(FAMILIES[6], QUALIFIERS[7], VALUES[7]); 926 put.addColumn(FAMILIES[7], QUALIFIERS[7], VALUES[7]); 927 put.addColumn(FAMILIES[9], QUALIFIERS[0], VALUES[0]); 928 ht.put(put); 929 930 // Get multiple columns across multiple families and get empties around it 931 singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); 932 933 // Scan multiple columns across multiple families and scan empties around it 934 singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); 935 936 //////////////////////////////////////////////////////////////////////////// 937 // Flush the table again 938 //////////////////////////////////////////////////////////////////////////// 939 940 TEST_UTIL.flush(); 941 942 // Redo tests again 943 singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); 944 singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); 945 946 // Insert more data to memstore 947 put = new Put(ROWS[0]); 948 put.addColumn(FAMILIES[6], QUALIFIERS[5], VALUES[5]); 949 put.addColumn(FAMILIES[6], QUALIFIERS[8], VALUES[8]); 950 put.addColumn(FAMILIES[6], QUALIFIERS[9], VALUES[9]); 951 put.addColumn(FAMILIES[4], QUALIFIERS[3], VALUES[3]); 952 ht.put(put); 953 954 //////////////////////////////////////////////////////////////////////////// 955 // Delete a storefile column 956 //////////////////////////////////////////////////////////////////////////// 957 delete = new Delete(ROWS[0]); 958 delete.addColumns(FAMILIES[6], QUALIFIERS[7]); 959 ht.delete(delete); 960 961 // Try to get deleted column 962 get = new Get(ROWS[0]); 963 get.addColumn(FAMILIES[6], QUALIFIERS[7]); 964 result = ht.get(get); 965 assertEmptyResult(result); 966 967 // Try to scan deleted column 968 scan = new Scan(); 969 scan.addColumn(FAMILIES[6], QUALIFIERS[7]); 970 result = getSingleScanResult(ht, scan); 971 assertNullResult(result); 972 973 // Make sure we can still get a column before it and after it 974 get = new Get(ROWS[0]); 975 get.addColumn(FAMILIES[6], QUALIFIERS[6]); 976 result = ht.get(get); 977 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); 978 979 get = new Get(ROWS[0]); 980 get.addColumn(FAMILIES[6], QUALIFIERS[8]); 981 result = ht.get(get); 982 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]); 983 984 // Make sure we can still scan a column before it and after it 985 scan = new Scan(); 986 scan.addColumn(FAMILIES[6], QUALIFIERS[6]); 987 result = getSingleScanResult(ht, scan); 988 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); 989 990 scan = new Scan(); 991 scan.addColumn(FAMILIES[6], QUALIFIERS[8]); 992 result = getSingleScanResult(ht, scan); 993 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]); 994 995 //////////////////////////////////////////////////////////////////////////// 996 // Delete a memstore column 997 //////////////////////////////////////////////////////////////////////////// 998 delete = new Delete(ROWS[0]); 999 delete.addColumns(FAMILIES[6], QUALIFIERS[8]); 1000 ht.delete(delete); 1001 1002 // Try to get deleted column 1003 get = new Get(ROWS[0]); 1004 get.addColumn(FAMILIES[6], QUALIFIERS[8]); 1005 result = ht.get(get); 1006 assertEmptyResult(result); 1007 1008 // Try to scan deleted column 1009 scan = new Scan(); 1010 scan.addColumn(FAMILIES[6], QUALIFIERS[8]); 1011 result = getSingleScanResult(ht, scan); 1012 assertNullResult(result); 1013 1014 // Make sure we can still get a column before it and after it 1015 get = new Get(ROWS[0]); 1016 get.addColumn(FAMILIES[6], QUALIFIERS[6]); 1017 result = ht.get(get); 1018 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); 1019 1020 get = new Get(ROWS[0]); 1021 get.addColumn(FAMILIES[6], QUALIFIERS[9]); 1022 result = ht.get(get); 1023 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); 1024 1025 // Make sure we can still scan a column before it and after it 1026 scan = new Scan(); 1027 scan.addColumn(FAMILIES[6], QUALIFIERS[6]); 1028 result = getSingleScanResult(ht, scan); 1029 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); 1030 1031 scan = new Scan(); 1032 scan.addColumn(FAMILIES[6], QUALIFIERS[9]); 1033 result = getSingleScanResult(ht, scan); 1034 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); 1035 1036 //////////////////////////////////////////////////////////////////////////// 1037 // Delete joint storefile/memstore family 1038 //////////////////////////////////////////////////////////////////////////// 1039 1040 delete = new Delete(ROWS[0]); 1041 delete.addFamily(FAMILIES[4]); 1042 ht.delete(delete); 1043 1044 // Try to get storefile column in deleted family 1045 get = new Get(ROWS[0]); 1046 get.addColumn(FAMILIES[4], QUALIFIERS[4]); 1047 result = ht.get(get); 1048 assertEmptyResult(result); 1049 1050 // Try to get memstore column in deleted family 1051 get = new Get(ROWS[0]); 1052 get.addColumn(FAMILIES[4], QUALIFIERS[3]); 1053 result = ht.get(get); 1054 assertEmptyResult(result); 1055 1056 // Try to get deleted family 1057 get = new Get(ROWS[0]); 1058 get.addFamily(FAMILIES[4]); 1059 result = ht.get(get); 1060 assertEmptyResult(result); 1061 1062 // Try to scan storefile column in deleted family 1063 scan = new Scan(); 1064 scan.addColumn(FAMILIES[4], QUALIFIERS[4]); 1065 result = getSingleScanResult(ht, scan); 1066 assertNullResult(result); 1067 1068 // Try to scan memstore column in deleted family 1069 scan = new Scan(); 1070 scan.addColumn(FAMILIES[4], QUALIFIERS[3]); 1071 result = getSingleScanResult(ht, scan); 1072 assertNullResult(result); 1073 1074 // Try to scan deleted family 1075 scan = new Scan(); 1076 scan.addFamily(FAMILIES[4]); 1077 result = getSingleScanResult(ht, scan); 1078 assertNullResult(result); 1079 1080 // Make sure we can still get another family 1081 get = new Get(ROWS[0]); 1082 get.addColumn(FAMILIES[2], QUALIFIERS[2]); 1083 result = ht.get(get); 1084 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); 1085 1086 get = new Get(ROWS[0]); 1087 get.addColumn(FAMILIES[6], QUALIFIERS[9]); 1088 result = ht.get(get); 1089 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); 1090 1091 // Make sure we can still scan another family 1092 scan = new Scan(); 1093 scan.addColumn(FAMILIES[6], QUALIFIERS[6]); 1094 result = getSingleScanResult(ht, scan); 1095 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); 1096 1097 scan = new Scan(); 1098 scan.addColumn(FAMILIES[6], QUALIFIERS[9]); 1099 result = getSingleScanResult(ht, scan); 1100 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); 1101 1102 //////////////////////////////////////////////////////////////////////////// 1103 // Flush everything and rerun delete tests 1104 //////////////////////////////////////////////////////////////////////////// 1105 1106 TEST_UTIL.flush(); 1107 1108 // Try to get storefile column in deleted family 1109 get = new Get(ROWS[0]); 1110 get.addColumn(FAMILIES[4], QUALIFIERS[4]); 1111 result = ht.get(get); 1112 assertEmptyResult(result); 1113 1114 // Try to get memstore column in deleted family 1115 get = new Get(ROWS[0]); 1116 get.addColumn(FAMILIES[4], QUALIFIERS[3]); 1117 result = ht.get(get); 1118 assertEmptyResult(result); 1119 1120 // Try to get deleted family 1121 get = new Get(ROWS[0]); 1122 get.addFamily(FAMILIES[4]); 1123 result = ht.get(get); 1124 assertEmptyResult(result); 1125 1126 // Try to scan storefile column in deleted family 1127 scan = new Scan(); 1128 scan.addColumn(FAMILIES[4], QUALIFIERS[4]); 1129 result = getSingleScanResult(ht, scan); 1130 assertNullResult(result); 1131 1132 // Try to scan memstore column in deleted family 1133 scan = new Scan(); 1134 scan.addColumn(FAMILIES[4], QUALIFIERS[3]); 1135 result = getSingleScanResult(ht, scan); 1136 assertNullResult(result); 1137 1138 // Try to scan deleted family 1139 scan = new Scan(); 1140 scan.addFamily(FAMILIES[4]); 1141 result = getSingleScanResult(ht, scan); 1142 assertNullResult(result); 1143 1144 // Make sure we can still get another family 1145 get = new Get(ROWS[0]); 1146 get.addColumn(FAMILIES[2], QUALIFIERS[2]); 1147 result = ht.get(get); 1148 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); 1149 1150 get = new Get(ROWS[0]); 1151 get.addColumn(FAMILIES[6], QUALIFIERS[9]); 1152 result = ht.get(get); 1153 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); 1154 1155 // Make sure we can still scan another family 1156 scan = new Scan(); 1157 scan.addColumn(FAMILIES[6], QUALIFIERS[6]); 1158 result = getSingleScanResult(ht, scan); 1159 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); 1160 1161 scan = new Scan(); 1162 scan.addColumn(FAMILIES[6], QUALIFIERS[9]); 1163 result = getSingleScanResult(ht, scan); 1164 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); 1165 } 1166 } 1167 1168 @Test(expected = NullPointerException.class) 1169 public void testNullTableName() throws IOException { 1170 // Null table name (should NOT work) 1171 TEST_UTIL.createTable((TableName)null, FAMILY); 1172 fail("Creating a table with null name passed, should have failed"); 1173 } 1174 1175 @Test(expected = IllegalArgumentException.class) 1176 public void testNullFamilyName() throws IOException { 1177 final TableName tableName = TableName.valueOf(name.getMethodName()); 1178 1179 // Null family (should NOT work) 1180 TEST_UTIL.createTable(tableName, new byte[][]{null}); 1181 fail("Creating a table with a null family passed, should fail"); 1182 } 1183 1184 @Test 1185 public void testNullRowAndQualifier() throws Exception { 1186 final TableName tableName = TableName.valueOf(name.getMethodName()); 1187 1188 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 1189 1190 // Null row (should NOT work) 1191 try { 1192 Put put = new Put((byte[]) null); 1193 put.addColumn(FAMILY, QUALIFIER, VALUE); 1194 ht.put(put); 1195 fail("Inserting a null row worked, should throw exception"); 1196 } catch (Exception e) { 1197 } 1198 1199 // Null qualifier (should work) 1200 { 1201 Put put = new Put(ROW); 1202 put.addColumn(FAMILY, null, VALUE); 1203 ht.put(put); 1204 1205 getTestNull(ht, ROW, FAMILY, VALUE); 1206 1207 scanTestNull(ht, ROW, FAMILY, VALUE); 1208 1209 Delete delete = new Delete(ROW); 1210 delete.addColumns(FAMILY, null); 1211 ht.delete(delete); 1212 1213 Get get = new Get(ROW); 1214 Result result = ht.get(get); 1215 assertEmptyResult(result); 1216 } 1217 } 1218 } 1219 1220 @Test 1221 public void testNullEmptyQualifier() throws Exception { 1222 final TableName tableName = TableName.valueOf(name.getMethodName()); 1223 1224 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 1225 1226 // Empty qualifier, byte[0] instead of null (should work) 1227 try { 1228 Put put = new Put(ROW); 1229 put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); 1230 ht.put(put); 1231 1232 getTestNull(ht, ROW, FAMILY, VALUE); 1233 1234 scanTestNull(ht, ROW, FAMILY, VALUE); 1235 1236 // Flush and try again 1237 1238 TEST_UTIL.flush(); 1239 1240 getTestNull(ht, ROW, FAMILY, VALUE); 1241 1242 scanTestNull(ht, ROW, FAMILY, VALUE); 1243 1244 Delete delete = new Delete(ROW); 1245 delete.addColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY); 1246 ht.delete(delete); 1247 1248 Get get = new Get(ROW); 1249 Result result = ht.get(get); 1250 assertEmptyResult(result); 1251 1252 } catch (Exception e) { 1253 throw new IOException("Using a row with null qualifier should not throw exception"); 1254 } 1255 } 1256 } 1257 1258 @Test 1259 public void testNullValue() throws IOException { 1260 final TableName tableName = TableName.valueOf(name.getMethodName()); 1261 1262 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 1263 // Null value 1264 try { 1265 Put put = new Put(ROW); 1266 put.addColumn(FAMILY, QUALIFIER, null); 1267 ht.put(put); 1268 1269 Get get = new Get(ROW); 1270 get.addColumn(FAMILY, QUALIFIER); 1271 Result result = ht.get(get); 1272 assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); 1273 1274 Scan scan = new Scan(); 1275 scan.addColumn(FAMILY, QUALIFIER); 1276 result = getSingleScanResult(ht, scan); 1277 assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); 1278 1279 Delete delete = new Delete(ROW); 1280 delete.addColumns(FAMILY, QUALIFIER); 1281 ht.delete(delete); 1282 1283 get = new Get(ROW); 1284 result = ht.get(get); 1285 assertEmptyResult(result); 1286 1287 } catch (Exception e) { 1288 throw new IOException("Null values should be allowed, but threw exception"); 1289 } 1290 } 1291 } 1292 1293 @Test 1294 public void testNullQualifier() throws Exception { 1295 final TableName tableName = TableName.valueOf(name.getMethodName()); 1296 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 1297 1298 // Work for Put 1299 Put put = new Put(ROW); 1300 put.addColumn(FAMILY, null, VALUE); 1301 table.put(put); 1302 1303 // Work for Get, Scan 1304 getTestNull(table, ROW, FAMILY, VALUE); 1305 scanTestNull(table, ROW, FAMILY, VALUE); 1306 1307 // Work for Delete 1308 Delete delete = new Delete(ROW); 1309 delete.addColumns(FAMILY, null); 1310 table.delete(delete); 1311 1312 Get get = new Get(ROW); 1313 Result result = table.get(get); 1314 assertEmptyResult(result); 1315 1316 // Work for Increment/Append 1317 Increment increment = new Increment(ROW); 1318 increment.addColumn(FAMILY, null, 1L); 1319 table.increment(increment); 1320 getTestNull(table, ROW, FAMILY, 1L); 1321 1322 table.incrementColumnValue(ROW, FAMILY, null, 1L); 1323 getTestNull(table, ROW, FAMILY, 2L); 1324 1325 delete = new Delete(ROW); 1326 delete.addColumns(FAMILY, null); 1327 table.delete(delete); 1328 1329 Append append = new Append(ROW); 1330 append.addColumn(FAMILY, null, VALUE); 1331 table.append(append); 1332 getTestNull(table, ROW, FAMILY, VALUE); 1333 1334 // Work for checkAndMutate using thenPut, thenMutate and thenDelete 1335 put = new Put(ROW); 1336 put.addColumn(FAMILY, null, Bytes.toBytes("checkAndPut")); 1337 table.put(put); 1338 table.checkAndMutate(ROW, FAMILY).ifEquals(VALUE).thenPut(put); 1339 1340 RowMutations mutate = new RowMutations(ROW); 1341 mutate.add(new Put(ROW).addColumn(FAMILY, null, Bytes.toBytes("checkAndMutate"))); 1342 table.checkAndMutate(ROW, FAMILY).ifEquals(Bytes.toBytes("checkAndPut")).thenMutate(mutate); 1343 1344 delete = new Delete(ROW); 1345 delete.addColumns(FAMILY, null); 1346 table.checkAndMutate(ROW, FAMILY).ifEquals(Bytes.toBytes("checkAndMutate")) 1347 .thenDelete(delete); 1348 } 1349 } 1350 1351 @Test 1352 public void testVersions() throws Exception { 1353 final TableName tableName = TableName.valueOf(name.getMethodName()); 1354 1355 long [] STAMPS = makeStamps(20); 1356 byte [][] VALUES = makeNAscii(VALUE, 20); 1357 1358 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 1359 1360 // Insert 4 versions of same column 1361 Put put = new Put(ROW); 1362 put.addColumn(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 1363 put.addColumn(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 1364 put.addColumn(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 1365 put.addColumn(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 1366 ht.put(put); 1367 1368 // Verify we can get each one properly 1369 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 1370 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 1371 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 1372 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 1373 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 1374 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 1375 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 1376 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 1377 1378 // Verify we don't accidentally get others 1379 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 1380 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 1381 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 1382 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 1383 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 1384 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 1385 1386 // Ensure maxVersions in query is respected 1387 Get get = new Get(ROW); 1388 get.addColumn(FAMILY, QUALIFIER); 1389 get.setMaxVersions(2); 1390 Result result = ht.get(get); 1391 assertNResult(result, ROW, FAMILY, QUALIFIER, 1392 new long [] {STAMPS[4], STAMPS[5]}, 1393 new byte[][] {VALUES[4], VALUES[5]}, 1394 0, 1); 1395 1396 Scan scan = new Scan(ROW); 1397 scan.addColumn(FAMILY, QUALIFIER); 1398 scan.setMaxVersions(2); 1399 result = getSingleScanResult(ht, scan); 1400 assertNResult(result, ROW, FAMILY, QUALIFIER, 1401 new long[]{STAMPS[4], STAMPS[5]}, 1402 new byte[][]{VALUES[4], VALUES[5]}, 1403 0, 1); 1404 1405 // Flush and redo 1406 1407 TEST_UTIL.flush(); 1408 1409 // Verify we can get each one properly 1410 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 1411 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 1412 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 1413 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 1414 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 1415 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 1416 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 1417 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 1418 1419 // Verify we don't accidentally get others 1420 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 1421 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 1422 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 1423 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 1424 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 1425 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 1426 1427 // Ensure maxVersions in query is respected 1428 get = new Get(ROW); 1429 get.addColumn(FAMILY, QUALIFIER); 1430 get.setMaxVersions(2); 1431 result = ht.get(get); 1432 assertNResult(result, ROW, FAMILY, QUALIFIER, 1433 new long [] {STAMPS[4], STAMPS[5]}, 1434 new byte[][] {VALUES[4], VALUES[5]}, 1435 0, 1); 1436 1437 scan = new Scan(ROW); 1438 scan.addColumn(FAMILY, QUALIFIER); 1439 scan.setMaxVersions(2); 1440 result = getSingleScanResult(ht, scan); 1441 assertNResult(result, ROW, FAMILY, QUALIFIER, 1442 new long[]{STAMPS[4], STAMPS[5]}, 1443 new byte[][]{VALUES[4], VALUES[5]}, 1444 0, 1); 1445 1446 1447 // Add some memstore and retest 1448 1449 // Insert 4 more versions of same column and a dupe 1450 put = new Put(ROW); 1451 put.addColumn(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 1452 put.addColumn(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]); 1453 put.addColumn(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); 1454 put.addColumn(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]); 1455 ht.put(put); 1456 1457 // Ensure maxVersions in query is respected 1458 get = new Get(ROW); 1459 get.addColumn(FAMILY, QUALIFIER); 1460 get.setMaxVersions(); 1461 result = ht.get(get); 1462 assertNResult(result, ROW, FAMILY, QUALIFIER, 1463 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], 1464 STAMPS[8]}, 1465 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], 1466 VALUES[8]}, 1467 0, 7); 1468 1469 scan = new Scan(ROW); 1470 scan.addColumn(FAMILY, QUALIFIER); 1471 scan.setMaxVersions(); 1472 result = getSingleScanResult(ht, scan); 1473 assertNResult(result, ROW, FAMILY, QUALIFIER, 1474 new long[]{STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], 1475 STAMPS[8]}, 1476 new byte[][]{VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], 1477 VALUES[8]},0, 7); 1478 1479 get = new Get(ROW); 1480 get.setMaxVersions(); 1481 result = ht.get(get); 1482 assertNResult(result, ROW, FAMILY, QUALIFIER, 1483 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], 1484 STAMPS[8]}, 1485 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], 1486 VALUES[8]}, 1487 0, 7); 1488 1489 scan = new Scan(ROW); 1490 scan.setMaxVersions(); 1491 result = getSingleScanResult(ht, scan); 1492 assertNResult(result, ROW, FAMILY, QUALIFIER, 1493 new long[]{STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], 1494 STAMPS[8]}, 1495 new byte[][]{VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], 1496 VALUES[8]},0, 7); 1497 1498 // Verify we can get each one properly 1499 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 1500 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 1501 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 1502 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); 1503 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 1504 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 1505 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 1506 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); 1507 1508 // Verify we don't accidentally get others 1509 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 1510 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]); 1511 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 1512 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]); 1513 1514 // Ensure maxVersions of table is respected 1515 1516 TEST_UTIL.flush(); 1517 1518 // Insert 4 more versions of same column and a dupe 1519 put = new Put(ROW); 1520 put.addColumn(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]); 1521 put.addColumn(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]); 1522 put.addColumn(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]); 1523 put.addColumn(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]); 1524 ht.put(put); 1525 1526 get = new Get(ROW); 1527 get.addColumn(FAMILY, QUALIFIER); 1528 get.setMaxVersions(Integer.MAX_VALUE); 1529 result = ht.get(get); 1530 assertNResult(result, ROW, FAMILY, QUALIFIER, 1531 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], 1532 STAMPS[11], STAMPS[13], STAMPS[15]}, 1533 new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], 1534 VALUES[11], VALUES[13], VALUES[15]}, 1535 0, 9); 1536 1537 scan = new Scan(ROW); 1538 scan.addColumn(FAMILY, QUALIFIER); 1539 scan.setMaxVersions(Integer.MAX_VALUE); 1540 result = getSingleScanResult(ht, scan); 1541 assertNResult(result, ROW, FAMILY, QUALIFIER, 1542 new long[]{STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], 1543 STAMPS[11], STAMPS[13], STAMPS[15]}, 1544 new byte[][]{VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], 1545 VALUES[11], VALUES[13], VALUES[15]},0, 9); 1546 1547 // Delete a version in the memstore and a version in a storefile 1548 Delete delete = new Delete(ROW); 1549 delete.addColumn(FAMILY, QUALIFIER, STAMPS[11]); 1550 delete.addColumn(FAMILY, QUALIFIER, STAMPS[7]); 1551 ht.delete(delete); 1552 1553 // Test that it's gone 1554 get = new Get(ROW); 1555 get.addColumn(FAMILY, QUALIFIER); 1556 get.setMaxVersions(Integer.MAX_VALUE); 1557 result = ht.get(get); 1558 assertNResult(result, ROW, FAMILY, QUALIFIER, 1559 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], 1560 STAMPS[9], STAMPS[13], STAMPS[15]}, 1561 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], 1562 VALUES[9], VALUES[13], VALUES[15]}, 1563 0, 9); 1564 1565 scan = new Scan(ROW); 1566 scan.addColumn(FAMILY, QUALIFIER); 1567 scan.setMaxVersions(Integer.MAX_VALUE); 1568 result = getSingleScanResult(ht, scan); 1569 assertNResult(result, ROW, FAMILY, QUALIFIER, 1570 new long[]{STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], 1571 STAMPS[9], STAMPS[13], STAMPS[15]}, 1572 new byte[][]{VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], 1573 VALUES[9], VALUES[13], VALUES[15]},0, 9); 1574 } 1575 } 1576 1577 @Test 1578 @SuppressWarnings("checkstyle:MethodLength") 1579 public void testVersionLimits() throws Exception { 1580 final TableName tableName = TableName.valueOf(name.getMethodName()); 1581 byte [][] FAMILIES = makeNAscii(FAMILY, 3); 1582 int [] LIMITS = {1,3,5}; 1583 long [] STAMPS = makeStamps(10); 1584 byte [][] VALUES = makeNAscii(VALUE, 10); 1585 try (Table ht = TEST_UTIL.createTable(tableName, FAMILIES, LIMITS)) { 1586 1587 // Insert limit + 1 on each family 1588 Put put = new Put(ROW); 1589 put.addColumn(FAMILIES[0], QUALIFIER, STAMPS[0], VALUES[0]); 1590 put.addColumn(FAMILIES[0], QUALIFIER, STAMPS[1], VALUES[1]); 1591 put.addColumn(FAMILIES[1], QUALIFIER, STAMPS[0], VALUES[0]); 1592 put.addColumn(FAMILIES[1], QUALIFIER, STAMPS[1], VALUES[1]); 1593 put.addColumn(FAMILIES[1], QUALIFIER, STAMPS[2], VALUES[2]); 1594 put.addColumn(FAMILIES[1], QUALIFIER, STAMPS[3], VALUES[3]); 1595 put.addColumn(FAMILIES[2], QUALIFIER, STAMPS[0], VALUES[0]); 1596 put.addColumn(FAMILIES[2], QUALIFIER, STAMPS[1], VALUES[1]); 1597 put.addColumn(FAMILIES[2], QUALIFIER, STAMPS[2], VALUES[2]); 1598 put.addColumn(FAMILIES[2], QUALIFIER, STAMPS[3], VALUES[3]); 1599 put.addColumn(FAMILIES[2], QUALIFIER, STAMPS[4], VALUES[4]); 1600 put.addColumn(FAMILIES[2], QUALIFIER, STAMPS[5], VALUES[5]); 1601 put.addColumn(FAMILIES[2], QUALIFIER, STAMPS[6], VALUES[6]); 1602 ht.put(put); 1603 1604 // Verify we only get the right number out of each 1605 1606 // Family0 1607 1608 Get get = new Get(ROW); 1609 get.addColumn(FAMILIES[0], QUALIFIER); 1610 get.setMaxVersions(Integer.MAX_VALUE); 1611 Result result = ht.get(get); 1612 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 1613 new long [] {STAMPS[1]}, 1614 new byte[][] {VALUES[1]}, 1615 0, 0); 1616 1617 get = new Get(ROW); 1618 get.addFamily(FAMILIES[0]); 1619 get.setMaxVersions(Integer.MAX_VALUE); 1620 result = ht.get(get); 1621 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 1622 new long [] {STAMPS[1]}, 1623 new byte[][] {VALUES[1]}, 1624 0, 0); 1625 1626 Scan scan = new Scan(ROW); 1627 scan.addColumn(FAMILIES[0], QUALIFIER); 1628 scan.setMaxVersions(Integer.MAX_VALUE); 1629 result = getSingleScanResult(ht, scan); 1630 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 1631 new long[]{STAMPS[1]}, 1632 new byte[][]{VALUES[1]}, 1633 0, 0); 1634 1635 scan = new Scan(ROW); 1636 scan.addFamily(FAMILIES[0]); 1637 scan.setMaxVersions(Integer.MAX_VALUE); 1638 result = getSingleScanResult(ht, scan); 1639 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 1640 new long[]{STAMPS[1]}, 1641 new byte[][]{VALUES[1]}, 1642 0, 0); 1643 1644 // Family1 1645 1646 get = new Get(ROW); 1647 get.addColumn(FAMILIES[1], QUALIFIER); 1648 get.setMaxVersions(Integer.MAX_VALUE); 1649 result = ht.get(get); 1650 assertNResult(result, ROW, FAMILIES[1], QUALIFIER, 1651 new long [] {STAMPS[1], STAMPS[2], STAMPS[3]}, 1652 new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, 1653 0, 2); 1654 1655 get = new Get(ROW); 1656 get.addFamily(FAMILIES[1]); 1657 get.setMaxVersions(Integer.MAX_VALUE); 1658 result = ht.get(get); 1659 assertNResult(result, ROW, FAMILIES[1], QUALIFIER, 1660 new long [] {STAMPS[1], STAMPS[2], STAMPS[3]}, 1661 new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, 1662 0, 2); 1663 1664 scan = new Scan(ROW); 1665 scan.addColumn(FAMILIES[1], QUALIFIER); 1666 scan.setMaxVersions(Integer.MAX_VALUE); 1667 result = getSingleScanResult(ht, scan); 1668 assertNResult(result, ROW, FAMILIES[1], QUALIFIER, 1669 new long[]{STAMPS[1], STAMPS[2], STAMPS[3]}, 1670 new byte[][]{VALUES[1], VALUES[2], VALUES[3]}, 1671 0, 2); 1672 1673 scan = new Scan(ROW); 1674 scan.addFamily(FAMILIES[1]); 1675 scan.setMaxVersions(Integer.MAX_VALUE); 1676 result = getSingleScanResult(ht, scan); 1677 assertNResult(result, ROW, FAMILIES[1], QUALIFIER, 1678 new long[]{STAMPS[1], STAMPS[2], STAMPS[3]}, 1679 new byte[][]{VALUES[1], VALUES[2], VALUES[3]}, 1680 0, 2); 1681 1682 // Family2 1683 1684 get = new Get(ROW); 1685 get.addColumn(FAMILIES[2], QUALIFIER); 1686 get.setMaxVersions(Integer.MAX_VALUE); 1687 result = ht.get(get); 1688 assertNResult(result, ROW, FAMILIES[2], QUALIFIER, 1689 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, 1690 new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, 1691 0, 4); 1692 1693 get = new Get(ROW); 1694 get.addFamily(FAMILIES[2]); 1695 get.setMaxVersions(Integer.MAX_VALUE); 1696 result = ht.get(get); 1697 assertNResult(result, ROW, FAMILIES[2], QUALIFIER, 1698 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, 1699 new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, 1700 0, 4); 1701 1702 scan = new Scan(ROW); 1703 scan.addColumn(FAMILIES[2], QUALIFIER); 1704 scan.setMaxVersions(Integer.MAX_VALUE); 1705 result = getSingleScanResult(ht, scan); 1706 assertNResult(result, ROW, FAMILIES[2], QUALIFIER, 1707 new long[]{STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, 1708 new byte[][]{VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, 1709 0, 4); 1710 1711 scan = new Scan(ROW); 1712 scan.addFamily(FAMILIES[2]); 1713 scan.setMaxVersions(Integer.MAX_VALUE); 1714 result = getSingleScanResult(ht, scan); 1715 assertNResult(result, ROW, FAMILIES[2], QUALIFIER, 1716 new long[]{STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, 1717 new byte[][]{VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, 1718 0, 4); 1719 1720 // Try all families 1721 1722 get = new Get(ROW); 1723 get.setMaxVersions(Integer.MAX_VALUE); 1724 result = ht.get(get); 1725 assertTrue("Expected 9 keys but received " + result.size(), 1726 result.size() == 9); 1727 1728 get = new Get(ROW); 1729 get.addFamily(FAMILIES[0]); 1730 get.addFamily(FAMILIES[1]); 1731 get.addFamily(FAMILIES[2]); 1732 get.setMaxVersions(Integer.MAX_VALUE); 1733 result = ht.get(get); 1734 assertTrue("Expected 9 keys but received " + result.size(), 1735 result.size() == 9); 1736 1737 get = new Get(ROW); 1738 get.addColumn(FAMILIES[0], QUALIFIER); 1739 get.addColumn(FAMILIES[1], QUALIFIER); 1740 get.addColumn(FAMILIES[2], QUALIFIER); 1741 get.setMaxVersions(Integer.MAX_VALUE); 1742 result = ht.get(get); 1743 assertTrue("Expected 9 keys but received " + result.size(), 1744 result.size() == 9); 1745 1746 scan = new Scan(ROW); 1747 scan.setMaxVersions(Integer.MAX_VALUE); 1748 result = getSingleScanResult(ht, scan); 1749 assertTrue("Expected 9 keys but received " + result.size(), 1750 result.size() == 9); 1751 1752 scan = new Scan(ROW); 1753 scan.setMaxVersions(Integer.MAX_VALUE); 1754 scan.addFamily(FAMILIES[0]); 1755 scan.addFamily(FAMILIES[1]); 1756 scan.addFamily(FAMILIES[2]); 1757 result = getSingleScanResult(ht, scan); 1758 assertTrue("Expected 9 keys but received " + result.size(), 1759 result.size() == 9); 1760 1761 scan = new Scan(ROW); 1762 scan.setMaxVersions(Integer.MAX_VALUE); 1763 scan.addColumn(FAMILIES[0], QUALIFIER); 1764 scan.addColumn(FAMILIES[1], QUALIFIER); 1765 scan.addColumn(FAMILIES[2], QUALIFIER); 1766 result = getSingleScanResult(ht, scan); 1767 assertTrue("Expected 9 keys but received " + result.size(), 1768 result.size() == 9); 1769 } 1770 } 1771 1772 @Test 1773 public void testDeleteFamilyVersion() throws Exception { 1774 try (Admin admin = TEST_UTIL.getAdmin()) { 1775 final TableName tableName = TableName.valueOf(name.getMethodName()); 1776 1777 byte[][] QUALIFIERS = makeNAscii(QUALIFIER, 1); 1778 byte[][] VALUES = makeN(VALUE, 5); 1779 long[] ts = {1000, 2000, 3000, 4000, 5000}; 1780 1781 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 5)) { 1782 1783 Put put = new Put(ROW); 1784 for (int q = 0; q < 1; q++) { 1785 for (int t = 0; t < 5; t++) { 1786 put.addColumn(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]); 1787 } 1788 } 1789 ht.put(put); 1790 admin.flush(tableName); 1791 1792 Delete delete = new Delete(ROW); 1793 delete.addFamilyVersion(FAMILY, ts[1]); // delete version '2000' 1794 delete.addFamilyVersion(FAMILY, ts[3]); // delete version '4000' 1795 ht.delete(delete); 1796 admin.flush(tableName); 1797 1798 for (int i = 0; i < 1; i++) { 1799 Get get = new Get(ROW); 1800 get.addColumn(FAMILY, QUALIFIERS[i]); 1801 get.setMaxVersions(Integer.MAX_VALUE); 1802 Result result = ht.get(get); 1803 // verify version '1000'/'3000'/'5000' remains for all columns 1804 assertNResult(result, ROW, FAMILY, QUALIFIERS[i], 1805 new long[]{ts[0], ts[2], ts[4]}, 1806 new byte[][]{VALUES[0], VALUES[2], VALUES[4]}, 1807 0, 2); 1808 } 1809 } 1810 } 1811 } 1812 1813 @Test 1814 public void testDeleteFamilyVersionWithOtherDeletes() throws Exception { 1815 final TableName tableName = TableName.valueOf(name.getMethodName()); 1816 1817 byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 5); 1818 byte [][] VALUES = makeN(VALUE, 5); 1819 long [] ts = {1000, 2000, 3000, 4000, 5000}; 1820 1821 try (Admin admin = TEST_UTIL.getAdmin(); 1822 Table ht = TEST_UTIL.createTable(tableName, FAMILY, 5)) { 1823 Put put = null; 1824 Result result = null; 1825 Get get = null; 1826 Delete delete = null; 1827 1828 // 1. put on ROW 1829 put = new Put(ROW); 1830 for (int q = 0; q < 5; q++) { 1831 for (int t = 0; t < 5; t++) { 1832 put.addColumn(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]); 1833 } 1834 } 1835 ht.put(put); 1836 admin.flush(tableName); 1837 1838 // 2. put on ROWS[0] 1839 byte[] ROW2 = Bytes.toBytes("myRowForTest"); 1840 put = new Put(ROW2); 1841 for (int q = 0; q < 5; q++) { 1842 for (int t = 0; t < 5; t++) { 1843 put.addColumn(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]); 1844 } 1845 } 1846 ht.put(put); 1847 admin.flush(tableName); 1848 1849 // 3. delete on ROW 1850 delete = new Delete(ROW); 1851 // delete version <= 2000 of all columns 1852 // note: addFamily must be the first since it will mask 1853 // the subsequent other type deletes! 1854 delete.addFamily(FAMILY, ts[1]); 1855 // delete version '4000' of all columns 1856 delete.addFamilyVersion(FAMILY, ts[3]); 1857 // delete version <= 3000 of column 0 1858 delete.addColumns(FAMILY, QUALIFIERS[0], ts[2]); 1859 // delete version <= 5000 of column 2 1860 delete.addColumns(FAMILY, QUALIFIERS[2], ts[4]); 1861 // delete version 5000 of column 4 1862 delete.addColumn(FAMILY, QUALIFIERS[4], ts[4]); 1863 ht.delete(delete); 1864 admin.flush(tableName); 1865 1866 // 4. delete on ROWS[0] 1867 delete = new Delete(ROW2); 1868 delete.addFamilyVersion(FAMILY, ts[1]); // delete version '2000' 1869 delete.addFamilyVersion(FAMILY, ts[3]); // delete version '4000' 1870 ht.delete(delete); 1871 admin.flush(tableName); 1872 1873 // 5. check ROW 1874 get = new Get(ROW); 1875 get.addColumn(FAMILY, QUALIFIERS[0]); 1876 get.setMaxVersions(Integer.MAX_VALUE); 1877 result = ht.get(get); 1878 assertNResult(result, ROW, FAMILY, QUALIFIERS[0], 1879 new long[]{ts[4]}, 1880 new byte[][]{VALUES[4]}, 1881 0, 0); 1882 1883 get = new Get(ROW); 1884 get.addColumn(FAMILY, QUALIFIERS[1]); 1885 get.setMaxVersions(Integer.MAX_VALUE); 1886 result = ht.get(get); 1887 assertNResult(result, ROW, FAMILY, QUALIFIERS[1], 1888 new long[]{ts[2], ts[4]}, 1889 new byte[][]{VALUES[2], VALUES[4]}, 1890 0, 1); 1891 1892 get = new Get(ROW); 1893 get.addColumn(FAMILY, QUALIFIERS[2]); 1894 get.setMaxVersions(Integer.MAX_VALUE); 1895 result = ht.get(get); 1896 assertEquals(0, result.size()); 1897 1898 get = new Get(ROW); 1899 get.addColumn(FAMILY, QUALIFIERS[3]); 1900 get.setMaxVersions(Integer.MAX_VALUE); 1901 result = ht.get(get); 1902 assertNResult(result, ROW, FAMILY, QUALIFIERS[3], 1903 new long[]{ts[2], ts[4]}, 1904 new byte[][]{VALUES[2], VALUES[4]}, 1905 0, 1); 1906 1907 get = new Get(ROW); 1908 get.addColumn(FAMILY, QUALIFIERS[4]); 1909 get.setMaxVersions(Integer.MAX_VALUE); 1910 result = ht.get(get); 1911 assertNResult(result, ROW, FAMILY, QUALIFIERS[4], 1912 new long[]{ts[2]}, 1913 new byte[][]{VALUES[2]}, 1914 0, 0); 1915 1916 // 6. check ROWS[0] 1917 for (int i = 0; i < 5; i++) { 1918 get = new Get(ROW2); 1919 get.addColumn(FAMILY, QUALIFIERS[i]); 1920 get.readVersions(Integer.MAX_VALUE); 1921 result = ht.get(get); 1922 // verify version '1000'/'3000'/'5000' remains for all columns 1923 assertNResult(result, ROW2, FAMILY, QUALIFIERS[i], 1924 new long[]{ts[0], ts[2], ts[4]}, 1925 new byte[][]{VALUES[0], VALUES[2], VALUES[4]}, 1926 0, 2); 1927 } 1928 } 1929 } 1930 1931 @Test 1932 public void testDeleteWithFailed() throws Exception { 1933 final TableName tableName = TableName.valueOf(name.getMethodName()); 1934 1935 byte [][] FAMILIES = makeNAscii(FAMILY, 3); 1936 byte [][] VALUES = makeN(VALUE, 5); 1937 long [] ts = {1000, 2000, 3000, 4000, 5000}; 1938 1939 try (Table ht = TEST_UTIL.createTable(tableName, FAMILIES, 3)) { 1940 Put put = new Put(ROW); 1941 put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); 1942 ht.put(put); 1943 1944 // delete wrong family 1945 Delete delete = new Delete(ROW); 1946 delete.addFamily(FAMILIES[1], ts[0]); 1947 ht.delete(delete); 1948 1949 Get get = new Get(ROW); 1950 get.addFamily(FAMILIES[0]); 1951 get.readAllVersions(); 1952 Result result = ht.get(get); 1953 assertTrue(Bytes.equals(result.getValue(FAMILIES[0], QUALIFIER), VALUES[0])); 1954 } 1955 } 1956 1957 @Test 1958 public void testDeletes() throws Exception { 1959 final TableName tableName = TableName.valueOf(name.getMethodName()); 1960 1961 byte [][] ROWS = makeNAscii(ROW, 6); 1962 byte [][] FAMILIES = makeNAscii(FAMILY, 3); 1963 byte [][] VALUES = makeN(VALUE, 5); 1964 long [] ts = {1000, 2000, 3000, 4000, 5000}; 1965 1966 try (Table ht = TEST_UTIL.createTable(tableName, FAMILIES, 3)) { 1967 1968 Put put = new Put(ROW); 1969 put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); 1970 put.addColumn(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]); 1971 ht.put(put); 1972 1973 Delete delete = new Delete(ROW); 1974 delete.addFamily(FAMILIES[0], ts[0]); 1975 ht.delete(delete); 1976 1977 Get get = new Get(ROW); 1978 get.addFamily(FAMILIES[0]); 1979 get.setMaxVersions(Integer.MAX_VALUE); 1980 Result result = ht.get(get); 1981 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 1982 new long [] {ts[1]}, 1983 new byte[][] {VALUES[1]}, 1984 0, 0); 1985 1986 Scan scan = new Scan(ROW); 1987 scan.addFamily(FAMILIES[0]); 1988 scan.setMaxVersions(Integer.MAX_VALUE); 1989 result = getSingleScanResult(ht, scan); 1990 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 1991 new long[]{ts[1]}, 1992 new byte[][]{VALUES[1]}, 1993 0, 0); 1994 1995 // Test delete latest version 1996 put = new Put(ROW); 1997 put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); 1998 put.addColumn(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]); 1999 put.addColumn(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]); 2000 put.addColumn(FAMILIES[0], null, ts[4], VALUES[4]); 2001 put.addColumn(FAMILIES[0], null, ts[2], VALUES[2]); 2002 put.addColumn(FAMILIES[0], null, ts[3], VALUES[3]); 2003 ht.put(put); 2004 2005 delete = new Delete(ROW); 2006 delete.addColumn(FAMILIES[0], QUALIFIER); // ts[4] 2007 ht.delete(delete); 2008 2009 get = new Get(ROW); 2010 get.addColumn(FAMILIES[0], QUALIFIER); 2011 get.setMaxVersions(Integer.MAX_VALUE); 2012 result = ht.get(get); 2013 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 2014 new long [] {ts[1], ts[2], ts[3]}, 2015 new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, 2016 0, 2); 2017 2018 scan = new Scan(ROW); 2019 scan.addColumn(FAMILIES[0], QUALIFIER); 2020 scan.setMaxVersions(Integer.MAX_VALUE); 2021 result = getSingleScanResult(ht, scan); 2022 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 2023 new long[]{ts[1], ts[2], ts[3]}, 2024 new byte[][]{VALUES[1], VALUES[2], VALUES[3]}, 2025 0, 2); 2026 2027 // Test for HBASE-1847 2028 delete = new Delete(ROW); 2029 delete.addColumn(FAMILIES[0], null); 2030 ht.delete(delete); 2031 2032 // Cleanup null qualifier 2033 delete = new Delete(ROW); 2034 delete.addColumns(FAMILIES[0], null); 2035 ht.delete(delete); 2036 2037 // Expected client behavior might be that you can re-put deleted values 2038 // But alas, this is not to be. We can't put them back in either case. 2039 2040 put = new Put(ROW); 2041 put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); // 1000 2042 put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); // 5000 2043 ht.put(put); 2044 2045 2046 // It used to be due to the internal implementation of Get, that 2047 // the Get() call would return ts[4] UNLIKE the Scan below. With 2048 // the switch to using Scan for Get this is no longer the case. 2049 get = new Get(ROW); 2050 get.addFamily(FAMILIES[0]); 2051 get.setMaxVersions(Integer.MAX_VALUE); 2052 result = ht.get(get); 2053 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 2054 new long [] {ts[1], ts[2], ts[3]}, 2055 new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, 2056 0, 2); 2057 2058 // The Scanner returns the previous values, the expected-naive-unexpected behavior 2059 2060 scan = new Scan(ROW); 2061 scan.addFamily(FAMILIES[0]); 2062 scan.setMaxVersions(Integer.MAX_VALUE); 2063 result = getSingleScanResult(ht, scan); 2064 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, 2065 new long[]{ts[1], ts[2], ts[3]}, 2066 new byte[][]{VALUES[1], VALUES[2], VALUES[3]}, 2067 0, 2); 2068 2069 // Test deleting an entire family from one row but not the other various ways 2070 2071 put = new Put(ROWS[0]); 2072 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 2073 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 2074 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 2075 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 2076 ht.put(put); 2077 2078 put = new Put(ROWS[1]); 2079 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 2080 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 2081 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 2082 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 2083 ht.put(put); 2084 2085 put = new Put(ROWS[2]); 2086 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 2087 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 2088 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 2089 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 2090 ht.put(put); 2091 2092 // Assert that above went in. 2093 get = new Get(ROWS[2]); 2094 get.addFamily(FAMILIES[1]); 2095 get.addFamily(FAMILIES[2]); 2096 get.setMaxVersions(Integer.MAX_VALUE); 2097 result = ht.get(get); 2098 assertTrue("Expected 4 key but received " + result.size() + ": " + result, 2099 result.size() == 4); 2100 2101 delete = new Delete(ROWS[0]); 2102 delete.addFamily(FAMILIES[2]); 2103 ht.delete(delete); 2104 2105 delete = new Delete(ROWS[1]); 2106 delete.addColumns(FAMILIES[1], QUALIFIER); 2107 ht.delete(delete); 2108 2109 delete = new Delete(ROWS[2]); 2110 delete.addColumn(FAMILIES[1], QUALIFIER); 2111 delete.addColumn(FAMILIES[1], QUALIFIER); 2112 delete.addColumn(FAMILIES[2], QUALIFIER); 2113 ht.delete(delete); 2114 2115 get = new Get(ROWS[0]); 2116 get.addFamily(FAMILIES[1]); 2117 get.addFamily(FAMILIES[2]); 2118 get.setMaxVersions(Integer.MAX_VALUE); 2119 result = ht.get(get); 2120 assertTrue("Expected 2 keys but received " + result.size(), 2121 result.size() == 2); 2122 assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, 2123 new long [] {ts[0], ts[1]}, 2124 new byte[][] {VALUES[0], VALUES[1]}, 2125 0, 1); 2126 2127 scan = new Scan(ROWS[0]); 2128 scan.addFamily(FAMILIES[1]); 2129 scan.addFamily(FAMILIES[2]); 2130 scan.setMaxVersions(Integer.MAX_VALUE); 2131 result = getSingleScanResult(ht, scan); 2132 assertTrue("Expected 2 keys but received " + result.size(), 2133 result.size() == 2); 2134 assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, 2135 new long[]{ts[0], ts[1]}, 2136 new byte[][]{VALUES[0], VALUES[1]}, 2137 0, 1); 2138 2139 get = new Get(ROWS[1]); 2140 get.addFamily(FAMILIES[1]); 2141 get.addFamily(FAMILIES[2]); 2142 get.setMaxVersions(Integer.MAX_VALUE); 2143 result = ht.get(get); 2144 assertTrue("Expected 2 keys but received " + result.size(), 2145 result.size() == 2); 2146 2147 scan = new Scan(ROWS[1]); 2148 scan.addFamily(FAMILIES[1]); 2149 scan.addFamily(FAMILIES[2]); 2150 scan.setMaxVersions(Integer.MAX_VALUE); 2151 result = getSingleScanResult(ht, scan); 2152 assertTrue("Expected 2 keys but received " + result.size(), 2153 result.size() == 2); 2154 2155 get = new Get(ROWS[2]); 2156 get.addFamily(FAMILIES[1]); 2157 get.addFamily(FAMILIES[2]); 2158 get.setMaxVersions(Integer.MAX_VALUE); 2159 result = ht.get(get); 2160 assertEquals(1, result.size()); 2161 assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER, 2162 new long [] {ts[2]}, 2163 new byte[][] {VALUES[2]}, 2164 0, 0); 2165 2166 scan = new Scan(ROWS[2]); 2167 scan.addFamily(FAMILIES[1]); 2168 scan.addFamily(FAMILIES[2]); 2169 scan.setMaxVersions(Integer.MAX_VALUE); 2170 result = getSingleScanResult(ht, scan); 2171 assertEquals(1, result.size()); 2172 assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER, 2173 new long[]{ts[2]}, 2174 new byte[][]{VALUES[2]}, 2175 0, 0); 2176 2177 // Test if we delete the family first in one row (HBASE-1541) 2178 2179 delete = new Delete(ROWS[3]); 2180 delete.addFamily(FAMILIES[1]); 2181 ht.delete(delete); 2182 2183 put = new Put(ROWS[3]); 2184 put.addColumn(FAMILIES[2], QUALIFIER, VALUES[0]); 2185 ht.put(put); 2186 2187 put = new Put(ROWS[4]); 2188 put.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]); 2189 put.addColumn(FAMILIES[2], QUALIFIER, VALUES[2]); 2190 ht.put(put); 2191 2192 get = new Get(ROWS[3]); 2193 get.addFamily(FAMILIES[1]); 2194 get.addFamily(FAMILIES[2]); 2195 get.setMaxVersions(Integer.MAX_VALUE); 2196 result = ht.get(get); 2197 assertTrue("Expected 1 key but received " + result.size(), 2198 result.size() == 1); 2199 2200 get = new Get(ROWS[4]); 2201 get.addFamily(FAMILIES[1]); 2202 get.addFamily(FAMILIES[2]); 2203 get.setMaxVersions(Integer.MAX_VALUE); 2204 result = ht.get(get); 2205 assertTrue("Expected 2 keys but received " + result.size(), 2206 result.size() == 2); 2207 2208 scan = new Scan(ROWS[3]); 2209 scan.addFamily(FAMILIES[1]); 2210 scan.addFamily(FAMILIES[2]); 2211 scan.setMaxVersions(Integer.MAX_VALUE); 2212 try (ResultScanner scanner = ht.getScanner(scan)) { 2213 result = scanner.next(); 2214 assertTrue("Expected 1 key but received " + result.size(), 2215 result.size() == 1); 2216 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[3])); 2217 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[0])); 2218 result = scanner.next(); 2219 assertTrue("Expected 2 keys but received " + result.size(), 2220 result.size() == 2); 2221 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[4])); 2222 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[1]), ROWS[4])); 2223 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[1])); 2224 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[1]), VALUES[2])); 2225 } 2226 2227 // Add test of bulk deleting. 2228 for (int i = 0; i < 10; i++) { 2229 byte[] bytes = Bytes.toBytes(i); 2230 put = new Put(bytes); 2231 put.setDurability(Durability.SKIP_WAL); 2232 put.addColumn(FAMILIES[0], QUALIFIER, bytes); 2233 ht.put(put); 2234 } 2235 for (int i = 0; i < 10; i++) { 2236 byte[] bytes = Bytes.toBytes(i); 2237 get = new Get(bytes); 2238 get.addFamily(FAMILIES[0]); 2239 result = ht.get(get); 2240 assertTrue(result.size() == 1); 2241 } 2242 ArrayList<Delete> deletes = new ArrayList<>(); 2243 for (int i = 0; i < 10; i++) { 2244 byte[] bytes = Bytes.toBytes(i); 2245 delete = new Delete(bytes); 2246 delete.addFamily(FAMILIES[0]); 2247 deletes.add(delete); 2248 } 2249 ht.delete(deletes); 2250 for (int i = 0; i < 10; i++) { 2251 byte[] bytes = Bytes.toBytes(i); 2252 get = new Get(bytes); 2253 get.addFamily(FAMILIES[0]); 2254 result = ht.get(get); 2255 assertTrue(result.isEmpty()); 2256 } 2257 } 2258 } 2259 2260 /** 2261 * Test batch operations with combination of valid and invalid args 2262 */ 2263 @Test 2264 public void testBatchOperationsWithErrors() throws Exception { 2265 final TableName tableName = TableName.valueOf(name.getMethodName()); 2266 try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] {FAMILY}, 10)) { 2267 2268 int NUM_OPS = 100; 2269 int FAILED_OPS = 50; 2270 2271 RetriesExhaustedWithDetailsException expectedException = null; 2272 IllegalArgumentException iae = null; 2273 2274 // 1.1 Put with no column families (local validation, runtime exception) 2275 List<Put> puts = new ArrayList<Put>(NUM_OPS); 2276 for (int i = 0; i != NUM_OPS; i++) { 2277 Put put = new Put(Bytes.toBytes(i)); 2278 puts.add(put); 2279 } 2280 2281 try { 2282 foo.put(puts); 2283 } catch (IllegalArgumentException e) { 2284 iae = e; 2285 } 2286 assertNotNull(iae); 2287 assertEquals(NUM_OPS, puts.size()); 2288 2289 // 1.2 Put with invalid column family 2290 iae = null; 2291 puts.clear(); 2292 for (int i = 0; i != NUM_OPS; i++) { 2293 Put put = new Put(Bytes.toBytes(i)); 2294 put.addColumn((i % 2) == 0 ? FAMILY : INVALID_FAMILY, FAMILY, Bytes.toBytes(i)); 2295 puts.add(put); 2296 } 2297 2298 try { 2299 foo.put(puts); 2300 } catch (RetriesExhaustedWithDetailsException e) { 2301 expectedException = e; 2302 } 2303 assertNotNull(expectedException); 2304 assertEquals(FAILED_OPS, expectedException.exceptions.size()); 2305 assertTrue(expectedException.actions.contains(puts.get(1))); 2306 2307 // 2.1 Get non-existent rows 2308 List<Get> gets = new ArrayList<>(NUM_OPS); 2309 for (int i = 0; i < NUM_OPS; i++) { 2310 Get get = new Get(Bytes.toBytes(i)); 2311 // get.addColumn(FAMILY, FAMILY); 2312 gets.add(get); 2313 } 2314 Result[] getsResult = foo.get(gets); 2315 2316 assertNotNull(getsResult); 2317 assertEquals(NUM_OPS, getsResult.length); 2318 assertNull(getsResult[1].getRow()); 2319 2320 // 2.2 Get with invalid column family 2321 gets.clear(); 2322 getsResult = null; 2323 expectedException = null; 2324 for (int i = 0; i < NUM_OPS; i++) { 2325 Get get = new Get(Bytes.toBytes(i)); 2326 get.addColumn((i % 2) == 0 ? FAMILY : INVALID_FAMILY, FAMILY); 2327 gets.add(get); 2328 } 2329 try { 2330 getsResult = foo.get(gets); 2331 } catch (RetriesExhaustedWithDetailsException e) { 2332 expectedException = e; 2333 } 2334 assertNull(getsResult); 2335 assertNotNull(expectedException); 2336 assertEquals(FAILED_OPS, expectedException.exceptions.size()); 2337 assertTrue(expectedException.actions.contains(gets.get(1))); 2338 2339 // 3.1 Delete with invalid column family 2340 expectedException = null; 2341 List<Delete> deletes = new ArrayList<>(NUM_OPS); 2342 for (int i = 0; i < NUM_OPS; i++) { 2343 Delete delete = new Delete(Bytes.toBytes(i)); 2344 delete.addColumn((i % 2) == 0 ? FAMILY : INVALID_FAMILY, FAMILY); 2345 deletes.add(delete); 2346 } 2347 try { 2348 foo.delete(deletes); 2349 } catch (RetriesExhaustedWithDetailsException e) { 2350 expectedException = e; 2351 } 2352 assertEquals((NUM_OPS - FAILED_OPS), deletes.size()); 2353 assertNotNull(expectedException); 2354 assertEquals(FAILED_OPS, expectedException.exceptions.size()); 2355 assertTrue(expectedException.actions.contains(deletes.get(1))); 2356 2357 2358 // 3.2 Delete non-existent rows 2359 deletes.clear(); 2360 for (int i = 0; i < NUM_OPS; i++) { 2361 Delete delete = new Delete(Bytes.toBytes(i)); 2362 deletes.add(delete); 2363 } 2364 foo.delete(deletes); 2365 2366 assertTrue(deletes.isEmpty()); 2367 } 2368 } 2369 2370 /* 2371 * Baseline "scalability" test. 2372 * 2373 * Tests one hundred families, one million columns, one million versions 2374 */ 2375 @Ignore @Test 2376 public void testMillions() throws Exception { 2377 2378 // 100 families 2379 2380 // millions of columns 2381 2382 // millions of versions 2383 2384 } 2385 2386 @Ignore @Test 2387 public void testMultipleRegionsAndBatchPuts() throws Exception { 2388 // Two family table 2389 2390 // Insert lots of rows 2391 2392 // Insert to the same row with batched puts 2393 2394 // Insert to multiple rows with batched puts 2395 2396 // Split the table 2397 2398 // Get row from first region 2399 2400 // Get row from second region 2401 2402 // Scan all rows 2403 2404 // Insert to multiple regions with batched puts 2405 2406 // Get row from first region 2407 2408 // Get row from second region 2409 2410 // Scan all rows 2411 2412 2413 } 2414 2415 @Ignore @Test 2416 public void testMultipleRowMultipleFamily() throws Exception { 2417 2418 } 2419 2420 // 2421 // JIRA Testers 2422 // 2423 2424 /** 2425 * HBASE-867 2426 * If millions of columns in a column family, hbase scanner won't come up 2427 * 2428 * Test will create numRows rows, each with numColsPerRow columns 2429 * (1 version each), and attempt to scan them all. 2430 * 2431 * To test at scale, up numColsPerRow to the millions 2432 * (have not gotten that to work running as junit though) 2433 */ 2434 @Test 2435 public void testJiraTest867() throws Exception { 2436 int numRows = 10; 2437 int numColsPerRow = 2000; 2438 2439 final TableName tableName = TableName.valueOf(name.getMethodName()); 2440 2441 byte [][] ROWS = makeN(ROW, numRows); 2442 byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow); 2443 2444 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 2445 2446 // Insert rows 2447 2448 for (int i = 0; i < numRows; i++) { 2449 Put put = new Put(ROWS[i]); 2450 put.setDurability(Durability.SKIP_WAL); 2451 for (int j = 0; j < numColsPerRow; j++) { 2452 put.addColumn(FAMILY, QUALIFIERS[j], QUALIFIERS[j]); 2453 } 2454 assertTrue("Put expected to contain " + numColsPerRow + " columns but " + 2455 "only contains " + put.size(), put.size() == numColsPerRow); 2456 ht.put(put); 2457 } 2458 2459 // Get a row 2460 Get get = new Get(ROWS[numRows - 1]); 2461 Result result = ht.get(get); 2462 assertNumKeys(result, numColsPerRow); 2463 Cell[] keys = result.rawCells(); 2464 for (int i = 0; i < result.size(); i++) { 2465 assertKey(keys[i], ROWS[numRows - 1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]); 2466 } 2467 2468 // Scan the rows 2469 Scan scan = new Scan(); 2470 try (ResultScanner scanner = ht.getScanner(scan)) { 2471 int rowCount = 0; 2472 while ((result = scanner.next()) != null) { 2473 assertNumKeys(result, numColsPerRow); 2474 Cell[] kvs = result.rawCells(); 2475 for (int i = 0; i < numColsPerRow; i++) { 2476 assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]); 2477 } 2478 rowCount++; 2479 } 2480 assertTrue("Expected to scan " + numRows + " rows but actually scanned " 2481 + rowCount + " rows", rowCount == numRows); 2482 } 2483 2484 // flush and try again 2485 2486 TEST_UTIL.flush(); 2487 2488 // Get a row 2489 get = new Get(ROWS[numRows - 1]); 2490 result = ht.get(get); 2491 assertNumKeys(result, numColsPerRow); 2492 keys = result.rawCells(); 2493 for (int i = 0; i < result.size(); i++) { 2494 assertKey(keys[i], ROWS[numRows - 1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]); 2495 } 2496 2497 // Scan the rows 2498 scan = new Scan(); 2499 try (ResultScanner scanner = ht.getScanner(scan)) { 2500 int rowCount = 0; 2501 while ((result = scanner.next()) != null) { 2502 assertNumKeys(result, numColsPerRow); 2503 Cell[] kvs = result.rawCells(); 2504 for (int i = 0; i < numColsPerRow; i++) { 2505 assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]); 2506 } 2507 rowCount++; 2508 } 2509 assertTrue("Expected to scan " + numRows + " rows but actually scanned " 2510 + rowCount + " rows", rowCount == numRows); 2511 } 2512 } 2513 } 2514 2515 /** 2516 * HBASE-861 2517 * get with timestamp will return a value if there is a version with an 2518 * earlier timestamp 2519 */ 2520 @Test 2521 public void testJiraTest861() throws Exception { 2522 final TableName tableName = TableName.valueOf(name.getMethodName()); 2523 byte [][] VALUES = makeNAscii(VALUE, 7); 2524 long [] STAMPS = makeStamps(7); 2525 2526 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 2527 2528 // Insert three versions 2529 2530 Put put = new Put(ROW); 2531 put.addColumn(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 2532 put.addColumn(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2533 put.addColumn(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 2534 ht.put(put); 2535 2536 // Get the middle value 2537 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2538 2539 // Try to get one version before (expect fail) 2540 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]); 2541 2542 // Try to get one version after (expect fail) 2543 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]); 2544 2545 // Try same from storefile 2546 TEST_UTIL.flush(); 2547 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2548 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]); 2549 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]); 2550 2551 // Insert two more versions surrounding others, into memstore 2552 put = new Put(ROW); 2553 put.addColumn(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); 2554 put.addColumn(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]); 2555 ht.put(put); 2556 2557 // Check we can get everything we should and can't get what we shouldn't 2558 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); 2559 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]); 2560 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2561 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 2562 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 2563 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]); 2564 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]); 2565 2566 // Try same from two storefiles 2567 TEST_UTIL.flush(); 2568 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); 2569 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]); 2570 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2571 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 2572 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 2573 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]); 2574 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]); 2575 } 2576 } 2577 2578 /** 2579 * HBASE-33 2580 * Add a HTable get/obtainScanner method that retrieves all versions of a 2581 * particular column and row between two timestamps 2582 */ 2583 @Test 2584 public void testJiraTest33() throws Exception { 2585 final TableName tableName = TableName.valueOf(name.getMethodName()); 2586 byte [][] VALUES = makeNAscii(VALUE, 7); 2587 long [] STAMPS = makeStamps(7); 2588 2589 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 2590 2591 // Insert lots versions 2592 2593 Put put = new Put(ROW); 2594 put.addColumn(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); 2595 put.addColumn(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 2596 put.addColumn(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2597 put.addColumn(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 2598 put.addColumn(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 2599 put.addColumn(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 2600 ht.put(put); 2601 2602 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2603 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2); 2604 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2605 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3); 2606 2607 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2608 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2); 2609 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2610 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3); 2611 2612 // Try same from storefile 2613 TEST_UTIL.flush(); 2614 2615 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2616 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2); 2617 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2618 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3); 2619 2620 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2621 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2); 2622 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2623 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3); 2624 } 2625 } 2626 2627 /** 2628 * HBASE-1014 2629 * commit(BatchUpdate) method should return timestamp 2630 */ 2631 @Test 2632 public void testJiraTest1014() throws Exception { 2633 final TableName tableName = TableName.valueOf(name.getMethodName()); 2634 2635 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 2636 2637 long manualStamp = 12345; 2638 2639 // Insert lots versions 2640 2641 Put put = new Put(ROW); 2642 put.addColumn(FAMILY, QUALIFIER, manualStamp, VALUE); 2643 ht.put(put); 2644 2645 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, manualStamp, VALUE); 2646 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp - 1); 2647 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp + 1); 2648 } 2649 } 2650 2651 /** 2652 * HBASE-1182 2653 * Scan for columns > some timestamp 2654 */ 2655 @Test 2656 public void testJiraTest1182() throws Exception { 2657 final TableName tableName = TableName.valueOf(name.getMethodName()); 2658 byte [][] VALUES = makeNAscii(VALUE, 7); 2659 long [] STAMPS = makeStamps(7); 2660 2661 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 2662 2663 // Insert lots versions 2664 2665 Put put = new Put(ROW); 2666 put.addColumn(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); 2667 put.addColumn(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 2668 put.addColumn(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2669 put.addColumn(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 2670 put.addColumn(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 2671 put.addColumn(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 2672 ht.put(put); 2673 2674 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2675 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); 2676 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2677 2678 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2679 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); 2680 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2681 2682 // Try same from storefile 2683 TEST_UTIL.flush(); 2684 2685 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2686 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); 2687 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2688 2689 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2690 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); 2691 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); 2692 } 2693 } 2694 2695 /** 2696 * HBASE-52 2697 * Add a means of scanning over all versions 2698 */ 2699 @Test 2700 public void testJiraTest52() throws Exception { 2701 final TableName tableName = TableName.valueOf(name.getMethodName()); 2702 byte [][] VALUES = makeNAscii(VALUE, 7); 2703 long [] STAMPS = makeStamps(7); 2704 2705 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 2706 2707 // Insert lots versions 2708 2709 Put put = new Put(ROW); 2710 put.addColumn(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); 2711 put.addColumn(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 2712 put.addColumn(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 2713 put.addColumn(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 2714 put.addColumn(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 2715 put.addColumn(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 2716 ht.put(put); 2717 2718 getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2719 2720 scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2721 2722 // Try same from storefile 2723 TEST_UTIL.flush(); 2724 2725 getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2726 2727 scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); 2728 } 2729 } 2730 2731 // 2732 // Bulk Testers 2733 // 2734 2735 private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, 2736 byte [] family, byte [] qualifier, long [] stamps, byte [][] values, 2737 int start, int end) 2738 throws IOException { 2739 Get get = new Get(row); 2740 get.addColumn(family, qualifier); 2741 get.setMaxVersions(Integer.MAX_VALUE); 2742 get.setTimeRange(stamps[start+1], Long.MAX_VALUE); 2743 Result result = ht.get(get); 2744 assertNResult(result, row, family, qualifier, stamps, values, start+1, end); 2745 } 2746 2747 private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family, 2748 byte [] qualifier, long [] stamps, byte [][] values, int start, int end) 2749 throws IOException { 2750 Get get = new Get(row); 2751 get.addColumn(family, qualifier); 2752 get.setMaxVersions(Integer.MAX_VALUE); 2753 get.setTimeRange(stamps[start], stamps[end]+1); 2754 Result result = ht.get(get); 2755 assertNResult(result, row, family, qualifier, stamps, values, start, end); 2756 } 2757 2758 private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family, 2759 byte [] qualifier, long [] stamps, byte [][] values, int start, int end) 2760 throws IOException { 2761 Get get = new Get(row); 2762 get.addColumn(family, qualifier); 2763 get.setMaxVersions(Integer.MAX_VALUE); 2764 Result result = ht.get(get); 2765 assertNResult(result, row, family, qualifier, stamps, values, start, end); 2766 } 2767 2768 private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, 2769 byte [] family, byte [] qualifier, long [] stamps, byte [][] values, 2770 int start, int end) 2771 throws IOException { 2772 Scan scan = new Scan(row); 2773 scan.addColumn(family, qualifier); 2774 scan.setMaxVersions(Integer.MAX_VALUE); 2775 scan.setTimeRange(stamps[start+1], Long.MAX_VALUE); 2776 Result result = getSingleScanResult(ht, scan); 2777 assertNResult(result, row, family, qualifier, stamps, values, start+1, end); 2778 } 2779 2780 private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family, 2781 byte [] qualifier, long [] stamps, byte [][] values, int start, int end) 2782 throws IOException { 2783 Scan scan = new Scan(row); 2784 scan.addColumn(family, qualifier); 2785 scan.setMaxVersions(Integer.MAX_VALUE); 2786 scan.setTimeRange(stamps[start], stamps[end]+1); 2787 Result result = getSingleScanResult(ht, scan); 2788 assertNResult(result, row, family, qualifier, stamps, values, start, end); 2789 } 2790 2791 private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family, 2792 byte [] qualifier, long [] stamps, byte [][] values, int start, int end) 2793 throws IOException { 2794 Scan scan = new Scan(row); 2795 scan.addColumn(family, qualifier); 2796 scan.setMaxVersions(Integer.MAX_VALUE); 2797 Result result = getSingleScanResult(ht, scan); 2798 assertNResult(result, row, family, qualifier, stamps, values, start, end); 2799 } 2800 2801 private void getVersionAndVerify(Table ht, byte [] row, byte [] family, 2802 byte [] qualifier, long stamp, byte [] value) 2803 throws Exception { 2804 Get get = new Get(row); 2805 get.addColumn(family, qualifier); 2806 get.setTimestamp(stamp); 2807 get.setMaxVersions(Integer.MAX_VALUE); 2808 Result result = ht.get(get); 2809 assertSingleResult(result, row, family, qualifier, stamp, value); 2810 } 2811 2812 private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family, 2813 byte [] qualifier, long stamp) 2814 throws Exception { 2815 Get get = new Get(row); 2816 get.addColumn(family, qualifier); 2817 get.setTimestamp(stamp); 2818 get.setMaxVersions(Integer.MAX_VALUE); 2819 Result result = ht.get(get); 2820 assertEmptyResult(result); 2821 } 2822 2823 private void scanVersionAndVerify(Table ht, byte [] row, byte [] family, 2824 byte [] qualifier, long stamp, byte [] value) 2825 throws Exception { 2826 Scan scan = new Scan(row); 2827 scan.addColumn(family, qualifier); 2828 scan.setTimestamp(stamp); 2829 scan.setMaxVersions(Integer.MAX_VALUE); 2830 Result result = getSingleScanResult(ht, scan); 2831 assertSingleResult(result, row, family, qualifier, stamp, value); 2832 } 2833 2834 private void scanVersionAndVerifyMissing(Table ht, byte [] row, 2835 byte [] family, byte [] qualifier, long stamp) 2836 throws Exception { 2837 Scan scan = new Scan(row); 2838 scan.addColumn(family, qualifier); 2839 scan.setTimestamp(stamp); 2840 scan.setMaxVersions(Integer.MAX_VALUE); 2841 Result result = getSingleScanResult(ht, scan); 2842 assertNullResult(result); 2843 } 2844 2845 private void getTestNull(Table ht, byte [] row, byte [] family, 2846 byte [] value) 2847 throws Exception { 2848 2849 Get get = new Get(row); 2850 get.addColumn(family, null); 2851 Result result = ht.get(get); 2852 assertSingleResult(result, row, family, null, value); 2853 2854 get = new Get(row); 2855 get.addColumn(family, HConstants.EMPTY_BYTE_ARRAY); 2856 result = ht.get(get); 2857 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2858 2859 get = new Get(row); 2860 get.addFamily(family); 2861 result = ht.get(get); 2862 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2863 2864 get = new Get(row); 2865 result = ht.get(get); 2866 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2867 2868 } 2869 2870 private void getTestNull(Table ht, byte[] row, byte[] family, long value) throws Exception { 2871 Get get = new Get(row); 2872 get.addColumn(family, null); 2873 Result result = ht.get(get); 2874 assertSingleResult(result, row, family, null, value); 2875 2876 get = new Get(row); 2877 get.addColumn(family, HConstants.EMPTY_BYTE_ARRAY); 2878 result = ht.get(get); 2879 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2880 2881 get = new Get(row); 2882 get.addFamily(family); 2883 result = ht.get(get); 2884 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2885 2886 get = new Get(row); 2887 result = ht.get(get); 2888 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2889 } 2890 2891 private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value) 2892 throws Exception { 2893 scanTestNull(ht, row, family, value, false); 2894 } 2895 2896 private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value, 2897 boolean isReversedScan) throws Exception { 2898 2899 Scan scan = new Scan(); 2900 scan.setReversed(isReversedScan); 2901 scan.addColumn(family, null); 2902 Result result = getSingleScanResult(ht, scan); 2903 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2904 2905 scan = new Scan(); 2906 scan.setReversed(isReversedScan); 2907 scan.addColumn(family, HConstants.EMPTY_BYTE_ARRAY); 2908 result = getSingleScanResult(ht, scan); 2909 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2910 2911 scan = new Scan(); 2912 scan.setReversed(isReversedScan); 2913 scan.addFamily(family); 2914 result = getSingleScanResult(ht, scan); 2915 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2916 2917 scan = new Scan(); 2918 scan.setReversed(isReversedScan); 2919 result = getSingleScanResult(ht, scan); 2920 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value); 2921 2922 } 2923 2924 private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, 2925 byte [][] QUALIFIERS, byte [][] VALUES) 2926 throws Exception { 2927 2928 // Single column from memstore 2929 Get get = new Get(ROWS[0]); 2930 get.addColumn(FAMILIES[4], QUALIFIERS[0]); 2931 Result result = ht.get(get); 2932 assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]); 2933 2934 // Single column from storefile 2935 get = new Get(ROWS[0]); 2936 get.addColumn(FAMILIES[2], QUALIFIERS[2]); 2937 result = ht.get(get); 2938 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); 2939 2940 // Single column from storefile, family match 2941 get = new Get(ROWS[0]); 2942 get.addFamily(FAMILIES[7]); 2943 result = ht.get(get); 2944 assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]); 2945 2946 // Two columns, one from memstore one from storefile, same family, 2947 // wildcard match 2948 get = new Get(ROWS[0]); 2949 get.addFamily(FAMILIES[4]); 2950 result = ht.get(get); 2951 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], 2952 FAMILIES[4], QUALIFIERS[4], VALUES[4]); 2953 2954 // Two columns, one from memstore one from storefile, same family, 2955 // explicit match 2956 get = new Get(ROWS[0]); 2957 get.addColumn(FAMILIES[4], QUALIFIERS[0]); 2958 get.addColumn(FAMILIES[4], QUALIFIERS[4]); 2959 result = ht.get(get); 2960 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], 2961 FAMILIES[4], QUALIFIERS[4], VALUES[4]); 2962 2963 // Three column, one from memstore two from storefile, different families, 2964 // wildcard match 2965 get = new Get(ROWS[0]); 2966 get.addFamily(FAMILIES[4]); 2967 get.addFamily(FAMILIES[7]); 2968 result = ht.get(get); 2969 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 2970 new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} }); 2971 2972 // Multiple columns from everywhere storefile, many family, wildcard 2973 get = new Get(ROWS[0]); 2974 get.addFamily(FAMILIES[2]); 2975 get.addFamily(FAMILIES[4]); 2976 get.addFamily(FAMILIES[6]); 2977 get.addFamily(FAMILIES[7]); 2978 result = ht.get(get); 2979 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 2980 new int [][] { 2981 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} 2982 }); 2983 2984 // Multiple columns from everywhere storefile, many family, wildcard 2985 get = new Get(ROWS[0]); 2986 get.addColumn(FAMILIES[2], QUALIFIERS[2]); 2987 get.addColumn(FAMILIES[2], QUALIFIERS[4]); 2988 get.addColumn(FAMILIES[4], QUALIFIERS[0]); 2989 get.addColumn(FAMILIES[4], QUALIFIERS[4]); 2990 get.addColumn(FAMILIES[6], QUALIFIERS[6]); 2991 get.addColumn(FAMILIES[6], QUALIFIERS[7]); 2992 get.addColumn(FAMILIES[7], QUALIFIERS[7]); 2993 get.addColumn(FAMILIES[7], QUALIFIERS[8]); 2994 result = ht.get(get); 2995 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 2996 new int [][] { 2997 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} 2998 }); 2999 3000 // Everything 3001 get = new Get(ROWS[0]); 3002 result = ht.get(get); 3003 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 3004 new int [][] { 3005 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0} 3006 }); 3007 3008 // Get around inserted columns 3009 3010 get = new Get(ROWS[1]); 3011 result = ht.get(get); 3012 assertEmptyResult(result); 3013 3014 get = new Get(ROWS[0]); 3015 get.addColumn(FAMILIES[4], QUALIFIERS[3]); 3016 get.addColumn(FAMILIES[2], QUALIFIERS[3]); 3017 result = ht.get(get); 3018 assertEmptyResult(result); 3019 3020 } 3021 3022 private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, 3023 byte [][] QUALIFIERS, byte [][] VALUES) 3024 throws Exception { 3025 3026 // Single column from memstore 3027 Scan scan = new Scan(); 3028 scan.addColumn(FAMILIES[4], QUALIFIERS[0]); 3029 Result result = getSingleScanResult(ht, scan); 3030 assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]); 3031 3032 // Single column from storefile 3033 scan = new Scan(); 3034 scan.addColumn(FAMILIES[2], QUALIFIERS[2]); 3035 result = getSingleScanResult(ht, scan); 3036 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); 3037 3038 // Single column from storefile, family match 3039 scan = new Scan(); 3040 scan.addFamily(FAMILIES[7]); 3041 result = getSingleScanResult(ht, scan); 3042 assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]); 3043 3044 // Two columns, one from memstore one from storefile, same family, 3045 // wildcard match 3046 scan = new Scan(); 3047 scan.addFamily(FAMILIES[4]); 3048 result = getSingleScanResult(ht, scan); 3049 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], 3050 FAMILIES[4], QUALIFIERS[4], VALUES[4]); 3051 3052 // Two columns, one from memstore one from storefile, same family, 3053 // explicit match 3054 scan = new Scan(); 3055 scan.addColumn(FAMILIES[4], QUALIFIERS[0]); 3056 scan.addColumn(FAMILIES[4], QUALIFIERS[4]); 3057 result = getSingleScanResult(ht, scan); 3058 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], 3059 FAMILIES[4], QUALIFIERS[4], VALUES[4]); 3060 3061 // Three column, one from memstore two from storefile, different families, 3062 // wildcard match 3063 scan = new Scan(); 3064 scan.addFamily(FAMILIES[4]); 3065 scan.addFamily(FAMILIES[7]); 3066 result = getSingleScanResult(ht, scan); 3067 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 3068 new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} }); 3069 3070 // Multiple columns from everywhere storefile, many family, wildcard 3071 scan = new Scan(); 3072 scan.addFamily(FAMILIES[2]); 3073 scan.addFamily(FAMILIES[4]); 3074 scan.addFamily(FAMILIES[6]); 3075 scan.addFamily(FAMILIES[7]); 3076 result = getSingleScanResult(ht, scan); 3077 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 3078 new int [][] { 3079 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} 3080 }); 3081 3082 // Multiple columns from everywhere storefile, many family, wildcard 3083 scan = new Scan(); 3084 scan.addColumn(FAMILIES[2], QUALIFIERS[2]); 3085 scan.addColumn(FAMILIES[2], QUALIFIERS[4]); 3086 scan.addColumn(FAMILIES[4], QUALIFIERS[0]); 3087 scan.addColumn(FAMILIES[4], QUALIFIERS[4]); 3088 scan.addColumn(FAMILIES[6], QUALIFIERS[6]); 3089 scan.addColumn(FAMILIES[6], QUALIFIERS[7]); 3090 scan.addColumn(FAMILIES[7], QUALIFIERS[7]); 3091 scan.addColumn(FAMILIES[7], QUALIFIERS[8]); 3092 result = getSingleScanResult(ht, scan); 3093 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 3094 new int [][] { 3095 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} 3096 }); 3097 3098 // Everything 3099 scan = new Scan(); 3100 result = getSingleScanResult(ht, scan); 3101 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, 3102 new int [][] { 3103 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0} 3104 }); 3105 3106 // Scan around inserted columns 3107 3108 scan = new Scan(ROWS[1]); 3109 result = getSingleScanResult(ht, scan); 3110 assertNullResult(result); 3111 3112 scan = new Scan(); 3113 scan.addColumn(FAMILIES[4], QUALIFIERS[3]); 3114 scan.addColumn(FAMILIES[2], QUALIFIERS[3]); 3115 result = getSingleScanResult(ht, scan); 3116 assertNullResult(result); 3117 } 3118 3119 /** 3120 * Verify a single column using gets. 3121 * Expects family and qualifier arrays to be valid for at least 3122 * the range: idx-2 < idx < idx+2 3123 */ 3124 private void getVerifySingleColumn(Table ht, 3125 byte [][] ROWS, int ROWIDX, 3126 byte [][] FAMILIES, int FAMILYIDX, 3127 byte [][] QUALIFIERS, int QUALIFIERIDX, 3128 byte [][] VALUES, int VALUEIDX) 3129 throws Exception { 3130 3131 Get get = new Get(ROWS[ROWIDX]); 3132 Result result = ht.get(get); 3133 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3134 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3135 3136 get = new Get(ROWS[ROWIDX]); 3137 get.addFamily(FAMILIES[FAMILYIDX]); 3138 result = ht.get(get); 3139 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3140 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3141 3142 get = new Get(ROWS[ROWIDX]); 3143 get.addFamily(FAMILIES[FAMILYIDX-2]); 3144 get.addFamily(FAMILIES[FAMILYIDX]); 3145 get.addFamily(FAMILIES[FAMILYIDX+2]); 3146 result = ht.get(get); 3147 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3148 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3149 3150 get = new Get(ROWS[ROWIDX]); 3151 get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[0]); 3152 result = ht.get(get); 3153 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3154 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3155 3156 get = new Get(ROWS[ROWIDX]); 3157 get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[1]); 3158 get.addFamily(FAMILIES[FAMILYIDX]); 3159 result = ht.get(get); 3160 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3161 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3162 3163 get = new Get(ROWS[ROWIDX]); 3164 get.addFamily(FAMILIES[FAMILYIDX]); 3165 get.addColumn(FAMILIES[FAMILYIDX+1], QUALIFIERS[1]); 3166 get.addColumn(FAMILIES[FAMILYIDX-2], QUALIFIERS[1]); 3167 get.addFamily(FAMILIES[FAMILYIDX-1]); 3168 get.addFamily(FAMILIES[FAMILYIDX+2]); 3169 result = ht.get(get); 3170 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3171 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3172 3173 } 3174 3175 3176 /** 3177 * Verify a single column using scanners. 3178 * Expects family and qualifier arrays to be valid for at least 3179 * the range: idx-2 to idx+2 3180 * Expects row array to be valid for at least idx to idx+2 3181 */ 3182 private void scanVerifySingleColumn(Table ht, 3183 byte [][] ROWS, int ROWIDX, 3184 byte [][] FAMILIES, int FAMILYIDX, 3185 byte [][] QUALIFIERS, int QUALIFIERIDX, 3186 byte [][] VALUES, int VALUEIDX) 3187 throws Exception { 3188 3189 Scan scan = new Scan(); 3190 Result result = getSingleScanResult(ht, scan); 3191 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3192 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3193 3194 scan = new Scan(ROWS[ROWIDX]); 3195 result = getSingleScanResult(ht, scan); 3196 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3197 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3198 3199 scan = new Scan(ROWS[ROWIDX], ROWS[ROWIDX+1]); 3200 result = getSingleScanResult(ht, scan); 3201 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3202 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3203 3204 scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX+1]); 3205 result = getSingleScanResult(ht, scan); 3206 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3207 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3208 3209 scan = new Scan(); 3210 scan.addFamily(FAMILIES[FAMILYIDX]); 3211 result = getSingleScanResult(ht, scan); 3212 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3213 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3214 3215 scan = new Scan(); 3216 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]); 3217 result = getSingleScanResult(ht, scan); 3218 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3219 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3220 3221 scan = new Scan(); 3222 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]); 3223 scan.addFamily(FAMILIES[FAMILYIDX]); 3224 result = getSingleScanResult(ht, scan); 3225 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3226 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3227 3228 scan = new Scan(); 3229 scan.addColumn(FAMILIES[FAMILYIDX-1], QUALIFIERS[QUALIFIERIDX+1]); 3230 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]); 3231 scan.addFamily(FAMILIES[FAMILYIDX+1]); 3232 result = getSingleScanResult(ht, scan); 3233 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], 3234 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); 3235 3236 } 3237 3238 /** 3239 * Verify we do not read any values by accident around a single column 3240 * Same requirements as getVerifySingleColumn 3241 */ 3242 private void getVerifySingleEmpty(Table ht, 3243 byte [][] ROWS, int ROWIDX, 3244 byte [][] FAMILIES, int FAMILYIDX, 3245 byte [][] QUALIFIERS, int QUALIFIERIDX) 3246 throws Exception { 3247 3248 Get get = new Get(ROWS[ROWIDX]); 3249 get.addFamily(FAMILIES[4]); 3250 get.addColumn(FAMILIES[4], QUALIFIERS[1]); 3251 Result result = ht.get(get); 3252 assertEmptyResult(result); 3253 3254 get = new Get(ROWS[ROWIDX]); 3255 get.addFamily(FAMILIES[4]); 3256 get.addColumn(FAMILIES[4], QUALIFIERS[2]); 3257 result = ht.get(get); 3258 assertEmptyResult(result); 3259 3260 get = new Get(ROWS[ROWIDX]); 3261 get.addFamily(FAMILIES[3]); 3262 get.addColumn(FAMILIES[4], QUALIFIERS[2]); 3263 get.addFamily(FAMILIES[5]); 3264 result = ht.get(get); 3265 assertEmptyResult(result); 3266 3267 get = new Get(ROWS[ROWIDX+1]); 3268 result = ht.get(get); 3269 assertEmptyResult(result); 3270 3271 } 3272 3273 private void scanVerifySingleEmpty(Table ht, 3274 byte [][] ROWS, int ROWIDX, 3275 byte [][] FAMILIES, int FAMILYIDX, 3276 byte [][] QUALIFIERS, int QUALIFIERIDX) 3277 throws Exception { 3278 3279 Scan scan = new Scan(ROWS[ROWIDX+1]); 3280 Result result = getSingleScanResult(ht, scan); 3281 assertNullResult(result); 3282 3283 scan = new Scan(ROWS[ROWIDX+1],ROWS[ROWIDX+2]); 3284 result = getSingleScanResult(ht, scan); 3285 assertNullResult(result); 3286 3287 scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX]); 3288 result = getSingleScanResult(ht, scan); 3289 assertNullResult(result); 3290 3291 scan = new Scan(); 3292 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]); 3293 scan.addFamily(FAMILIES[FAMILYIDX-1]); 3294 result = getSingleScanResult(ht, scan); 3295 assertNullResult(result); 3296 3297 } 3298 3299 // 3300 // Verifiers 3301 // 3302 3303 private void assertKey(Cell key, byte [] row, byte [] family, 3304 byte [] qualifier, byte [] value) 3305 throws Exception { 3306 assertTrue("Expected row [" + Bytes.toString(row) + "] " + 3307 "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]", 3308 equals(row, CellUtil.cloneRow(key))); 3309 assertTrue("Expected family [" + Bytes.toString(family) + "] " + 3310 "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]", 3311 equals(family, CellUtil.cloneFamily(key))); 3312 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " + 3313 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]", 3314 equals(qualifier, CellUtil.cloneQualifier(key))); 3315 assertTrue("Expected value [" + Bytes.toString(value) + "] " + 3316 "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]", 3317 equals(value, CellUtil.cloneValue(key))); 3318 } 3319 3320 static void assertIncrementKey(Cell key, byte [] row, byte [] family, 3321 byte [] qualifier, long value) 3322 throws Exception { 3323 assertTrue("Expected row [" + Bytes.toString(row) + "] " + 3324 "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]", 3325 equals(row, CellUtil.cloneRow(key))); 3326 assertTrue("Expected family [" + Bytes.toString(family) + "] " + 3327 "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]", 3328 equals(family, CellUtil.cloneFamily(key))); 3329 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " + 3330 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]", 3331 equals(qualifier, CellUtil.cloneQualifier(key))); 3332 assertTrue("Expected value [" + value + "] " + 3333 "Got value [" + Bytes.toLong(CellUtil.cloneValue(key)) + "]", 3334 Bytes.toLong(CellUtil.cloneValue(key)) == value); 3335 } 3336 3337 private void assertNumKeys(Result result, int n) throws Exception { 3338 assertTrue("Expected " + n + " keys but got " + result.size(), 3339 result.size() == n); 3340 } 3341 3342 private void assertNResult(Result result, byte [] row, 3343 byte [][] families, byte [][] qualifiers, byte [][] values, 3344 int [][] idxs) 3345 throws Exception { 3346 assertTrue("Expected row [" + Bytes.toString(row) + "] " + 3347 "Got row [" + Bytes.toString(result.getRow()) +"]", 3348 equals(row, result.getRow())); 3349 assertTrue("Expected " + idxs.length + " keys but result contains " 3350 + result.size(), result.size() == idxs.length); 3351 3352 Cell [] keys = result.rawCells(); 3353 3354 for(int i=0;i<keys.length;i++) { 3355 byte [] family = families[idxs[i][0]]; 3356 byte [] qualifier = qualifiers[idxs[i][1]]; 3357 byte [] value = values[idxs[i][2]]; 3358 Cell key = keys[i]; 3359 3360 byte[] famb = CellUtil.cloneFamily(key); 3361 byte[] qualb = CellUtil.cloneQualifier(key); 3362 byte[] valb = CellUtil.cloneValue(key); 3363 assertTrue("(" + i + ") Expected family [" + Bytes.toString(family) 3364 + "] " + "Got family [" + Bytes.toString(famb) + "]", 3365 equals(family, famb)); 3366 assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier) 3367 + "] " + "Got qualifier [" + Bytes.toString(qualb) + "]", 3368 equals(qualifier, qualb)); 3369 assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] " 3370 + "Got value [" + Bytes.toString(valb) + "]", 3371 equals(value, valb)); 3372 } 3373 } 3374 3375 private void assertNResult(Result result, byte [] row, 3376 byte [] family, byte [] qualifier, long [] stamps, byte [][] values, 3377 int start, int end) 3378 throws IOException { 3379 assertTrue("Expected row [" + Bytes.toString(row) + "] " + 3380 "Got row [" + Bytes.toString(result.getRow()) +"]", 3381 equals(row, result.getRow())); 3382 int expectedResults = end - start + 1; 3383 assertEquals(expectedResults, result.size()); 3384 3385 Cell[] keys = result.rawCells(); 3386 3387 for (int i=0; i<keys.length; i++) { 3388 byte [] value = values[end-i]; 3389 long ts = stamps[end-i]; 3390 Cell key = keys[i]; 3391 3392 assertTrue("(" + i + ") Expected family [" + Bytes.toString(family) 3393 + "] " + "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]", 3394 CellUtil.matchingFamily(key, family)); 3395 assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier) 3396 + "] " + "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key))+ "]", 3397 CellUtil.matchingQualifier(key, qualifier)); 3398 assertTrue("Expected ts [" + ts + "] " + 3399 "Got ts [" + key.getTimestamp() + "]", ts == key.getTimestamp()); 3400 assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] " 3401 + "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]", 3402 CellUtil.matchingValue(key, value)); 3403 } 3404 } 3405 3406 /** 3407 * Validate that result contains two specified keys, exactly. 3408 * It is assumed key A sorts before key B. 3409 */ 3410 private void assertDoubleResult(Result result, byte [] row, 3411 byte [] familyA, byte [] qualifierA, byte [] valueA, 3412 byte [] familyB, byte [] qualifierB, byte [] valueB) 3413 throws Exception { 3414 assertTrue("Expected row [" + Bytes.toString(row) + "] " + 3415 "Got row [" + Bytes.toString(result.getRow()) +"]", 3416 equals(row, result.getRow())); 3417 assertTrue("Expected two keys but result contains " + result.size(), 3418 result.size() == 2); 3419 Cell [] kv = result.rawCells(); 3420 Cell kvA = kv[0]; 3421 assertTrue("(A) Expected family [" + Bytes.toString(familyA) + "] " + 3422 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvA)) + "]", 3423 equals(familyA, CellUtil.cloneFamily(kvA))); 3424 assertTrue("(A) Expected qualifier [" + Bytes.toString(qualifierA) + "] " + 3425 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvA)) + "]", 3426 equals(qualifierA, CellUtil.cloneQualifier(kvA))); 3427 assertTrue("(A) Expected value [" + Bytes.toString(valueA) + "] " + 3428 "Got value [" + Bytes.toString(CellUtil.cloneValue(kvA)) + "]", 3429 equals(valueA, CellUtil.cloneValue(kvA))); 3430 Cell kvB = kv[1]; 3431 assertTrue("(B) Expected family [" + Bytes.toString(familyB) + "] " + 3432 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvB)) + "]", 3433 equals(familyB, CellUtil.cloneFamily(kvB))); 3434 assertTrue("(B) Expected qualifier [" + Bytes.toString(qualifierB) + "] " + 3435 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvB)) + "]", 3436 equals(qualifierB, CellUtil.cloneQualifier(kvB))); 3437 assertTrue("(B) Expected value [" + Bytes.toString(valueB) + "] " + 3438 "Got value [" + Bytes.toString(CellUtil.cloneValue(kvB)) + "]", 3439 equals(valueB, CellUtil.cloneValue(kvB))); 3440 } 3441 3442 private void assertSingleResult(Result result, byte [] row, byte [] family, 3443 byte [] qualifier, byte [] value) 3444 throws Exception { 3445 assertTrue("Expected row [" + Bytes.toString(row) + "] " + 3446 "Got row [" + Bytes.toString(result.getRow()) +"]", 3447 equals(row, result.getRow())); 3448 assertTrue("Expected a single key but result contains " + result.size(), 3449 result.size() == 1); 3450 Cell kv = result.rawCells()[0]; 3451 assertTrue("Expected family [" + Bytes.toString(family) + "] " + 3452 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]", 3453 equals(family, CellUtil.cloneFamily(kv))); 3454 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " + 3455 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]", 3456 equals(qualifier, CellUtil.cloneQualifier(kv))); 3457 assertTrue("Expected value [" + Bytes.toString(value) + "] " + 3458 "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]", 3459 equals(value, CellUtil.cloneValue(kv))); 3460 } 3461 3462 private void assertSingleResult(Result result, byte[] row, byte[] family, byte[] qualifier, 3463 long value) throws Exception { 3464 assertTrue( 3465 "Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(result.getRow()) 3466 + "]", equals(row, result.getRow())); 3467 assertTrue("Expected a single key but result contains " + result.size(), result.size() == 1); 3468 Cell kv = result.rawCells()[0]; 3469 assertTrue( 3470 "Expected family [" + Bytes.toString(family) + "] " + "Got family [" 3471 + Bytes.toString(CellUtil.cloneFamily(kv)) + "]", 3472 equals(family, CellUtil.cloneFamily(kv))); 3473 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " + "Got qualifier [" 3474 + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]", 3475 equals(qualifier, CellUtil.cloneQualifier(kv))); 3476 assertTrue( 3477 "Expected value [" + value + "] " + "Got value [" + Bytes.toLong(CellUtil.cloneValue(kv)) 3478 + "]", value == Bytes.toLong(CellUtil.cloneValue(kv))); 3479 } 3480 3481 private void assertSingleResult(Result result, byte [] row, byte [] family, 3482 byte [] qualifier, long ts, byte [] value) 3483 throws Exception { 3484 assertTrue("Expected row [" + Bytes.toString(row) + "] " + 3485 "Got row [" + Bytes.toString(result.getRow()) +"]", 3486 equals(row, result.getRow())); 3487 assertTrue("Expected a single key but result contains " + result.size(), 3488 result.size() == 1); 3489 Cell kv = result.rawCells()[0]; 3490 assertTrue("Expected family [" + Bytes.toString(family) + "] " + 3491 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]", 3492 equals(family, CellUtil.cloneFamily(kv))); 3493 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " + 3494 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]", 3495 equals(qualifier, CellUtil.cloneQualifier(kv))); 3496 assertTrue("Expected ts [" + ts + "] " + 3497 "Got ts [" + kv.getTimestamp() + "]", ts == kv.getTimestamp()); 3498 assertTrue("Expected value [" + Bytes.toString(value) + "] " + 3499 "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]", 3500 equals(value, CellUtil.cloneValue(kv))); 3501 } 3502 3503 private void assertEmptyResult(Result result) throws Exception { 3504 assertTrue("expected an empty result but result contains " + 3505 result.size() + " keys", result.isEmpty()); 3506 } 3507 3508 private void assertNullResult(Result result) throws Exception { 3509 assertTrue("expected null result but received a non-null result", 3510 result == null); 3511 } 3512 3513 // 3514 // Helpers 3515 // 3516 3517 private Result getSingleScanResult(Table ht, Scan scan) throws IOException { 3518 try (ResultScanner scanner = ht.getScanner(scan)) { 3519 Result result = scanner.next(); 3520 return result; 3521 } 3522 } 3523 3524 private byte [][] makeNAscii(byte [] base, int n) { 3525 if(n > 256) { 3526 return makeNBig(base, n); 3527 } 3528 byte [][] ret = new byte[n][]; 3529 for(int i=0;i<n;i++) { 3530 byte [] tail = Bytes.toBytes(Integer.toString(i)); 3531 ret[i] = Bytes.add(base, tail); 3532 } 3533 return ret; 3534 } 3535 3536 private byte [][] makeN(byte [] base, int n) { 3537 if (n > 256) { 3538 return makeNBig(base, n); 3539 } 3540 byte [][] ret = new byte[n][]; 3541 for(int i=0;i<n;i++) { 3542 ret[i] = Bytes.add(base, new byte[]{(byte)i}); 3543 } 3544 return ret; 3545 } 3546 3547 private byte [][] makeNBig(byte [] base, int n) { 3548 byte [][] ret = new byte[n][]; 3549 for(int i=0;i<n;i++) { 3550 int byteA = (i % 256); 3551 int byteB = (i >> 8); 3552 ret[i] = Bytes.add(base, new byte[]{(byte)byteB,(byte)byteA}); 3553 } 3554 return ret; 3555 } 3556 3557 private long [] makeStamps(int n) { 3558 long [] stamps = new long[n]; 3559 for (int i = 0; i < n; i++) { 3560 stamps[i] = i+1L; 3561 } 3562 return stamps; 3563 } 3564 3565 static boolean equals(byte [] left, byte [] right) { 3566 if (left == null && right == null) return true; 3567 if (left == null && right.length == 0) return true; 3568 if (right == null && left.length == 0) return true; 3569 return Bytes.equals(left, right); 3570 } 3571 3572 @Test 3573 public void testDuplicateVersions() throws Exception { 3574 final TableName tableName = TableName.valueOf(name.getMethodName()); 3575 3576 long [] STAMPS = makeStamps(20); 3577 byte [][] VALUES = makeNAscii(VALUE, 20); 3578 3579 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 3580 3581 // Insert 4 versions of same column 3582 Put put = new Put(ROW); 3583 put.addColumn(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 3584 put.addColumn(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 3585 put.addColumn(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 3586 put.addColumn(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 3587 ht.put(put); 3588 3589 // Verify we can get each one properly 3590 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 3591 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 3592 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 3593 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 3594 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 3595 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 3596 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 3597 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 3598 3599 // Verify we don't accidentally get others 3600 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 3601 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 3602 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 3603 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 3604 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 3605 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 3606 3607 // Ensure maxVersions in query is respected 3608 Get get = new Get(ROW); 3609 get.addColumn(FAMILY, QUALIFIER); 3610 get.setMaxVersions(2); 3611 Result result = ht.get(get); 3612 assertNResult(result, ROW, FAMILY, QUALIFIER, 3613 new long [] {STAMPS[4], STAMPS[5]}, 3614 new byte[][] {VALUES[4], VALUES[5]}, 3615 0, 1); 3616 3617 Scan scan = new Scan(ROW); 3618 scan.addColumn(FAMILY, QUALIFIER); 3619 scan.setMaxVersions(2); 3620 result = getSingleScanResult(ht, scan); 3621 assertNResult(result, ROW, FAMILY, QUALIFIER, 3622 new long[]{STAMPS[4], STAMPS[5]}, 3623 new byte[][]{VALUES[4], VALUES[5]}, 3624 0, 1); 3625 3626 // Flush and redo 3627 3628 TEST_UTIL.flush(); 3629 3630 // Verify we can get each one properly 3631 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 3632 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 3633 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 3634 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 3635 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 3636 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 3637 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); 3638 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); 3639 3640 // Verify we don't accidentally get others 3641 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 3642 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 3643 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 3644 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 3645 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); 3646 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); 3647 3648 // Ensure maxVersions in query is respected 3649 get = new Get(ROW); 3650 get.addColumn(FAMILY, QUALIFIER); 3651 get.setMaxVersions(2); 3652 result = ht.get(get); 3653 assertNResult(result, ROW, FAMILY, QUALIFIER, 3654 new long [] {STAMPS[4], STAMPS[5]}, 3655 new byte[][] {VALUES[4], VALUES[5]}, 3656 0, 1); 3657 3658 scan = new Scan(ROW); 3659 scan.addColumn(FAMILY, QUALIFIER); 3660 scan.setMaxVersions(2); 3661 result = getSingleScanResult(ht, scan); 3662 assertNResult(result, ROW, FAMILY, QUALIFIER, 3663 new long[]{STAMPS[4], STAMPS[5]}, 3664 new byte[][]{VALUES[4], VALUES[5]}, 3665 0, 1); 3666 3667 3668 // Add some memstore and retest 3669 3670 // Insert 4 more versions of same column and a dupe 3671 put = new Put(ROW); 3672 put.addColumn(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); 3673 put.addColumn(FAMILY, QUALIFIER, STAMPS[4], VALUES[14]); 3674 put.addColumn(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]); 3675 put.addColumn(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); 3676 put.addColumn(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]); 3677 ht.put(put); 3678 3679 // Ensure maxVersions in query is respected 3680 get = new Get(ROW); 3681 get.addColumn(FAMILY, QUALIFIER); 3682 get.setMaxVersions(7); 3683 result = ht.get(get); 3684 assertNResult(result, ROW, FAMILY, QUALIFIER, 3685 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, 3686 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], 3687 VALUES[8]}, 3688 0, 6); 3689 3690 scan = new Scan(ROW); 3691 scan.addColumn(FAMILY, QUALIFIER); 3692 scan.setMaxVersions(7); 3693 result = getSingleScanResult(ht, scan); 3694 assertNResult(result, ROW, FAMILY, QUALIFIER, 3695 new long[]{STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, 3696 new byte[][]{VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]}, 3697 0, 6); 3698 3699 get = new Get(ROW); 3700 get.setMaxVersions(7); 3701 result = ht.get(get); 3702 assertNResult(result, ROW, FAMILY, QUALIFIER, 3703 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, 3704 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], 3705 VALUES[8]}, 3706 0, 6); 3707 3708 scan = new Scan(ROW); 3709 scan.setMaxVersions(7); 3710 result = getSingleScanResult(ht, scan); 3711 assertNResult(result, ROW, FAMILY, QUALIFIER, 3712 new long[]{STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, 3713 new byte[][]{VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]}, 3714 0, 6); 3715 3716 // Verify we can get each one properly 3717 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 3718 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 3719 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]); 3720 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); 3721 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); 3722 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); 3723 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]); 3724 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); 3725 3726 // Verify we don't accidentally get others 3727 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 3728 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]); 3729 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); 3730 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]); 3731 3732 // Ensure maxVersions of table is respected 3733 3734 TEST_UTIL.flush(); 3735 3736 // Insert 4 more versions of same column and a dupe 3737 put = new Put(ROW); 3738 put.addColumn(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]); 3739 put.addColumn(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]); 3740 put.addColumn(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]); 3741 put.addColumn(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]); 3742 ht.put(put); 3743 3744 get = new Get(ROW); 3745 get.addColumn(FAMILY, QUALIFIER); 3746 get.setMaxVersions(Integer.MAX_VALUE); 3747 result = ht.get(get); 3748 assertNResult(result, ROW, FAMILY, QUALIFIER, 3749 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], 3750 STAMPS[11], STAMPS[13], STAMPS[15]}, 3751 new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], 3752 VALUES[9], VALUES[11], VALUES[13], VALUES[15]}, 3753 0, 9); 3754 3755 scan = new Scan(ROW); 3756 scan.addColumn(FAMILY, QUALIFIER); 3757 scan.setMaxVersions(Integer.MAX_VALUE); 3758 result = getSingleScanResult(ht, scan); 3759 assertNResult(result, ROW, FAMILY, QUALIFIER, 3760 new long[]{STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], 3761 STAMPS[11], STAMPS[13], STAMPS[15]}, 3762 new byte[][]{VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], 3763 VALUES[11], VALUES[13], VALUES[15]},0, 9); 3764 3765 // Delete a version in the memstore and a version in a storefile 3766 Delete delete = new Delete(ROW); 3767 delete.addColumn(FAMILY, QUALIFIER, STAMPS[11]); 3768 delete.addColumn(FAMILY, QUALIFIER, STAMPS[7]); 3769 ht.delete(delete); 3770 3771 // Test that it's gone 3772 get = new Get(ROW); 3773 get.addColumn(FAMILY, QUALIFIER); 3774 get.setMaxVersions(Integer.MAX_VALUE); 3775 result = ht.get(get); 3776 assertNResult(result, ROW, FAMILY, QUALIFIER, 3777 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], 3778 STAMPS[9], STAMPS[13], STAMPS[15]}, 3779 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], 3780 VALUES[8], VALUES[9], VALUES[13], VALUES[15]}, 3781 0, 9); 3782 3783 scan = new Scan(ROW); 3784 scan.addColumn(FAMILY, QUALIFIER); 3785 scan.setMaxVersions(Integer.MAX_VALUE); 3786 result = getSingleScanResult(ht, scan); 3787 assertNResult(result, ROW, FAMILY, QUALIFIER, 3788 new long[]{STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], 3789 STAMPS[9], STAMPS[13], STAMPS[15]}, 3790 new byte[][]{VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], 3791 VALUES[9], VALUES[13], VALUES[15]},0,9); 3792 } 3793 } 3794 3795 @Test 3796 public void testUpdates() throws Exception { 3797 final TableName tableName = TableName.valueOf(name.getMethodName()); 3798 try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10)) { 3799 3800 // Write a column with values at timestamp 1, 2 and 3 3801 byte[] row = Bytes.toBytes("row1"); 3802 byte[] qualifier = Bytes.toBytes("myCol"); 3803 Put put = new Put(row); 3804 put.addColumn(FAMILY, qualifier, 1L, Bytes.toBytes("AAA")); 3805 hTable.put(put); 3806 3807 put = new Put(row); 3808 put.addColumn(FAMILY, qualifier, 2L, Bytes.toBytes("BBB")); 3809 hTable.put(put); 3810 3811 put = new Put(row); 3812 put.addColumn(FAMILY, qualifier, 3L, Bytes.toBytes("EEE")); 3813 hTable.put(put); 3814 3815 Get get = new Get(row); 3816 get.addColumn(FAMILY, qualifier); 3817 get.setMaxVersions(); 3818 3819 // Check that the column indeed has the right values at timestamps 1 and 3820 // 2 3821 Result result = hTable.get(get); 3822 NavigableMap<Long, byte[]> navigableMap = 3823 result.getMap().get(FAMILY).get(qualifier); 3824 assertEquals("AAA", Bytes.toString(navigableMap.get(1L))); 3825 assertEquals("BBB", Bytes.toString(navigableMap.get(2L))); 3826 3827 // Update the value at timestamp 1 3828 put = new Put(row); 3829 put.addColumn(FAMILY, qualifier, 1L, Bytes.toBytes("CCC")); 3830 hTable.put(put); 3831 3832 // Update the value at timestamp 2 3833 put = new Put(row); 3834 put.addColumn(FAMILY, qualifier, 2L, Bytes.toBytes("DDD")); 3835 hTable.put(put); 3836 3837 // Check that the values at timestamp 2 and 1 got updated 3838 result = hTable.get(get); 3839 navigableMap = result.getMap().get(FAMILY).get(qualifier); 3840 assertEquals("CCC", Bytes.toString(navigableMap.get(1L))); 3841 assertEquals("DDD", Bytes.toString(navigableMap.get(2L))); 3842 } 3843 } 3844 3845 @Test 3846 public void testUpdatesWithMajorCompaction() throws Exception { 3847 final TableName tableName = TableName.valueOf(name.getMethodName()); 3848 try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10); 3849 Admin admin = TEST_UTIL.getAdmin()) { 3850 3851 // Write a column with values at timestamp 1, 2 and 3 3852 byte[] row = Bytes.toBytes("row2"); 3853 byte[] qualifier = Bytes.toBytes("myCol"); 3854 Put put = new Put(row); 3855 put.addColumn(FAMILY, qualifier, 1L, Bytes.toBytes("AAA")); 3856 hTable.put(put); 3857 3858 put = new Put(row); 3859 put.addColumn(FAMILY, qualifier, 2L, Bytes.toBytes("BBB")); 3860 hTable.put(put); 3861 3862 put = new Put(row); 3863 put.addColumn(FAMILY, qualifier, 3L, Bytes.toBytes("EEE")); 3864 hTable.put(put); 3865 3866 Get get = new Get(row); 3867 get.addColumn(FAMILY, qualifier); 3868 get.setMaxVersions(); 3869 3870 // Check that the column indeed has the right values at timestamps 1 and 3871 // 2 3872 Result result = hTable.get(get); 3873 NavigableMap<Long, byte[]> navigableMap = 3874 result.getMap().get(FAMILY).get(qualifier); 3875 assertEquals("AAA", Bytes.toString(navigableMap.get(1L))); 3876 assertEquals("BBB", Bytes.toString(navigableMap.get(2L))); 3877 3878 // Trigger a major compaction 3879 admin.flush(tableName); 3880 admin.majorCompact(tableName); 3881 Thread.sleep(6000); 3882 3883 // Update the value at timestamp 1 3884 put = new Put(row); 3885 put.addColumn(FAMILY, qualifier, 1L, Bytes.toBytes("CCC")); 3886 hTable.put(put); 3887 3888 // Update the value at timestamp 2 3889 put = new Put(row); 3890 put.addColumn(FAMILY, qualifier, 2L, Bytes.toBytes("DDD")); 3891 hTable.put(put); 3892 3893 // Trigger a major compaction 3894 admin.flush(tableName); 3895 admin.majorCompact(tableName); 3896 Thread.sleep(6000); 3897 3898 // Check that the values at timestamp 2 and 1 got updated 3899 result = hTable.get(get); 3900 navigableMap = result.getMap().get(FAMILY).get(qualifier); 3901 assertEquals("CCC", Bytes.toString(navigableMap.get(1L))); 3902 assertEquals("DDD", Bytes.toString(navigableMap.get(2L))); 3903 } 3904 } 3905 3906 @Test 3907 public void testMajorCompactionBetweenTwoUpdates() throws Exception { 3908 final TableName tableName = TableName.valueOf(name.getMethodName()); 3909 try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10); 3910 Admin admin = TEST_UTIL.getAdmin()) { 3911 3912 // Write a column with values at timestamp 1, 2 and 3 3913 byte[] row = Bytes.toBytes("row3"); 3914 byte[] qualifier = Bytes.toBytes("myCol"); 3915 Put put = new Put(row); 3916 put.addColumn(FAMILY, qualifier, 1L, Bytes.toBytes("AAA")); 3917 hTable.put(put); 3918 3919 put = new Put(row); 3920 put.addColumn(FAMILY, qualifier, 2L, Bytes.toBytes("BBB")); 3921 hTable.put(put); 3922 3923 put = new Put(row); 3924 put.addColumn(FAMILY, qualifier, 3L, Bytes.toBytes("EEE")); 3925 hTable.put(put); 3926 3927 Get get = new Get(row); 3928 get.addColumn(FAMILY, qualifier); 3929 get.setMaxVersions(); 3930 3931 // Check that the column indeed has the right values at timestamps 1 and 3932 // 2 3933 Result result = hTable.get(get); 3934 NavigableMap<Long, byte[]> navigableMap = 3935 result.getMap().get(FAMILY).get(qualifier); 3936 assertEquals("AAA", Bytes.toString(navigableMap.get(1L))); 3937 assertEquals("BBB", Bytes.toString(navigableMap.get(2L))); 3938 3939 // Trigger a major compaction 3940 admin.flush(tableName); 3941 admin.majorCompact(tableName); 3942 Thread.sleep(6000); 3943 3944 // Update the value at timestamp 1 3945 put = new Put(row); 3946 put.addColumn(FAMILY, qualifier, 1L, Bytes.toBytes("CCC")); 3947 hTable.put(put); 3948 3949 // Trigger a major compaction 3950 admin.flush(tableName); 3951 admin.majorCompact(tableName); 3952 Thread.sleep(6000); 3953 3954 // Update the value at timestamp 2 3955 put = new Put(row); 3956 put.addColumn(FAMILY, qualifier, 2L, Bytes.toBytes("DDD")); 3957 hTable.put(put); 3958 3959 // Trigger a major compaction 3960 admin.flush(tableName); 3961 admin.majorCompact(tableName); 3962 Thread.sleep(6000); 3963 3964 // Check that the values at timestamp 2 and 1 got updated 3965 result = hTable.get(get); 3966 navigableMap = result.getMap().get(FAMILY).get(qualifier); 3967 3968 assertEquals("CCC", Bytes.toString(navigableMap.get(1L))); 3969 assertEquals("DDD", Bytes.toString(navigableMap.get(2L))); 3970 } 3971 } 3972 3973 @Test 3974 public void testGet_EmptyTable() throws IOException { 3975 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 3976 Get get = new Get(ROW); 3977 get.addFamily(FAMILY); 3978 Result r = table.get(get); 3979 assertTrue(r.isEmpty()); 3980 } 3981 } 3982 3983 @Test 3984 public void testGet_NullQualifier() throws IOException { 3985 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 3986 Put put = new Put(ROW); 3987 put.addColumn(FAMILY, QUALIFIER, VALUE); 3988 table.put(put); 3989 3990 put = new Put(ROW); 3991 put.addColumn(FAMILY, null, VALUE); 3992 table.put(put); 3993 LOG.info("Row put"); 3994 3995 Get get = new Get(ROW); 3996 get.addColumn(FAMILY, null); 3997 Result r = table.get(get); 3998 assertEquals(1, r.size()); 3999 4000 get = new Get(ROW); 4001 get.addFamily(FAMILY); 4002 r = table.get(get); 4003 assertEquals(2, r.size()); 4004 } 4005 } 4006 4007 @Test 4008 public void testGet_NonExistentRow() throws IOException { 4009 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 4010 Put put = new Put(ROW); 4011 put.addColumn(FAMILY, QUALIFIER, VALUE); 4012 table.put(put); 4013 LOG.info("Row put"); 4014 4015 Get get = new Get(ROW); 4016 get.addFamily(FAMILY); 4017 Result r = table.get(get); 4018 assertFalse(r.isEmpty()); 4019 System.out.println("Row retrieved successfully"); 4020 4021 byte[] missingrow = Bytes.toBytes("missingrow"); 4022 get = new Get(missingrow); 4023 get.addFamily(FAMILY); 4024 r = table.get(get); 4025 assertTrue(r.isEmpty()); 4026 LOG.info("Row missing as it should be"); 4027 } 4028 } 4029 4030 @Test 4031 public void testPut() throws IOException { 4032 final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents"); 4033 final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam"); 4034 final byte [] row1 = Bytes.toBytes("row1"); 4035 final byte [] row2 = Bytes.toBytes("row2"); 4036 final byte [] value = Bytes.toBytes("abcd"); 4037 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), 4038 new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) { 4039 Put put = new Put(row1); 4040 put.addColumn(CONTENTS_FAMILY, null, value); 4041 table.put(put); 4042 4043 put = new Put(row2); 4044 put.addColumn(CONTENTS_FAMILY, null, value); 4045 4046 assertEquals(1, put.size()); 4047 assertEquals(1, put.getFamilyCellMap().get(CONTENTS_FAMILY).size()); 4048 4049 // KeyValue v1 expectation. Cast for now until we go all Cell all the time. TODO 4050 KeyValue kv = (KeyValue) put.getFamilyCellMap().get(CONTENTS_FAMILY).get(0); 4051 4052 assertTrue(Bytes.equals(CellUtil.cloneFamily(kv), CONTENTS_FAMILY)); 4053 // will it return null or an empty byte array? 4054 assertTrue(Bytes.equals(CellUtil.cloneQualifier(kv), new byte[0])); 4055 4056 assertTrue(Bytes.equals(CellUtil.cloneValue(kv), value)); 4057 4058 table.put(put); 4059 4060 Scan scan = new Scan(); 4061 scan.addColumn(CONTENTS_FAMILY, null); 4062 try (ResultScanner scanner = table.getScanner(scan)) { 4063 for (Result r : scanner) { 4064 for (Cell key : r.rawCells()) { 4065 System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString()); 4066 } 4067 } 4068 } 4069 } 4070 } 4071 4072 @Test 4073 public void testPutNoCF() throws IOException { 4074 final byte[] BAD_FAM = Bytes.toBytes("BAD_CF"); 4075 final byte[] VAL = Bytes.toBytes(100); 4076 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 4077 boolean caughtNSCFE = false; 4078 4079 try { 4080 Put p = new Put(ROW); 4081 p.addColumn(BAD_FAM, QUALIFIER, VAL); 4082 table.put(p); 4083 } catch (Exception e) { 4084 caughtNSCFE = e instanceof NoSuchColumnFamilyException; 4085 } 4086 assertTrue("Should throw NoSuchColumnFamilyException", caughtNSCFE); 4087 } 4088 } 4089 4090 @Test 4091 public void testRowsPut() throws IOException { 4092 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents"); 4093 final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam"); 4094 final int NB_BATCH_ROWS = 10; 4095 final byte[] value = Bytes.toBytes("abcd"); 4096 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), 4097 new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY })) { 4098 ArrayList<Put> rowsUpdate = new ArrayList<Put>(); 4099 for (int i = 0; i < NB_BATCH_ROWS; i++) { 4100 byte[] row = Bytes.toBytes("row" + i); 4101 Put put = new Put(row); 4102 put.setDurability(Durability.SKIP_WAL); 4103 put.addColumn(CONTENTS_FAMILY, null, value); 4104 rowsUpdate.add(put); 4105 } 4106 table.put(rowsUpdate); 4107 Scan scan = new Scan(); 4108 scan.addFamily(CONTENTS_FAMILY); 4109 try (ResultScanner scanner = table.getScanner(scan)) { 4110 int nbRows = 0; 4111 for (@SuppressWarnings("unused") 4112 Result row : scanner) { 4113 nbRows++; 4114 } 4115 assertEquals(NB_BATCH_ROWS, nbRows); 4116 } 4117 } 4118 } 4119 4120 @Test 4121 public void testRowsPutBufferedManyManyFlushes() throws IOException { 4122 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents"); 4123 final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam"); 4124 final byte[] value = Bytes.toBytes("abcd"); 4125 final int NB_BATCH_ROWS = 10; 4126 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), 4127 new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) { 4128 ArrayList<Put> rowsUpdate = new ArrayList<Put>(); 4129 for (int i = 0; i < NB_BATCH_ROWS * 10; i++) { 4130 byte[] row = Bytes.toBytes("row" + i); 4131 Put put = new Put(row); 4132 put.setDurability(Durability.SKIP_WAL); 4133 put.addColumn(CONTENTS_FAMILY, null, value); 4134 rowsUpdate.add(put); 4135 } 4136 table.put(rowsUpdate); 4137 4138 Scan scan = new Scan(); 4139 scan.addFamily(CONTENTS_FAMILY); 4140 try (ResultScanner scanner = table.getScanner(scan)) { 4141 int nbRows = 0; 4142 for (@SuppressWarnings("unused") 4143 Result row : scanner) { 4144 nbRows++; 4145 } 4146 assertEquals(NB_BATCH_ROWS * 10, nbRows); 4147 } 4148 } 4149 } 4150 4151 @Test 4152 public void testAddKeyValue() throws IOException { 4153 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents"); 4154 final byte[] value = Bytes.toBytes("abcd"); 4155 final byte[] row1 = Bytes.toBytes("row1"); 4156 final byte[] row2 = Bytes.toBytes("row2"); 4157 byte[] qualifier = Bytes.toBytes("qf1"); 4158 Put put = new Put(row1); 4159 4160 // Adding KeyValue with the same row 4161 KeyValue kv = new KeyValue(row1, CONTENTS_FAMILY, qualifier, value); 4162 boolean ok = true; 4163 try { 4164 put.add(kv); 4165 } catch (IOException e) { 4166 ok = false; 4167 } 4168 assertEquals(true, ok); 4169 4170 // Adding KeyValue with the different row 4171 kv = new KeyValue(row2, CONTENTS_FAMILY, qualifier, value); 4172 ok = false; 4173 try { 4174 put.add(kv); 4175 } catch (IOException e) { 4176 ok = true; 4177 } 4178 assertEquals(true, ok); 4179 } 4180 4181 /** 4182 * test for HBASE-737 4183 */ 4184 @Test 4185 public void testHBase737 () throws IOException { 4186 final byte [] FAM1 = Bytes.toBytes("fam1"); 4187 final byte [] FAM2 = Bytes.toBytes("fam2"); 4188 // Open table 4189 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), 4190 new byte [][] {FAM1, FAM2})) { 4191 // Insert some values 4192 Put put = new Put(ROW); 4193 put.addColumn(FAM1, Bytes.toBytes("letters"), Bytes.toBytes("abcdefg")); 4194 table.put(put); 4195 try { 4196 Thread.sleep(1000); 4197 } catch (InterruptedException i) { 4198 //ignore 4199 } 4200 4201 put = new Put(ROW); 4202 put.addColumn(FAM1, Bytes.toBytes("numbers"), Bytes.toBytes("123456")); 4203 table.put(put); 4204 4205 try { 4206 Thread.sleep(1000); 4207 } catch (InterruptedException i) { 4208 //ignore 4209 } 4210 4211 put = new Put(ROW); 4212 put.addColumn(FAM2, Bytes.toBytes("letters"), Bytes.toBytes("hijklmnop")); 4213 table.put(put); 4214 4215 long[] times = new long[3]; 4216 4217 // First scan the memstore 4218 4219 Scan scan = new Scan(); 4220 scan.addFamily(FAM1); 4221 scan.addFamily(FAM2); 4222 try (ResultScanner s = table.getScanner(scan)) { 4223 int index = 0; 4224 Result r = null; 4225 while ((r = s.next()) != null) { 4226 for (Cell key : r.rawCells()) { 4227 times[index++] = key.getTimestamp(); 4228 } 4229 } 4230 } 4231 for (int i = 0; i < times.length - 1; i++) { 4232 for (int j = i + 1; j < times.length; j++) { 4233 assertTrue(times[j] > times[i]); 4234 } 4235 } 4236 4237 // Flush data to disk and try again 4238 TEST_UTIL.flush(); 4239 4240 // Reset times 4241 for (int i = 0; i < times.length; i++) { 4242 times[i] = 0; 4243 } 4244 4245 try { 4246 Thread.sleep(1000); 4247 } catch (InterruptedException i) { 4248 //ignore 4249 } 4250 scan = new Scan(); 4251 scan.addFamily(FAM1); 4252 scan.addFamily(FAM2); 4253 try (ResultScanner s = table.getScanner(scan)) { 4254 int index = 0; 4255 Result r = null; 4256 while ((r = s.next()) != null) { 4257 for (Cell key : r.rawCells()) { 4258 times[index++] = key.getTimestamp(); 4259 } 4260 } 4261 for (int i = 0; i < times.length - 1; i++) { 4262 for (int j = i + 1; j < times.length; j++) { 4263 assertTrue(times[j] > times[i]); 4264 } 4265 } 4266 } 4267 } 4268 } 4269 4270 @Test 4271 public void testListTables() throws IOException, InterruptedException { 4272 final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1"); 4273 final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2"); 4274 final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3"); 4275 TableName [] tables = new TableName[] { tableName1, tableName2, tableName3 }; 4276 for (int i = 0; i < tables.length; i++) { 4277 TEST_UTIL.createTable(tables[i], FAMILY); 4278 } 4279 try (Admin admin = TEST_UTIL.getAdmin()) { 4280 List<TableDescriptor> ts = admin.listTableDescriptors(); 4281 HashSet<TableDescriptor> result = new HashSet<>(ts); 4282 int size = result.size(); 4283 assertTrue(size >= tables.length); 4284 for (int i = 0; i < tables.length && i < size; i++) { 4285 boolean found = false; 4286 for (int j = 0; j < ts.size(); j++) { 4287 if (ts.get(j).getTableName().equals(tables[i])) { 4288 found = true; 4289 break; 4290 } 4291 } 4292 assertTrue("Not found: " + tables[i], found); 4293 } 4294 } 4295 } 4296 4297 /** 4298 * simple test that just executes parts of the client 4299 * API that accept a pre-created Connection instance 4300 */ 4301 @Test 4302 public void testUnmanagedHConnection() throws IOException { 4303 final TableName tableName = TableName.valueOf(name.getMethodName()); 4304 TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); 4305 try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); 4306 Table t = conn.getTable(tableName); 4307 Admin admin = conn.getAdmin()) { 4308 assertTrue(admin.tableExists(tableName)); 4309 assertTrue(t.get(new Get(ROW)).isEmpty()); 4310 } 4311 } 4312 4313 /** 4314 * test of that unmanaged HConnections are able to reconnect 4315 * properly (see HBASE-5058) 4316 */ 4317 @Test 4318 public void testUnmanagedHConnectionReconnect() throws Exception { 4319 final TableName tableName = TableName.valueOf(name.getMethodName()); 4320 TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); 4321 try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { 4322 try (Table t = conn.getTable(tableName); 4323 Admin admin = conn.getAdmin()) { 4324 assertTrue(admin.tableExists(tableName)); 4325 assertTrue(t.get(new Get(ROW)).isEmpty()); 4326 } 4327 4328 // stop the master 4329 MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); 4330 cluster.stopMaster(0, false); 4331 cluster.waitOnMaster(0); 4332 4333 // start up a new master 4334 cluster.startMaster(); 4335 assertTrue(cluster.waitForActiveAndReadyMaster()); 4336 4337 // test that the same unmanaged connection works with a new 4338 // Admin and can connect to the new master; 4339 boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration()); 4340 try (Admin admin = conn.getAdmin()) { 4341 assertTrue(admin.tableExists(tableName)); 4342 assertTrue(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)) 4343 .getLiveServerMetrics().size() == SLAVES + (tablesOnMaster ? 1 : 0)); 4344 } 4345 } 4346 } 4347 4348 @Test 4349 public void testMiscHTableStuff() throws IOException { 4350 final TableName tableAname = TableName.valueOf(name.getMethodName() + "A"); 4351 final TableName tableBname = TableName.valueOf(name.getMethodName() + "B"); 4352 final byte[] attrName = Bytes.toBytes("TESTATTR"); 4353 final byte[] attrValue = Bytes.toBytes("somevalue"); 4354 byte[] value = Bytes.toBytes("value"); 4355 4356 try (Table a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY); 4357 Table b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY)) { 4358 Put put = new Put(ROW); 4359 put.addColumn(HConstants.CATALOG_FAMILY, null, value); 4360 a.put(put); 4361 4362 // open a new connection to A and a connection to b 4363 try (Table newA = TEST_UTIL.getConnection().getTable(tableAname)) { 4364 4365 // copy data from A to B 4366 Scan scan = new Scan(); 4367 scan.addFamily(HConstants.CATALOG_FAMILY); 4368 try (ResultScanner s = newA.getScanner(scan)) { 4369 for (Result r : s) { 4370 put = new Put(r.getRow()); 4371 put.setDurability(Durability.SKIP_WAL); 4372 for (Cell kv : r.rawCells()) { 4373 put.add(kv); 4374 } 4375 b.put(put); 4376 } 4377 } 4378 } 4379 4380 // Opening a new connection to A will cause the tables to be reloaded 4381 try (Table anotherA = TEST_UTIL.getConnection().getTable(tableAname)) { 4382 Get get = new Get(ROW); 4383 get.addFamily(HConstants.CATALOG_FAMILY); 4384 anotherA.get(get); 4385 } 4386 4387 // We can still access A through newA because it has the table information 4388 // cached. And if it needs to recalibrate, that will cause the information 4389 // to be reloaded. 4390 4391 // Test user metadata 4392 try (Admin admin = TEST_UTIL.getAdmin()) { 4393 // make a modifiable descriptor 4394 HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor()); 4395 // offline the table 4396 admin.disableTable(tableAname); 4397 // add a user attribute to HTD 4398 desc.setValue(attrName, attrValue); 4399 // add a user attribute to HCD 4400 for (HColumnDescriptor c : desc.getFamilies()) { 4401 c.setValue(attrName, attrValue); 4402 } 4403 // update metadata for all regions of this table 4404 admin.modifyTable(desc); 4405 // enable the table 4406 admin.enableTable(tableAname); 4407 } 4408 4409 // Test that attribute changes were applied 4410 HTableDescriptor desc = a.getTableDescriptor(); 4411 assertEquals("wrong table descriptor returned", desc.getTableName(), tableAname); 4412 // check HTD attribute 4413 value = desc.getValue(attrName); 4414 assertFalse("missing HTD attribute value", value == null); 4415 assertFalse("HTD attribute value is incorrect", 4416 Bytes.compareTo(value, attrValue) != 0); 4417 // check HCD attribute 4418 for (HColumnDescriptor c : desc.getFamilies()) { 4419 value = c.getValue(attrName); 4420 assertFalse("missing HCD attribute value", value == null); 4421 assertFalse("HCD attribute value is incorrect", 4422 Bytes.compareTo(value, attrValue) != 0); 4423 } 4424 } 4425 } 4426 4427 @Test 4428 public void testGetClosestRowBefore() throws IOException, InterruptedException { 4429 final TableName tableName = TableName.valueOf(name.getMethodName()); 4430 final byte[] firstRow = Bytes.toBytes("row111"); 4431 final byte[] secondRow = Bytes.toBytes("row222"); 4432 final byte[] thirdRow = Bytes.toBytes("row333"); 4433 final byte[] forthRow = Bytes.toBytes("row444"); 4434 final byte[] beforeFirstRow = Bytes.toBytes("row"); 4435 final byte[] beforeSecondRow = Bytes.toBytes("row22"); 4436 final byte[] beforeThirdRow = Bytes.toBytes("row33"); 4437 final byte[] beforeForthRow = Bytes.toBytes("row44"); 4438 4439 try (Table table = 4440 TEST_UTIL.createTable(tableName, 4441 new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024); 4442 RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 4443 4444 // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow 4445 // in Store.rowAtOrBeforeFromStoreFile 4446 String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); 4447 HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); 4448 Put put1 = new Put(firstRow); 4449 Put put2 = new Put(secondRow); 4450 Put put3 = new Put(thirdRow); 4451 Put put4 = new Put(forthRow); 4452 byte[] one = new byte[] { 1 }; 4453 byte[] two = new byte[] { 2 }; 4454 byte[] three = new byte[] { 3 }; 4455 byte[] four = new byte[] { 4 }; 4456 4457 put1.addColumn(HConstants.CATALOG_FAMILY, null, one); 4458 put2.addColumn(HConstants.CATALOG_FAMILY, null, two); 4459 put3.addColumn(HConstants.CATALOG_FAMILY, null, three); 4460 put4.addColumn(HConstants.CATALOG_FAMILY, null, four); 4461 table.put(put1); 4462 table.put(put2); 4463 table.put(put3); 4464 table.put(put4); 4465 region.flush(true); 4466 4467 Result result; 4468 4469 // Test before first that null is returned 4470 result = getReverseScanResult(table, beforeFirstRow, 4471 HConstants.CATALOG_FAMILY); 4472 assertNull(result); 4473 4474 // Test at first that first is returned 4475 result = getReverseScanResult(table, firstRow, HConstants.CATALOG_FAMILY); 4476 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4477 assertTrue(Bytes.equals(result.getRow(), firstRow)); 4478 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one)); 4479 4480 // Test in between first and second that first is returned 4481 result = getReverseScanResult(table, beforeSecondRow, HConstants.CATALOG_FAMILY); 4482 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4483 assertTrue(Bytes.equals(result.getRow(), firstRow)); 4484 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one)); 4485 4486 // Test at second make sure second is returned 4487 result = getReverseScanResult(table, secondRow, HConstants.CATALOG_FAMILY); 4488 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4489 assertTrue(Bytes.equals(result.getRow(), secondRow)); 4490 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two)); 4491 4492 // Test in second and third, make sure second is returned 4493 result = getReverseScanResult(table, beforeThirdRow, HConstants.CATALOG_FAMILY); 4494 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4495 assertTrue(Bytes.equals(result.getRow(), secondRow)); 4496 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two)); 4497 4498 // Test at third make sure third is returned 4499 result = getReverseScanResult(table, thirdRow, HConstants.CATALOG_FAMILY); 4500 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4501 assertTrue(Bytes.equals(result.getRow(), thirdRow)); 4502 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three)); 4503 4504 // Test in third and forth, make sure third is returned 4505 result = getReverseScanResult(table, beforeForthRow, HConstants.CATALOG_FAMILY); 4506 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4507 assertTrue(Bytes.equals(result.getRow(), thirdRow)); 4508 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three)); 4509 4510 // Test at forth make sure forth is returned 4511 result = getReverseScanResult(table, forthRow, HConstants.CATALOG_FAMILY); 4512 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4513 assertTrue(Bytes.equals(result.getRow(), forthRow)); 4514 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four)); 4515 4516 // Test after forth make sure forth is returned 4517 result = getReverseScanResult(table, Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY); 4518 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); 4519 assertTrue(Bytes.equals(result.getRow(), forthRow)); 4520 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four)); 4521 } 4522 } 4523 4524 private Result getReverseScanResult(Table table, byte[] row, byte[] fam) throws IOException { 4525 Scan scan = new Scan(row); 4526 scan.setSmall(true); 4527 scan.setReversed(true); 4528 scan.setCaching(1); 4529 scan.addFamily(fam); 4530 try (ResultScanner scanner = table.getScanner(scan)) { 4531 return scanner.next(); 4532 } 4533 } 4534 4535 /** 4536 * For HBASE-2156 4537 */ 4538 @Test 4539 public void testScanVariableReuse() throws Exception { 4540 Scan scan = new Scan(); 4541 scan.addFamily(FAMILY); 4542 scan.addColumn(FAMILY, ROW); 4543 4544 assertTrue(scan.getFamilyMap().get(FAMILY).size() == 1); 4545 4546 scan = new Scan(); 4547 scan.addFamily(FAMILY); 4548 4549 assertTrue(scan.getFamilyMap().get(FAMILY) == null); 4550 assertTrue(scan.getFamilyMap().containsKey(FAMILY)); 4551 } 4552 4553 @Test 4554 public void testMultiRowMutation() throws Exception { 4555 LOG.info("Starting testMultiRowMutation"); 4556 final TableName tableName = TableName.valueOf(name.getMethodName()); 4557 final byte [] ROW1 = Bytes.toBytes("testRow1"); 4558 4559 try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { 4560 Put p = new Put(ROW); 4561 p.addColumn(FAMILY, QUALIFIER, VALUE); 4562 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p); 4563 4564 p = new Put(ROW1); 4565 p.addColumn(FAMILY, QUALIFIER, VALUE); 4566 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, p); 4567 4568 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder(); 4569 mrmBuilder.addMutationRequest(m1); 4570 mrmBuilder.addMutationRequest(m2); 4571 MutateRowsRequest mrm = mrmBuilder.build(); 4572 CoprocessorRpcChannel channel = t.coprocessorService(ROW); 4573 MultiRowMutationService.BlockingInterface service = 4574 MultiRowMutationService.newBlockingStub(channel); 4575 service.mutateRows(null, mrm); 4576 Get g = new Get(ROW); 4577 Result r = t.get(g); 4578 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER))); 4579 g = new Get(ROW1); 4580 r = t.get(g); 4581 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER))); 4582 } 4583 } 4584 4585 @Test 4586 public void testRowMutation() throws Exception { 4587 LOG.info("Starting testRowMutation"); 4588 final TableName tableName = TableName.valueOf(name.getMethodName()); 4589 try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { 4590 byte[][] QUALIFIERS = new byte[][]{ 4591 Bytes.toBytes("a"), Bytes.toBytes("b") 4592 }; 4593 RowMutations arm = new RowMutations(ROW); 4594 Put p = new Put(ROW); 4595 p.addColumn(FAMILY, QUALIFIERS[0], VALUE); 4596 arm.add(p); 4597 t.mutateRow(arm); 4598 4599 Get g = new Get(ROW); 4600 Result r = t.get(g); 4601 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0]))); 4602 4603 arm = new RowMutations(ROW); 4604 p = new Put(ROW); 4605 p.addColumn(FAMILY, QUALIFIERS[1], VALUE); 4606 arm.add(p); 4607 Delete d = new Delete(ROW); 4608 d.addColumns(FAMILY, QUALIFIERS[0]); 4609 arm.add(d); 4610 // TODO: Trying mutateRow again. The batch was failing with a one try only. 4611 t.mutateRow(arm); 4612 r = t.get(g); 4613 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1]))); 4614 assertNull(r.getValue(FAMILY, QUALIFIERS[0])); 4615 4616 //Test that we get a region level exception 4617 try { 4618 arm = new RowMutations(ROW); 4619 p = new Put(ROW); 4620 p.addColumn(new byte[]{'b', 'o', 'g', 'u', 's'}, QUALIFIERS[0], VALUE); 4621 arm.add(p); 4622 t.mutateRow(arm); 4623 fail("Expected NoSuchColumnFamilyException"); 4624 } catch (RetriesExhaustedWithDetailsException e) { 4625 for (Throwable rootCause : e.getCauses()) { 4626 if (rootCause instanceof NoSuchColumnFamilyException) { 4627 return; 4628 } 4629 } 4630 throw e; 4631 } 4632 } 4633 } 4634 4635 @Test 4636 public void testBatchAppendWithReturnResultFalse() throws Exception { 4637 LOG.info("Starting testBatchAppendWithReturnResultFalse"); 4638 final TableName tableName = TableName.valueOf(name.getMethodName()); 4639 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 4640 Append append1 = new Append(Bytes.toBytes("row1")); 4641 append1.setReturnResults(false); 4642 append1.addColumn(FAMILY, Bytes.toBytes("f1"), Bytes.toBytes("value1")); 4643 Append append2 = new Append(Bytes.toBytes("row1")); 4644 append2.setReturnResults(false); 4645 append2.addColumn(FAMILY, Bytes.toBytes("f1"), Bytes.toBytes("value2")); 4646 List<Append> appends = new ArrayList<>(); 4647 appends.add(append1); 4648 appends.add(append2); 4649 Object[] results = new Object[2]; 4650 table.batch(appends, results); 4651 assertTrue(results.length == 2); 4652 for (Object r : results) { 4653 Result result = (Result) r; 4654 assertTrue(result.isEmpty()); 4655 } 4656 } 4657 } 4658 4659 @Test 4660 public void testAppend() throws Exception { 4661 LOG.info("Starting testAppend"); 4662 final TableName tableName = TableName.valueOf(name.getMethodName()); 4663 try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { 4664 byte[] v1 = Bytes.toBytes("42"); 4665 byte[] v2 = Bytes.toBytes("23"); 4666 byte[][] QUALIFIERS = new byte[][]{ 4667 Bytes.toBytes("b"), Bytes.toBytes("a"), Bytes.toBytes("c") 4668 }; 4669 Append a = new Append(ROW); 4670 a.addColumn(FAMILY, QUALIFIERS[0], v1); 4671 a.addColumn(FAMILY, QUALIFIERS[1], v2); 4672 a.setReturnResults(false); 4673 assertEmptyResult(t.append(a)); 4674 4675 a = new Append(ROW); 4676 a.addColumn(FAMILY, QUALIFIERS[0], v2); 4677 a.addColumn(FAMILY, QUALIFIERS[1], v1); 4678 a.addColumn(FAMILY, QUALIFIERS[2], v2); 4679 Result r = t.append(a); 4680 assertEquals(0, Bytes.compareTo(Bytes.add(v1, v2), r.getValue(FAMILY, QUALIFIERS[0]))); 4681 assertEquals(0, Bytes.compareTo(Bytes.add(v2, v1), r.getValue(FAMILY, QUALIFIERS[1]))); 4682 // QUALIFIERS[2] previously not exist, verify both value and timestamp are correct 4683 assertEquals(0, Bytes.compareTo(v2, r.getValue(FAMILY, QUALIFIERS[2]))); 4684 assertEquals(r.getColumnLatestCell(FAMILY, QUALIFIERS[0]).getTimestamp(), 4685 r.getColumnLatestCell(FAMILY, QUALIFIERS[2]).getTimestamp()); 4686 } 4687 } 4688 private List<Result> doAppend(final boolean walUsed) throws IOException { 4689 LOG.info("Starting testAppend, walUsed is " + walUsed); 4690 final TableName TABLENAME = 4691 TableName.valueOf(walUsed ? "testAppendWithWAL" : "testAppendWithoutWAL"); 4692 try (Table t = TEST_UTIL.createTable(TABLENAME, FAMILY)) { 4693 final byte[] row1 = Bytes.toBytes("c"); 4694 final byte[] row2 = Bytes.toBytes("b"); 4695 final byte[] row3 = Bytes.toBytes("a"); 4696 final byte[] qual = Bytes.toBytes("qual"); 4697 Put put_0 = new Put(row2); 4698 put_0.addColumn(FAMILY, qual, Bytes.toBytes("put")); 4699 Put put_1 = new Put(row3); 4700 put_1.addColumn(FAMILY, qual, Bytes.toBytes("put")); 4701 Append append_0 = new Append(row1); 4702 append_0.addColumn(FAMILY, qual, Bytes.toBytes("i")); 4703 Append append_1 = new Append(row1); 4704 append_1.addColumn(FAMILY, qual, Bytes.toBytes("k")); 4705 Append append_2 = new Append(row1); 4706 append_2.addColumn(FAMILY, qual, Bytes.toBytes("e")); 4707 if (!walUsed) { 4708 append_2.setDurability(Durability.SKIP_WAL); 4709 } 4710 Append append_3 = new Append(row1); 4711 append_3.addColumn(FAMILY, qual, Bytes.toBytes("a")); 4712 Scan s = new Scan(); 4713 s.setCaching(1); 4714 t.append(append_0); 4715 t.put(put_0); 4716 t.put(put_1); 4717 List<Result> results = new LinkedList<>(); 4718 try (ResultScanner scanner = t.getScanner(s)) { 4719 t.append(append_1); 4720 t.append(append_2); 4721 t.append(append_3); 4722 for (Result r : scanner) { 4723 results.add(r); 4724 } 4725 } 4726 TEST_UTIL.deleteTable(TABLENAME); 4727 return results; 4728 } 4729 } 4730 4731 @Test 4732 public void testAppendWithoutWAL() throws Exception { 4733 List<Result> resultsWithWal = doAppend(true); 4734 List<Result> resultsWithoutWal = doAppend(false); 4735 assertEquals(resultsWithWal.size(), resultsWithoutWal.size()); 4736 for (int i = 0; i != resultsWithWal.size(); ++i) { 4737 Result resultWithWal = resultsWithWal.get(i); 4738 Result resultWithoutWal = resultsWithoutWal.get(i); 4739 assertEquals(resultWithWal.rawCells().length, resultWithoutWal.rawCells().length); 4740 for (int j = 0; j != resultWithWal.rawCells().length; ++j) { 4741 Cell cellWithWal = resultWithWal.rawCells()[j]; 4742 Cell cellWithoutWal = resultWithoutWal.rawCells()[j]; 4743 assertTrue(Bytes.equals(CellUtil.cloneRow(cellWithWal), 4744 CellUtil.cloneRow(cellWithoutWal))); 4745 assertTrue(Bytes.equals(CellUtil.cloneFamily(cellWithWal), 4746 CellUtil.cloneFamily(cellWithoutWal))); 4747 assertTrue(Bytes.equals(CellUtil.cloneQualifier(cellWithWal), 4748 CellUtil.cloneQualifier(cellWithoutWal))); 4749 assertTrue(Bytes.equals(CellUtil.cloneValue(cellWithWal), 4750 CellUtil.cloneValue(cellWithoutWal))); 4751 } 4752 } 4753 } 4754 4755 @Test 4756 public void testClientPoolRoundRobin() throws IOException { 4757 final TableName tableName = TableName.valueOf(name.getMethodName()); 4758 4759 int poolSize = 3; 4760 int numVersions = poolSize * 2; 4761 Configuration conf = TEST_UTIL.getConfiguration(); 4762 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin"); 4763 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); 4764 4765 try (Table table = 4766 TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, Integer.MAX_VALUE)) { 4767 4768 final long ts = EnvironmentEdgeManager.currentTime(); 4769 Get get = new Get(ROW); 4770 get.addColumn(FAMILY, QUALIFIER); 4771 get.setMaxVersions(); 4772 4773 for (int versions = 1; versions <= numVersions; versions++) { 4774 Put put = new Put(ROW); 4775 put.addColumn(FAMILY, QUALIFIER, ts + versions, VALUE); 4776 table.put(put); 4777 4778 Result result = table.get(get); 4779 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY) 4780 .get(QUALIFIER); 4781 4782 assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":" 4783 + Bytes.toString(QUALIFIER) + " did not match", versions, navigableMap.size()); 4784 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) { 4785 assertTrue("The value at time " + entry.getKey() 4786 + " did not match what was put", 4787 Bytes.equals(VALUE, entry.getValue())); 4788 } 4789 } 4790 } 4791 } 4792 4793 @Ignore ("Flakey: HBASE-8989") @Test 4794 public void testClientPoolThreadLocal() throws IOException { 4795 final TableName tableName = TableName.valueOf(name.getMethodName()); 4796 4797 int poolSize = Integer.MAX_VALUE; 4798 int numVersions = 3; 4799 Configuration conf = TEST_UTIL.getConfiguration(); 4800 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local"); 4801 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); 4802 4803 try (final Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, 3)) { 4804 4805 final long ts = EnvironmentEdgeManager.currentTime(); 4806 final Get get = new Get(ROW); 4807 get.addColumn(FAMILY, QUALIFIER); 4808 get.setMaxVersions(); 4809 4810 for (int versions = 1; versions <= numVersions; versions++) { 4811 Put put = new Put(ROW); 4812 put.addColumn(FAMILY, QUALIFIER, ts + versions, VALUE); 4813 table.put(put); 4814 4815 Result result = table.get(get); 4816 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY) 4817 .get(QUALIFIER); 4818 4819 assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":" 4820 + Bytes.toString(QUALIFIER) + " did not match", versions, navigableMap.size()); 4821 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) { 4822 assertTrue("The value at time " + entry.getKey() 4823 + " did not match what was put", 4824 Bytes.equals(VALUE, entry.getValue())); 4825 } 4826 } 4827 4828 final Object waitLock = new Object(); 4829 ExecutorService executorService = Executors.newFixedThreadPool(numVersions); 4830 final AtomicReference<AssertionError> error = new AtomicReference<>(null); 4831 for (int versions = numVersions; versions < numVersions * 2; versions++) { 4832 final int versionsCopy = versions; 4833 executorService.submit(new Callable<Void>() { 4834 @Override 4835 public Void call() { 4836 try { 4837 Put put = new Put(ROW); 4838 put.addColumn(FAMILY, QUALIFIER, ts + versionsCopy, VALUE); 4839 table.put(put); 4840 4841 Result result = table.get(get); 4842 NavigableMap<Long, byte[]> navigableMap = result.getMap() 4843 .get(FAMILY).get(QUALIFIER); 4844 4845 assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":" 4846 + Bytes.toString(QUALIFIER) + " did not match " + versionsCopy, versionsCopy, 4847 navigableMap.size()); 4848 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) { 4849 assertTrue("The value at time " + entry.getKey() 4850 + " did not match what was put", 4851 Bytes.equals(VALUE, entry.getValue())); 4852 } 4853 synchronized (waitLock) { 4854 waitLock.wait(); 4855 } 4856 } catch (Exception e) { 4857 } catch (AssertionError e) { 4858 // the error happens in a thread, it won't fail the test, 4859 // need to pass it to the caller for proper handling. 4860 error.set(e); 4861 LOG.error(e.toString(), e); 4862 } 4863 4864 return null; 4865 } 4866 }); 4867 } 4868 synchronized (waitLock) { 4869 waitLock.notifyAll(); 4870 } 4871 executorService.shutdownNow(); 4872 assertNull(error.get()); 4873 } 4874 } 4875 4876 @Test 4877 public void testCheckAndPut() throws IOException { 4878 final byte [] anotherrow = Bytes.toBytes("anotherrow"); 4879 final byte [] value2 = Bytes.toBytes("abcd"); 4880 4881 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 4882 Put put1 = new Put(ROW); 4883 put1.addColumn(FAMILY, QUALIFIER, VALUE); 4884 4885 // row doesn't exist, so using non-null value should be considered "not match". 4886 boolean ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4887 .ifEquals(VALUE).thenPut(put1); 4888 assertFalse(ok); 4889 4890 // row doesn't exist, so using "ifNotExists" should be considered "match". 4891 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); 4892 assertTrue(ok); 4893 4894 // row now exists, so using "ifNotExists" should be considered "not match". 4895 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put1); 4896 assertFalse(ok); 4897 4898 Put put2 = new Put(ROW); 4899 put2.addColumn(FAMILY, QUALIFIER, value2); 4900 4901 // row now exists, use the matching value to check 4902 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(VALUE).thenPut(put2); 4903 assertTrue(ok); 4904 4905 Put put3 = new Put(anotherrow); 4906 put3.addColumn(FAMILY, QUALIFIER, VALUE); 4907 4908 // try to do CheckAndPut on different rows 4909 try { 4910 table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifEquals(value2).thenPut(put3); 4911 fail("trying to check and modify different rows should have failed."); 4912 } catch (Exception e) { 4913 } 4914 } 4915 } 4916 4917 @Test 4918 public void testCheckAndMutateWithTimeRange() throws IOException { 4919 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 4920 final long ts = System.currentTimeMillis() / 2; 4921 Put put = new Put(ROW); 4922 put.addColumn(FAMILY, QUALIFIER, ts, VALUE); 4923 4924 boolean ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4925 .ifNotExists() 4926 .thenPut(put); 4927 assertTrue(ok); 4928 4929 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4930 .timeRange(TimeRange.at(ts + 10000)) 4931 .ifEquals(VALUE) 4932 .thenPut(put); 4933 assertFalse(ok); 4934 4935 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4936 .timeRange(TimeRange.from(ts + 10000)) 4937 .ifEquals(VALUE) 4938 .thenPut(put); 4939 assertFalse(ok); 4940 4941 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4942 .timeRange(TimeRange.between(ts + 10000, ts + 20000)) 4943 .ifEquals(VALUE) 4944 .thenPut(put); 4945 assertFalse(ok); 4946 4947 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4948 .timeRange(TimeRange.until(ts)) 4949 .ifEquals(VALUE) 4950 .thenPut(put); 4951 assertFalse(ok); 4952 4953 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4954 .timeRange(TimeRange.at(ts)) 4955 .ifEquals(VALUE) 4956 .thenPut(put); 4957 assertTrue(ok); 4958 4959 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4960 .timeRange(TimeRange.from(ts)) 4961 .ifEquals(VALUE) 4962 .thenPut(put); 4963 assertTrue(ok); 4964 4965 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4966 .timeRange(TimeRange.between(ts, ts + 20000)) 4967 .ifEquals(VALUE) 4968 .thenPut(put); 4969 assertTrue(ok); 4970 4971 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4972 .timeRange(TimeRange.until(ts + 10000)) 4973 .ifEquals(VALUE) 4974 .thenPut(put); 4975 assertTrue(ok); 4976 4977 RowMutations rm = new RowMutations(ROW) 4978 .add((Mutation) put); 4979 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4980 .timeRange(TimeRange.at(ts + 10000)) 4981 .ifEquals(VALUE) 4982 .thenMutate(rm); 4983 assertFalse(ok); 4984 4985 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4986 .timeRange(TimeRange.at(ts)) 4987 .ifEquals(VALUE) 4988 .thenMutate(rm); 4989 assertTrue(ok); 4990 4991 Delete delete = new Delete(ROW) 4992 .addColumn(FAMILY, QUALIFIER); 4993 4994 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 4995 .timeRange(TimeRange.at(ts + 10000)) 4996 .ifEquals(VALUE) 4997 .thenDelete(delete); 4998 assertFalse(ok); 4999 5000 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5001 .timeRange(TimeRange.at(ts)) 5002 .ifEquals(VALUE) 5003 .thenDelete(delete); 5004 assertTrue(ok); 5005 } 5006 } 5007 5008 @Test 5009 public void testCheckAndPutWithCompareOp() throws IOException { 5010 final byte [] value1 = Bytes.toBytes("aaaa"); 5011 final byte [] value2 = Bytes.toBytes("bbbb"); 5012 final byte [] value3 = Bytes.toBytes("cccc"); 5013 final byte [] value4 = Bytes.toBytes("dddd"); 5014 5015 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 5016 5017 Put put2 = new Put(ROW); 5018 put2.addColumn(FAMILY, QUALIFIER, value2); 5019 5020 Put put3 = new Put(ROW); 5021 put3.addColumn(FAMILY, QUALIFIER, value3); 5022 5023 // row doesn't exist, so using "ifNotExists" should be considered "match". 5024 boolean ok = 5025 table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER).ifNotExists().thenPut(put2); 5026 assertTrue(ok); 5027 5028 // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL 5029 // turns out "match" 5030 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5031 .ifMatches(CompareOperator.GREATER, value1).thenPut(put2); 5032 assertFalse(ok); 5033 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5034 .ifMatches(CompareOperator.EQUAL, value1).thenPut(put2); 5035 assertFalse(ok); 5036 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5037 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenPut(put2); 5038 assertFalse(ok); 5039 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5040 .ifMatches(CompareOperator.LESS, value1).thenPut(put2); 5041 assertTrue(ok); 5042 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5043 .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenPut(put2); 5044 assertTrue(ok); 5045 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5046 .ifMatches(CompareOperator.NOT_EQUAL, value1).thenPut(put3); 5047 assertTrue(ok); 5048 5049 // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL 5050 // turns out "match" 5051 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5052 .ifMatches(CompareOperator.LESS, value4).thenPut(put3); 5053 assertFalse(ok); 5054 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5055 .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenPut(put3); 5056 assertFalse(ok); 5057 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5058 .ifMatches(CompareOperator.EQUAL, value4).thenPut(put3); 5059 assertFalse(ok); 5060 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5061 .ifMatches(CompareOperator.GREATER, value4).thenPut(put3); 5062 assertTrue(ok); 5063 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5064 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenPut(put3); 5065 assertTrue(ok); 5066 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5067 .ifMatches(CompareOperator.NOT_EQUAL, value4).thenPut(put2); 5068 assertTrue(ok); 5069 5070 // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL 5071 // turns out "match" 5072 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5073 .ifMatches(CompareOperator.GREATER, value2).thenPut(put2); 5074 assertFalse(ok); 5075 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5076 .ifMatches(CompareOperator.NOT_EQUAL, value2).thenPut(put2); 5077 assertFalse(ok); 5078 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5079 .ifMatches(CompareOperator.LESS, value2).thenPut(put2); 5080 assertFalse(ok); 5081 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5082 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenPut(put2); 5083 assertTrue(ok); 5084 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5085 .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenPut(put2); 5086 assertTrue(ok); 5087 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5088 .ifMatches(CompareOperator.EQUAL, value2).thenPut(put3); 5089 assertTrue(ok); 5090 } 5091 } 5092 5093 @Test 5094 public void testCheckAndDelete() throws IOException { 5095 final byte [] value1 = Bytes.toBytes("aaaa"); 5096 5097 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), 5098 FAMILY)) { 5099 5100 Put put = new Put(ROW); 5101 put.addColumn(FAMILY, QUALIFIER, value1); 5102 table.put(put); 5103 5104 Delete delete = new Delete(ROW); 5105 delete.addColumns(FAMILY, QUALIFIER); 5106 5107 boolean ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5108 .ifEquals(value1).thenDelete(delete); 5109 assertTrue(ok); 5110 } 5111 } 5112 5113 @Test 5114 public void testCheckAndDeleteWithCompareOp() throws IOException { 5115 final byte [] value1 = Bytes.toBytes("aaaa"); 5116 final byte [] value2 = Bytes.toBytes("bbbb"); 5117 final byte [] value3 = Bytes.toBytes("cccc"); 5118 final byte [] value4 = Bytes.toBytes("dddd"); 5119 5120 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), 5121 FAMILY)) { 5122 5123 Put put2 = new Put(ROW); 5124 put2.addColumn(FAMILY, QUALIFIER, value2); 5125 table.put(put2); 5126 5127 Put put3 = new Put(ROW); 5128 put3.addColumn(FAMILY, QUALIFIER, value3); 5129 5130 Delete delete = new Delete(ROW); 5131 delete.addColumns(FAMILY, QUALIFIER); 5132 5133 // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL 5134 // turns out "match" 5135 boolean ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5136 .ifMatches(CompareOperator.GREATER, value1).thenDelete(delete); 5137 assertFalse(ok); 5138 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5139 .ifMatches(CompareOperator.EQUAL, value1).thenDelete(delete); 5140 assertFalse(ok); 5141 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5142 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value1).thenDelete(delete); 5143 assertFalse(ok); 5144 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5145 .ifMatches(CompareOperator.LESS, value1).thenDelete(delete); 5146 assertTrue(ok); 5147 table.put(put2); 5148 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5149 .ifMatches(CompareOperator.LESS_OR_EQUAL, value1).thenDelete(delete); 5150 assertTrue(ok); 5151 table.put(put2); 5152 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5153 .ifMatches(CompareOperator.NOT_EQUAL, value1).thenDelete(delete); 5154 assertTrue(ok); 5155 5156 // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL 5157 // turns out "match" 5158 table.put(put3); 5159 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5160 .ifMatches(CompareOperator.LESS, value4).thenDelete(delete); 5161 assertFalse(ok); 5162 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5163 .ifMatches(CompareOperator.LESS_OR_EQUAL, value4).thenDelete(delete); 5164 assertFalse(ok); 5165 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5166 .ifMatches(CompareOperator.EQUAL, value4).thenDelete(delete); 5167 assertFalse(ok); 5168 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5169 .ifMatches(CompareOperator.GREATER, value4).thenDelete(delete); 5170 assertTrue(ok); 5171 table.put(put3); 5172 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5173 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value4).thenDelete(delete); 5174 assertTrue(ok); 5175 table.put(put3); 5176 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5177 .ifMatches(CompareOperator.NOT_EQUAL, value4).thenDelete(delete); 5178 assertTrue(ok); 5179 5180 // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL 5181 // turns out "match" 5182 table.put(put2); 5183 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5184 .ifMatches(CompareOperator.GREATER, value2).thenDelete(delete); 5185 assertFalse(ok); 5186 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5187 .ifMatches(CompareOperator.NOT_EQUAL, value2).thenDelete(delete); 5188 assertFalse(ok); 5189 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5190 .ifMatches(CompareOperator.LESS, value2).thenDelete(delete); 5191 assertFalse(ok); 5192 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5193 .ifMatches(CompareOperator.GREATER_OR_EQUAL, value2).thenDelete(delete); 5194 assertTrue(ok); 5195 table.put(put2); 5196 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5197 .ifMatches(CompareOperator.LESS_OR_EQUAL, value2).thenDelete(delete); 5198 assertTrue(ok); 5199 table.put(put2); 5200 ok = table.checkAndMutate(ROW, FAMILY).qualifier(QUALIFIER) 5201 .ifMatches(CompareOperator.EQUAL, value2).thenDelete(delete); 5202 assertTrue(ok); 5203 } 5204 } 5205 5206 /** 5207 * Test ScanMetrics 5208 */ 5209 @Test 5210 @SuppressWarnings ("unused") 5211 public void testScanMetrics() throws Exception { 5212 final TableName tableName = TableName.valueOf(name.getMethodName()); 5213 5214 // Set up test table: 5215 // Create table: 5216 try (Table ht = TEST_UTIL.createMultiRegionTable(tableName, FAMILY)) { 5217 int numOfRegions = -1; 5218 try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 5219 numOfRegions = r.getStartKeys().length; 5220 } 5221 // Create 3 rows in the table, with rowkeys starting with "zzz*" so that 5222 // scan are forced to hit all the regions. 5223 Put put1 = new Put(Bytes.toBytes("zzz1")); 5224 put1.addColumn(FAMILY, QUALIFIER, VALUE); 5225 Put put2 = new Put(Bytes.toBytes("zzz2")); 5226 put2.addColumn(FAMILY, QUALIFIER, VALUE); 5227 Put put3 = new Put(Bytes.toBytes("zzz3")); 5228 put3.addColumn(FAMILY, QUALIFIER, VALUE); 5229 ht.put(Arrays.asList(put1, put2, put3)); 5230 5231 Scan scan1 = new Scan(); 5232 int numRecords = 0; 5233 try (ResultScanner scanner = ht.getScanner(scan1)) { 5234 for (Result result : scanner) { 5235 numRecords++; 5236 } 5237 5238 LOG.info("test data has " + numRecords + " records."); 5239 5240 // by default, scan metrics collection is turned off 5241 assertEquals(null, scanner.getScanMetrics()); 5242 } 5243 5244 // turn on scan metrics 5245 Scan scan2 = new Scan(); 5246 scan2.setScanMetricsEnabled(true); 5247 scan2.setCaching(numRecords + 1); 5248 try (ResultScanner scanner = ht.getScanner(scan2)) { 5249 for (Result result : scanner.next(numRecords - 1)) { 5250 } 5251 } 5252 // closing the scanner will set the metrics. 5253 assertNotNull(scan2.getScanMetrics()); 5254 5255 // set caching to 1, because metrics are collected in each roundtrip only 5256 scan2 = new Scan(); 5257 scan2.setScanMetricsEnabled(true); 5258 scan2.setCaching(1); 5259 try (ResultScanner scanner = ht.getScanner(scan2)) { 5260 // per HBASE-5717, this should still collect even if you don't run all the way to 5261 // the end of the scanner. So this is asking for 2 of the 3 rows we inserted. 5262 for (Result result : scanner.next(numRecords - 1)) { 5263 } 5264 } 5265 ScanMetrics scanMetrics = scan2.getScanMetrics(); 5266 assertEquals("Did not access all the regions in the table", numOfRegions, 5267 scanMetrics.countOfRegions.get()); 5268 5269 // check byte counters 5270 scan2 = new Scan(); 5271 scan2.setScanMetricsEnabled(true); 5272 scan2.setCaching(1); 5273 try (ResultScanner scanner = ht.getScanner(scan2)) { 5274 int numBytes = 0; 5275 for (Result result : scanner.next(1)) { 5276 for (Cell cell : result.listCells()) { 5277 numBytes += PrivateCellUtil.estimatedSerializedSizeOf(cell); 5278 } 5279 } 5280 scanMetrics = scanner.getScanMetrics(); 5281 assertEquals("Did not count the result bytes", numBytes, 5282 scanMetrics.countOfBytesInResults.get()); 5283 } 5284 5285 // check byte counters on a small scan 5286 scan2 = new Scan(); 5287 scan2.setScanMetricsEnabled(true); 5288 scan2.setCaching(1); 5289 scan2.setSmall(true); 5290 try (ResultScanner scanner = ht.getScanner(scan2)) { 5291 int numBytes = 0; 5292 for (Result result : scanner.next(1)) { 5293 for (Cell cell : result.listCells()) { 5294 numBytes += PrivateCellUtil.estimatedSerializedSizeOf(cell); 5295 } 5296 } 5297 scanMetrics = scanner.getScanMetrics(); 5298 assertEquals("Did not count the result bytes", numBytes, 5299 scanMetrics.countOfBytesInResults.get()); 5300 } 5301 5302 // now, test that the metrics are still collected even if you don't call close, but do 5303 // run past the end of all the records 5304 /** There seems to be a timing issue here. Comment out for now. Fix when time. 5305 Scan scanWithoutClose = new Scan(); 5306 scanWithoutClose.setCaching(1); 5307 scanWithoutClose.setScanMetricsEnabled(true); 5308 ResultScanner scannerWithoutClose = ht.getScanner(scanWithoutClose); 5309 for (Result result : scannerWithoutClose.next(numRecords + 1)) { 5310 } 5311 ScanMetrics scanMetricsWithoutClose = getScanMetrics(scanWithoutClose); 5312 assertEquals("Did not access all the regions in the table", numOfRegions, 5313 scanMetricsWithoutClose.countOfRegions.get()); 5314 */ 5315 5316 // finally, 5317 // test that the metrics are collected correctly if you both run past all the records, 5318 // AND close the scanner 5319 Scan scanWithClose = new Scan(); 5320 // make sure we can set caching up to the number of a scanned values 5321 scanWithClose.setCaching(numRecords); 5322 scanWithClose.setScanMetricsEnabled(true); 5323 try (ResultScanner scannerWithClose = ht.getScanner(scanWithClose)) { 5324 for (Result result : scannerWithClose.next(numRecords + 1)) { 5325 } 5326 } 5327 ScanMetrics scanMetricsWithClose = scanWithClose.getScanMetrics(); 5328 assertEquals("Did not access all the regions in the table", numOfRegions, 5329 scanMetricsWithClose.countOfRegions.get()); 5330 } 5331 } 5332 5333 private ScanMetrics getScanMetrics(Scan scan) throws Exception { 5334 byte[] serializedMetrics = scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA); 5335 assertTrue("Serialized metrics were not found.", serializedMetrics != null); 5336 5337 ScanMetrics scanMetrics = ProtobufUtil.toScanMetrics(serializedMetrics); 5338 5339 return scanMetrics; 5340 } 5341 5342 /** 5343 * Tests that cache on write works all the way up from the client-side. 5344 * 5345 * Performs inserts, flushes, and compactions, verifying changes in the block 5346 * cache along the way. 5347 */ 5348 @Test 5349 public void testCacheOnWriteEvictOnClose() throws Exception { 5350 final TableName tableName = TableName.valueOf(name.getMethodName()); 5351 byte [] data = Bytes.toBytes("data"); 5352 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 5353 try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 5354 // get the block cache and region 5355 String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); 5356 5357 HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName) 5358 .getRegion(regionName); 5359 HStore store = region.getStores().iterator().next(); 5360 CacheConfig cacheConf = store.getCacheConfig(); 5361 cacheConf.setCacheDataOnWrite(true); 5362 cacheConf.setEvictOnClose(true); 5363 BlockCache cache = cacheConf.getBlockCache().get(); 5364 5365 // establish baseline stats 5366 long startBlockCount = cache.getBlockCount(); 5367 long startBlockHits = cache.getStats().getHitCount(); 5368 long startBlockMiss = cache.getStats().getMissCount(); 5369 5370 // wait till baseline is stable, (minimal 500 ms) 5371 for (int i = 0; i < 5; i++) { 5372 Thread.sleep(100); 5373 if (startBlockCount != cache.getBlockCount() 5374 || startBlockHits != cache.getStats().getHitCount() 5375 || startBlockMiss != cache.getStats().getMissCount()) { 5376 startBlockCount = cache.getBlockCount(); 5377 startBlockHits = cache.getStats().getHitCount(); 5378 startBlockMiss = cache.getStats().getMissCount(); 5379 i = -1; 5380 } 5381 } 5382 5383 // insert data 5384 Put put = new Put(ROW); 5385 put.addColumn(FAMILY, QUALIFIER, data); 5386 table.put(put); 5387 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); 5388 // data was in memstore so don't expect any changes 5389 assertEquals(startBlockCount, cache.getBlockCount()); 5390 assertEquals(startBlockHits, cache.getStats().getHitCount()); 5391 assertEquals(startBlockMiss, cache.getStats().getMissCount()); 5392 // flush the data 5393 System.out.println("Flushing cache"); 5394 region.flush(true); 5395 // expect one more block in cache, no change in hits/misses 5396 long expectedBlockCount = startBlockCount + 1; 5397 long expectedBlockHits = startBlockHits; 5398 long expectedBlockMiss = startBlockMiss; 5399 assertEquals(expectedBlockCount, cache.getBlockCount()); 5400 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 5401 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 5402 // read the data and expect same blocks, one new hit, no misses 5403 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); 5404 assertEquals(expectedBlockCount, cache.getBlockCount()); 5405 assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); 5406 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 5407 // insert a second column, read the row, no new blocks, one new hit 5408 byte[] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER); 5409 byte[] data2 = Bytes.add(data, data); 5410 put = new Put(ROW); 5411 put.addColumn(FAMILY, QUALIFIER2, data2); 5412 table.put(put); 5413 Result r = table.get(new Get(ROW)); 5414 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); 5415 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); 5416 assertEquals(expectedBlockCount, cache.getBlockCount()); 5417 assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); 5418 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 5419 // flush, one new block 5420 System.out.println("Flushing cache"); 5421 region.flush(true); 5422 assertEquals(++expectedBlockCount, cache.getBlockCount()); 5423 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 5424 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 5425 // compact, net minus two blocks, two hits, no misses 5426 System.out.println("Compacting"); 5427 assertEquals(2, store.getStorefilesCount()); 5428 store.triggerMajorCompaction(); 5429 region.compact(true); 5430 store.closeAndArchiveCompactedFiles(); 5431 waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max 5432 assertEquals(1, store.getStorefilesCount()); 5433 expectedBlockCount -= 2; // evicted two blocks, cached none 5434 assertEquals(expectedBlockCount, cache.getBlockCount()); 5435 expectedBlockHits += 2; 5436 assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); 5437 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 5438 // read the row, this should be a cache miss because we don't cache data 5439 // blocks on compaction 5440 r = table.get(new Get(ROW)); 5441 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); 5442 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); 5443 expectedBlockCount += 1; // cached one data block 5444 assertEquals(expectedBlockCount, cache.getBlockCount()); 5445 assertEquals(expectedBlockHits, cache.getStats().getHitCount()); 5446 assertEquals(++expectedBlockMiss, cache.getStats().getMissCount()); 5447 } 5448 } 5449 } 5450 5451 private void waitForStoreFileCount(HStore store, int count, int timeout) 5452 throws InterruptedException { 5453 long start = System.currentTimeMillis(); 5454 while (start + timeout > System.currentTimeMillis() && store.getStorefilesCount() != count) { 5455 Thread.sleep(100); 5456 } 5457 System.out.println("start=" + start + ", now=" + System.currentTimeMillis() + ", cur=" + 5458 store.getStorefilesCount()); 5459 assertEquals(count, store.getStorefilesCount()); 5460 } 5461 5462 @Test 5463 /** 5464 * Tests the non cached version of getRegionLocator by moving a region. 5465 */ 5466 public void testNonCachedGetRegionLocation() throws Exception { 5467 // Test Initialization. 5468 final TableName tableName = TableName.valueOf(name.getMethodName()); 5469 byte [] family1 = Bytes.toBytes("f1"); 5470 byte [] family2 = Bytes.toBytes("f2"); 5471 try (Table table = TEST_UTIL.createTable(tableName, new byte[][] {family1, family2}, 10); 5472 Admin admin = TEST_UTIL.getAdmin(); 5473 RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 5474 List<HRegionLocation> allRegionLocations = locator.getAllRegionLocations(); 5475 assertEquals(1, allRegionLocations.size()); 5476 HRegionInfo regionInfo = allRegionLocations.get(0).getRegionInfo(); 5477 ServerName addrBefore = allRegionLocations.get(0).getServerName(); 5478 // Verify region location before move. 5479 HRegionLocation addrCache = locator.getRegionLocation(regionInfo.getStartKey(), false); 5480 HRegionLocation addrNoCache = locator.getRegionLocation(regionInfo.getStartKey(), true); 5481 5482 assertEquals(addrBefore.getPort(), addrCache.getPort()); 5483 assertEquals(addrBefore.getPort(), addrNoCache.getPort()); 5484 5485 ServerName addrAfter = null; 5486 // Now move the region to a different server. 5487 for (int i = 0; i < SLAVES; i++) { 5488 HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(i); 5489 ServerName addr = regionServer.getServerName(); 5490 if (addr.getPort() != addrBefore.getPort()) { 5491 admin.move(regionInfo.getEncodedNameAsBytes(), addr); 5492 // Wait for the region to move. 5493 Thread.sleep(5000); 5494 addrAfter = addr; 5495 break; 5496 } 5497 } 5498 5499 // Verify the region was moved. 5500 addrCache = locator.getRegionLocation(regionInfo.getStartKey(), false); 5501 addrNoCache = locator.getRegionLocation(regionInfo.getStartKey(), true); 5502 assertNotNull(addrAfter); 5503 assertTrue(addrAfter.getPort() != addrCache.getPort()); 5504 assertEquals(addrAfter.getPort(), addrNoCache.getPort()); 5505 } 5506 } 5507 5508 @Test 5509 /** 5510 * Tests getRegionsInRange by creating some regions over which a range of 5511 * keys spans; then changing the key range. 5512 */ 5513 public void testGetRegionsInRange() throws Exception { 5514 // Test Initialization. 5515 byte [] startKey = Bytes.toBytes("ddc"); 5516 byte [] endKey = Bytes.toBytes("mmm"); 5517 TableName tableName = TableName.valueOf(name.getMethodName()); 5518 TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10); 5519 5520 int numOfRegions = -1; 5521 try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 5522 numOfRegions = r.getStartKeys().length; 5523 } 5524 assertEquals(26, numOfRegions); 5525 5526 // Get the regions in this range 5527 List<HRegionLocation> regionsList = getRegionsInRange(tableName, startKey, endKey); 5528 assertEquals(10, regionsList.size()); 5529 5530 // Change the start key 5531 startKey = Bytes.toBytes("fff"); 5532 regionsList = getRegionsInRange(tableName, startKey, endKey); 5533 assertEquals(7, regionsList.size()); 5534 5535 // Change the end key 5536 endKey = Bytes.toBytes("nnn"); 5537 regionsList = getRegionsInRange(tableName, startKey, endKey); 5538 assertEquals(8, regionsList.size()); 5539 5540 // Empty start key 5541 regionsList = getRegionsInRange(tableName, HConstants.EMPTY_START_ROW, endKey); 5542 assertEquals(13, regionsList.size()); 5543 5544 // Empty end key 5545 regionsList = getRegionsInRange(tableName, startKey, HConstants.EMPTY_END_ROW); 5546 assertEquals(21, regionsList.size()); 5547 5548 // Both start and end keys empty 5549 regionsList = getRegionsInRange(tableName, HConstants.EMPTY_START_ROW, 5550 HConstants.EMPTY_END_ROW); 5551 assertEquals(26, regionsList.size()); 5552 5553 // Change the end key to somewhere in the last block 5554 endKey = Bytes.toBytes("zzz1"); 5555 regionsList = getRegionsInRange(tableName, startKey, endKey); 5556 assertEquals(21, regionsList.size()); 5557 5558 // Change the start key to somewhere in the first block 5559 startKey = Bytes.toBytes("aac"); 5560 regionsList = getRegionsInRange(tableName, startKey, endKey); 5561 assertEquals(26, regionsList.size()); 5562 5563 // Make start and end key the same 5564 startKey = endKey = Bytes.toBytes("ccc"); 5565 regionsList = getRegionsInRange(tableName, startKey, endKey); 5566 assertEquals(1, regionsList.size()); 5567 } 5568 5569 private List<HRegionLocation> getRegionsInRange(TableName tableName, byte[] startKey, 5570 byte[] endKey) throws IOException { 5571 List<HRegionLocation> regionsInRange = new ArrayList<>(); 5572 byte[] currentKey = startKey; 5573 final boolean endKeyIsEndOfTable = Bytes.equals(endKey, HConstants.EMPTY_END_ROW); 5574 try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 5575 do { 5576 HRegionLocation regionLocation = r.getRegionLocation(currentKey); 5577 regionsInRange.add(regionLocation); 5578 currentKey = regionLocation.getRegionInfo().getEndKey(); 5579 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) 5580 && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0)); 5581 return regionsInRange; 5582 } 5583 } 5584 5585 @Test 5586 public void testJira6912() throws Exception { 5587 final TableName tableName = TableName.valueOf(name.getMethodName()); 5588 try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] {FAMILY}, 10)) { 5589 5590 List<Put> puts = new ArrayList<Put>(); 5591 for (int i = 0; i != 100; i++) { 5592 Put put = new Put(Bytes.toBytes(i)); 5593 put.addColumn(FAMILY, FAMILY, Bytes.toBytes(i)); 5594 puts.add(put); 5595 } 5596 foo.put(puts); 5597 // If i comment this out it works 5598 TEST_UTIL.flush(); 5599 5600 Scan scan = new Scan(); 5601 scan.setStartRow(Bytes.toBytes(1)); 5602 scan.setStopRow(Bytes.toBytes(3)); 5603 scan.addColumn(FAMILY, FAMILY); 5604 scan.setFilter(new RowFilter(CompareOperator.NOT_EQUAL, 5605 new BinaryComparator(Bytes.toBytes(1)))); 5606 5607 try (ResultScanner scanner = foo.getScanner(scan)) { 5608 Result[] bar = scanner.next(100); 5609 assertEquals(1, bar.length); 5610 } 5611 } 5612 } 5613 5614 @Test 5615 public void testScan_NullQualifier() throws IOException { 5616 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 5617 Put put = new Put(ROW); 5618 put.addColumn(FAMILY, QUALIFIER, VALUE); 5619 table.put(put); 5620 5621 put = new Put(ROW); 5622 put.addColumn(FAMILY, null, VALUE); 5623 table.put(put); 5624 LOG.info("Row put"); 5625 5626 Scan scan = new Scan(); 5627 scan.addColumn(FAMILY, null); 5628 5629 ResultScanner scanner = table.getScanner(scan); 5630 Result[] bar = scanner.next(100); 5631 assertEquals(1, bar.length); 5632 assertEquals(1, bar[0].size()); 5633 5634 scan = new Scan(); 5635 scan.addFamily(FAMILY); 5636 5637 scanner = table.getScanner(scan); 5638 bar = scanner.next(100); 5639 assertEquals(1, bar.length); 5640 assertEquals(2, bar[0].size()); 5641 } 5642 } 5643 5644 @Test 5645 public void testNegativeTimestamp() throws IOException { 5646 try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { 5647 5648 try { 5649 Put put = new Put(ROW, -1); 5650 put.addColumn(FAMILY, QUALIFIER, VALUE); 5651 table.put(put); 5652 fail("Negative timestamps should not have been allowed"); 5653 } catch (IllegalArgumentException ex) { 5654 assertTrue(ex.getMessage().contains("negative")); 5655 } 5656 5657 try { 5658 Put put = new Put(ROW); 5659 long ts = -1; 5660 put.addColumn(FAMILY, QUALIFIER, ts, VALUE); 5661 table.put(put); 5662 fail("Negative timestamps should not have been allowed"); 5663 } catch (IllegalArgumentException ex) { 5664 assertTrue(ex.getMessage().contains("negative")); 5665 } 5666 5667 try { 5668 Delete delete = new Delete(ROW, -1); 5669 table.delete(delete); 5670 fail("Negative timestamps should not have been allowed"); 5671 } catch (IllegalArgumentException ex) { 5672 assertTrue(ex.getMessage().contains("negative")); 5673 } 5674 5675 try { 5676 Delete delete = new Delete(ROW); 5677 delete.addFamily(FAMILY, -1); 5678 table.delete(delete); 5679 fail("Negative timestamps should not have been allowed"); 5680 } catch (IllegalArgumentException ex) { 5681 assertTrue(ex.getMessage().contains("negative")); 5682 } 5683 5684 try { 5685 Scan scan = new Scan(); 5686 scan.setTimeRange(-1, 1); 5687 table.getScanner(scan); 5688 fail("Negative timestamps should not have been allowed"); 5689 } catch (IllegalArgumentException ex) { 5690 assertTrue(ex.getMessage().contains("negative")); 5691 } 5692 5693 // KeyValue should allow negative timestamps for backwards compat. Otherwise, if the user 5694 // already has negative timestamps in cluster data, HBase won't be able to handle that 5695 try { 5696 new KeyValue(Bytes.toBytes(42), Bytes.toBytes(42), Bytes.toBytes(42), -1, 5697 Bytes.toBytes(42)); 5698 } catch (IllegalArgumentException ex) { 5699 fail("KeyValue SHOULD allow negative timestamps"); 5700 } 5701 5702 } 5703 } 5704 5705 @Test 5706 public void testRawScanRespectsVersions() throws Exception { 5707 final TableName tableName = TableName.valueOf(name.getMethodName()); 5708 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 5709 byte[] row = Bytes.toBytes("row"); 5710 5711 // put the same row 4 times, with different values 5712 Put p = new Put(row); 5713 p.addColumn(FAMILY, QUALIFIER, 10, VALUE); 5714 table.put(p); 5715 p = new Put(row); 5716 p.addColumn(FAMILY, QUALIFIER, 11, ArrayUtils.add(VALUE, (byte) 2)); 5717 table.put(p); 5718 5719 p = new Put(row); 5720 p.addColumn(FAMILY, QUALIFIER, 12, ArrayUtils.add(VALUE, (byte) 3)); 5721 table.put(p); 5722 5723 p = new Put(row); 5724 p.addColumn(FAMILY, QUALIFIER, 13, ArrayUtils.add(VALUE, (byte) 4)); 5725 table.put(p); 5726 5727 int versions = 4; 5728 Scan s = new Scan(row); 5729 // get all the possible versions 5730 s.setMaxVersions(); 5731 s.setRaw(true); 5732 5733 try (ResultScanner scanner = table.getScanner(s)) { 5734 int count = 0; 5735 for (Result r : scanner) { 5736 assertEquals("Found an unexpected number of results for the row!", versions, 5737 r.listCells().size()); 5738 count++; 5739 } 5740 assertEquals("Found more than a single row when raw scanning the table with a single row!", 5741 1, count); 5742 } 5743 5744 // then if we decrease the number of versions, but keep the scan raw, we should see exactly 5745 // that number of versions 5746 versions = 2; 5747 s.setMaxVersions(versions); 5748 try (ResultScanner scanner = table.getScanner(s)) { 5749 int count = 0; 5750 for (Result r : scanner) { 5751 assertEquals("Found an unexpected number of results for the row!", versions, 5752 r.listCells().size()); 5753 count++; 5754 } 5755 assertEquals("Found more than a single row when raw scanning the table with a single row!", 5756 1, count); 5757 } 5758 5759 // finally, if we turn off raw scanning, but max out the number of versions, we should go back 5760 // to seeing just three 5761 versions = 3; 5762 s.setMaxVersions(versions); 5763 try (ResultScanner scanner = table.getScanner(s)) { 5764 int count = 0; 5765 for (Result r : scanner) { 5766 assertEquals("Found an unexpected number of results for the row!", versions, 5767 r.listCells().size()); 5768 count++; 5769 } 5770 assertEquals("Found more than a single row when raw scanning the table with a single row!", 5771 1, count); 5772 } 5773 5774 } 5775 TEST_UTIL.deleteTable(tableName); 5776 } 5777 5778 @Test 5779 public void testEmptyFilterList() throws Exception { 5780 // Test Initialization. 5781 final TableName tableName = TableName.valueOf(name.getMethodName()); 5782 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 5783 5784 // Insert one row each region 5785 Put put = new Put(Bytes.toBytes("row")); 5786 put.addColumn(FAMILY, QUALIFIER, VALUE); 5787 table.put(put); 5788 5789 List<Result> scanResults = new LinkedList<>(); 5790 Scan scan = new Scan(); 5791 scan.setFilter(new FilterList()); 5792 try (ResultScanner scanner = table.getScanner(scan)) { 5793 for (Result r : scanner) { 5794 scanResults.add(r); 5795 } 5796 } 5797 assertEquals(1, scanResults.size()); 5798 Get g = new Get(Bytes.toBytes("row")); 5799 g.setFilter(new FilterList()); 5800 Result getResult = table.get(g); 5801 Result scanResult = scanResults.get(0); 5802 assertEquals(scanResult.rawCells().length, getResult.rawCells().length); 5803 for (int i = 0; i != scanResult.rawCells().length; ++i) { 5804 Cell scanCell = scanResult.rawCells()[i]; 5805 Cell getCell = getResult.rawCells()[i]; 5806 assertEquals(0, Bytes.compareTo(CellUtil.cloneRow(scanCell), 5807 CellUtil.cloneRow(getCell))); 5808 assertEquals(0, Bytes.compareTo(CellUtil.cloneFamily(scanCell), 5809 CellUtil.cloneFamily(getCell))); 5810 assertEquals(0, Bytes.compareTo(CellUtil.cloneQualifier(scanCell), 5811 CellUtil.cloneQualifier(getCell))); 5812 assertEquals(0, Bytes.compareTo(CellUtil.cloneValue(scanCell), 5813 CellUtil.cloneValue(getCell))); 5814 } 5815 } 5816 } 5817 5818 @Test 5819 public void testSmallScan() throws Exception { 5820 // Test Initialization. 5821 final TableName tableName = TableName.valueOf(name.getMethodName()); 5822 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 5823 5824 // Insert one row each region 5825 int insertNum = 10; 5826 for (int i = 0; i < 10; i++) { 5827 Put put = new Put(Bytes.toBytes("row" + String.format("%03d", i))); 5828 put.addColumn(FAMILY, QUALIFIER, VALUE); 5829 table.put(put); 5830 } 5831 5832 // normal scan 5833 try (ResultScanner scanner = table.getScanner(new Scan())) { 5834 int count = 0; 5835 for (Result r : scanner) { 5836 assertTrue(!r.isEmpty()); 5837 count++; 5838 } 5839 assertEquals(insertNum, count); 5840 } 5841 5842 // small scan 5843 Scan scan = new Scan(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); 5844 scan.setSmall(true); 5845 scan.setCaching(2); 5846 try (ResultScanner scanner = table.getScanner(scan)) { 5847 int count = 0; 5848 for (Result r : scanner) { 5849 assertTrue(!r.isEmpty()); 5850 count++; 5851 } 5852 assertEquals(insertNum, count); 5853 } 5854 } 5855 } 5856 5857 @Test 5858 public void testSuperSimpleWithReverseScan() throws Exception { 5859 final TableName tableName = TableName.valueOf(name.getMethodName()); 5860 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 5861 Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); 5862 put.addColumn(FAMILY, QUALIFIER, VALUE); 5863 ht.put(put); 5864 put = new Put(Bytes.toBytes("0-b11111-0000000000000000002")); 5865 put.addColumn(FAMILY, QUALIFIER, VALUE); 5866 ht.put(put); 5867 put = new Put(Bytes.toBytes("0-b11111-0000000000000000004")); 5868 put.addColumn(FAMILY, QUALIFIER, VALUE); 5869 ht.put(put); 5870 put = new Put(Bytes.toBytes("0-b11111-0000000000000000006")); 5871 put.addColumn(FAMILY, QUALIFIER, VALUE); 5872 ht.put(put); 5873 put = new Put(Bytes.toBytes("0-b11111-0000000000000000008")); 5874 put.addColumn(FAMILY, QUALIFIER, VALUE); 5875 ht.put(put); 5876 put = new Put(Bytes.toBytes("0-b22222-0000000000000000001")); 5877 put.addColumn(FAMILY, QUALIFIER, VALUE); 5878 ht.put(put); 5879 put = new Put(Bytes.toBytes("0-b22222-0000000000000000003")); 5880 put.addColumn(FAMILY, QUALIFIER, VALUE); 5881 ht.put(put); 5882 put = new Put(Bytes.toBytes("0-b22222-0000000000000000005")); 5883 put.addColumn(FAMILY, QUALIFIER, VALUE); 5884 ht.put(put); 5885 put = new Put(Bytes.toBytes("0-b22222-0000000000000000007")); 5886 put.addColumn(FAMILY, QUALIFIER, VALUE); 5887 ht.put(put); 5888 put = new Put(Bytes.toBytes("0-b22222-0000000000000000009")); 5889 put.addColumn(FAMILY, QUALIFIER, VALUE); 5890 ht.put(put); 5891 Scan scan = new Scan(Bytes.toBytes("0-b11111-9223372036854775807"), 5892 Bytes.toBytes("0-b11111-0000000000000000000")); 5893 scan.setReversed(true); 5894 try (ResultScanner scanner = ht.getScanner(scan)) { 5895 Result result = scanner.next(); 5896 assertTrue(Bytes.equals(result.getRow(), 5897 Bytes.toBytes("0-b11111-0000000000000000008"))); 5898 } 5899 } 5900 } 5901 5902 @Test 5903 public void testFiltersWithReverseScan() throws Exception { 5904 final TableName tableName = TableName.valueOf(name.getMethodName()); 5905 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 5906 byte[][] ROWS = makeN(ROW, 10); 5907 byte[][] QUALIFIERS = {Bytes.toBytes("col0-<d2v1>-<d3v2>"), 5908 Bytes.toBytes("col1-<d2v1>-<d3v2>"), 5909 Bytes.toBytes("col2-<d2v1>-<d3v2>"), 5910 Bytes.toBytes("col3-<d2v1>-<d3v2>"), 5911 Bytes.toBytes("col4-<d2v1>-<d3v2>"), 5912 Bytes.toBytes("col5-<d2v1>-<d3v2>"), 5913 Bytes.toBytes("col6-<d2v1>-<d3v2>"), 5914 Bytes.toBytes("col7-<d2v1>-<d3v2>"), 5915 Bytes.toBytes("col8-<d2v1>-<d3v2>"), 5916 Bytes.toBytes("col9-<d2v1>-<d3v2>")}; 5917 for (int i = 0; i < 10; i++) { 5918 Put put = new Put(ROWS[i]); 5919 put.addColumn(FAMILY, QUALIFIERS[i], VALUE); 5920 ht.put(put); 5921 } 5922 Scan scan = new Scan(); 5923 scan.setReversed(true); 5924 scan.addFamily(FAMILY); 5925 Filter filter = new QualifierFilter(CompareOperator.EQUAL, 5926 new RegexStringComparator("col[1-5]")); 5927 scan.setFilter(filter); 5928 try (ResultScanner scanner = ht.getScanner(scan)) { 5929 int expectedIndex = 5; 5930 for (Result result : scanner) { 5931 assertEquals(1, result.size()); 5932 Cell c = result.rawCells()[0]; 5933 assertTrue(Bytes.equals(c.getRowArray(), c.getRowOffset(), c.getRowLength(), 5934 ROWS[expectedIndex], 0, ROWS[expectedIndex].length)); 5935 assertTrue(Bytes.equals(c.getQualifierArray(), c.getQualifierOffset(), 5936 c.getQualifierLength(), QUALIFIERS[expectedIndex], 0, 5937 QUALIFIERS[expectedIndex].length)); 5938 expectedIndex--; 5939 } 5940 assertEquals(0, expectedIndex); 5941 } 5942 } 5943 } 5944 5945 @Test 5946 public void testKeyOnlyFilterWithReverseScan() throws Exception { 5947 final TableName tableName = TableName.valueOf(name.getMethodName()); 5948 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 5949 byte[][] ROWS = makeN(ROW, 10); 5950 byte[][] QUALIFIERS = {Bytes.toBytes("col0-<d2v1>-<d3v2>"), 5951 Bytes.toBytes("col1-<d2v1>-<d3v2>"), 5952 Bytes.toBytes("col2-<d2v1>-<d3v2>"), 5953 Bytes.toBytes("col3-<d2v1>-<d3v2>"), 5954 Bytes.toBytes("col4-<d2v1>-<d3v2>"), 5955 Bytes.toBytes("col5-<d2v1>-<d3v2>"), 5956 Bytes.toBytes("col6-<d2v1>-<d3v2>"), 5957 Bytes.toBytes("col7-<d2v1>-<d3v2>"), 5958 Bytes.toBytes("col8-<d2v1>-<d3v2>"), 5959 Bytes.toBytes("col9-<d2v1>-<d3v2>")}; 5960 for (int i = 0; i < 10; i++) { 5961 Put put = new Put(ROWS[i]); 5962 put.addColumn(FAMILY, QUALIFIERS[i], VALUE); 5963 ht.put(put); 5964 } 5965 Scan scan = new Scan(); 5966 scan.setReversed(true); 5967 scan.addFamily(FAMILY); 5968 Filter filter = new KeyOnlyFilter(true); 5969 scan.setFilter(filter); 5970 try (ResultScanner scanner = ht.getScanner(scan)) { 5971 int count = 0; 5972 for (Result result : ht.getScanner(scan)) { 5973 assertEquals(1, result.size()); 5974 assertEquals(Bytes.SIZEOF_INT, result.rawCells()[0].getValueLength()); 5975 assertEquals(VALUE.length, Bytes.toInt(CellUtil.cloneValue(result.rawCells()[0]))); 5976 count++; 5977 } 5978 assertEquals(10, count); 5979 } 5980 } 5981 } 5982 5983 /** 5984 * Test simple table and non-existent row cases. 5985 */ 5986 @Test 5987 public void testSimpleMissingWithReverseScan() throws Exception { 5988 final TableName tableName = TableName.valueOf(name.getMethodName()); 5989 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 5990 byte[][] ROWS = makeN(ROW, 4); 5991 5992 // Try to get a row on an empty table 5993 Scan scan = new Scan(); 5994 scan.setReversed(true); 5995 Result result = getSingleScanResult(ht, scan); 5996 assertNullResult(result); 5997 5998 scan = new Scan(ROWS[0]); 5999 scan.setReversed(true); 6000 result = getSingleScanResult(ht, scan); 6001 assertNullResult(result); 6002 6003 scan = new Scan(ROWS[0], ROWS[1]); 6004 scan.setReversed(true); 6005 result = getSingleScanResult(ht, scan); 6006 assertNullResult(result); 6007 6008 scan = new Scan(); 6009 scan.setReversed(true); 6010 scan.addFamily(FAMILY); 6011 result = getSingleScanResult(ht, scan); 6012 assertNullResult(result); 6013 6014 scan = new Scan(); 6015 scan.setReversed(true); 6016 scan.addColumn(FAMILY, QUALIFIER); 6017 result = getSingleScanResult(ht, scan); 6018 assertNullResult(result); 6019 6020 // Insert a row 6021 6022 Put put = new Put(ROWS[2]); 6023 put.addColumn(FAMILY, QUALIFIER, VALUE); 6024 ht.put(put); 6025 6026 // Make sure we can scan the row 6027 scan = new Scan(); 6028 scan.setReversed(true); 6029 result = getSingleScanResult(ht, scan); 6030 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 6031 6032 scan = new Scan(ROWS[3], ROWS[0]); 6033 scan.setReversed(true); 6034 result = getSingleScanResult(ht, scan); 6035 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 6036 6037 scan = new Scan(ROWS[2], ROWS[1]); 6038 scan.setReversed(true); 6039 result = getSingleScanResult(ht, scan); 6040 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); 6041 6042 // Try to scan empty rows around it 6043 // Introduced MemStore#shouldSeekForReverseScan to fix the following 6044 scan = new Scan(ROWS[1]); 6045 scan.setReversed(true); 6046 result = getSingleScanResult(ht, scan); 6047 assertNullResult(result); 6048 } 6049 } 6050 6051 @Test 6052 public void testNullWithReverseScan() throws Exception { 6053 final TableName tableName = TableName.valueOf(name.getMethodName()); 6054 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { 6055 // Null qualifier (should work) 6056 Put put = new Put(ROW); 6057 put.addColumn(FAMILY, null, VALUE); 6058 ht.put(put); 6059 scanTestNull(ht, ROW, FAMILY, VALUE, true); 6060 Delete delete = new Delete(ROW); 6061 delete.addColumns(FAMILY, null); 6062 ht.delete(delete); 6063 } 6064 6065 // Use a new table 6066 try (Table ht = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName() + "2"), FAMILY)) { 6067 // Empty qualifier, byte[0] instead of null (should work) 6068 Put put = new Put(ROW); 6069 put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); 6070 ht.put(put); 6071 scanTestNull(ht, ROW, FAMILY, VALUE, true); 6072 TEST_UTIL.flush(); 6073 scanTestNull(ht, ROW, FAMILY, VALUE, true); 6074 Delete delete = new Delete(ROW); 6075 delete.addColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY); 6076 ht.delete(delete); 6077 // Null value 6078 put = new Put(ROW); 6079 put.addColumn(FAMILY, QUALIFIER, null); 6080 ht.put(put); 6081 Scan scan = new Scan(); 6082 scan.setReversed(true); 6083 scan.addColumn(FAMILY, QUALIFIER); 6084 Result result = getSingleScanResult(ht, scan); 6085 assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); 6086 } 6087 } 6088 6089 @Test 6090 @SuppressWarnings("checkstyle:MethodLength") 6091 public void testDeletesWithReverseScan() throws Exception { 6092 final TableName tableName = TableName.valueOf(name.getMethodName()); 6093 byte[][] ROWS = makeNAscii(ROW, 6); 6094 byte[][] FAMILIES = makeNAscii(FAMILY, 3); 6095 byte[][] VALUES = makeN(VALUE, 5); 6096 long[] ts = { 1000, 2000, 3000, 4000, 5000 }; 6097 try (Table ht = TEST_UTIL.createTable(tableName, FAMILIES, 3)) { 6098 6099 Put put = new Put(ROW); 6100 put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); 6101 put.addColumn(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]); 6102 ht.put(put); 6103 6104 Delete delete = new Delete(ROW); 6105 delete.addFamily(FAMILIES[0], ts[0]); 6106 ht.delete(delete); 6107 6108 Scan scan = new Scan(ROW); 6109 scan.setReversed(true); 6110 scan.addFamily(FAMILIES[0]); 6111 scan.setMaxVersions(Integer.MAX_VALUE); 6112 Result result = getSingleScanResult(ht, scan); 6113 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[]{ts[1]}, 6114 new byte[][]{VALUES[1]}, 0, 0); 6115 6116 // Test delete latest version 6117 put = new Put(ROW); 6118 put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); 6119 put.addColumn(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]); 6120 put.addColumn(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]); 6121 put.addColumn(FAMILIES[0], null, ts[4], VALUES[4]); 6122 put.addColumn(FAMILIES[0], null, ts[2], VALUES[2]); 6123 put.addColumn(FAMILIES[0], null, ts[3], VALUES[3]); 6124 ht.put(put); 6125 6126 delete = new Delete(ROW); 6127 delete.addColumn(FAMILIES[0], QUALIFIER); // ts[4] 6128 ht.delete(delete); 6129 6130 scan = new Scan(ROW); 6131 scan.setReversed(true); 6132 scan.addColumn(FAMILIES[0], QUALIFIER); 6133 scan.setMaxVersions(Integer.MAX_VALUE); 6134 result = getSingleScanResult(ht, scan); 6135 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[]{ts[1], 6136 ts[2], ts[3]}, new byte[][]{VALUES[1], VALUES[2], VALUES[3]}, 0, 2); 6137 6138 // Test for HBASE-1847 6139 delete = new Delete(ROW); 6140 delete.addColumn(FAMILIES[0], null); 6141 ht.delete(delete); 6142 6143 // Cleanup null qualifier 6144 delete = new Delete(ROW); 6145 delete.addColumns(FAMILIES[0], null); 6146 ht.delete(delete); 6147 6148 // Expected client behavior might be that you can re-put deleted values 6149 // But alas, this is not to be. We can't put them back in either case. 6150 6151 put = new Put(ROW); 6152 put.addColumn(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); 6153 put.addColumn(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); 6154 ht.put(put); 6155 6156 // The Scanner returns the previous values, the expected-naive-unexpected 6157 // behavior 6158 6159 scan = new Scan(ROW); 6160 scan.setReversed(true); 6161 scan.addFamily(FAMILIES[0]); 6162 scan.setMaxVersions(Integer.MAX_VALUE); 6163 result = getSingleScanResult(ht, scan); 6164 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[]{ts[1], 6165 ts[2], ts[3]}, new byte[][]{VALUES[1], VALUES[2], VALUES[3]}, 0, 2); 6166 6167 // Test deleting an entire family from one row but not the other various 6168 // ways 6169 6170 put = new Put(ROWS[0]); 6171 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 6172 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 6173 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 6174 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 6175 ht.put(put); 6176 6177 put = new Put(ROWS[1]); 6178 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 6179 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 6180 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 6181 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 6182 ht.put(put); 6183 6184 put = new Put(ROWS[2]); 6185 put.addColumn(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]); 6186 put.addColumn(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]); 6187 put.addColumn(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]); 6188 put.addColumn(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]); 6189 ht.put(put); 6190 6191 delete = new Delete(ROWS[0]); 6192 delete.addFamily(FAMILIES[2]); 6193 ht.delete(delete); 6194 6195 delete = new Delete(ROWS[1]); 6196 delete.addColumns(FAMILIES[1], QUALIFIER); 6197 ht.delete(delete); 6198 6199 delete = new Delete(ROWS[2]); 6200 delete.addColumn(FAMILIES[1], QUALIFIER); 6201 delete.addColumn(FAMILIES[1], QUALIFIER); 6202 delete.addColumn(FAMILIES[2], QUALIFIER); 6203 ht.delete(delete); 6204 6205 scan = new Scan(ROWS[0]); 6206 scan.setReversed(true); 6207 scan.addFamily(FAMILIES[1]); 6208 scan.addFamily(FAMILIES[2]); 6209 scan.setMaxVersions(Integer.MAX_VALUE); 6210 result = getSingleScanResult(ht, scan); 6211 assertTrue("Expected 2 keys but received " + result.size(), 6212 result.size() == 2); 6213 assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, new long[]{ts[0], 6214 ts[1]}, new byte[][]{VALUES[0], VALUES[1]}, 0, 1); 6215 6216 scan = new Scan(ROWS[1]); 6217 scan.setReversed(true); 6218 scan.addFamily(FAMILIES[1]); 6219 scan.addFamily(FAMILIES[2]); 6220 scan.setMaxVersions(Integer.MAX_VALUE); 6221 result = getSingleScanResult(ht, scan); 6222 assertTrue("Expected 2 keys but received " + result.size(), 6223 result.size() == 2); 6224 6225 scan = new Scan(ROWS[2]); 6226 scan.setReversed(true); 6227 scan.addFamily(FAMILIES[1]); 6228 scan.addFamily(FAMILIES[2]); 6229 scan.setMaxVersions(Integer.MAX_VALUE); 6230 result = getSingleScanResult(ht, scan); 6231 assertEquals(1, result.size()); 6232 assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER, 6233 new long[]{ts[2]}, new byte[][]{VALUES[2]}, 0, 0); 6234 6235 // Test if we delete the family first in one row (HBASE-1541) 6236 6237 delete = new Delete(ROWS[3]); 6238 delete.addFamily(FAMILIES[1]); 6239 ht.delete(delete); 6240 6241 put = new Put(ROWS[3]); 6242 put.addColumn(FAMILIES[2], QUALIFIER, VALUES[0]); 6243 ht.put(put); 6244 6245 put = new Put(ROWS[4]); 6246 put.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]); 6247 put.addColumn(FAMILIES[2], QUALIFIER, VALUES[2]); 6248 ht.put(put); 6249 6250 scan = new Scan(ROWS[4]); 6251 scan.setReversed(true); 6252 scan.addFamily(FAMILIES[1]); 6253 scan.addFamily(FAMILIES[2]); 6254 scan.setMaxVersions(Integer.MAX_VALUE); 6255 try (ResultScanner scanner = ht.getScanner(scan)) { 6256 result = scanner.next(); 6257 assertTrue("Expected 2 keys but received " + result.size(), 6258 result.size() == 2); 6259 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[4])); 6260 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[1]), ROWS[4])); 6261 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[1])); 6262 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[1]), VALUES[2])); 6263 result = scanner.next(); 6264 assertTrue("Expected 1 key but received " + result.size(), 6265 result.size() == 1); 6266 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[3])); 6267 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[0])); 6268 } 6269 } 6270 } 6271 6272 /** 6273 * Tests reversed scan under multi regions 6274 */ 6275 @Test 6276 public void testReversedScanUnderMultiRegions() throws Exception { 6277 // Test Initialization. 6278 final TableName tableName = TableName.valueOf(name.getMethodName()); 6279 byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY; 6280 byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), 6281 Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), 6282 Bytes.toBytes("006"), 6283 Bytes.add(Bytes.toBytes("006"), Bytes.multiple(maxByteArray, 8)), 6284 Bytes.toBytes("007"), 6285 Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)), 6286 Bytes.toBytes("008"), Bytes.multiple(maxByteArray, 2) }; 6287 try (Table table = TEST_UTIL.createTable(tableName, FAMILY, splitRows)) { 6288 TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); 6289 6290 try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 6291 assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); 6292 } 6293 // Insert one row each region 6294 int insertNum = splitRows.length; 6295 for (int i = 0; i < insertNum; i++) { 6296 Put put = new Put(splitRows[i]); 6297 put.addColumn(FAMILY, QUALIFIER, VALUE); 6298 table.put(put); 6299 } 6300 6301 // scan forward 6302 try (ResultScanner scanner = table.getScanner(new Scan())) { 6303 int count = 0; 6304 for (Result r : scanner) { 6305 assertTrue(!r.isEmpty()); 6306 count++; 6307 } 6308 assertEquals(insertNum, count); 6309 } 6310 6311 // scan backward 6312 Scan scan = new Scan(); 6313 scan.setReversed(true); 6314 try (ResultScanner scanner = table.getScanner(scan)) { 6315 int count = 0; 6316 byte[] lastRow = null; 6317 for (Result r : scanner) { 6318 assertTrue(!r.isEmpty()); 6319 count++; 6320 byte[] thisRow = r.getRow(); 6321 if (lastRow != null) { 6322 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow) 6323 + ",this row=" + Bytes.toString(thisRow), 6324 Bytes.compareTo(thisRow, lastRow) < 0); 6325 } 6326 lastRow = thisRow; 6327 } 6328 assertEquals(insertNum, count); 6329 } 6330 } 6331 } 6332 6333 /** 6334 * Tests reversed scan under multi regions 6335 */ 6336 @Test 6337 public void testSmallReversedScanUnderMultiRegions() throws Exception { 6338 // Test Initialization. 6339 final TableName tableName = TableName.valueOf(name.getMethodName()); 6340 byte[][] splitRows = new byte[][]{ 6341 Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"), 6342 Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")}; 6343 try (Table table = TEST_UTIL.createTable(tableName, FAMILY, splitRows)) { 6344 TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); 6345 6346 try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) { 6347 assertEquals(splitRows.length + 1, l.getAllRegionLocations().size()); 6348 } 6349 for (byte[] splitRow : splitRows) { 6350 Put put = new Put(splitRow); 6351 put.addColumn(FAMILY, QUALIFIER, VALUE); 6352 table.put(put); 6353 6354 byte[] nextRow = Bytes.copy(splitRow); 6355 nextRow[nextRow.length - 1]++; 6356 6357 put = new Put(nextRow); 6358 put.addColumn(FAMILY, QUALIFIER, VALUE); 6359 table.put(put); 6360 } 6361 6362 // scan forward 6363 try (ResultScanner scanner = table.getScanner(new Scan())) { 6364 int count = 0; 6365 for (Result r : scanner) { 6366 assertTrue(!r.isEmpty()); 6367 count++; 6368 } 6369 assertEquals(12, count); 6370 } 6371 6372 reverseScanTest(table, false); 6373 reverseScanTest(table, true); 6374 } 6375 } 6376 6377 private void reverseScanTest(Table table, boolean small) throws IOException { 6378 // scan backward 6379 Scan scan = new Scan(); 6380 scan.setReversed(true); 6381 try (ResultScanner scanner = table.getScanner(scan)) { 6382 int count = 0; 6383 byte[] lastRow = null; 6384 for (Result r : scanner) { 6385 assertTrue(!r.isEmpty()); 6386 count++; 6387 byte[] thisRow = r.getRow(); 6388 if (lastRow != null) { 6389 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow) 6390 + ",this row=" + Bytes.toString(thisRow), 6391 Bytes.compareTo(thisRow, lastRow) < 0); 6392 } 6393 lastRow = thisRow; 6394 } 6395 assertEquals(12, count); 6396 } 6397 6398 scan = new Scan(); 6399 scan.setSmall(small); 6400 scan.setReversed(true); 6401 scan.setStartRow(Bytes.toBytes("002")); 6402 try (ResultScanner scanner = table.getScanner(scan)) { 6403 int count = 0; 6404 byte[] lastRow = null; 6405 for (Result r : scanner) { 6406 assertTrue(!r.isEmpty()); 6407 count++; 6408 byte[] thisRow = r.getRow(); 6409 if (lastRow != null) { 6410 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow) 6411 + ",this row=" + Bytes.toString(thisRow), 6412 Bytes.compareTo(thisRow, lastRow) < 0); 6413 } 6414 lastRow = thisRow; 6415 } 6416 assertEquals(3, count); // 000 001 002 6417 } 6418 6419 scan = new Scan(); 6420 scan.setSmall(small); 6421 scan.setReversed(true); 6422 scan.setStartRow(Bytes.toBytes("002")); 6423 scan.setStopRow(Bytes.toBytes("000")); 6424 try (ResultScanner scanner = table.getScanner(scan)) { 6425 int count = 0; 6426 byte[] lastRow = null; 6427 for (Result r : scanner) { 6428 assertTrue(!r.isEmpty()); 6429 count++; 6430 byte[] thisRow = r.getRow(); 6431 if (lastRow != null) { 6432 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow) 6433 + ",this row=" + Bytes.toString(thisRow), 6434 Bytes.compareTo(thisRow, lastRow) < 0); 6435 } 6436 lastRow = thisRow; 6437 } 6438 assertEquals(2, count); // 001 002 6439 } 6440 6441 scan = new Scan(); 6442 scan.setSmall(small); 6443 scan.setReversed(true); 6444 scan.setStartRow(Bytes.toBytes("001")); 6445 try (ResultScanner scanner = table.getScanner(scan)) { 6446 int count = 0; 6447 byte[] lastRow = null; 6448 for (Result r : scanner) { 6449 assertTrue(!r.isEmpty()); 6450 count++; 6451 byte[] thisRow = r.getRow(); 6452 if (lastRow != null) { 6453 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow) 6454 + ",this row=" + Bytes.toString(thisRow), 6455 Bytes.compareTo(thisRow, lastRow) < 0); 6456 } 6457 lastRow = thisRow; 6458 } 6459 assertEquals(2, count); // 000 001 6460 } 6461 6462 scan = new Scan(); 6463 scan.setSmall(small); 6464 scan.setReversed(true); 6465 scan.setStartRow(Bytes.toBytes("000")); 6466 try (ResultScanner scanner = table.getScanner(scan)) { 6467 int count = 0; 6468 byte[] lastRow = null; 6469 for (Result r : scanner) { 6470 assertTrue(!r.isEmpty()); 6471 count++; 6472 byte[] thisRow = r.getRow(); 6473 if (lastRow != null) { 6474 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow) 6475 + ",this row=" + Bytes.toString(thisRow), 6476 Bytes.compareTo(thisRow, lastRow) < 0); 6477 } 6478 lastRow = thisRow; 6479 } 6480 assertEquals(1, count); // 000 6481 } 6482 6483 scan = new Scan(); 6484 scan.setSmall(small); 6485 scan.setReversed(true); 6486 scan.setStartRow(Bytes.toBytes("006")); 6487 scan.setStopRow(Bytes.toBytes("002")); 6488 try (ResultScanner scanner = table.getScanner(scan)) { 6489 int count = 0; 6490 byte[] lastRow = null; 6491 for (Result r : scanner) { 6492 assertTrue(!r.isEmpty()); 6493 count++; 6494 byte[] thisRow = r.getRow(); 6495 if (lastRow != null) { 6496 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow) 6497 + ",this row=" + Bytes.toString(thisRow), 6498 Bytes.compareTo(thisRow, lastRow) < 0); 6499 } 6500 lastRow = thisRow; 6501 } 6502 assertEquals(4, count); // 003 004 005 006 6503 } 6504 } 6505 6506 private static Pair<byte[][], byte[][]> getStartEndKeys(List<RegionLocations> regions) { 6507 final byte[][] startKeyList = new byte[regions.size()][]; 6508 final byte[][] endKeyList = new byte[regions.size()][]; 6509 6510 for (int i = 0; i < regions.size(); i++) { 6511 RegionInfo region = regions.get(i).getRegionLocation().getRegion(); 6512 startKeyList[i] = region.getStartKey(); 6513 endKeyList[i] = region.getEndKey(); 6514 } 6515 6516 return new Pair<>(startKeyList, endKeyList); 6517 } 6518 6519 @Test 6520 public void testFilterAllRecords() throws IOException { 6521 Scan scan = new Scan(); 6522 scan.setBatch(1); 6523 scan.setCaching(1); 6524 // Filter out any records 6525 scan.setFilter(new FilterList(new FirstKeyOnlyFilter(), new InclusiveStopFilter(new byte[0]))); 6526 try (Table table = TEST_UTIL.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME)) { 6527 try (ResultScanner s = table.getScanner(scan)) { 6528 assertNull(s.next()); 6529 } 6530 } 6531 } 6532 6533 @Test 6534 public void testRegionCache() throws IOException { 6535 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); 6536 HColumnDescriptor fam = new HColumnDescriptor(FAMILY); 6537 htd.addFamily(fam); 6538 byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE; 6539 try (Admin admin = TEST_UTIL.getAdmin()) { 6540 admin.createTable(htd, KEYS); 6541 HRegionLocator locator = 6542 (HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName()); 6543 List<HRegionLocation> results = locator.getAllRegionLocations(); 6544 int number = ((ConnectionImplementation) admin.getConnection()) 6545 .getNumberOfCachedRegionLocations(htd.getTableName()); 6546 assertEquals(results.size(), number); 6547 ConnectionImplementation conn = ((ConnectionImplementation) admin.getConnection()); 6548 assertNotNull("Can't get cached location for row aaa", 6549 conn.getCachedLocation(htd.getTableName(), Bytes.toBytes("aaa"))); 6550 for (byte[] startKey : HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE) { 6551 assertNotNull("Can't get cached location for row " + 6552 Bytes.toString(startKey), (conn.getCachedLocation(htd.getTableName(), startKey))); 6553 } 6554 } 6555 } 6556 6557 @Test 6558 public void testCellSizeLimit() throws IOException { 6559 final TableName tableName = TableName.valueOf("testCellSizeLimit"); 6560 HTableDescriptor htd = new HTableDescriptor(tableName); 6561 htd.setConfiguration(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024)); // 10K 6562 HColumnDescriptor fam = new HColumnDescriptor(FAMILY); 6563 htd.addFamily(fam); 6564 try (Admin admin = TEST_UTIL.getAdmin()) { 6565 admin.createTable(htd); 6566 } 6567 // Will succeed 6568 try (Table t = TEST_UTIL.getConnection().getTable(tableName)) { 6569 t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, Bytes.toBytes(0L))); 6570 t.increment(new Increment(ROW).addColumn(FAMILY, QUALIFIER, 1L)); 6571 } 6572 // Will succeed 6573 try (Table t = TEST_UTIL.getConnection().getTable(tableName)) { 6574 t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, new byte[9*1024])); 6575 } 6576 // Will fail 6577 try (Table t = TEST_UTIL.getConnection().getTable(tableName)) { 6578 try { 6579 t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, new byte[10 * 1024])); 6580 fail("Oversize cell failed to trigger exception"); 6581 } catch (IOException e) { 6582 // expected 6583 } 6584 try { 6585 t.append(new Append(ROW).addColumn(FAMILY, QUALIFIER, new byte[2 * 1024])); 6586 fail("Oversize cell failed to trigger exception"); 6587 } catch (IOException e) { 6588 // expected 6589 } 6590 } 6591 } 6592 6593 @Test 6594 public void testDeleteSpecifiedVersionOfSpecifiedColumn() throws Exception { 6595 try (Admin admin = TEST_UTIL.getAdmin()) { 6596 final TableName tableName = TableName.valueOf(name.getMethodName()); 6597 6598 byte[][] VALUES = makeN(VALUE, 5); 6599 long[] ts = {1000, 2000, 3000, 4000, 5000}; 6600 6601 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 5)) { 6602 6603 Put put = new Put(ROW); 6604 // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER 6605 for (int t = 0; t < 4; t++) { 6606 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUES[t]); 6607 } 6608 ht.put(put); 6609 6610 Delete delete = new Delete(ROW); 6611 // Delete version 3000 of column FAMILY:QUALIFIER 6612 delete.addColumn(FAMILY, QUALIFIER, ts[2]); 6613 ht.delete(delete); 6614 6615 Get get = new Get(ROW); 6616 get.addColumn(FAMILY, QUALIFIER); 6617 get.setMaxVersions(Integer.MAX_VALUE); 6618 Result result = ht.get(get); 6619 // verify version 1000,2000,4000 remains for column FAMILY:QUALIFIER 6620 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[0], ts[1], ts[3]}, new byte[][]{ 6621 VALUES[0], VALUES[1], VALUES[3]}, 0, 2); 6622 6623 delete = new Delete(ROW); 6624 // Delete a version 5000 of column FAMILY:QUALIFIER which didn't exist 6625 delete.addColumn(FAMILY, QUALIFIER, ts[4]); 6626 ht.delete(delete); 6627 6628 get = new Get(ROW); 6629 get.addColumn(FAMILY, QUALIFIER); 6630 get.setMaxVersions(Integer.MAX_VALUE); 6631 result = ht.get(get); 6632 // verify version 1000,2000,4000 remains for column FAMILY:QUALIFIER 6633 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[0], ts[1], ts[3]}, new byte[][]{ 6634 VALUES[0], VALUES[1], VALUES[3]}, 0, 2); 6635 } 6636 } 6637 } 6638 6639 @Test 6640 public void testDeleteLatestVersionOfSpecifiedColumn() throws Exception { 6641 try (Admin admin = TEST_UTIL.getAdmin()) { 6642 final TableName tableName = TableName.valueOf(name.getMethodName()); 6643 6644 byte[][] VALUES = makeN(VALUE, 5); 6645 long[] ts = {1000, 2000, 3000, 4000, 5000}; 6646 6647 try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 5)) { 6648 6649 Put put = new Put(ROW); 6650 // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER 6651 for (int t = 0; t < 4; t++) { 6652 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUES[t]); 6653 } 6654 ht.put(put); 6655 6656 Delete delete = new Delete(ROW); 6657 // Delete latest version of column FAMILY:QUALIFIER 6658 delete.addColumn(FAMILY, QUALIFIER); 6659 ht.delete(delete); 6660 6661 Get get = new Get(ROW); 6662 get.addColumn(FAMILY, QUALIFIER); 6663 get.setMaxVersions(Integer.MAX_VALUE); 6664 Result result = ht.get(get); 6665 // verify version 1000,2000,3000 remains for column FAMILY:QUALIFIER 6666 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[0], ts[1], ts[2]}, new byte[][]{ 6667 VALUES[0], VALUES[1], VALUES[2]}, 0, 2); 6668 6669 delete = new Delete(ROW); 6670 // Delete two latest version of column FAMILY:QUALIFIER 6671 delete.addColumn(FAMILY, QUALIFIER); 6672 delete.addColumn(FAMILY, QUALIFIER); 6673 ht.delete(delete); 6674 6675 get = new Get(ROW); 6676 get.addColumn(FAMILY, QUALIFIER); 6677 get.setMaxVersions(Integer.MAX_VALUE); 6678 result = ht.get(get); 6679 // verify version 1000 remains for column FAMILY:QUALIFIER 6680 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[0]}, new byte[][]{VALUES[0]}, 6681 0, 0); 6682 6683 put = new Put(ROW); 6684 // Put a version 5000 of column FAMILY:QUALIFIER 6685 put.addColumn(FAMILY, QUALIFIER, ts[4], VALUES[4]); 6686 ht.put(put); 6687 6688 get = new Get(ROW); 6689 get.addColumn(FAMILY, QUALIFIER); 6690 get.setMaxVersions(Integer.MAX_VALUE); 6691 result = ht.get(get); 6692 // verify version 1000,5000 remains for column FAMILY:QUALIFIER 6693 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[0], ts[4]}, new byte[][]{ 6694 VALUES[0], VALUES[4]}, 0, 1); 6695 } 6696 } 6697 } 6698 6699 /** 6700 * Test for HBASE-17125 6701 */ 6702 @Test 6703 public void testReadWithFilter() throws Exception { 6704 try (Admin admin = TEST_UTIL.getAdmin()) { 6705 final TableName tableName = TableName.valueOf(name.getMethodName()); 6706 try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 3)) { 6707 6708 byte[] VALUEA = Bytes.toBytes("value-a"); 6709 byte[] VALUEB = Bytes.toBytes("value-b"); 6710 long[] ts = {1000, 2000, 3000, 4000}; 6711 6712 Put put = new Put(ROW); 6713 // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER 6714 for (int t = 0; t <= 3; t++) { 6715 if (t <= 1) { 6716 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUEA); 6717 } else { 6718 put.addColumn(FAMILY, QUALIFIER, ts[t], VALUEB); 6719 } 6720 } 6721 table.put(put); 6722 6723 Scan scan = 6724 new Scan().setFilter(new ValueFilter(CompareOperator.EQUAL, 6725 new SubstringComparator("value-a"))) 6726 .setMaxVersions(3); 6727 ResultScanner scanner = table.getScanner(scan); 6728 Result result = scanner.next(); 6729 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 6730 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[1]}, new byte[][]{VALUEA}, 0, 6731 0); 6732 6733 Get get = 6734 new Get(ROW) 6735 .setFilter(new ValueFilter(CompareOperator.EQUAL, 6736 new SubstringComparator("value-a"))) 6737 .setMaxVersions(3); 6738 result = table.get(get); 6739 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 6740 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[1]}, new byte[][]{VALUEA}, 0, 6741 0); 6742 6743 // Test with max versions 1, it should still read ts[1] 6744 scan = 6745 new Scan().setFilter(new ValueFilter(CompareOperator.EQUAL, 6746 new SubstringComparator("value-a"))) 6747 .setMaxVersions(1); 6748 scanner = table.getScanner(scan); 6749 result = scanner.next(); 6750 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 6751 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[1]}, new byte[][]{VALUEA}, 0, 6752 0); 6753 6754 // Test with max versions 1, it should still read ts[1] 6755 get = 6756 new Get(ROW) 6757 .setFilter(new ValueFilter(CompareOperator.EQUAL, 6758 new SubstringComparator("value-a"))) 6759 .setMaxVersions(1); 6760 result = table.get(get); 6761 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 6762 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[1]}, new byte[][]{VALUEA}, 0, 6763 0); 6764 6765 // Test with max versions 5, it should still read ts[1] 6766 scan = 6767 new Scan().setFilter(new ValueFilter(CompareOperator.EQUAL, 6768 new SubstringComparator("value-a"))) 6769 .setMaxVersions(5); 6770 scanner = table.getScanner(scan); 6771 result = scanner.next(); 6772 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 6773 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[1]}, new byte[][]{VALUEA}, 0, 6774 0); 6775 6776 // Test with max versions 5, it should still read ts[1] 6777 get = 6778 new Get(ROW) 6779 .setFilter(new ValueFilter(CompareOperator.EQUAL, 6780 new SubstringComparator("value-a"))) 6781 .setMaxVersions(5); 6782 result = table.get(get); 6783 // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3 6784 assertNResult(result, ROW, FAMILY, QUALIFIER, new long[]{ts[1]}, new byte[][]{VALUEA}, 0, 6785 0); 6786 } 6787 } 6788 } 6789 6790 @Test 6791 public void testCellUtilTypeMethods() throws IOException { 6792 final TableName tableName = TableName.valueOf(name.getMethodName()); 6793 try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { 6794 6795 final byte[] row = Bytes.toBytes("p"); 6796 Put p = new Put(row); 6797 p.addColumn(FAMILY, QUALIFIER, VALUE); 6798 table.put(p); 6799 6800 try (ResultScanner scanner = table.getScanner(new Scan())) { 6801 Result result = scanner.next(); 6802 assertNotNull(result); 6803 CellScanner cs = result.cellScanner(); 6804 assertTrue(cs.advance()); 6805 Cell c = cs.current(); 6806 assertTrue(CellUtil.isPut(c)); 6807 assertFalse(CellUtil.isDelete(c)); 6808 assertFalse(cs.advance()); 6809 assertNull(scanner.next()); 6810 } 6811 6812 Delete d = new Delete(row); 6813 d.addColumn(FAMILY, QUALIFIER); 6814 table.delete(d); 6815 6816 Scan scan = new Scan(); 6817 scan.setRaw(true); 6818 try (ResultScanner scanner = table.getScanner(scan)) { 6819 Result result = scanner.next(); 6820 assertNotNull(result); 6821 CellScanner cs = result.cellScanner(); 6822 assertTrue(cs.advance()); 6823 6824 // First cell should be the delete (masking the Put) 6825 Cell c = cs.current(); 6826 assertTrue("Cell should be a Delete: " + c, CellUtil.isDelete(c)); 6827 assertFalse("Cell should not be a Put: " + c, CellUtil.isPut(c)); 6828 6829 // Second cell should be the original Put 6830 assertTrue(cs.advance()); 6831 c = cs.current(); 6832 assertFalse("Cell should not be a Delete: " + c, CellUtil.isDelete(c)); 6833 assertTrue("Cell should be a Put: " + c, CellUtil.isPut(c)); 6834 6835 // No more cells in this row 6836 assertFalse(cs.advance()); 6837 6838 // No more results in this scan 6839 assertNull(scanner.next()); 6840 } 6841 } 6842 } 6843 6844 @Test(expected = DoNotRetryIOException.class) 6845 public void testCreateTableWithZeroRegionReplicas() throws Exception { 6846 TableName tableName = TableName.valueOf(name.getMethodName()); 6847 TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) 6848 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))) 6849 .setRegionReplication(0) 6850 .build(); 6851 6852 TEST_UTIL.getAdmin().createTable(desc); 6853 } 6854 6855 @Test(expected = DoNotRetryIOException.class) 6856 public void testModifyTableWithZeroRegionReplicas() throws Exception { 6857 TableName tableName = TableName.valueOf(name.getMethodName()); 6858 TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) 6859 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))) 6860 .build(); 6861 6862 TEST_UTIL.getAdmin().createTable(desc); 6863 TableDescriptor newDesc = TableDescriptorBuilder.newBuilder(desc) 6864 .setRegionReplication(0) 6865 .build(); 6866 6867 TEST_UTIL.getAdmin().modifyTable(newDesc); 6868 } 6869 6870 @Test(timeout = 60000) 6871 public void testModifyTableWithMemstoreData() throws Exception { 6872 TableName tableName = TableName.valueOf(name.getMethodName()); 6873 createTableAndValidateTableSchemaModification(tableName, true); 6874 } 6875 6876 @Test(timeout = 60000) 6877 public void testDeleteCFWithMemstoreData() throws Exception { 6878 TableName tableName = TableName.valueOf(name.getMethodName()); 6879 createTableAndValidateTableSchemaModification(tableName, false); 6880 } 6881 6882 /** 6883 * Create table and validate online schema modification 6884 * @param tableName Table name 6885 * @param modifyTable Modify table if true otherwise delete column family 6886 * @throws IOException in case of failures 6887 */ 6888 private void createTableAndValidateTableSchemaModification(TableName tableName, 6889 boolean modifyTable) throws Exception { 6890 Admin admin = TEST_UTIL.getAdmin(); 6891 // Create table with two Cfs 6892 byte[] cf1 = Bytes.toBytes("cf1"); 6893 byte[] cf2 = Bytes.toBytes("cf2"); 6894 TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(tableName) 6895 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf1)) 6896 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf2)).build(); 6897 admin.createTable(tableDesc); 6898 6899 Table t = TEST_UTIL.getConnection().getTable(tableName); 6900 // Insert few records and flush the table 6901 t.put(new Put(ROW).addColumn(cf1, QUALIFIER, Bytes.toBytes("val1"))); 6902 t.put(new Put(ROW).addColumn(cf2, QUALIFIER, Bytes.toBytes("val2"))); 6903 admin.flush(tableName); 6904 Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), tableName); 6905 List<Path> regionDirs = FSUtils.getRegionDirs(TEST_UTIL.getTestFileSystem(), tableDir); 6906 assertTrue(regionDirs.size() == 1); 6907 List<Path> familyDirs = FSUtils.getFamilyDirs(TEST_UTIL.getTestFileSystem(), regionDirs.get(0)); 6908 assertTrue(familyDirs.size() == 2); 6909 6910 // Insert record but dont flush the table 6911 t.put(new Put(ROW).addColumn(cf1, QUALIFIER, Bytes.toBytes("val2"))); 6912 t.put(new Put(ROW).addColumn(cf2, QUALIFIER, Bytes.toBytes("val2"))); 6913 6914 if (modifyTable) { 6915 tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).removeColumnFamily(cf2).build(); 6916 admin.modifyTable(tableDesc); 6917 } else { 6918 admin.deleteColumnFamily(tableName, cf2); 6919 } 6920 // After table modification or delete family there should be only one CF in FS 6921 familyDirs = FSUtils.getFamilyDirs(TEST_UTIL.getTestFileSystem(), regionDirs.get(0)); 6922 assertTrue("CF dir count should be 1, but was " + familyDirs.size(), familyDirs.size() == 1); 6923 } 6924}