1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import static org.junit.Assert.assertArrayEquals;
22 import static org.junit.Assert.assertEquals;
23 import static org.junit.Assert.assertFalse;
24 import static org.junit.Assert.assertNotNull;
25 import static org.junit.Assert.assertNull;
26 import static org.junit.Assert.assertSame;
27 import static org.junit.Assert.assertTrue;
28 import static org.junit.Assert.fail;
29
30 import java.io.IOException;
31 import java.lang.reflect.Method;
32 import java.util.ArrayList;
33 import java.util.Arrays;
34 import java.util.HashSet;
35 import java.util.Iterator;
36 import java.util.List;
37 import java.util.Map;
38 import java.util.NavigableMap;
39 import java.util.UUID;
40 import java.util.concurrent.Callable;
41 import java.util.concurrent.ExecutorService;
42 import java.util.concurrent.Executors;
43 import java.util.concurrent.atomic.AtomicReference;
44
45 import org.apache.commons.lang.ArrayUtils;
46 import org.apache.commons.logging.Log;
47 import org.apache.commons.logging.LogFactory;
48 import org.apache.commons.logging.impl.Log4JLogger;
49 import org.apache.hadoop.conf.Configuration;
50 import org.apache.hadoop.hbase.Abortable;
51 import org.apache.hadoop.hbase.Cell;
52 import org.apache.hadoop.hbase.CellUtil;
53 import org.apache.hadoop.hbase.DoNotRetryIOException;
54 import org.apache.hadoop.hbase.HBaseTestingUtility;
55 import org.apache.hadoop.hbase.HColumnDescriptor;
56 import org.apache.hadoop.hbase.HConstants;
57 import org.apache.hadoop.hbase.HRegionInfo;
58 import org.apache.hadoop.hbase.HRegionLocation;
59 import org.apache.hadoop.hbase.HTableDescriptor;
60 import org.apache.hadoop.hbase.KeyValue;
61 import org.apache.hadoop.hbase.LargeTests;
62 import org.apache.hadoop.hbase.MiniHBaseCluster;
63 import org.apache.hadoop.hbase.ServerName;
64 import org.apache.hadoop.hbase.TableName;
65 import org.apache.hadoop.hbase.Waiter;
66 import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
67 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
68 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
69 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
70 import org.apache.hadoop.hbase.filter.BinaryComparator;
71 import org.apache.hadoop.hbase.filter.CompareFilter;
72 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
73 import org.apache.hadoop.hbase.filter.Filter;
74 import org.apache.hadoop.hbase.filter.FilterList;
75 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
76 import org.apache.hadoop.hbase.filter.PrefixFilter;
77 import org.apache.hadoop.hbase.filter.QualifierFilter;
78 import org.apache.hadoop.hbase.filter.RegexStringComparator;
79 import org.apache.hadoop.hbase.filter.RowFilter;
80 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
81 import org.apache.hadoop.hbase.filter.WhileMatchFilter;
82 import org.apache.hadoop.hbase.io.hfile.BlockCache;
83 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
84 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
85 import org.apache.hadoop.hbase.ipc.RpcClient;
86 import org.apache.hadoop.hbase.ipc.RpcServer;
87 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
88 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
89 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
90 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
91 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
92 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
93 import org.apache.hadoop.hbase.regionserver.HRegion;
94 import org.apache.hadoop.hbase.regionserver.HRegionServer;
95 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
96 import org.apache.hadoop.hbase.regionserver.Store;
97 import org.apache.hadoop.hbase.util.Bytes;
98 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
99 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
100 import org.apache.log4j.Level;
101 import org.junit.After;
102 import org.junit.AfterClass;
103 import org.junit.Before;
104 import org.junit.BeforeClass;
105 import org.junit.Ignore;
106 import org.junit.Test;
107 import org.junit.experimental.categories.Category;
108
109
110
111
112
113
114 @Category(LargeTests.class)
115 @SuppressWarnings ("deprecation")
116 public class TestFromClientSide {
117 final Log LOG = LogFactory.getLog(getClass());
118 protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
119 private static byte [] ROW = Bytes.toBytes("testRow");
120 private static byte [] FAMILY = Bytes.toBytes("testFamily");
121 private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
122 private static byte [] VALUE = Bytes.toBytes("testValue");
123 protected static int SLAVES = 3;
124
125
126
127
128 @BeforeClass
129 public static void setUpBeforeClass() throws Exception {
130 ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
131 ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
132 ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
133 Configuration conf = TEST_UTIL.getConfiguration();
134 conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
135 MultiRowMutationEndpoint.class.getName());
136
137 TEST_UTIL.startMiniCluster(SLAVES);
138 }
139
140
141
142
143 @AfterClass
144 public static void tearDownAfterClass() throws Exception {
145 TEST_UTIL.shutdownMiniCluster();
146 }
147
148
149
150
151 @Before
152 public void setUp() throws Exception {
153
154 }
155
156
157
158
159 @After
160 public void tearDown() throws Exception {
161
162 }
163
164
165
166
167 @Test
168 public void testKeepDeletedCells() throws Exception {
169 final byte[] TABLENAME = Bytes.toBytes("testKeepDeletesCells");
170 final byte[] FAMILY = Bytes.toBytes("family");
171 final byte[] C0 = Bytes.toBytes("c0");
172
173 final byte[] T1 = Bytes.toBytes("T1");
174 final byte[] T2 = Bytes.toBytes("T2");
175 final byte[] T3 = Bytes.toBytes("T3");
176 HColumnDescriptor hcd = new HColumnDescriptor(FAMILY)
177 .setKeepDeletedCells(true).setMaxVersions(3);
178
179 HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TABLENAME));
180 desc.addFamily(hcd);
181 TEST_UTIL.getHBaseAdmin().createTable(desc);
182 Configuration c = TEST_UTIL.getConfiguration();
183 HTable h = new HTable(c, TABLENAME);
184
185 long ts = System.currentTimeMillis();
186 Put p = new Put(T1, ts);
187 p.add(FAMILY, C0, T1);
188 h.put(p);
189 p = new Put(T1, ts+2);
190 p.add(FAMILY, C0, T2);
191 h.put(p);
192 p = new Put(T1, ts+4);
193 p.add(FAMILY, C0, T3);
194 h.put(p);
195
196 Delete d = new Delete(T1, ts+3);
197 h.delete(d);
198
199 d = new Delete(T1, ts+3);
200 d.deleteColumns(FAMILY, C0, ts+3);
201 h.delete(d);
202
203 Get g = new Get(T1);
204
205 g.setTimeRange(0, ts+3);
206 Result r = h.get(g);
207 assertArrayEquals(T2, r.getValue(FAMILY, C0));
208
209 Scan s = new Scan(T1);
210 s.setTimeRange(0, ts+3);
211 s.setMaxVersions();
212 ResultScanner scanner = h.getScanner(s);
213 Cell[] kvs = scanner.next().rawCells();
214 assertArrayEquals(T2, CellUtil.cloneValue(kvs[0]));
215 assertArrayEquals(T1, CellUtil.cloneValue(kvs[1]));
216 scanner.close();
217
218 s = new Scan(T1);
219 s.setRaw(true);
220 s.setMaxVersions();
221 scanner = h.getScanner(s);
222 kvs = scanner.next().rawCells();
223 assertTrue(CellUtil.isDeleteFamily(kvs[0]));
224 assertArrayEquals(T3, CellUtil.cloneValue(kvs[1]));
225 assertTrue(CellUtil.isDelete(kvs[2]));
226 assertArrayEquals(T2, CellUtil.cloneValue(kvs[3]));
227 assertArrayEquals(T1, CellUtil.cloneValue(kvs[4]));
228 scanner.close();
229 h.close();
230 }
231
232
233
234
235 @Test
236 public void testPurgeFutureDeletes() throws Exception {
237 final byte[] TABLENAME = Bytes.toBytes("testPurgeFutureDeletes");
238 final byte[] ROW = Bytes.toBytes("row");
239 final byte[] FAMILY = Bytes.toBytes("family");
240 final byte[] COLUMN = Bytes.toBytes("column");
241 final byte[] VALUE = Bytes.toBytes("value");
242
243 HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY);
244
245
246 long ts = System.currentTimeMillis() * 2;
247 Put put = new Put(ROW, ts);
248 put.add(FAMILY, COLUMN, VALUE);
249 table.put(put);
250
251 Get get = new Get(ROW);
252 Result result = table.get(get);
253 assertArrayEquals(VALUE, result.getValue(FAMILY, COLUMN));
254
255 Delete del = new Delete(ROW);
256 del.deleteColumn(FAMILY, COLUMN, ts);
257 table.delete(del);
258
259 get = new Get(ROW);
260 result = table.get(get);
261 assertNull(result.getValue(FAMILY, COLUMN));
262
263
264 TEST_UTIL.getHBaseAdmin().flush(TABLENAME);
265 TEST_UTIL.getHBaseAdmin().majorCompact(TABLENAME);
266
267
268 TEST_UTIL.waitFor(6000, new Waiter.Predicate<IOException>() {
269 @Override
270 public boolean evaluate() throws IOException {
271 try {
272 return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) ==
273 AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
274 } catch (InterruptedException e) {
275 throw new IOException(e);
276 }
277 }
278 });
279
280 put = new Put(ROW, ts);
281 put.add(FAMILY, COLUMN, VALUE);
282 table.put(put);
283
284 get = new Get(ROW);
285 result = table.get(get);
286 assertArrayEquals(VALUE, result.getValue(FAMILY, COLUMN));
287
288 table.close();
289 }
290
291 @Test
292 public void testSharedZooKeeper() throws Exception {
293 Configuration newConfig = new Configuration(TEST_UTIL.getConfiguration());
294 newConfig.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "12345");
295
296
297 ZooKeeperWatcher z0 = new ZooKeeperWatcher(
298 newConfig, "hconnection", new Abortable() {
299 @Override public void abort(String why, Throwable e) {}
300 @Override public boolean isAborted() {return false;}
301 });
302 z0.getRecoverableZooKeeper().getZooKeeper().exists("/oldZooKeeperWatcher", false);
303 z0.close();
304
305
306 HConnectionManager.HConnectionImplementation connection1 =
307 (HConnectionManager.HConnectionImplementation)
308 HConnectionManager.getConnection(newConfig);
309
310 ZooKeeperKeepAliveConnection z1 = connection1.getKeepAliveZooKeeperWatcher();
311 z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1", false);
312
313 z1.close();
314
315
316
317 z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1afterclose", false);
318
319
320 ZooKeeperKeepAliveConnection z2 = connection1.getKeepAliveZooKeeperWatcher();
321 assertTrue(
322 "ZooKeeperKeepAliveConnection equals on same connection", z1 == z2);
323
324
325
326 Configuration newConfig2 = new Configuration(TEST_UTIL.getConfiguration());
327 newConfig2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "6789");
328 HConnectionManager.HConnectionImplementation connection2 =
329 (HConnectionManager.HConnectionImplementation)
330 HConnectionManager.getConnection(newConfig2);
331
332 assertTrue("connections should be different ", connection1 != connection2);
333
334 ZooKeeperKeepAliveConnection z3 = connection2.getKeepAliveZooKeeperWatcher();
335 assertTrue(
336 "ZooKeeperKeepAliveConnection should be different" +
337 " on different connections", z1 != z3);
338
339
340 Method m = HConnectionManager.HConnectionImplementation.class.
341 getDeclaredMethod("closeZooKeeperWatcher");
342 m.setAccessible(true);
343 m.invoke(connection2);
344
345 ZooKeeperKeepAliveConnection z4 = connection2.getKeepAliveZooKeeperWatcher();
346 assertTrue(
347 "ZooKeeperKeepAliveConnection should be recreated" +
348 " when previous connections was closed"
349 , z3 != z4);
350
351
352 z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false);
353 z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
354
355
356 HConnectionManager.deleteConnection(newConfig);
357 try {
358 z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false);
359 assertTrue("We should not have a valid connection for z2", false);
360 } catch (Exception e){
361 }
362
363 z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
364
365
366
367 HConnectionManager.deleteConnection(newConfig2);
368 try {
369 z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
370 assertTrue("We should not have a valid connection for z4", false);
371 } catch (Exception e){
372 }
373 }
374
375
376
377
378 @Test
379 public void testRegionCachePreWarm() throws Exception {
380 LOG.info("Starting testRegionCachePreWarm");
381 final TableName TABLENAME =
382 TableName.valueOf("testCachePrewarm");
383 Configuration conf = TEST_UTIL.getConfiguration();
384
385
386
387 TEST_UTIL.createTable(TABLENAME, FAMILY);
388
389
390 HTable.setRegionCachePrefetch(conf, TABLENAME, false);
391 assertFalse("The table is disabled for region cache prefetch",
392 HTable.getRegionCachePrefetch(conf, TABLENAME));
393
394 HTable table = new HTable(conf, TABLENAME);
395
396
397 TEST_UTIL.createMultiRegions(table, FAMILY);
398
399
400 TEST_UTIL.countRows(table);
401 table.getConnection().clearRegionCache();
402 assertEquals("Clearing cache should have 0 cached ", 0,
403 HConnectionManager.getCachedRegionCount(conf, TABLENAME));
404
405
406 Get g = new Get(Bytes.toBytes("aaa"));
407 table.get(g);
408
409
410 assertEquals("Number of cached region is incorrect ", 1,
411 HConnectionManager.getCachedRegionCount(conf, TABLENAME));
412
413
414 HTable.setRegionCachePrefetch(conf, TABLENAME, true);
415 assertTrue("The table is enabled for region cache prefetch",
416 HTable.getRegionCachePrefetch(conf, TABLENAME));
417
418 HTable.setRegionCachePrefetch(conf, TABLENAME, false);
419 assertFalse("The table is disabled for region cache prefetch",
420 HTable.getRegionCachePrefetch(conf, TABLENAME));
421
422 HTable.setRegionCachePrefetch(conf, TABLENAME, true);
423 assertTrue("The table is enabled for region cache prefetch",
424 HTable.getRegionCachePrefetch(conf, TABLENAME));
425
426 table.getConnection().clearRegionCache();
427
428 assertEquals("Number of cached region is incorrect ", 0,
429 HConnectionManager.getCachedRegionCount(conf, TABLENAME));
430
431
432 Get g2 = new Get(Bytes.toBytes("bbb"));
433 table.get(g2);
434
435
436 int prefetchRegionNumber = conf.getInt("hbase.client.prefetch.limit", 10);
437
438
439 LOG.info("Testing how many regions cached");
440 assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
441 HConnectionManager.getCachedRegionCount(conf, TABLENAME));
442
443 table.getConnection().clearRegionCache();
444
445 Get g3 = new Get(Bytes.toBytes("abc"));
446 table.get(g3);
447 assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
448 HConnectionManager.getCachedRegionCount(conf, TABLENAME));
449
450 LOG.info("Finishing testRegionCachePreWarm");
451 }
452
453 @Test
454 public void testPrefetchDisableToggle() throws Exception {
455 final TableName TABLENAME = TableName.valueOf("testPrefetchDisableToggle");
456 Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
457 conf.setBoolean(HConstants.HBASE_CLIENT_PREFETCH, false);
458
459
460 HConnection connection = HConnectionManager.createConnection(conf);
461 assertFalse("The table is not disabled for region cache prefetch",
462 ((HConnectionImplementation)connection).getRegionCachePrefetch(TABLENAME));
463 }
464
465
466
467
468
469 @Test
470 public void testGetConfiguration() throws Exception {
471 byte[] TABLE = Bytes.toBytes("testGetConfiguration");
472 byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
473 Configuration conf = TEST_UTIL.getConfiguration();
474 HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
475 assertSame(conf, table.getConfiguration());
476 }
477
478
479
480
481
482
483
484 @Test
485 public void testWeirdCacheBehaviour() throws Exception {
486 byte [] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
487 byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
488 Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
489 Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
490 HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
491 String value = "this is the value";
492 String value2 = "this is some other value";
493 String keyPrefix1 = UUID.randomUUID().toString();
494 String keyPrefix2 = UUID.randomUUID().toString();
495 String keyPrefix3 = UUID.randomUUID().toString();
496 putRows(ht, 3, value, keyPrefix1);
497 putRows(ht, 3, value, keyPrefix2);
498 putRows(ht, 3, value, keyPrefix3);
499 ht.flushCommits();
500 putRows(ht, 3, value2, keyPrefix1);
501 putRows(ht, 3, value2, keyPrefix2);
502 putRows(ht, 3, value2, keyPrefix3);
503 HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
504 System.out.println("Checking values for key: " + keyPrefix1);
505 assertEquals("Got back incorrect number of rows from scan", 3,
506 getNumberOfRows(keyPrefix1, value2, table));
507 System.out.println("Checking values for key: " + keyPrefix2);
508 assertEquals("Got back incorrect number of rows from scan", 3,
509 getNumberOfRows(keyPrefix2, value2, table));
510 System.out.println("Checking values for key: " + keyPrefix3);
511 assertEquals("Got back incorrect number of rows from scan", 3,
512 getNumberOfRows(keyPrefix3, value2, table));
513 deleteColumns(ht, value2, keyPrefix1);
514 deleteColumns(ht, value2, keyPrefix2);
515 deleteColumns(ht, value2, keyPrefix3);
516 System.out.println("Starting important checks.....");
517 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
518 0, getNumberOfRows(keyPrefix1, value2, table));
519 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
520 0, getNumberOfRows(keyPrefix2, value2, table));
521 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
522 0, getNumberOfRows(keyPrefix3, value2, table));
523 ht.setScannerCaching(0);
524 assertEquals("Got back incorrect number of rows from scan", 0,
525 getNumberOfRows(keyPrefix1, value2, table)); ht.setScannerCaching(100);
526 assertEquals("Got back incorrect number of rows from scan", 0,
527 getNumberOfRows(keyPrefix2, value2, table));
528 }
529
530 private void deleteColumns(HTable ht, String value, String keyPrefix)
531 throws IOException {
532 ResultScanner scanner = buildScanner(keyPrefix, value, ht);
533 Iterator<Result> it = scanner.iterator();
534 int count = 0;
535 while (it.hasNext()) {
536 Result result = it.next();
537 Delete delete = new Delete(result.getRow());
538 delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
539 ht.delete(delete);
540 count++;
541 }
542 assertEquals("Did not perform correct number of deletes", 3, count);
543 }
544
545 private int getNumberOfRows(String keyPrefix, String value, HTable ht)
546 throws Exception {
547 ResultScanner resultScanner = buildScanner(keyPrefix, value, ht);
548 Iterator<Result> scanner = resultScanner.iterator();
549 int numberOfResults = 0;
550 while (scanner.hasNext()) {
551 Result result = scanner.next();
552 System.out.println("Got back key: " + Bytes.toString(result.getRow()));
553 for (Cell kv : result.rawCells()) {
554 System.out.println("kv=" + kv.toString() + ", "
555 + Bytes.toString(CellUtil.cloneValue(kv)));
556 }
557 numberOfResults++;
558 }
559 return numberOfResults;
560 }
561
562 private ResultScanner buildScanner(String keyPrefix, String value, HTable ht)
563 throws IOException {
564
565 FilterList allFilters = new FilterList(
566 allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
567 SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes
568 .toBytes("trans-tags"), Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes
569 .toBytes(value));
570 filter.setFilterIfMissing(true);
571 allFilters.addFilter(filter);
572
573
574
575
576
577 Scan scan = new Scan();
578 scan.addFamily(Bytes.toBytes("trans-blob"));
579 scan.addFamily(Bytes.toBytes("trans-type"));
580 scan.addFamily(Bytes.toBytes("trans-date"));
581 scan.addFamily(Bytes.toBytes("trans-tags"));
582 scan.addFamily(Bytes.toBytes("trans-group"));
583 scan.setFilter(allFilters);
584
585 return ht.getScanner(scan);
586 }
587
588 private void putRows(HTable ht, int numRows, String value, String key)
589 throws IOException {
590 for (int i = 0; i < numRows; i++) {
591 String row = key + "_" + UUID.randomUUID().toString();
592 System.out.println(String.format("Saving row: %s, with value %s", row,
593 value));
594 Put put = new Put(Bytes.toBytes(row));
595 put.setDurability(Durability.SKIP_WAL);
596 put.add(Bytes.toBytes("trans-blob"), null, Bytes
597 .toBytes("value for blob"));
598 put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
599 put.add(Bytes.toBytes("trans-date"), null, Bytes
600 .toBytes("20090921010101999"));
601 put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"), Bytes
602 .toBytes(value));
603 put.add(Bytes.toBytes("trans-group"), null, Bytes
604 .toBytes("adhocTransactionGroupId"));
605 ht.put(put);
606 }
607 }
608
609
610
611
612
613
614
615
616 @Test
617 public void testFilterAcrossMultipleRegions()
618 throws IOException, InterruptedException {
619 byte [] name = Bytes.toBytes("testFilterAcrossMutlipleRegions");
620 HTable t = TEST_UTIL.createTable(name, FAMILY);
621 int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
622 assertRowCount(t, rowCount);
623
624 Map<HRegionInfo, ServerName> regions = splitTable(t);
625 assertRowCount(t, rowCount);
626
627 byte [] endKey = regions.keySet().iterator().next().getEndKey();
628
629
630 int endKeyCount = countRows(t, createScanWithRowFilter(endKey));
631 assertTrue(endKeyCount < rowCount);
632
633
634
635
636
637
638
639
640
641 byte [] key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] + 1)};
642 int plusOneCount = countRows(t, createScanWithRowFilter(key));
643 assertEquals(endKeyCount + 1, plusOneCount);
644 key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] + 2)};
645 int plusTwoCount = countRows(t, createScanWithRowFilter(key));
646 assertEquals(endKeyCount + 2, plusTwoCount);
647
648
649 key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] - 1)};
650 int minusOneCount = countRows(t, createScanWithRowFilter(key));
651 assertEquals(endKeyCount - 1, minusOneCount);
652
653
654
655 key = new byte [] {'a', 'a', 'a'};
656 int countBBB = countRows(t,
657 createScanWithRowFilter(key, null, CompareFilter.CompareOp.EQUAL));
658 assertEquals(1, countBBB);
659
660 int countGreater = countRows(t, createScanWithRowFilter(endKey, null,
661 CompareFilter.CompareOp.GREATER_OR_EQUAL));
662
663 assertEquals(0, countGreater);
664 countGreater = countRows(t, createScanWithRowFilter(endKey, endKey,
665 CompareFilter.CompareOp.GREATER_OR_EQUAL));
666 assertEquals(rowCount - endKeyCount, countGreater);
667 }
668
669
670
671
672
673 private Scan createScanWithRowFilter(final byte [] key) {
674 return createScanWithRowFilter(key, null, CompareFilter.CompareOp.LESS);
675 }
676
677
678
679
680
681
682
683 private Scan createScanWithRowFilter(final byte [] key,
684 final byte [] startRow, CompareFilter.CompareOp op) {
685
686 assertTrue(key != null && key.length > 0 &&
687 Bytes.BYTES_COMPARATOR.compare(key, new byte [] {'a', 'a', 'a'}) >= 0);
688 LOG.info("Key=" + Bytes.toString(key));
689 Scan s = startRow == null? new Scan(): new Scan(startRow);
690 Filter f = new RowFilter(op, new BinaryComparator(key));
691 f = new WhileMatchFilter(f);
692 s.setFilter(f);
693 return s;
694 }
695
696
697
698
699
700
701
702 private int countRows(final HTable t, final Scan s)
703 throws IOException {
704
705 ResultScanner scanner = t.getScanner(s);
706 int count = 0;
707 for (Result result: scanner) {
708 count++;
709 assertTrue(result.size() > 0);
710
711 }
712 return count;
713 }
714
715 private void assertRowCount(final HTable t, final int expected)
716 throws IOException {
717 assertEquals(expected, countRows(t, new Scan()));
718 }
719
720
721
722
723
724
725
726 private Map<HRegionInfo, ServerName> splitTable(final HTable t)
727 throws IOException, InterruptedException {
728
729 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
730 admin.split(t.getTableName());
731 admin.close();
732 Map<HRegionInfo, ServerName> regions = waitOnSplit(t);
733 assertTrue(regions.size() > 1);
734 return regions;
735 }
736
737
738
739
740
741
742
743 private Map<HRegionInfo, ServerName> waitOnSplit(final HTable t)
744 throws IOException {
745 Map<HRegionInfo, ServerName> regions = t.getRegionLocations();
746 int originalCount = regions.size();
747 for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) {
748 Thread.currentThread();
749 try {
750 Thread.sleep(1000);
751 } catch (InterruptedException e) {
752 e.printStackTrace();
753 }
754 regions = t.getRegionLocations();
755 if (regions.size() > originalCount) break;
756 }
757 return regions;
758 }
759
760 @Test
761 public void testSuperSimple() throws Exception {
762 byte [] TABLE = Bytes.toBytes("testSuperSimple");
763 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
764 Put put = new Put(ROW);
765 put.add(FAMILY, QUALIFIER, VALUE);
766 ht.put(put);
767 Scan scan = new Scan();
768 scan.addColumn(FAMILY, TABLE);
769 ResultScanner scanner = ht.getScanner(scan);
770 Result result = scanner.next();
771 assertTrue("Expected null result", result == null);
772 scanner.close();
773 }
774
775 @Test
776 public void testMaxKeyValueSize() throws Exception {
777 byte [] TABLE = Bytes.toBytes("testMaxKeyValueSize");
778 Configuration conf = TEST_UTIL.getConfiguration();
779 String oldMaxSize = conf.get("hbase.client.keyvalue.maxsize");
780 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
781 byte[] value = new byte[4 * 1024 * 1024];
782 Put put = new Put(ROW);
783 put.add(FAMILY, QUALIFIER, value);
784 ht.put(put);
785 try {
786 conf.setInt("hbase.client.keyvalue.maxsize", 2 * 1024 * 1024);
787 TABLE = Bytes.toBytes("testMaxKeyValueSize2");
788 ht = TEST_UTIL.createTable(TABLE, FAMILY);
789 put = new Put(ROW);
790 put.add(FAMILY, QUALIFIER, value);
791 ht.put(put);
792 fail("Inserting a too large KeyValue worked, should throw exception");
793 } catch(Exception e) {}
794 conf.set("hbase.client.keyvalue.maxsize", oldMaxSize);
795 }
796
797 @Test
798 public void testFilters() throws Exception {
799 byte [] TABLE = Bytes.toBytes("testFilters");
800 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
801 byte [][] ROWS = makeN(ROW, 10);
802 byte [][] QUALIFIERS = {
803 Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
804 Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"),
805 Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"),
806 Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"),
807 Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>")
808 };
809 for(int i=0;i<10;i++) {
810 Put put = new Put(ROWS[i]);
811 put.setDurability(Durability.SKIP_WAL);
812 put.add(FAMILY, QUALIFIERS[i], VALUE);
813 ht.put(put);
814 }
815 Scan scan = new Scan();
816 scan.addFamily(FAMILY);
817 Filter filter = new QualifierFilter(CompareOp.EQUAL,
818 new RegexStringComparator("col[1-5]"));
819 scan.setFilter(filter);
820 ResultScanner scanner = ht.getScanner(scan);
821 int expectedIndex = 1;
822 for(Result result : ht.getScanner(scan)) {
823 assertEquals(result.size(), 1);
824 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[expectedIndex]));
825 assertTrue(Bytes.equals(CellUtil.cloneQualifier(result.rawCells()[0]),
826 QUALIFIERS[expectedIndex]));
827 expectedIndex++;
828 }
829 assertEquals(expectedIndex, 6);
830 scanner.close();
831 }
832
833 @Test
834 public void testKeyOnlyFilter() throws Exception {
835 byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
836 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
837 byte [][] ROWS = makeN(ROW, 10);
838 byte [][] QUALIFIERS = {
839 Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
840 Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"),
841 Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"),
842 Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"),
843 Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>")
844 };
845 for(int i=0;i<10;i++) {
846 Put put = new Put(ROWS[i]);
847 put.setDurability(Durability.SKIP_WAL);
848 put.add(FAMILY, QUALIFIERS[i], VALUE);
849 ht.put(put);
850 }
851 Scan scan = new Scan();
852 scan.addFamily(FAMILY);
853 Filter filter = new KeyOnlyFilter(true);
854 scan.setFilter(filter);
855 ResultScanner scanner = ht.getScanner(scan);
856 int count = 0;
857 for(Result result : ht.getScanner(scan)) {
858 assertEquals(result.size(), 1);
859 assertEquals(result.rawCells()[0].getValueLength(), Bytes.SIZEOF_INT);
860 assertEquals(Bytes.toInt(CellUtil.cloneValue(result.rawCells()[0])), VALUE.length);
861 count++;
862 }
863 assertEquals(count, 10);
864 scanner.close();
865 }
866
867
868
869
870 @Test
871 public void testSimpleMissing() throws Exception {
872 byte [] TABLE = Bytes.toBytes("testSimpleMissing");
873 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
874 byte [][] ROWS = makeN(ROW, 4);
875
876
877 Get get = new Get(ROWS[0]);
878 Result result = ht.get(get);
879 assertEmptyResult(result);
880
881 get = new Get(ROWS[0]);
882 get.addFamily(FAMILY);
883 result = ht.get(get);
884 assertEmptyResult(result);
885
886 get = new Get(ROWS[0]);
887 get.addColumn(FAMILY, QUALIFIER);
888 result = ht.get(get);
889 assertEmptyResult(result);
890
891 Scan scan = new Scan();
892 result = getSingleScanResult(ht, scan);
893 assertNullResult(result);
894
895
896 scan = new Scan(ROWS[0]);
897 result = getSingleScanResult(ht, scan);
898 assertNullResult(result);
899
900 scan = new Scan(ROWS[0],ROWS[1]);
901 result = getSingleScanResult(ht, scan);
902 assertNullResult(result);
903
904 scan = new Scan();
905 scan.addFamily(FAMILY);
906 result = getSingleScanResult(ht, scan);
907 assertNullResult(result);
908
909 scan = new Scan();
910 scan.addColumn(FAMILY, QUALIFIER);
911 result = getSingleScanResult(ht, scan);
912 assertNullResult(result);
913
914
915
916 Put put = new Put(ROWS[2]);
917 put.add(FAMILY, QUALIFIER, VALUE);
918 ht.put(put);
919
920
921
922 get = new Get(ROWS[1]);
923 result = ht.get(get);
924 assertEmptyResult(result);
925
926 get = new Get(ROWS[0]);
927 get.addFamily(FAMILY);
928 result = ht.get(get);
929 assertEmptyResult(result);
930
931 get = new Get(ROWS[3]);
932 get.addColumn(FAMILY, QUALIFIER);
933 result = ht.get(get);
934 assertEmptyResult(result);
935
936
937
938 scan = new Scan(ROWS[3]);
939 result = getSingleScanResult(ht, scan);
940 assertNullResult(result);
941
942 scan = new Scan(ROWS[0],ROWS[2]);
943 result = getSingleScanResult(ht, scan);
944 assertNullResult(result);
945
946
947
948 get = new Get(ROWS[2]);
949 result = ht.get(get);
950 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
951
952 get = new Get(ROWS[2]);
953 get.addFamily(FAMILY);
954 result = ht.get(get);
955 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
956
957 get = new Get(ROWS[2]);
958 get.addColumn(FAMILY, QUALIFIER);
959 result = ht.get(get);
960 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
961
962
963
964 scan = new Scan();
965 result = getSingleScanResult(ht, scan);
966 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
967
968 scan = new Scan(ROWS[0],ROWS[3]);
969 result = getSingleScanResult(ht, scan);
970 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
971
972 scan = new Scan(ROWS[2],ROWS[3]);
973 result = getSingleScanResult(ht, scan);
974 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
975 }
976
977
978
979
980
981 @Test
982 public void testSingleRowMultipleFamily() throws Exception {
983 byte [] TABLE = Bytes.toBytes("testSingleRowMultipleFamily");
984 byte [][] ROWS = makeN(ROW, 3);
985 byte [][] FAMILIES = makeNAscii(FAMILY, 10);
986 byte [][] QUALIFIERS = makeN(QUALIFIER, 10);
987 byte [][] VALUES = makeN(VALUE, 10);
988
989 HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
990
991 Get get;
992 Scan scan;
993 Delete delete;
994 Put put;
995 Result result;
996
997
998
999
1000
1001 put = new Put(ROWS[0]);
1002 put.add(FAMILIES[4], QUALIFIERS[0], VALUES[0]);
1003 ht.put(put);
1004
1005
1006 getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1007
1008
1009 scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1010
1011
1012 getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1013
1014
1015 scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1016
1017
1018
1019
1020
1021 TEST_UTIL.flush();
1022
1023
1024 getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1025 scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1026 getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1027 scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1028
1029
1030
1031
1032
1033
1034 put = new Put(ROWS[0]);
1035 put.add(FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1036 put.add(FAMILIES[2], QUALIFIERS[4], VALUES[4]);
1037 put.add(FAMILIES[4], QUALIFIERS[4], VALUES[4]);
1038 put.add(FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1039 put.add(FAMILIES[6], QUALIFIERS[7], VALUES[7]);
1040 put.add(FAMILIES[7], QUALIFIERS[7], VALUES[7]);
1041 put.add(FAMILIES[9], QUALIFIERS[0], VALUES[0]);
1042 ht.put(put);
1043
1044
1045 singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1046
1047
1048 singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1049
1050
1051
1052
1053
1054 TEST_UTIL.flush();
1055
1056
1057 singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1058 singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1059
1060
1061 put = new Put(ROWS[0]);
1062 put.add(FAMILIES[6], QUALIFIERS[5], VALUES[5]);
1063 put.add(FAMILIES[6], QUALIFIERS[8], VALUES[8]);
1064 put.add(FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1065 put.add(FAMILIES[4], QUALIFIERS[3], VALUES[3]);
1066 ht.put(put);
1067
1068
1069
1070
1071 delete = new Delete(ROWS[0]);
1072 delete.deleteColumns(FAMILIES[6], QUALIFIERS[7]);
1073 ht.delete(delete);
1074
1075
1076 get = new Get(ROWS[0]);
1077 get.addColumn(FAMILIES[6], QUALIFIERS[7]);
1078 result = ht.get(get);
1079 assertEmptyResult(result);
1080
1081
1082 scan = new Scan();
1083 scan.addColumn(FAMILIES[6], QUALIFIERS[7]);
1084 result = getSingleScanResult(ht, scan);
1085 assertNullResult(result);
1086
1087
1088 get = new Get(ROWS[0]);
1089 get.addColumn(FAMILIES[6], QUALIFIERS[6]);
1090 result = ht.get(get);
1091 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1092
1093 get = new Get(ROWS[0]);
1094 get.addColumn(FAMILIES[6], QUALIFIERS[8]);
1095 result = ht.get(get);
1096 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
1097
1098
1099 scan = new Scan();
1100 scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1101 result = getSingleScanResult(ht, scan);
1102 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1103
1104 scan = new Scan();
1105 scan.addColumn(FAMILIES[6], QUALIFIERS[8]);
1106 result = getSingleScanResult(ht, scan);
1107 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
1108
1109
1110
1111
1112 delete = new Delete(ROWS[0]);
1113 delete.deleteColumns(FAMILIES[6], QUALIFIERS[8]);
1114 ht.delete(delete);
1115
1116
1117 get = new Get(ROWS[0]);
1118 get.addColumn(FAMILIES[6], QUALIFIERS[8]);
1119 result = ht.get(get);
1120 assertEmptyResult(result);
1121
1122
1123 scan = new Scan();
1124 scan.addColumn(FAMILIES[6], QUALIFIERS[8]);
1125 result = getSingleScanResult(ht, scan);
1126 assertNullResult(result);
1127
1128
1129 get = new Get(ROWS[0]);
1130 get.addColumn(FAMILIES[6], QUALIFIERS[6]);
1131 result = ht.get(get);
1132 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1133
1134 get = new Get(ROWS[0]);
1135 get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1136 result = ht.get(get);
1137 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1138
1139
1140 scan = new Scan();
1141 scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1142 result = getSingleScanResult(ht, scan);
1143 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1144
1145 scan = new Scan();
1146 scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1147 result = getSingleScanResult(ht, scan);
1148 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1149
1150
1151
1152
1153
1154 delete = new Delete(ROWS[0]);
1155 delete.deleteFamily(FAMILIES[4]);
1156 ht.delete(delete);
1157
1158
1159 get = new Get(ROWS[0]);
1160 get.addColumn(FAMILIES[4], QUALIFIERS[4]);
1161 result = ht.get(get);
1162 assertEmptyResult(result);
1163
1164
1165 get = new Get(ROWS[0]);
1166 get.addColumn(FAMILIES[4], QUALIFIERS[3]);
1167 result = ht.get(get);
1168 assertEmptyResult(result);
1169
1170
1171 get = new Get(ROWS[0]);
1172 get.addFamily(FAMILIES[4]);
1173 result = ht.get(get);
1174 assertEmptyResult(result);
1175
1176
1177 scan = new Scan();
1178 scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
1179 result = getSingleScanResult(ht, scan);
1180 assertNullResult(result);
1181
1182
1183 scan = new Scan();
1184 scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
1185 result = getSingleScanResult(ht, scan);
1186 assertNullResult(result);
1187
1188
1189 scan = new Scan();
1190 scan.addFamily(FAMILIES[4]);
1191 result = getSingleScanResult(ht, scan);
1192 assertNullResult(result);
1193
1194
1195 get = new Get(ROWS[0]);
1196 get.addColumn(FAMILIES[2], QUALIFIERS[2]);
1197 result = ht.get(get);
1198 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1199
1200 get = new Get(ROWS[0]);
1201 get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1202 result = ht.get(get);
1203 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1204
1205
1206 scan = new Scan();
1207 scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1208 result = getSingleScanResult(ht, scan);
1209 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1210
1211 scan = new Scan();
1212 scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1213 result = getSingleScanResult(ht, scan);
1214 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1215
1216
1217
1218
1219
1220 TEST_UTIL.flush();
1221
1222
1223 get = new Get(ROWS[0]);
1224 get.addColumn(FAMILIES[4], QUALIFIERS[4]);
1225 result = ht.get(get);
1226 assertEmptyResult(result);
1227
1228
1229 get = new Get(ROWS[0]);
1230 get.addColumn(FAMILIES[4], QUALIFIERS[3]);
1231 result = ht.get(get);
1232 assertEmptyResult(result);
1233
1234
1235 get = new Get(ROWS[0]);
1236 get.addFamily(FAMILIES[4]);
1237 result = ht.get(get);
1238 assertEmptyResult(result);
1239
1240
1241 scan = new Scan();
1242 scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
1243 result = getSingleScanResult(ht, scan);
1244 assertNullResult(result);
1245
1246
1247 scan = new Scan();
1248 scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
1249 result = getSingleScanResult(ht, scan);
1250 assertNullResult(result);
1251
1252
1253 scan = new Scan();
1254 scan.addFamily(FAMILIES[4]);
1255 result = getSingleScanResult(ht, scan);
1256 assertNullResult(result);
1257
1258
1259 get = new Get(ROWS[0]);
1260 get.addColumn(FAMILIES[2], QUALIFIERS[2]);
1261 result = ht.get(get);
1262 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1263
1264 get = new Get(ROWS[0]);
1265 get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1266 result = ht.get(get);
1267 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1268
1269
1270 scan = new Scan();
1271 scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1272 result = getSingleScanResult(ht, scan);
1273 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1274
1275 scan = new Scan();
1276 scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1277 result = getSingleScanResult(ht, scan);
1278 assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1279
1280 }
1281
1282 @Test
1283 public void testNull() throws Exception {
1284 byte [] TABLE = Bytes.toBytes("testNull");
1285
1286
1287 try {
1288 TEST_UTIL.createTable((TableName)null, FAMILY);
1289 fail("Creating a table with null name passed, should have failed");
1290 } catch(Exception e) {}
1291
1292
1293 try {
1294 TEST_UTIL.createTable(TABLE, (byte[])null);
1295 fail("Creating a table with a null family passed, should fail");
1296 } catch(Exception e) {}
1297
1298 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
1299
1300
1301 try {
1302 Put put = new Put((byte[])null);
1303 put.add(FAMILY, QUALIFIER, VALUE);
1304 ht.put(put);
1305 fail("Inserting a null row worked, should throw exception");
1306 } catch(Exception e) {}
1307
1308
1309 {
1310 Put put = new Put(ROW);
1311 put.add(FAMILY, null, VALUE);
1312 ht.put(put);
1313
1314 getTestNull(ht, ROW, FAMILY, VALUE);
1315
1316 scanTestNull(ht, ROW, FAMILY, VALUE);
1317
1318 Delete delete = new Delete(ROW);
1319 delete.deleteColumns(FAMILY, null);
1320 ht.delete(delete);
1321
1322 Get get = new Get(ROW);
1323 Result result = ht.get(get);
1324 assertEmptyResult(result);
1325 }
1326
1327
1328 byte [] TABLE2 = Bytes.toBytes("testNull2");
1329 ht = TEST_UTIL.createTable(TABLE2, FAMILY);
1330
1331
1332 try {
1333 Put put = new Put(ROW);
1334 put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
1335 ht.put(put);
1336
1337 getTestNull(ht, ROW, FAMILY, VALUE);
1338
1339 scanTestNull(ht, ROW, FAMILY, VALUE);
1340
1341
1342
1343 TEST_UTIL.flush();
1344
1345 getTestNull(ht, ROW, FAMILY, VALUE);
1346
1347 scanTestNull(ht, ROW, FAMILY, VALUE);
1348
1349 Delete delete = new Delete(ROW);
1350 delete.deleteColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY);
1351 ht.delete(delete);
1352
1353 Get get = new Get(ROW);
1354 Result result = ht.get(get);
1355 assertEmptyResult(result);
1356
1357 } catch(Exception e) {
1358 throw new IOException("Using a row with null qualifier threw exception, should ");
1359 }
1360
1361
1362 try {
1363 Put put = new Put(ROW);
1364 put.add(FAMILY, QUALIFIER, null);
1365 ht.put(put);
1366
1367 Get get = new Get(ROW);
1368 get.addColumn(FAMILY, QUALIFIER);
1369 Result result = ht.get(get);
1370 assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
1371
1372 Scan scan = new Scan();
1373 scan.addColumn(FAMILY, QUALIFIER);
1374 result = getSingleScanResult(ht, scan);
1375 assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
1376
1377 Delete delete = new Delete(ROW);
1378 delete.deleteColumns(FAMILY, QUALIFIER);
1379 ht.delete(delete);
1380
1381 get = new Get(ROW);
1382 result = ht.get(get);
1383 assertEmptyResult(result);
1384
1385 } catch(Exception e) {
1386 throw new IOException("Null values should be allowed, but threw exception");
1387 }
1388 }
1389
1390 @Test
1391 public void testVersions() throws Exception {
1392 byte [] TABLE = Bytes.toBytes("testVersions");
1393
1394 long [] STAMPS = makeStamps(20);
1395 byte [][] VALUES = makeNAscii(VALUE, 20);
1396
1397 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
1398
1399
1400 Put put = new Put(ROW);
1401 put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1402 put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1403 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1404 put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1405 ht.put(put);
1406
1407
1408 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1409 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1410 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1411 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1412 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1413 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1414 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1415 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1416
1417
1418 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1419 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1420 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1421 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1422 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1423 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1424
1425
1426 Get get = new Get(ROW);
1427 get.addColumn(FAMILY, QUALIFIER);
1428 get.setMaxVersions(2);
1429 Result result = ht.get(get);
1430 assertNResult(result, ROW, FAMILY, QUALIFIER,
1431 new long [] {STAMPS[4], STAMPS[5]},
1432 new byte[][] {VALUES[4], VALUES[5]},
1433 0, 1);
1434
1435 Scan scan = new Scan(ROW);
1436 scan.addColumn(FAMILY, QUALIFIER);
1437 scan.setMaxVersions(2);
1438 result = getSingleScanResult(ht, scan);
1439 assertNResult(result, ROW, FAMILY, QUALIFIER,
1440 new long [] {STAMPS[4], STAMPS[5]},
1441 new byte[][] {VALUES[4], VALUES[5]},
1442 0, 1);
1443
1444
1445
1446 TEST_UTIL.flush();
1447
1448
1449 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1450 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1451 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1452 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1453 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1454 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1455 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1456 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1457
1458
1459 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1460 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1461 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1462 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1463 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1464 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1465
1466
1467 get = new Get(ROW);
1468 get.addColumn(FAMILY, QUALIFIER);
1469 get.setMaxVersions(2);
1470 result = ht.get(get);
1471 assertNResult(result, ROW, FAMILY, QUALIFIER,
1472 new long [] {STAMPS[4], STAMPS[5]},
1473 new byte[][] {VALUES[4], VALUES[5]},
1474 0, 1);
1475
1476 scan = new Scan(ROW);
1477 scan.addColumn(FAMILY, QUALIFIER);
1478 scan.setMaxVersions(2);
1479 result = getSingleScanResult(ht, scan);
1480 assertNResult(result, ROW, FAMILY, QUALIFIER,
1481 new long [] {STAMPS[4], STAMPS[5]},
1482 new byte[][] {VALUES[4], VALUES[5]},
1483 0, 1);
1484
1485
1486
1487
1488
1489 put = new Put(ROW);
1490 put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
1491 put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
1492 put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1493 put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]);
1494 ht.put(put);
1495
1496
1497 get = new Get(ROW);
1498 get.addColumn(FAMILY, QUALIFIER);
1499 get.setMaxVersions();
1500 result = ht.get(get);
1501 assertNResult(result, ROW, FAMILY, QUALIFIER,
1502 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1503 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1504 0, 7);
1505
1506 scan = new Scan(ROW);
1507 scan.addColumn(FAMILY, QUALIFIER);
1508 scan.setMaxVersions();
1509 result = getSingleScanResult(ht, scan);
1510 assertNResult(result, ROW, FAMILY, QUALIFIER,
1511 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1512 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1513 0, 7);
1514
1515 get = new Get(ROW);
1516 get.setMaxVersions();
1517 result = ht.get(get);
1518 assertNResult(result, ROW, FAMILY, QUALIFIER,
1519 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1520 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1521 0, 7);
1522
1523 scan = new Scan(ROW);
1524 scan.setMaxVersions();
1525 result = getSingleScanResult(ht, scan);
1526 assertNResult(result, ROW, FAMILY, QUALIFIER,
1527 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1528 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1529 0, 7);
1530
1531
1532 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1533 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1534 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1535 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1536 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1537 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1538 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1539 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1540
1541
1542 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1543 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
1544 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1545 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
1546
1547
1548
1549 TEST_UTIL.flush();
1550
1551
1552 put = new Put(ROW);
1553 put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]);
1554 put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]);
1555 put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]);
1556 put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
1557 ht.put(put);
1558
1559 get = new Get(ROW);
1560 get.addColumn(FAMILY, QUALIFIER);
1561 get.setMaxVersions(Integer.MAX_VALUE);
1562 result = ht.get(get);
1563 assertNResult(result, ROW, FAMILY, QUALIFIER,
1564 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
1565 new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
1566 0, 9);
1567
1568 scan = new Scan(ROW);
1569 scan.addColumn(FAMILY, QUALIFIER);
1570 scan.setMaxVersions(Integer.MAX_VALUE);
1571 result = getSingleScanResult(ht, scan);
1572 assertNResult(result, ROW, FAMILY, QUALIFIER,
1573 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
1574 new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
1575 0, 9);
1576
1577
1578 Delete delete = new Delete(ROW);
1579 delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
1580 delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
1581 ht.delete(delete);
1582
1583
1584 get = new Get(ROW);
1585 get.addColumn(FAMILY, QUALIFIER);
1586 get.setMaxVersions(Integer.MAX_VALUE);
1587 result = ht.get(get);
1588 assertNResult(result, ROW, FAMILY, QUALIFIER,
1589 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
1590 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
1591 0, 9);
1592
1593 scan = new Scan(ROW);
1594 scan.addColumn(FAMILY, QUALIFIER);
1595 scan.setMaxVersions(Integer.MAX_VALUE);
1596 result = getSingleScanResult(ht, scan);
1597 assertNResult(result, ROW, FAMILY, QUALIFIER,
1598 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
1599 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
1600 0, 9);
1601
1602 }
1603
1604 @Test
1605 public void testVersionLimits() throws Exception {
1606 byte [] TABLE = Bytes.toBytes("testVersionLimits");
1607 byte [][] FAMILIES = makeNAscii(FAMILY, 3);
1608 int [] LIMITS = {1,3,5};
1609 long [] STAMPS = makeStamps(10);
1610 byte [][] VALUES = makeNAscii(VALUE, 10);
1611 HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS);
1612
1613
1614 Put put = new Put(ROW);
1615 put.add(FAMILIES[0], QUALIFIER, STAMPS[0], VALUES[0]);
1616 put.add(FAMILIES[0], QUALIFIER, STAMPS[1], VALUES[1]);
1617 put.add(FAMILIES[1], QUALIFIER, STAMPS[0], VALUES[0]);
1618 put.add(FAMILIES[1], QUALIFIER, STAMPS[1], VALUES[1]);
1619 put.add(FAMILIES[1], QUALIFIER, STAMPS[2], VALUES[2]);
1620 put.add(FAMILIES[1], QUALIFIER, STAMPS[3], VALUES[3]);
1621 put.add(FAMILIES[2], QUALIFIER, STAMPS[0], VALUES[0]);
1622 put.add(FAMILIES[2], QUALIFIER, STAMPS[1], VALUES[1]);
1623 put.add(FAMILIES[2], QUALIFIER, STAMPS[2], VALUES[2]);
1624 put.add(FAMILIES[2], QUALIFIER, STAMPS[3], VALUES[3]);
1625 put.add(FAMILIES[2], QUALIFIER, STAMPS[4], VALUES[4]);
1626 put.add(FAMILIES[2], QUALIFIER, STAMPS[5], VALUES[5]);
1627 put.add(FAMILIES[2], QUALIFIER, STAMPS[6], VALUES[6]);
1628 ht.put(put);
1629
1630
1631
1632
1633
1634 Get get = new Get(ROW);
1635 get.addColumn(FAMILIES[0], QUALIFIER);
1636 get.setMaxVersions(Integer.MAX_VALUE);
1637 Result result = ht.get(get);
1638 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1639 new long [] {STAMPS[1]},
1640 new byte[][] {VALUES[1]},
1641 0, 0);
1642
1643 get = new Get(ROW);
1644 get.addFamily(FAMILIES[0]);
1645 get.setMaxVersions(Integer.MAX_VALUE);
1646 result = ht.get(get);
1647 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1648 new long [] {STAMPS[1]},
1649 new byte[][] {VALUES[1]},
1650 0, 0);
1651
1652 Scan scan = new Scan(ROW);
1653 scan.addColumn(FAMILIES[0], QUALIFIER);
1654 scan.setMaxVersions(Integer.MAX_VALUE);
1655 result = getSingleScanResult(ht, scan);
1656 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1657 new long [] {STAMPS[1]},
1658 new byte[][] {VALUES[1]},
1659 0, 0);
1660
1661 scan = new Scan(ROW);
1662 scan.addFamily(FAMILIES[0]);
1663 scan.setMaxVersions(Integer.MAX_VALUE);
1664 result = getSingleScanResult(ht, scan);
1665 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1666 new long [] {STAMPS[1]},
1667 new byte[][] {VALUES[1]},
1668 0, 0);
1669
1670
1671
1672 get = new Get(ROW);
1673 get.addColumn(FAMILIES[1], QUALIFIER);
1674 get.setMaxVersions(Integer.MAX_VALUE);
1675 result = ht.get(get);
1676 assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1677 new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1678 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1679 0, 2);
1680
1681 get = new Get(ROW);
1682 get.addFamily(FAMILIES[1]);
1683 get.setMaxVersions(Integer.MAX_VALUE);
1684 result = ht.get(get);
1685 assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1686 new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1687 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1688 0, 2);
1689
1690 scan = new Scan(ROW);
1691 scan.addColumn(FAMILIES[1], QUALIFIER);
1692 scan.setMaxVersions(Integer.MAX_VALUE);
1693 result = getSingleScanResult(ht, scan);
1694 assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1695 new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1696 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1697 0, 2);
1698
1699 scan = new Scan(ROW);
1700 scan.addFamily(FAMILIES[1]);
1701 scan.setMaxVersions(Integer.MAX_VALUE);
1702 result = getSingleScanResult(ht, scan);
1703 assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1704 new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1705 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1706 0, 2);
1707
1708
1709
1710 get = new Get(ROW);
1711 get.addColumn(FAMILIES[2], QUALIFIER);
1712 get.setMaxVersions(Integer.MAX_VALUE);
1713 result = ht.get(get);
1714 assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1715 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1716 new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1717 0, 4);
1718
1719 get = new Get(ROW);
1720 get.addFamily(FAMILIES[2]);
1721 get.setMaxVersions(Integer.MAX_VALUE);
1722 result = ht.get(get);
1723 assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1724 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1725 new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1726 0, 4);
1727
1728 scan = new Scan(ROW);
1729 scan.addColumn(FAMILIES[2], QUALIFIER);
1730 scan.setMaxVersions(Integer.MAX_VALUE);
1731 result = getSingleScanResult(ht, scan);
1732 assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1733 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1734 new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1735 0, 4);
1736
1737 scan = new Scan(ROW);
1738 scan.addFamily(FAMILIES[2]);
1739 scan.setMaxVersions(Integer.MAX_VALUE);
1740 result = getSingleScanResult(ht, scan);
1741 assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1742 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1743 new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1744 0, 4);
1745
1746
1747
1748 get = new Get(ROW);
1749 get.setMaxVersions(Integer.MAX_VALUE);
1750 result = ht.get(get);
1751 assertTrue("Expected 9 keys but received " + result.size(),
1752 result.size() == 9);
1753
1754 get = new Get(ROW);
1755 get.addFamily(FAMILIES[0]);
1756 get.addFamily(FAMILIES[1]);
1757 get.addFamily(FAMILIES[2]);
1758 get.setMaxVersions(Integer.MAX_VALUE);
1759 result = ht.get(get);
1760 assertTrue("Expected 9 keys but received " + result.size(),
1761 result.size() == 9);
1762
1763 get = new Get(ROW);
1764 get.addColumn(FAMILIES[0], QUALIFIER);
1765 get.addColumn(FAMILIES[1], QUALIFIER);
1766 get.addColumn(FAMILIES[2], QUALIFIER);
1767 get.setMaxVersions(Integer.MAX_VALUE);
1768 result = ht.get(get);
1769 assertTrue("Expected 9 keys but received " + result.size(),
1770 result.size() == 9);
1771
1772 scan = new Scan(ROW);
1773 scan.setMaxVersions(Integer.MAX_VALUE);
1774 result = getSingleScanResult(ht, scan);
1775 assertTrue("Expected 9 keys but received " + result.size(),
1776 result.size() == 9);
1777
1778 scan = new Scan(ROW);
1779 scan.setMaxVersions(Integer.MAX_VALUE);
1780 scan.addFamily(FAMILIES[0]);
1781 scan.addFamily(FAMILIES[1]);
1782 scan.addFamily(FAMILIES[2]);
1783 result = getSingleScanResult(ht, scan);
1784 assertTrue("Expected 9 keys but received " + result.size(),
1785 result.size() == 9);
1786
1787 scan = new Scan(ROW);
1788 scan.setMaxVersions(Integer.MAX_VALUE);
1789 scan.addColumn(FAMILIES[0], QUALIFIER);
1790 scan.addColumn(FAMILIES[1], QUALIFIER);
1791 scan.addColumn(FAMILIES[2], QUALIFIER);
1792 result = getSingleScanResult(ht, scan);
1793 assertTrue("Expected 9 keys but received " + result.size(),
1794 result.size() == 9);
1795
1796 }
1797
1798 @Test
1799 public void testDeleteFamilyVersion() throws Exception {
1800 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
1801 byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersion");
1802
1803 byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 1);
1804 byte [][] VALUES = makeN(VALUE, 5);
1805 long [] ts = {1000, 2000, 3000, 4000, 5000};
1806
1807 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
1808
1809 Put put = new Put(ROW);
1810 for (int q = 0; q < 1; q++)
1811 for (int t = 0; t < 5; t++)
1812 put.add(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]);
1813 ht.put(put);
1814 admin.flush(TABLE);
1815
1816 Delete delete = new Delete(ROW);
1817 delete.deleteFamilyVersion(FAMILY, ts[1]);
1818 delete.deleteFamilyVersion(FAMILY, ts[3]);
1819 ht.delete(delete);
1820 admin.flush(TABLE);
1821
1822 for (int i = 0; i < 1; i++) {
1823 Get get = new Get(ROW);
1824 get.addColumn(FAMILY, QUALIFIERS[i]);
1825 get.setMaxVersions(Integer.MAX_VALUE);
1826 Result result = ht.get(get);
1827
1828 assertNResult(result, ROW, FAMILY, QUALIFIERS[i],
1829 new long [] {ts[0], ts[2], ts[4]},
1830 new byte[][] {VALUES[0], VALUES[2], VALUES[4]},
1831 0, 2);
1832 }
1833 ht.close();
1834 admin.close();
1835 }
1836
1837 @Test
1838 public void testDeleteFamilyVersionWithOtherDeletes() throws Exception {
1839 byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersionWithOtherDeletes");
1840
1841 byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 5);
1842 byte [][] VALUES = makeN(VALUE, 5);
1843 long [] ts = {1000, 2000, 3000, 4000, 5000};
1844
1845 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
1846 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
1847 Put put = null;
1848 Result result = null;
1849 Get get = null;
1850 Delete delete = null;
1851
1852
1853 put = new Put(ROW);
1854 for (int q = 0; q < 5; q++)
1855 for (int t = 0; t < 5; t++)
1856 put.add(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]);
1857 ht.put(put);
1858 admin.flush(TABLE);
1859
1860
1861 byte [] ROW2 = Bytes.toBytes("myRowForTest");
1862 put = new Put(ROW2);
1863 for (int q = 0; q < 5; q++)
1864 for (int t = 0; t < 5; t++)
1865 put.add(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]);
1866 ht.put(put);
1867 admin.flush(TABLE);
1868
1869
1870 delete = new Delete(ROW);
1871
1872
1873
1874 delete.deleteFamily(FAMILY, ts[1]);
1875
1876 delete.deleteFamilyVersion(FAMILY, ts[3]);
1877
1878 delete.deleteColumns(FAMILY, QUALIFIERS[0], ts[2]);
1879
1880 delete.deleteColumns(FAMILY, QUALIFIERS[2], ts[4]);
1881
1882 delete.deleteColumn(FAMILY, QUALIFIERS[4], ts[4]);
1883 ht.delete(delete);
1884 admin.flush(TABLE);
1885
1886
1887 delete = new Delete(ROW2);
1888 delete.deleteFamilyVersion(FAMILY, ts[1]);
1889 delete.deleteFamilyVersion(FAMILY, ts[3]);
1890 ht.delete(delete);
1891 admin.flush(TABLE);
1892
1893
1894 get = new Get(ROW);
1895 get.addColumn(FAMILY, QUALIFIERS[0]);
1896 get.setMaxVersions(Integer.MAX_VALUE);
1897 result = ht.get(get);
1898 assertNResult(result, ROW, FAMILY, QUALIFIERS[0],
1899 new long [] {ts[4]},
1900 new byte[][] {VALUES[4]},
1901 0, 0);
1902
1903 get = new Get(ROW);
1904 get.addColumn(FAMILY, QUALIFIERS[1]);
1905 get.setMaxVersions(Integer.MAX_VALUE);
1906 result = ht.get(get);
1907 assertNResult(result, ROW, FAMILY, QUALIFIERS[1],
1908 new long [] {ts[2], ts[4]},
1909 new byte[][] {VALUES[2], VALUES[4]},
1910 0, 1);
1911
1912 get = new Get(ROW);
1913 get.addColumn(FAMILY, QUALIFIERS[2]);
1914 get.setMaxVersions(Integer.MAX_VALUE);
1915 result = ht.get(get);
1916 assertEquals(0, result.size());
1917
1918 get = new Get(ROW);
1919 get.addColumn(FAMILY, QUALIFIERS[3]);
1920 get.setMaxVersions(Integer.MAX_VALUE);
1921 result = ht.get(get);
1922 assertNResult(result, ROW, FAMILY, QUALIFIERS[3],
1923 new long [] {ts[2], ts[4]},
1924 new byte[][] {VALUES[2], VALUES[4]},
1925 0, 1);
1926
1927 get = new Get(ROW);
1928 get.addColumn(FAMILY, QUALIFIERS[4]);
1929 get.setMaxVersions(Integer.MAX_VALUE);
1930 result = ht.get(get);
1931 assertNResult(result, ROW, FAMILY, QUALIFIERS[4],
1932 new long [] {ts[2]},
1933 new byte[][] {VALUES[2]},
1934 0, 0);
1935
1936
1937 for (int i = 0; i < 5; i++) {
1938 get = new Get(ROW2);
1939 get.addColumn(FAMILY, QUALIFIERS[i]);
1940 get.setMaxVersions(Integer.MAX_VALUE);
1941 result = ht.get(get);
1942
1943 assertNResult(result, ROW2, FAMILY, QUALIFIERS[i],
1944 new long [] {ts[0], ts[2], ts[4]},
1945 new byte[][] {VALUES[0], VALUES[2], VALUES[4]},
1946 0, 2);
1947 }
1948 ht.close();
1949 admin.close();
1950 }
1951
1952 @Test
1953 public void testDeletes() throws Exception {
1954 byte [] TABLE = Bytes.toBytes("testDeletes");
1955
1956 byte [][] ROWS = makeNAscii(ROW, 6);
1957 byte [][] FAMILIES = makeNAscii(FAMILY, 3);
1958 byte [][] VALUES = makeN(VALUE, 5);
1959 long [] ts = {1000, 2000, 3000, 4000, 5000};
1960
1961 HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3);
1962
1963 Put put = new Put(ROW);
1964 put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
1965 put.add(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]);
1966 ht.put(put);
1967
1968 Delete delete = new Delete(ROW);
1969 delete.deleteFamily(FAMILIES[0], ts[0]);
1970 ht.delete(delete);
1971
1972 Get get = new Get(ROW);
1973 get.addFamily(FAMILIES[0]);
1974 get.setMaxVersions(Integer.MAX_VALUE);
1975 Result result = ht.get(get);
1976 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1977 new long [] {ts[1]},
1978 new byte[][] {VALUES[1]},
1979 0, 0);
1980
1981 Scan scan = new Scan(ROW);
1982 scan.addFamily(FAMILIES[0]);
1983 scan.setMaxVersions(Integer.MAX_VALUE);
1984 result = getSingleScanResult(ht, scan);
1985 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1986 new long [] {ts[1]},
1987 new byte[][] {VALUES[1]},
1988 0, 0);
1989
1990
1991 put = new Put(ROW);
1992 put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]);
1993 put.add(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]);
1994 put.add(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]);
1995 put.add(FAMILIES[0], null, ts[4], VALUES[4]);
1996 put.add(FAMILIES[0], null, ts[2], VALUES[2]);
1997 put.add(FAMILIES[0], null, ts[3], VALUES[3]);
1998 ht.put(put);
1999
2000 delete = new Delete(ROW);
2001 delete.deleteColumn(FAMILIES[0], QUALIFIER);
2002 ht.delete(delete);
2003
2004 get = new Get(ROW);
2005 get.addColumn(FAMILIES[0], QUALIFIER);
2006 get.setMaxVersions(Integer.MAX_VALUE);
2007 result = ht.get(get);
2008 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2009 new long [] {ts[1], ts[2], ts[3]},
2010 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2011 0, 2);
2012
2013 scan = new Scan(ROW);
2014 scan.addColumn(FAMILIES[0], QUALIFIER);
2015 scan.setMaxVersions(Integer.MAX_VALUE);
2016 result = getSingleScanResult(ht, scan);
2017 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2018 new long [] {ts[1], ts[2], ts[3]},
2019 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2020 0, 2);
2021
2022
2023 delete = new Delete(ROW);
2024 delete.deleteColumn(FAMILIES[0], null);
2025 ht.delete(delete);
2026
2027
2028 delete = new Delete(ROW);
2029 delete.deleteColumns(FAMILIES[0], null);
2030 ht.delete(delete);
2031
2032
2033
2034
2035 put = new Put(ROW);
2036 put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
2037 put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]);
2038 ht.put(put);
2039
2040
2041
2042
2043
2044 get = new Get(ROW);
2045 get.addFamily(FAMILIES[0]);
2046 get.setMaxVersions(Integer.MAX_VALUE);
2047 result = ht.get(get);
2048 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2049 new long [] {ts[1], ts[2], ts[3]},
2050 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2051 0, 2);
2052
2053
2054
2055 scan = new Scan(ROW);
2056 scan.addFamily(FAMILIES[0]);
2057 scan.setMaxVersions(Integer.MAX_VALUE);
2058 result = getSingleScanResult(ht, scan);
2059 assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2060 new long [] {ts[1], ts[2], ts[3]},
2061 new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2062 0, 2);
2063
2064
2065
2066 put = new Put(ROWS[0]);
2067 put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
2068 put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
2069 put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
2070 put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
2071 ht.put(put);
2072
2073 put = new Put(ROWS[1]);
2074 put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
2075 put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
2076 put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
2077 put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
2078 ht.put(put);
2079
2080 put = new Put(ROWS[2]);
2081 put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
2082 put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
2083 put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
2084 put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
2085 ht.put(put);
2086
2087
2088 get = new Get(ROWS[2]);
2089 get.addFamily(FAMILIES[1]);
2090 get.addFamily(FAMILIES[2]);
2091 get.setMaxVersions(Integer.MAX_VALUE);
2092 result = ht.get(get);
2093 assertTrue("Expected 4 key but received " + result.size() + ": " + result,
2094 result.size() == 4);
2095
2096 delete = new Delete(ROWS[0]);
2097 delete.deleteFamily(FAMILIES[2]);
2098 ht.delete(delete);
2099
2100 delete = new Delete(ROWS[1]);
2101 delete.deleteColumns(FAMILIES[1], QUALIFIER);
2102 ht.delete(delete);
2103
2104 delete = new Delete(ROWS[2]);
2105 delete.deleteColumn(FAMILIES[1], QUALIFIER);
2106 delete.deleteColumn(FAMILIES[1], QUALIFIER);
2107 delete.deleteColumn(FAMILIES[2], QUALIFIER);
2108 ht.delete(delete);
2109
2110 get = new Get(ROWS[0]);
2111 get.addFamily(FAMILIES[1]);
2112 get.addFamily(FAMILIES[2]);
2113 get.setMaxVersions(Integer.MAX_VALUE);
2114 result = ht.get(get);
2115 assertTrue("Expected 2 keys but received " + result.size(),
2116 result.size() == 2);
2117 assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER,
2118 new long [] {ts[0], ts[1]},
2119 new byte[][] {VALUES[0], VALUES[1]},
2120 0, 1);
2121
2122 scan = new Scan(ROWS[0]);
2123 scan.addFamily(FAMILIES[1]);
2124 scan.addFamily(FAMILIES[2]);
2125 scan.setMaxVersions(Integer.MAX_VALUE);
2126 result = getSingleScanResult(ht, scan);
2127 assertTrue("Expected 2 keys but received " + result.size(),
2128 result.size() == 2);
2129 assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER,
2130 new long [] {ts[0], ts[1]},
2131 new byte[][] {VALUES[0], VALUES[1]},
2132 0, 1);
2133
2134 get = new Get(ROWS[1]);
2135 get.addFamily(FAMILIES[1]);
2136 get.addFamily(FAMILIES[2]);
2137 get.setMaxVersions(Integer.MAX_VALUE);
2138 result = ht.get(get);
2139 assertTrue("Expected 2 keys but received " + result.size(),
2140 result.size() == 2);
2141
2142 scan = new Scan(ROWS[1]);
2143 scan.addFamily(FAMILIES[1]);
2144 scan.addFamily(FAMILIES[2]);
2145 scan.setMaxVersions(Integer.MAX_VALUE);
2146 result = getSingleScanResult(ht, scan);
2147 assertTrue("Expected 2 keys but received " + result.size(),
2148 result.size() == 2);
2149
2150 get = new Get(ROWS[2]);
2151 get.addFamily(FAMILIES[1]);
2152 get.addFamily(FAMILIES[2]);
2153 get.setMaxVersions(Integer.MAX_VALUE);
2154 result = ht.get(get);
2155 assertEquals(1, result.size());
2156 assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
2157 new long [] {ts[2]},
2158 new byte[][] {VALUES[2]},
2159 0, 0);
2160
2161 scan = new Scan(ROWS[2]);
2162 scan.addFamily(FAMILIES[1]);
2163 scan.addFamily(FAMILIES[2]);
2164 scan.setMaxVersions(Integer.MAX_VALUE);
2165 result = getSingleScanResult(ht, scan);
2166 assertEquals(1, result.size());
2167 assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
2168 new long [] {ts[2]},
2169 new byte[][] {VALUES[2]},
2170 0, 0);
2171
2172
2173
2174 delete = new Delete(ROWS[3]);
2175 delete.deleteFamily(FAMILIES[1]);
2176 ht.delete(delete);
2177
2178 put = new Put(ROWS[3]);
2179 put.add(FAMILIES[2], QUALIFIER, VALUES[0]);
2180 ht.put(put);
2181
2182 put = new Put(ROWS[4]);
2183 put.add(FAMILIES[1], QUALIFIER, VALUES[1]);
2184 put.add(FAMILIES[2], QUALIFIER, VALUES[2]);
2185 ht.put(put);
2186
2187 get = new Get(ROWS[3]);
2188 get.addFamily(FAMILIES[1]);
2189 get.addFamily(FAMILIES[2]);
2190 get.setMaxVersions(Integer.MAX_VALUE);
2191 result = ht.get(get);
2192 assertTrue("Expected 1 key but received " + result.size(),
2193 result.size() == 1);
2194
2195 get = new Get(ROWS[4]);
2196 get.addFamily(FAMILIES[1]);
2197 get.addFamily(FAMILIES[2]);
2198 get.setMaxVersions(Integer.MAX_VALUE);
2199 result = ht.get(get);
2200 assertTrue("Expected 2 keys but received " + result.size(),
2201 result.size() == 2);
2202
2203 scan = new Scan(ROWS[3]);
2204 scan.addFamily(FAMILIES[1]);
2205 scan.addFamily(FAMILIES[2]);
2206 scan.setMaxVersions(Integer.MAX_VALUE);
2207 ResultScanner scanner = ht.getScanner(scan);
2208 result = scanner.next();
2209 assertTrue("Expected 1 key but received " + result.size(),
2210 result.size() == 1);
2211 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[3]));
2212 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[0]));
2213 result = scanner.next();
2214 assertTrue("Expected 2 keys but received " + result.size(),
2215 result.size() == 2);
2216 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[4]));
2217 assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[1]), ROWS[4]));
2218 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[1]));
2219 assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[1]), VALUES[2]));
2220 scanner.close();
2221
2222
2223 for (int i = 0; i < 10; i++) {
2224 byte [] bytes = Bytes.toBytes(i);
2225 put = new Put(bytes);
2226 put.setDurability(Durability.SKIP_WAL);
2227 put.add(FAMILIES[0], QUALIFIER, bytes);
2228 ht.put(put);
2229 }
2230 for (int i = 0; i < 10; i++) {
2231 byte [] bytes = Bytes.toBytes(i);
2232 get = new Get(bytes);
2233 get.addFamily(FAMILIES[0]);
2234 result = ht.get(get);
2235 assertTrue(result.size() == 1);
2236 }
2237 ArrayList<Delete> deletes = new ArrayList<Delete>();
2238 for (int i = 0; i < 10; i++) {
2239 byte [] bytes = Bytes.toBytes(i);
2240 delete = new Delete(bytes);
2241 delete.deleteFamily(FAMILIES[0]);
2242 deletes.add(delete);
2243 }
2244 ht.delete(deletes);
2245 for (int i = 0; i < 10; i++) {
2246 byte [] bytes = Bytes.toBytes(i);
2247 get = new Get(bytes);
2248 get.addFamily(FAMILIES[0]);
2249 result = ht.get(get);
2250 assertTrue(result.size() == 0);
2251 }
2252 }
2253
2254
2255
2256
2257
2258
2259 @Ignore @Test
2260 public void testMillions() throws Exception {
2261
2262
2263
2264
2265
2266
2267
2268 }
2269
2270 @Ignore @Test
2271 public void testMultipleRegionsAndBatchPuts() throws Exception {
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297 }
2298
2299 @Ignore @Test
2300 public void testMultipleRowMultipleFamily() throws Exception {
2301
2302 }
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318 @Test
2319 public void testJiraTest867() throws Exception {
2320 int numRows = 10;
2321 int numColsPerRow = 2000;
2322
2323 byte [] TABLE = Bytes.toBytes("testJiraTest867");
2324
2325 byte [][] ROWS = makeN(ROW, numRows);
2326 byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
2327
2328 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
2329
2330
2331
2332 for(int i=0;i<numRows;i++) {
2333 Put put = new Put(ROWS[i]);
2334 put.setDurability(Durability.SKIP_WAL);
2335 for(int j=0;j<numColsPerRow;j++) {
2336 put.add(FAMILY, QUALIFIERS[j], QUALIFIERS[j]);
2337 }
2338 assertTrue("Put expected to contain " + numColsPerRow + " columns but " +
2339 "only contains " + put.size(), put.size() == numColsPerRow);
2340 ht.put(put);
2341 }
2342
2343
2344 Get get = new Get(ROWS[numRows-1]);
2345 Result result = ht.get(get);
2346 assertNumKeys(result, numColsPerRow);
2347 Cell [] keys = result.rawCells();
2348 for(int i=0;i<result.size();i++) {
2349 assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2350 }
2351
2352
2353 Scan scan = new Scan();
2354 ResultScanner scanner = ht.getScanner(scan);
2355 int rowCount = 0;
2356 while((result = scanner.next()) != null) {
2357 assertNumKeys(result, numColsPerRow);
2358 Cell [] kvs = result.rawCells();
2359 for(int i=0;i<numColsPerRow;i++) {
2360 assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2361 }
2362 rowCount++;
2363 }
2364 scanner.close();
2365 assertTrue("Expected to scan " + numRows + " rows but actually scanned "
2366 + rowCount + " rows", rowCount == numRows);
2367
2368
2369
2370 TEST_UTIL.flush();
2371
2372
2373 get = new Get(ROWS[numRows-1]);
2374 result = ht.get(get);
2375 assertNumKeys(result, numColsPerRow);
2376 keys = result.rawCells();
2377 for(int i=0;i<result.size();i++) {
2378 assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2379 }
2380
2381
2382 scan = new Scan();
2383 scanner = ht.getScanner(scan);
2384 rowCount = 0;
2385 while((result = scanner.next()) != null) {
2386 assertNumKeys(result, numColsPerRow);
2387 Cell [] kvs = result.rawCells();
2388 for(int i=0;i<numColsPerRow;i++) {
2389 assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2390 }
2391 rowCount++;
2392 }
2393 scanner.close();
2394 assertTrue("Expected to scan " + numRows + " rows but actually scanned "
2395 + rowCount + " rows", rowCount == numRows);
2396
2397 }
2398
2399
2400
2401
2402
2403
2404 @Test
2405 public void testJiraTest861() throws Exception {
2406
2407 byte [] TABLE = Bytes.toBytes("testJiraTest861");
2408 byte [][] VALUES = makeNAscii(VALUE, 7);
2409 long [] STAMPS = makeStamps(7);
2410
2411 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2412
2413
2414
2415 Put put = new Put(ROW);
2416 put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2417 put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2418 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2419 ht.put(put);
2420
2421
2422 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2423
2424
2425 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2426
2427
2428 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2429
2430
2431 TEST_UTIL.flush();
2432 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2433 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2434 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2435
2436
2437 put = new Put(ROW);
2438 put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2439 put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2440 ht.put(put);
2441
2442
2443 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2444 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2445 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2446 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2447 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2448 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2449 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2450
2451
2452 TEST_UTIL.flush();
2453 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2454 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2455 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2456 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2457 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2458 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2459 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2460
2461 }
2462
2463
2464
2465
2466
2467
2468 @Test
2469 public void testJiraTest33() throws Exception {
2470
2471 byte [] TABLE = Bytes.toBytes("testJiraTest33");
2472 byte [][] VALUES = makeNAscii(VALUE, 7);
2473 long [] STAMPS = makeStamps(7);
2474
2475 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2476
2477
2478
2479 Put put = new Put(ROW);
2480 put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2481 put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2482 put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2483 put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2484 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2485 put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2486 ht.put(put);
2487
2488 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2489 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2490 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2491 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2492
2493 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2494 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2495 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2496 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2497
2498
2499 TEST_UTIL.flush();
2500
2501 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2502 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2503 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2504 getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2505
2506 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2507 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2508 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2509 scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2510
2511 }
2512
2513
2514
2515
2516
2517 @Test
2518 public void testJiraTest1014() throws Exception {
2519
2520 byte [] TABLE = Bytes.toBytes("testJiraTest1014");
2521
2522 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2523
2524 long manualStamp = 12345;
2525
2526
2527
2528 Put put = new Put(ROW);
2529 put.add(FAMILY, QUALIFIER, manualStamp, VALUE);
2530 ht.put(put);
2531
2532 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, manualStamp, VALUE);
2533 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp-1);
2534 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp+1);
2535
2536 }
2537
2538
2539
2540
2541
2542 @Test
2543 public void testJiraTest1182() throws Exception {
2544
2545 byte [] TABLE = Bytes.toBytes("testJiraTest1182");
2546 byte [][] VALUES = makeNAscii(VALUE, 7);
2547 long [] STAMPS = makeStamps(7);
2548
2549 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2550
2551
2552
2553 Put put = new Put(ROW);
2554 put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2555 put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2556 put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2557 put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2558 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2559 put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2560 ht.put(put);
2561
2562 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2563 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2564 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2565
2566 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2567 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2568 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2569
2570
2571 TEST_UTIL.flush();
2572
2573 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2574 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2575 getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2576
2577 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2578 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2579 scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2580 }
2581
2582
2583
2584
2585
2586 @Test
2587 public void testJiraTest52() throws Exception {
2588 byte [] TABLE = Bytes.toBytes("testJiraTest52");
2589 byte [][] VALUES = makeNAscii(VALUE, 7);
2590 long [] STAMPS = makeStamps(7);
2591
2592 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2593
2594
2595
2596 Put put = new Put(ROW);
2597 put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2598 put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2599 put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2600 put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2601 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2602 put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2603 ht.put(put);
2604
2605 getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2606
2607 scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2608
2609
2610 TEST_UTIL.flush();
2611
2612 getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2613
2614 scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2615 }
2616
2617
2618
2619
2620
2621 private void getVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row,
2622 byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
2623 int start, int end)
2624 throws IOException {
2625 Get get = new Get(row);
2626 get.addColumn(family, qualifier);
2627 get.setMaxVersions(Integer.MAX_VALUE);
2628 get.setTimeRange(stamps[start+1], Long.MAX_VALUE);
2629 Result result = ht.get(get);
2630 assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
2631 }
2632
2633 private void getVersionRangeAndVerify(HTable ht, byte [] row, byte [] family,
2634 byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2635 throws IOException {
2636 Get get = new Get(row);
2637 get.addColumn(family, qualifier);
2638 get.setMaxVersions(Integer.MAX_VALUE);
2639 get.setTimeRange(stamps[start], stamps[end]+1);
2640 Result result = ht.get(get);
2641 assertNResult(result, row, family, qualifier, stamps, values, start, end);
2642 }
2643
2644 private void getAllVersionsAndVerify(HTable ht, byte [] row, byte [] family,
2645 byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2646 throws IOException {
2647 Get get = new Get(row);
2648 get.addColumn(family, qualifier);
2649 get.setMaxVersions(Integer.MAX_VALUE);
2650 Result result = ht.get(get);
2651 assertNResult(result, row, family, qualifier, stamps, values, start, end);
2652 }
2653
2654 private void scanVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row,
2655 byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
2656 int start, int end)
2657 throws IOException {
2658 Scan scan = new Scan(row);
2659 scan.addColumn(family, qualifier);
2660 scan.setMaxVersions(Integer.MAX_VALUE);
2661 scan.setTimeRange(stamps[start+1], Long.MAX_VALUE);
2662 Result result = getSingleScanResult(ht, scan);
2663 assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
2664 }
2665
2666 private void scanVersionRangeAndVerify(HTable ht, byte [] row, byte [] family,
2667 byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2668 throws IOException {
2669 Scan scan = new Scan(row);
2670 scan.addColumn(family, qualifier);
2671 scan.setMaxVersions(Integer.MAX_VALUE);
2672 scan.setTimeRange(stamps[start], stamps[end]+1);
2673 Result result = getSingleScanResult(ht, scan);
2674 assertNResult(result, row, family, qualifier, stamps, values, start, end);
2675 }
2676
2677 private void scanAllVersionsAndVerify(HTable ht, byte [] row, byte [] family,
2678 byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2679 throws IOException {
2680 Scan scan = new Scan(row);
2681 scan.addColumn(family, qualifier);
2682 scan.setMaxVersions(Integer.MAX_VALUE);
2683 Result result = getSingleScanResult(ht, scan);
2684 assertNResult(result, row, family, qualifier, stamps, values, start, end);
2685 }
2686
2687 private void getVersionAndVerify(HTable ht, byte [] row, byte [] family,
2688 byte [] qualifier, long stamp, byte [] value)
2689 throws Exception {
2690 Get get = new Get(row);
2691 get.addColumn(family, qualifier);
2692 get.setTimeStamp(stamp);
2693 get.setMaxVersions(Integer.MAX_VALUE);
2694 Result result = ht.get(get);
2695 assertSingleResult(result, row, family, qualifier, stamp, value);
2696 }
2697
2698 private void getVersionAndVerifyMissing(HTable ht, byte [] row, byte [] family,
2699 byte [] qualifier, long stamp)
2700 throws Exception {
2701 Get get = new Get(row);
2702 get.addColumn(family, qualifier);
2703 get.setTimeStamp(stamp);
2704 get.setMaxVersions(Integer.MAX_VALUE);
2705 Result result = ht.get(get);
2706 assertEmptyResult(result);
2707 }
2708
2709 private void scanVersionAndVerify(HTable ht, byte [] row, byte [] family,
2710 byte [] qualifier, long stamp, byte [] value)
2711 throws Exception {
2712 Scan scan = new Scan(row);
2713 scan.addColumn(family, qualifier);
2714 scan.setTimeStamp(stamp);
2715 scan.setMaxVersions(Integer.MAX_VALUE);
2716 Result result = getSingleScanResult(ht, scan);
2717 assertSingleResult(result, row, family, qualifier, stamp, value);
2718 }
2719
2720 private void scanVersionAndVerifyMissing(HTable ht, byte [] row,
2721 byte [] family, byte [] qualifier, long stamp)
2722 throws Exception {
2723 Scan scan = new Scan(row);
2724 scan.addColumn(family, qualifier);
2725 scan.setTimeStamp(stamp);
2726 scan.setMaxVersions(Integer.MAX_VALUE);
2727 Result result = getSingleScanResult(ht, scan);
2728 assertNullResult(result);
2729 }
2730
2731 private void getTestNull(HTable ht, byte [] row, byte [] family,
2732 byte [] value)
2733 throws Exception {
2734
2735 Get get = new Get(row);
2736 get.addColumn(family, null);
2737 Result result = ht.get(get);
2738 assertSingleResult(result, row, family, null, value);
2739
2740 get = new Get(row);
2741 get.addColumn(family, HConstants.EMPTY_BYTE_ARRAY);
2742 result = ht.get(get);
2743 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2744
2745 get = new Get(row);
2746 get.addFamily(family);
2747 result = ht.get(get);
2748 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2749
2750 get = new Get(row);
2751 result = ht.get(get);
2752 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2753
2754 }
2755
2756 private void scanTestNull(HTable ht, byte[] row, byte[] family, byte[] value)
2757 throws Exception {
2758 scanTestNull(ht, row, family, value, false);
2759 }
2760
2761 private void scanTestNull(HTable ht, byte[] row, byte[] family, byte[] value,
2762 boolean isReversedScan) throws Exception {
2763
2764 Scan scan = new Scan();
2765 scan.setReversed(isReversedScan);
2766 scan.addColumn(family, null);
2767 Result result = getSingleScanResult(ht, scan);
2768 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2769
2770 scan = new Scan();
2771 scan.setReversed(isReversedScan);
2772 scan.addColumn(family, HConstants.EMPTY_BYTE_ARRAY);
2773 result = getSingleScanResult(ht, scan);
2774 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2775
2776 scan = new Scan();
2777 scan.setReversed(isReversedScan);
2778 scan.addFamily(family);
2779 result = getSingleScanResult(ht, scan);
2780 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2781
2782 scan = new Scan();
2783 scan.setReversed(isReversedScan);
2784 result = getSingleScanResult(ht, scan);
2785 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2786
2787 }
2788
2789 private void singleRowGetTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
2790 byte [][] QUALIFIERS, byte [][] VALUES)
2791 throws Exception {
2792
2793
2794 Get get = new Get(ROWS[0]);
2795 get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2796 Result result = ht.get(get);
2797 assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
2798
2799
2800 get = new Get(ROWS[0]);
2801 get.addColumn(FAMILIES[2], QUALIFIERS[2]);
2802 result = ht.get(get);
2803 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
2804
2805
2806 get = new Get(ROWS[0]);
2807 get.addFamily(FAMILIES[7]);
2808 result = ht.get(get);
2809 assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
2810
2811
2812
2813 get = new Get(ROWS[0]);
2814 get.addFamily(FAMILIES[4]);
2815 result = ht.get(get);
2816 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2817 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2818
2819
2820
2821 get = new Get(ROWS[0]);
2822 get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2823 get.addColumn(FAMILIES[4], QUALIFIERS[4]);
2824 result = ht.get(get);
2825 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2826 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2827
2828
2829
2830 get = new Get(ROWS[0]);
2831 get.addFamily(FAMILIES[4]);
2832 get.addFamily(FAMILIES[7]);
2833 result = ht.get(get);
2834 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2835 new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
2836
2837
2838 get = new Get(ROWS[0]);
2839 get.addFamily(FAMILIES[2]);
2840 get.addFamily(FAMILIES[4]);
2841 get.addFamily(FAMILIES[6]);
2842 get.addFamily(FAMILIES[7]);
2843 result = ht.get(get);
2844 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2845 new int [][] {
2846 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2847 });
2848
2849
2850 get = new Get(ROWS[0]);
2851 get.addColumn(FAMILIES[2], QUALIFIERS[2]);
2852 get.addColumn(FAMILIES[2], QUALIFIERS[4]);
2853 get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2854 get.addColumn(FAMILIES[4], QUALIFIERS[4]);
2855 get.addColumn(FAMILIES[6], QUALIFIERS[6]);
2856 get.addColumn(FAMILIES[6], QUALIFIERS[7]);
2857 get.addColumn(FAMILIES[7], QUALIFIERS[7]);
2858 get.addColumn(FAMILIES[7], QUALIFIERS[8]);
2859 result = ht.get(get);
2860 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2861 new int [][] {
2862 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2863 });
2864
2865
2866 get = new Get(ROWS[0]);
2867 result = ht.get(get);
2868 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2869 new int [][] {
2870 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
2871 });
2872
2873
2874
2875 get = new Get(ROWS[1]);
2876 result = ht.get(get);
2877 assertEmptyResult(result);
2878
2879 get = new Get(ROWS[0]);
2880 get.addColumn(FAMILIES[4], QUALIFIERS[3]);
2881 get.addColumn(FAMILIES[2], QUALIFIERS[3]);
2882 result = ht.get(get);
2883 assertEmptyResult(result);
2884
2885 }
2886
2887 private void singleRowScanTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
2888 byte [][] QUALIFIERS, byte [][] VALUES)
2889 throws Exception {
2890
2891
2892 Scan scan = new Scan();
2893 scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2894 Result result = getSingleScanResult(ht, scan);
2895 assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
2896
2897
2898 scan = new Scan();
2899 scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
2900 result = getSingleScanResult(ht, scan);
2901 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
2902
2903
2904 scan = new Scan();
2905 scan.addFamily(FAMILIES[7]);
2906 result = getSingleScanResult(ht, scan);
2907 assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
2908
2909
2910
2911 scan = new Scan();
2912 scan.addFamily(FAMILIES[4]);
2913 result = getSingleScanResult(ht, scan);
2914 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2915 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2916
2917
2918
2919 scan = new Scan();
2920 scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2921 scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
2922 result = getSingleScanResult(ht, scan);
2923 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2924 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2925
2926
2927
2928 scan = new Scan();
2929 scan.addFamily(FAMILIES[4]);
2930 scan.addFamily(FAMILIES[7]);
2931 result = getSingleScanResult(ht, scan);
2932 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2933 new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
2934
2935
2936 scan = new Scan();
2937 scan.addFamily(FAMILIES[2]);
2938 scan.addFamily(FAMILIES[4]);
2939 scan.addFamily(FAMILIES[6]);
2940 scan.addFamily(FAMILIES[7]);
2941 result = getSingleScanResult(ht, scan);
2942 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2943 new int [][] {
2944 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2945 });
2946
2947
2948 scan = new Scan();
2949 scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
2950 scan.addColumn(FAMILIES[2], QUALIFIERS[4]);
2951 scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2952 scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
2953 scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
2954 scan.addColumn(FAMILIES[6], QUALIFIERS[7]);
2955 scan.addColumn(FAMILIES[7], QUALIFIERS[7]);
2956 scan.addColumn(FAMILIES[7], QUALIFIERS[8]);
2957 result = getSingleScanResult(ht, scan);
2958 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2959 new int [][] {
2960 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2961 });
2962
2963
2964 scan = new Scan();
2965 result = getSingleScanResult(ht, scan);
2966 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2967 new int [][] {
2968 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
2969 });
2970
2971
2972
2973 scan = new Scan(ROWS[1]);
2974 result = getSingleScanResult(ht, scan);
2975 assertNullResult(result);
2976
2977 scan = new Scan();
2978 scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
2979 scan.addColumn(FAMILIES[2], QUALIFIERS[3]);
2980 result = getSingleScanResult(ht, scan);
2981 assertNullResult(result);
2982 }
2983
2984
2985
2986
2987
2988
2989 private void getVerifySingleColumn(HTable ht,
2990 byte [][] ROWS, int ROWIDX,
2991 byte [][] FAMILIES, int FAMILYIDX,
2992 byte [][] QUALIFIERS, int QUALIFIERIDX,
2993 byte [][] VALUES, int VALUEIDX)
2994 throws Exception {
2995
2996 Get get = new Get(ROWS[ROWIDX]);
2997 Result result = ht.get(get);
2998 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2999 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3000
3001 get = new Get(ROWS[ROWIDX]);
3002 get.addFamily(FAMILIES[FAMILYIDX]);
3003 result = ht.get(get);
3004 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3005 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3006
3007 get = new Get(ROWS[ROWIDX]);
3008 get.addFamily(FAMILIES[FAMILYIDX-2]);
3009 get.addFamily(FAMILIES[FAMILYIDX]);
3010 get.addFamily(FAMILIES[FAMILYIDX+2]);
3011 result = ht.get(get);
3012 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3013 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3014
3015 get = new Get(ROWS[ROWIDX]);
3016 get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[0]);
3017 result = ht.get(get);
3018 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3019 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3020
3021 get = new Get(ROWS[ROWIDX]);
3022 get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[1]);
3023 get.addFamily(FAMILIES[FAMILYIDX]);
3024 result = ht.get(get);
3025 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3026 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3027
3028 get = new Get(ROWS[ROWIDX]);
3029 get.addFamily(FAMILIES[FAMILYIDX]);
3030 get.addColumn(FAMILIES[FAMILYIDX+1], QUALIFIERS[1]);
3031 get.addColumn(FAMILIES[FAMILYIDX-2], QUALIFIERS[1]);
3032 get.addFamily(FAMILIES[FAMILYIDX-1]);
3033 get.addFamily(FAMILIES[FAMILYIDX+2]);
3034 result = ht.get(get);
3035 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3036 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3037
3038 }
3039
3040
3041
3042
3043
3044
3045
3046
3047 private void scanVerifySingleColumn(HTable ht,
3048 byte [][] ROWS, int ROWIDX,
3049 byte [][] FAMILIES, int FAMILYIDX,
3050 byte [][] QUALIFIERS, int QUALIFIERIDX,
3051 byte [][] VALUES, int VALUEIDX)
3052 throws Exception {
3053
3054 Scan scan = new Scan();
3055 Result result = getSingleScanResult(ht, scan);
3056 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3057 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3058
3059 scan = new Scan(ROWS[ROWIDX]);
3060 result = getSingleScanResult(ht, scan);
3061 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3062 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3063
3064 scan = new Scan(ROWS[ROWIDX], ROWS[ROWIDX+1]);
3065 result = getSingleScanResult(ht, scan);
3066 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3067 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3068
3069 scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX+1]);
3070 result = getSingleScanResult(ht, scan);
3071 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3072 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3073
3074 scan = new Scan();
3075 scan.addFamily(FAMILIES[FAMILYIDX]);
3076 result = getSingleScanResult(ht, scan);
3077 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3078 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3079
3080 scan = new Scan();
3081 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
3082 result = getSingleScanResult(ht, scan);
3083 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3084 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3085
3086 scan = new Scan();
3087 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
3088 scan.addFamily(FAMILIES[FAMILYIDX]);
3089 result = getSingleScanResult(ht, scan);
3090 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3091 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3092
3093 scan = new Scan();
3094 scan.addColumn(FAMILIES[FAMILYIDX-1], QUALIFIERS[QUALIFIERIDX+1]);
3095 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
3096 scan.addFamily(FAMILIES[FAMILYIDX+1]);
3097 result = getSingleScanResult(ht, scan);
3098 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3099 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3100
3101 }
3102
3103
3104
3105
3106
3107 private void getVerifySingleEmpty(HTable ht,
3108 byte [][] ROWS, int ROWIDX,
3109 byte [][] FAMILIES, int FAMILYIDX,
3110 byte [][] QUALIFIERS, int QUALIFIERIDX)
3111 throws Exception {
3112
3113 Get get = new Get(ROWS[ROWIDX]);
3114 get.addFamily(FAMILIES[4]);
3115 get.addColumn(FAMILIES[4], QUALIFIERS[1]);
3116 Result result = ht.get(get);
3117 assertEmptyResult(result);
3118
3119 get = new Get(ROWS[ROWIDX]);
3120 get.addFamily(FAMILIES[4]);
3121 get.addColumn(FAMILIES[4], QUALIFIERS[2]);
3122 result = ht.get(get);
3123 assertEmptyResult(result);
3124
3125 get = new Get(ROWS[ROWIDX]);
3126 get.addFamily(FAMILIES[3]);
3127 get.addColumn(FAMILIES[4], QUALIFIERS[2]);
3128 get.addFamily(FAMILIES[5]);
3129 result = ht.get(get);
3130 assertEmptyResult(result);
3131
3132 get = new Get(ROWS[ROWIDX+1]);
3133 result = ht.get(get);
3134 assertEmptyResult(result);
3135
3136 }
3137
3138 private void scanVerifySingleEmpty(HTable ht,
3139 byte [][] ROWS, int ROWIDX,
3140 byte [][] FAMILIES, int FAMILYIDX,
3141 byte [][] QUALIFIERS, int QUALIFIERIDX)
3142 throws Exception {
3143
3144 Scan scan = new Scan(ROWS[ROWIDX+1]);
3145 Result result = getSingleScanResult(ht, scan);
3146 assertNullResult(result);
3147
3148 scan = new Scan(ROWS[ROWIDX+1],ROWS[ROWIDX+2]);
3149 result = getSingleScanResult(ht, scan);
3150 assertNullResult(result);
3151
3152 scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX]);
3153 result = getSingleScanResult(ht, scan);
3154 assertNullResult(result);
3155
3156 scan = new Scan();
3157 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
3158 scan.addFamily(FAMILIES[FAMILYIDX-1]);
3159 result = getSingleScanResult(ht, scan);
3160 assertNullResult(result);
3161
3162 }
3163
3164
3165
3166
3167
3168 private void assertKey(Cell key, byte [] row, byte [] family,
3169 byte [] qualifier, byte [] value)
3170 throws Exception {
3171 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3172 "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
3173 equals(row, CellUtil.cloneRow(key)));
3174 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3175 "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3176 equals(family, CellUtil.cloneFamily(key)));
3177 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3178 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]",
3179 equals(qualifier, CellUtil.cloneQualifier(key)));
3180 assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3181 "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]",
3182 equals(value, CellUtil.cloneValue(key)));
3183 }
3184
3185 private void assertIncrementKey(Cell key, byte [] row, byte [] family,
3186 byte [] qualifier, long value)
3187 throws Exception {
3188 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3189 "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
3190 equals(row, CellUtil.cloneRow(key)));
3191 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3192 "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3193 equals(family, CellUtil.cloneFamily(key)));
3194 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3195 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]",
3196 equals(qualifier, CellUtil.cloneQualifier(key)));
3197 assertTrue("Expected value [" + value + "] " +
3198 "Got value [" + Bytes.toLong(CellUtil.cloneValue(key)) + "]",
3199 Bytes.toLong(CellUtil.cloneValue(key)) == value);
3200 }
3201
3202 private void assertNumKeys(Result result, int n) throws Exception {
3203 assertTrue("Expected " + n + " keys but got " + result.size(),
3204 result.size() == n);
3205 }
3206
3207 private void assertNResult(Result result, byte [] row,
3208 byte [][] families, byte [][] qualifiers, byte [][] values,
3209 int [][] idxs)
3210 throws Exception {
3211 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3212 "Got row [" + Bytes.toString(result.getRow()) +"]",
3213 equals(row, result.getRow()));
3214 assertTrue("Expected " + idxs.length + " keys but result contains "
3215 + result.size(), result.size() == idxs.length);
3216
3217 Cell [] keys = result.rawCells();
3218
3219 for(int i=0;i<keys.length;i++) {
3220 byte [] family = families[idxs[i][0]];
3221 byte [] qualifier = qualifiers[idxs[i][1]];
3222 byte [] value = values[idxs[i][2]];
3223 Cell key = keys[i];
3224
3225 byte[] famb = CellUtil.cloneFamily(key);
3226 byte[] qualb = CellUtil.cloneQualifier(key);
3227 byte[] valb = CellUtil.cloneValue(key);
3228 assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
3229 + "] " + "Got family [" + Bytes.toString(famb) + "]",
3230 equals(family, famb));
3231 assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
3232 + "] " + "Got qualifier [" + Bytes.toString(qualb) + "]",
3233 equals(qualifier, qualb));
3234 assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
3235 + "Got value [" + Bytes.toString(valb) + "]",
3236 equals(value, valb));
3237 }
3238 }
3239
3240 private void assertNResult(Result result, byte [] row,
3241 byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
3242 int start, int end)
3243 throws IOException {
3244 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3245 "Got row [" + Bytes.toString(result.getRow()) +"]",
3246 equals(row, result.getRow()));
3247 int expectedResults = end - start + 1;
3248 assertEquals(expectedResults, result.size());
3249
3250 Cell[] keys = result.rawCells();
3251
3252 for (int i=0; i<keys.length; i++) {
3253 byte [] value = values[end-i];
3254 long ts = stamps[end-i];
3255 Cell key = keys[i];
3256
3257 assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
3258 + "] " + "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3259 CellUtil.matchingFamily(key, family));
3260 assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
3261 + "] " + "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key))+ "]",
3262 CellUtil.matchingQualifier(key, qualifier));
3263 assertTrue("Expected ts [" + ts + "] " +
3264 "Got ts [" + key.getTimestamp() + "]", ts == key.getTimestamp());
3265 assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
3266 + "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]",
3267 CellUtil.matchingValue(key, value));
3268 }
3269 }
3270
3271
3272
3273
3274
3275 private void assertDoubleResult(Result result, byte [] row,
3276 byte [] familyA, byte [] qualifierA, byte [] valueA,
3277 byte [] familyB, byte [] qualifierB, byte [] valueB)
3278 throws Exception {
3279 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3280 "Got row [" + Bytes.toString(result.getRow()) +"]",
3281 equals(row, result.getRow()));
3282 assertTrue("Expected two keys but result contains " + result.size(),
3283 result.size() == 2);
3284 Cell [] kv = result.rawCells();
3285 Cell kvA = kv[0];
3286 assertTrue("(A) Expected family [" + Bytes.toString(familyA) + "] " +
3287 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvA)) + "]",
3288 equals(familyA, CellUtil.cloneFamily(kvA)));
3289 assertTrue("(A) Expected qualifier [" + Bytes.toString(qualifierA) + "] " +
3290 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvA)) + "]",
3291 equals(qualifierA, CellUtil.cloneQualifier(kvA)));
3292 assertTrue("(A) Expected value [" + Bytes.toString(valueA) + "] " +
3293 "Got value [" + Bytes.toString(CellUtil.cloneValue(kvA)) + "]",
3294 equals(valueA, CellUtil.cloneValue(kvA)));
3295 Cell kvB = kv[1];
3296 assertTrue("(B) Expected family [" + Bytes.toString(familyB) + "] " +
3297 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvB)) + "]",
3298 equals(familyB, CellUtil.cloneFamily(kvB)));
3299 assertTrue("(B) Expected qualifier [" + Bytes.toString(qualifierB) + "] " +
3300 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvB)) + "]",
3301 equals(qualifierB, CellUtil.cloneQualifier(kvB)));
3302 assertTrue("(B) Expected value [" + Bytes.toString(valueB) + "] " +
3303 "Got value [" + Bytes.toString(CellUtil.cloneValue(kvB)) + "]",
3304 equals(valueB, CellUtil.cloneValue(kvB)));
3305 }
3306
3307 private void assertSingleResult(Result result, byte [] row, byte [] family,
3308 byte [] qualifier, byte [] value)
3309 throws Exception {
3310 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3311 "Got row [" + Bytes.toString(result.getRow()) +"]",
3312 equals(row, result.getRow()));
3313 assertTrue("Expected a single key but result contains " + result.size(),
3314 result.size() == 1);
3315 Cell kv = result.rawCells()[0];
3316 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3317 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]",
3318 equals(family, CellUtil.cloneFamily(kv)));
3319 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3320 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]",
3321 equals(qualifier, CellUtil.cloneQualifier(kv)));
3322 assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3323 "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]",
3324 equals(value, CellUtil.cloneValue(kv)));
3325 }
3326
3327 private void assertSingleResult(Result result, byte [] row, byte [] family,
3328 byte [] qualifier, long ts, byte [] value)
3329 throws Exception {
3330 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3331 "Got row [" + Bytes.toString(result.getRow()) +"]",
3332 equals(row, result.getRow()));
3333 assertTrue("Expected a single key but result contains " + result.size(),
3334 result.size() == 1);
3335 Cell kv = result.rawCells()[0];
3336 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3337 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]",
3338 equals(family, CellUtil.cloneFamily(kv)));
3339 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3340 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]",
3341 equals(qualifier, CellUtil.cloneQualifier(kv)));
3342 assertTrue("Expected ts [" + ts + "] " +
3343 "Got ts [" + kv.getTimestamp() + "]", ts == kv.getTimestamp());
3344 assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3345 "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]",
3346 equals(value, CellUtil.cloneValue(kv)));
3347 }
3348
3349 private void assertEmptyResult(Result result) throws Exception {
3350 assertTrue("expected an empty result but result contains " +
3351 result.size() + " keys", result.isEmpty());
3352 }
3353
3354 private void assertNullResult(Result result) throws Exception {
3355 assertTrue("expected null result but received a non-null result",
3356 result == null);
3357 }
3358
3359
3360
3361
3362
3363 private Result getSingleScanResult(HTable ht, Scan scan) throws IOException {
3364 ResultScanner scanner = ht.getScanner(scan);
3365 Result result = scanner.next();
3366 scanner.close();
3367 return result;
3368 }
3369
3370 private byte [][] makeNAscii(byte [] base, int n) {
3371 if(n > 256) {
3372 return makeNBig(base, n);
3373 }
3374 byte [][] ret = new byte[n][];
3375 for(int i=0;i<n;i++) {
3376 byte [] tail = Bytes.toBytes(Integer.toString(i));
3377 ret[i] = Bytes.add(base, tail);
3378 }
3379 return ret;
3380 }
3381
3382 private byte [][] makeN(byte [] base, int n) {
3383 if (n > 256) {
3384 return makeNBig(base, n);
3385 }
3386 byte [][] ret = new byte[n][];
3387 for(int i=0;i<n;i++) {
3388 ret[i] = Bytes.add(base, new byte[]{(byte)i});
3389 }
3390 return ret;
3391 }
3392
3393 private byte [][] makeNBig(byte [] base, int n) {
3394 byte [][] ret = new byte[n][];
3395 for(int i=0;i<n;i++) {
3396 int byteA = (i % 256);
3397 int byteB = (i >> 8);
3398 ret[i] = Bytes.add(base, new byte[]{(byte)byteB,(byte)byteA});
3399 }
3400 return ret;
3401 }
3402
3403 private long [] makeStamps(int n) {
3404 long [] stamps = new long[n];
3405 for(int i=0;i<n;i++) stamps[i] = i+1;
3406 return stamps;
3407 }
3408
3409 private boolean equals(byte [] left, byte [] right) {
3410 if (left == null && right == null) return true;
3411 if (left == null && right.length == 0) return true;
3412 if (right == null && left.length == 0) return true;
3413 return Bytes.equals(left, right);
3414 }
3415
3416 @Test
3417 public void testDuplicateVersions() throws Exception {
3418 byte [] TABLE = Bytes.toBytes("testDuplicateVersions");
3419
3420 long [] STAMPS = makeStamps(20);
3421 byte [][] VALUES = makeNAscii(VALUE, 20);
3422
3423 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3424
3425
3426 Put put = new Put(ROW);
3427 put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3428 put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3429 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3430 put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3431 ht.put(put);
3432
3433
3434 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3435 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3436 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3437 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3438 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3439 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3440 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3441 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3442
3443
3444 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3445 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3446 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3447 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3448 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3449 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3450
3451
3452 Get get = new Get(ROW);
3453 get.addColumn(FAMILY, QUALIFIER);
3454 get.setMaxVersions(2);
3455 Result result = ht.get(get);
3456 assertNResult(result, ROW, FAMILY, QUALIFIER,
3457 new long [] {STAMPS[4], STAMPS[5]},
3458 new byte[][] {VALUES[4], VALUES[5]},
3459 0, 1);
3460
3461 Scan scan = new Scan(ROW);
3462 scan.addColumn(FAMILY, QUALIFIER);
3463 scan.setMaxVersions(2);
3464 result = getSingleScanResult(ht, scan);
3465 assertNResult(result, ROW, FAMILY, QUALIFIER,
3466 new long [] {STAMPS[4], STAMPS[5]},
3467 new byte[][] {VALUES[4], VALUES[5]},
3468 0, 1);
3469
3470
3471
3472 TEST_UTIL.flush();
3473
3474
3475 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3476 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3477 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3478 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3479 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3480 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3481 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3482 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3483
3484
3485 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3486 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3487 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3488 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3489 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3490 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3491
3492
3493 get = new Get(ROW);
3494 get.addColumn(FAMILY, QUALIFIER);
3495 get.setMaxVersions(2);
3496 result = ht.get(get);
3497 assertNResult(result, ROW, FAMILY, QUALIFIER,
3498 new long [] {STAMPS[4], STAMPS[5]},
3499 new byte[][] {VALUES[4], VALUES[5]},
3500 0, 1);
3501
3502 scan = new Scan(ROW);
3503 scan.addColumn(FAMILY, QUALIFIER);
3504 scan.setMaxVersions(2);
3505 result = getSingleScanResult(ht, scan);
3506 assertNResult(result, ROW, FAMILY, QUALIFIER,
3507 new long [] {STAMPS[4], STAMPS[5]},
3508 new byte[][] {VALUES[4], VALUES[5]},
3509 0, 1);
3510
3511
3512
3513
3514
3515 put = new Put(ROW);
3516 put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
3517 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3518 put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
3519 put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3520 put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]);
3521 ht.put(put);
3522
3523
3524 get = new Get(ROW);
3525 get.addColumn(FAMILY, QUALIFIER);
3526 get.setMaxVersions(7);
3527 result = ht.get(get);
3528 assertNResult(result, ROW, FAMILY, QUALIFIER,
3529 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3530 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3531 0, 6);
3532
3533 scan = new Scan(ROW);
3534 scan.addColumn(FAMILY, QUALIFIER);
3535 scan.setMaxVersions(7);
3536 result = getSingleScanResult(ht, scan);
3537 assertNResult(result, ROW, FAMILY, QUALIFIER,
3538 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3539 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3540 0, 6);
3541
3542 get = new Get(ROW);
3543 get.setMaxVersions(7);
3544 result = ht.get(get);
3545 assertNResult(result, ROW, FAMILY, QUALIFIER,
3546 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3547 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3548 0, 6);
3549
3550 scan = new Scan(ROW);
3551 scan.setMaxVersions(7);
3552 result = getSingleScanResult(ht, scan);
3553 assertNResult(result, ROW, FAMILY, QUALIFIER,
3554 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3555 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3556 0, 6);
3557
3558
3559 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3560 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3561 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3562 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3563 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3564 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3565 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3566 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3567
3568
3569 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3570 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3571 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3572 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3573
3574
3575
3576 TEST_UTIL.flush();
3577
3578
3579 put = new Put(ROW);
3580 put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]);
3581 put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]);
3582 put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]);
3583 put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
3584 ht.put(put);
3585
3586 get = new Get(ROW);
3587 get.addColumn(FAMILY, QUALIFIER);
3588 get.setMaxVersions(Integer.MAX_VALUE);
3589 result = ht.get(get);
3590 assertNResult(result, ROW, FAMILY, QUALIFIER,
3591 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3592 new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3593 0, 9);
3594
3595 scan = new Scan(ROW);
3596 scan.addColumn(FAMILY, QUALIFIER);
3597 scan.setMaxVersions(Integer.MAX_VALUE);
3598 result = getSingleScanResult(ht, scan);
3599 assertNResult(result, ROW, FAMILY, QUALIFIER,
3600 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3601 new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3602 0, 9);
3603
3604
3605 Delete delete = new Delete(ROW);
3606 delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
3607 delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
3608 ht.delete(delete);
3609
3610
3611 get = new Get(ROW);
3612 get.addColumn(FAMILY, QUALIFIER);
3613 get.setMaxVersions(Integer.MAX_VALUE);
3614 result = ht.get(get);
3615 assertNResult(result, ROW, FAMILY, QUALIFIER,
3616 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3617 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3618 0, 9);
3619
3620 scan = new Scan(ROW);
3621 scan.addColumn(FAMILY, QUALIFIER);
3622 scan.setMaxVersions(Integer.MAX_VALUE);
3623 result = getSingleScanResult(ht, scan);
3624 assertNResult(result, ROW, FAMILY, QUALIFIER,
3625 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3626 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3627 0, 9);
3628 }
3629
3630 @Test
3631 public void testUpdates() throws Exception {
3632
3633 byte [] TABLE = Bytes.toBytes("testUpdates");
3634 HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3635
3636
3637 byte[] row = Bytes.toBytes("row1");
3638 byte[] qualifier = Bytes.toBytes("myCol");
3639 Put put = new Put(row);
3640 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3641 hTable.put(put);
3642
3643 put = new Put(row);
3644 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3645 hTable.put(put);
3646
3647 put = new Put(row);
3648 put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3649 hTable.put(put);
3650
3651 Get get = new Get(row);
3652 get.addColumn(FAMILY, qualifier);
3653 get.setMaxVersions();
3654
3655
3656
3657 Result result = hTable.get(get);
3658 NavigableMap<Long, byte[]> navigableMap =
3659 result.getMap().get(FAMILY).get(qualifier);
3660 assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3661 assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3662
3663
3664 put = new Put(row);
3665 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3666 hTable.put(put);
3667
3668
3669 put = new Put(row);
3670 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3671 hTable.put(put);
3672
3673
3674 result = hTable.get(get);
3675 navigableMap = result.getMap().get(FAMILY).get(qualifier);
3676 assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3677 assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3678 }
3679
3680 @Test
3681 public void testUpdatesWithMajorCompaction() throws Exception {
3682
3683 String tableName = "testUpdatesWithMajorCompaction";
3684 byte [] TABLE = Bytes.toBytes(tableName);
3685 HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3686 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3687
3688
3689 byte[] row = Bytes.toBytes("row2");
3690 byte[] qualifier = Bytes.toBytes("myCol");
3691 Put put = new Put(row);
3692 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3693 hTable.put(put);
3694
3695 put = new Put(row);
3696 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3697 hTable.put(put);
3698
3699 put = new Put(row);
3700 put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3701 hTable.put(put);
3702
3703 Get get = new Get(row);
3704 get.addColumn(FAMILY, qualifier);
3705 get.setMaxVersions();
3706
3707
3708
3709 Result result = hTable.get(get);
3710 NavigableMap<Long, byte[]> navigableMap =
3711 result.getMap().get(FAMILY).get(qualifier);
3712 assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3713 assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3714
3715
3716 admin.flush(tableName);
3717 admin.majorCompact(tableName);
3718 Thread.sleep(6000);
3719
3720
3721 put = new Put(row);
3722 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3723 hTable.put(put);
3724
3725
3726 put = new Put(row);
3727 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3728 hTable.put(put);
3729
3730
3731 admin.flush(tableName);
3732 admin.majorCompact(tableName);
3733 Thread.sleep(6000);
3734
3735
3736 result = hTable.get(get);
3737 navigableMap = result.getMap().get(FAMILY).get(qualifier);
3738 assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3739 assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3740 }
3741
3742 @Test
3743 public void testMajorCompactionBetweenTwoUpdates() throws Exception {
3744
3745 String tableName = "testMajorCompactionBetweenTwoUpdates";
3746 byte [] TABLE = Bytes.toBytes(tableName);
3747 HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3748 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3749
3750
3751 byte[] row = Bytes.toBytes("row3");
3752 byte[] qualifier = Bytes.toBytes("myCol");
3753 Put put = new Put(row);
3754 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3755 hTable.put(put);
3756
3757 put = new Put(row);
3758 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3759 hTable.put(put);
3760
3761 put = new Put(row);
3762 put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3763 hTable.put(put);
3764
3765 Get get = new Get(row);
3766 get.addColumn(FAMILY, qualifier);
3767 get.setMaxVersions();
3768
3769
3770
3771 Result result = hTable.get(get);
3772 NavigableMap<Long, byte[]> navigableMap =
3773 result.getMap().get(FAMILY).get(qualifier);
3774 assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3775 assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3776
3777
3778 admin.flush(tableName);
3779 admin.majorCompact(tableName);
3780 Thread.sleep(6000);
3781
3782
3783 put = new Put(row);
3784 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3785 hTable.put(put);
3786
3787
3788 admin.flush(tableName);
3789 admin.majorCompact(tableName);
3790 Thread.sleep(6000);
3791
3792
3793 put = new Put(row);
3794 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3795 hTable.put(put);
3796
3797
3798 admin.flush(tableName);
3799 admin.majorCompact(tableName);
3800 Thread.sleep(6000);
3801
3802
3803 result = hTable.get(get);
3804 navigableMap = result.getMap().get(FAMILY).get(qualifier);
3805
3806 assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3807 assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3808 }
3809
3810 @Test
3811 public void testGet_EmptyTable() throws IOException {
3812 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY);
3813 Get get = new Get(ROW);
3814 get.addFamily(FAMILY);
3815 Result r = table.get(get);
3816 assertTrue(r.isEmpty());
3817 }
3818
3819 @Test
3820 public void testGet_NullQualifier() throws IOException {
3821 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NullQualifier"), FAMILY);
3822 Put put = new Put(ROW);
3823 put.add(FAMILY, QUALIFIER, VALUE);
3824 table.put(put);
3825
3826 put = new Put(ROW);
3827 put.add(FAMILY, null, VALUE);
3828 table.put(put);
3829 LOG.info("Row put");
3830
3831 Get get = new Get(ROW);
3832 get.addColumn(FAMILY, null);
3833 Result r = table.get(get);
3834 assertEquals(1, r.size());
3835
3836 get = new Get(ROW);
3837 get.addFamily(FAMILY);
3838 r = table.get(get);
3839 assertEquals(2, r.size());
3840 }
3841
3842 @Test
3843 public void testGet_NonExistentRow() throws IOException {
3844 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY);
3845 Put put = new Put(ROW);
3846 put.add(FAMILY, QUALIFIER, VALUE);
3847 table.put(put);
3848 LOG.info("Row put");
3849
3850 Get get = new Get(ROW);
3851 get.addFamily(FAMILY);
3852 Result r = table.get(get);
3853 assertFalse(r.isEmpty());
3854 System.out.println("Row retrieved successfully");
3855
3856 byte [] missingrow = Bytes.toBytes("missingrow");
3857 get = new Get(missingrow);
3858 get.addFamily(FAMILY);
3859 r = table.get(get);
3860 assertTrue(r.isEmpty());
3861 LOG.info("Row missing as it should be");
3862 }
3863
3864 @Test
3865 public void testPut() throws IOException {
3866 final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
3867 final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
3868 final byte [] row1 = Bytes.toBytes("row1");
3869 final byte [] row2 = Bytes.toBytes("row2");
3870 final byte [] value = Bytes.toBytes("abcd");
3871 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
3872 new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
3873 Put put = new Put(row1);
3874 put.add(CONTENTS_FAMILY, null, value);
3875 table.put(put);
3876
3877 put = new Put(row2);
3878 put.add(CONTENTS_FAMILY, null, value);
3879
3880 assertEquals(put.size(), 1);
3881 assertEquals(put.getFamilyCellMap().get(CONTENTS_FAMILY).size(), 1);
3882
3883
3884 KeyValue kv = (KeyValue)put.getFamilyCellMap().get(CONTENTS_FAMILY).get(0);
3885
3886 assertTrue(Bytes.equals(kv.getFamily(), CONTENTS_FAMILY));
3887
3888 assertTrue(Bytes.equals(kv.getQualifier(), new byte[0]));
3889
3890 assertTrue(Bytes.equals(kv.getValue(), value));
3891
3892 table.put(put);
3893
3894 Scan scan = new Scan();
3895 scan.addColumn(CONTENTS_FAMILY, null);
3896 ResultScanner scanner = table.getScanner(scan);
3897 for (Result r : scanner) {
3898 for(Cell key : r.rawCells()) {
3899 System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString());
3900 }
3901 }
3902 }
3903
3904 @Test
3905 public void testPutNoCF() throws IOException {
3906 final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
3907 final byte[] VAL = Bytes.toBytes(100);
3908 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY});
3909
3910 boolean caughtNSCFE = false;
3911
3912 try {
3913 Put p = new Put(ROW);
3914 p.add(BAD_FAM, QUALIFIER, VAL);
3915 table.put(p);
3916 } catch (RetriesExhaustedWithDetailsException e) {
3917 caughtNSCFE = e.getCause(0) instanceof NoSuchColumnFamilyException;
3918 }
3919 assertTrue("Should throw NoSuchColumnFamilyException", caughtNSCFE);
3920
3921 }
3922
3923 @Test
3924 public void testRowsPut() throws IOException {
3925 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3926 final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
3927 final int NB_BATCH_ROWS = 10;
3928 final byte[] value = Bytes.toBytes("abcd");
3929 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
3930 new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
3931 ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3932 for (int i = 0; i < NB_BATCH_ROWS; i++) {
3933 byte[] row = Bytes.toBytes("row" + i);
3934 Put put = new Put(row);
3935 put.setDurability(Durability.SKIP_WAL);
3936 put.add(CONTENTS_FAMILY, null, value);
3937 rowsUpdate.add(put);
3938 }
3939 table.put(rowsUpdate);
3940 Scan scan = new Scan();
3941 scan.addFamily(CONTENTS_FAMILY);
3942 ResultScanner scanner = table.getScanner(scan);
3943 int nbRows = 0;
3944 for (@SuppressWarnings("unused")
3945 Result row : scanner)
3946 nbRows++;
3947 assertEquals(NB_BATCH_ROWS, nbRows);
3948 }
3949
3950 @Test
3951 public void testRowsPutBufferedOneFlush() throws IOException {
3952 final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
3953 final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
3954 final byte [] value = Bytes.toBytes("abcd");
3955 final int NB_BATCH_ROWS = 10;
3956 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
3957 new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
3958 table.setAutoFlush(false, true);
3959 ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3960 for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
3961 byte[] row = Bytes.toBytes("row" + i);
3962 Put put = new Put(row);
3963 put.setDurability(Durability.SKIP_WAL);
3964 put.add(CONTENTS_FAMILY, null, value);
3965 rowsUpdate.add(put);
3966 }
3967 table.put(rowsUpdate);
3968
3969 Scan scan = new Scan();
3970 scan.addFamily(CONTENTS_FAMILY);
3971 ResultScanner scanner = table.getScanner(scan);
3972 int nbRows = 0;
3973 for (@SuppressWarnings("unused")
3974 Result row : scanner)
3975 nbRows++;
3976 assertEquals(0, nbRows);
3977 scanner.close();
3978
3979 table.flushCommits();
3980
3981 scan = new Scan();
3982 scan.addFamily(CONTENTS_FAMILY);
3983 scanner = table.getScanner(scan);
3984 nbRows = 0;
3985 for (@SuppressWarnings("unused")
3986 Result row : scanner)
3987 nbRows++;
3988 assertEquals(NB_BATCH_ROWS * 10, nbRows);
3989 }
3990
3991 @Test
3992 public void testRowsPutBufferedManyManyFlushes() throws IOException {
3993 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3994 final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
3995 final byte[] value = Bytes.toBytes("abcd");
3996 final int NB_BATCH_ROWS = 10;
3997 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
3998 new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
3999 table.setAutoFlush(false, true);
4000 table.setWriteBufferSize(10);
4001 ArrayList<Put> rowsUpdate = new ArrayList<Put>();
4002 for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
4003 byte[] row = Bytes.toBytes("row" + i);
4004 Put put = new Put(row);
4005 put.setDurability(Durability.SKIP_WAL);
4006 put.add(CONTENTS_FAMILY, null, value);
4007 rowsUpdate.add(put);
4008 }
4009 table.put(rowsUpdate);
4010
4011 table.flushCommits();
4012
4013 Scan scan = new Scan();
4014 scan.addFamily(CONTENTS_FAMILY);
4015 ResultScanner scanner = table.getScanner(scan);
4016 int nbRows = 0;
4017 for (@SuppressWarnings("unused")
4018 Result row : scanner)
4019 nbRows++;
4020 assertEquals(NB_BATCH_ROWS * 10, nbRows);
4021 }
4022
4023 @Test
4024 public void testAddKeyValue() throws IOException {
4025 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
4026 final byte[] value = Bytes.toBytes("abcd");
4027 final byte[] row1 = Bytes.toBytes("row1");
4028 final byte[] row2 = Bytes.toBytes("row2");
4029 byte[] qualifier = Bytes.toBytes("qf1");
4030 Put put = new Put(row1);
4031
4032
4033 KeyValue kv = new KeyValue(row1, CONTENTS_FAMILY, qualifier, value);
4034 boolean ok = true;
4035 try {
4036 put.add(kv);
4037 } catch (IOException e) {
4038 ok = false;
4039 }
4040 assertEquals(true, ok);
4041
4042
4043 kv = new KeyValue(row2, CONTENTS_FAMILY, qualifier, value);
4044 ok = false;
4045 try {
4046 put.add(kv);
4047 } catch (IOException e) {
4048 ok = true;
4049 }
4050 assertEquals(true, ok);
4051 }
4052
4053
4054
4055
4056
4057 @Test
4058 public void testHBase737 () throws IOException {
4059 final byte [] FAM1 = Bytes.toBytes("fam1");
4060 final byte [] FAM2 = Bytes.toBytes("fam2");
4061
4062 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
4063 new byte [][] {FAM1, FAM2});
4064
4065 Put put = new Put(ROW);
4066 put.add(FAM1, Bytes.toBytes("letters"), Bytes.toBytes("abcdefg"));
4067 table.put(put);
4068 try {
4069 Thread.sleep(1000);
4070 } catch (InterruptedException i) {
4071
4072 }
4073
4074 put = new Put(ROW);
4075 put.add(FAM1, Bytes.toBytes("numbers"), Bytes.toBytes("123456"));
4076 table.put(put);
4077
4078 try {
4079 Thread.sleep(1000);
4080 } catch (InterruptedException i) {
4081
4082 }
4083
4084 put = new Put(ROW);
4085 put.add(FAM2, Bytes.toBytes("letters"), Bytes.toBytes("hijklmnop"));
4086 table.put(put);
4087
4088 long times[] = new long[3];
4089
4090
4091
4092 Scan scan = new Scan();
4093 scan.addFamily(FAM1);
4094 scan.addFamily(FAM2);
4095 ResultScanner s = table.getScanner(scan);
4096 try {
4097 int index = 0;
4098 Result r = null;
4099 while ((r = s.next()) != null) {
4100 for(Cell key : r.rawCells()) {
4101 times[index++] = key.getTimestamp();
4102 }
4103 }
4104 } finally {
4105 s.close();
4106 }
4107 for (int i = 0; i < times.length - 1; i++) {
4108 for (int j = i + 1; j < times.length; j++) {
4109 assertTrue(times[j] > times[i]);
4110 }
4111 }
4112
4113
4114 TEST_UTIL.flush();
4115
4116
4117 for(int i=0;i<times.length;i++) {
4118 times[i] = 0;
4119 }
4120
4121 try {
4122 Thread.sleep(1000);
4123 } catch (InterruptedException i) {
4124
4125 }
4126 scan = new Scan();
4127 scan.addFamily(FAM1);
4128 scan.addFamily(FAM2);
4129 s = table.getScanner(scan);
4130 try {
4131 int index = 0;
4132 Result r = null;
4133 while ((r = s.next()) != null) {
4134 for(Cell key : r.rawCells()) {
4135 times[index++] = key.getTimestamp();
4136 }
4137 }
4138 } finally {
4139 s.close();
4140 }
4141 for (int i = 0; i < times.length - 1; i++) {
4142 for (int j = i + 1; j < times.length; j++) {
4143 assertTrue(times[j] > times[i]);
4144 }
4145 }
4146 }
4147
4148 @Test
4149 public void testListTables() throws IOException, InterruptedException {
4150 byte [] t1 = Bytes.toBytes("testListTables1");
4151 byte [] t2 = Bytes.toBytes("testListTables2");
4152 byte [] t3 = Bytes.toBytes("testListTables3");
4153 byte [][] tables = new byte[][] { t1, t2, t3 };
4154 for (int i = 0; i < tables.length; i++) {
4155 TEST_UTIL.createTable(tables[i], FAMILY);
4156 }
4157 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
4158 HTableDescriptor[] ts = admin.listTables();
4159 HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
4160 for (int i = 0; i < ts.length; i++) {
4161 result.add(ts[i]);
4162 }
4163 int size = result.size();
4164 assertTrue(size >= tables.length);
4165 for (int i = 0; i < tables.length && i < size; i++) {
4166 boolean found = false;
4167 for (int j = 0; j < ts.length; j++) {
4168 if (Bytes.equals(ts[j].getTableName().getName(), tables[i])) {
4169 found = true;
4170 break;
4171 }
4172 }
4173 assertTrue("Not found: " + Bytes.toString(tables[i]), found);
4174 }
4175 }
4176
4177
4178
4179
4180
4181
4182
4183
4184 HTable createUnmangedHConnectionHTable(final byte [] tableName) throws IOException {
4185 TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
4186 HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
4187 return (HTable)conn.getTable(tableName);
4188 }
4189
4190
4191
4192
4193
4194
4195
4196 @Test
4197 public void testUnmanagedHConnection() throws IOException {
4198 final byte[] tableName = Bytes.toBytes("testUnmanagedHConnection");
4199 HTable t = createUnmangedHConnectionHTable(tableName);
4200 HBaseAdmin ha = new HBaseAdmin(t.getConnection());
4201 assertTrue(ha.tableExists(tableName));
4202 assertTrue(t.get(new Get(ROW)).isEmpty());
4203 }
4204
4205
4206
4207
4208
4209
4210
4211 @Test
4212 public void testUnmanagedHConnectionReconnect() throws Exception {
4213 final byte[] tableName = Bytes.toBytes("testUnmanagedHConnectionReconnect");
4214 HTable t = createUnmangedHConnectionHTable(tableName);
4215 HConnection conn = t.getConnection();
4216 HBaseAdmin ha = new HBaseAdmin(conn);
4217 assertTrue(ha.tableExists(tableName));
4218 assertTrue(t.get(new Get(ROW)).isEmpty());
4219
4220
4221 MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
4222 cluster.stopMaster(0, false);
4223 cluster.waitOnMaster(0);
4224
4225
4226 cluster.startMaster();
4227 assertTrue(cluster.waitForActiveAndReadyMaster());
4228
4229
4230
4231 HBaseAdmin newAdmin = new HBaseAdmin(conn);
4232 assertTrue(newAdmin.tableExists(tableName));
4233 assert(newAdmin.getClusterStatus().getServersSize() == SLAVES);
4234 }
4235
4236 @Test
4237 public void testMiscHTableStuff() throws IOException {
4238 final byte[] tableAname = Bytes.toBytes("testMiscHTableStuffA");
4239 final byte[] tableBname = Bytes.toBytes("testMiscHTableStuffB");
4240 final byte[] attrName = Bytes.toBytes("TESTATTR");
4241 final byte[] attrValue = Bytes.toBytes("somevalue");
4242 byte[] value = Bytes.toBytes("value");
4243
4244 HTable a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY);
4245 HTable b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY);
4246 Put put = new Put(ROW);
4247 put.add(HConstants.CATALOG_FAMILY, null, value);
4248 a.put(put);
4249
4250
4251 HTable newA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
4252
4253
4254 Scan scan = new Scan();
4255 scan.addFamily(HConstants.CATALOG_FAMILY);
4256 ResultScanner s = newA.getScanner(scan);
4257 try {
4258 for (Result r : s) {
4259 put = new Put(r.getRow());
4260 put.setDurability(Durability.SKIP_WAL);
4261 for (Cell kv : r.rawCells()) {
4262 put.add(kv);
4263 }
4264 b.put(put);
4265 }
4266 } finally {
4267 s.close();
4268 }
4269
4270
4271 HTable anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
4272 Get get = new Get(ROW);
4273 get.addFamily(HConstants.CATALOG_FAMILY);
4274 anotherA.get(get);
4275
4276
4277
4278
4279
4280
4281 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
4282
4283 HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor());
4284
4285 admin.disableTable(tableAname);
4286
4287 desc.setValue(attrName, attrValue);
4288
4289 for (HColumnDescriptor c : desc.getFamilies())
4290 c.setValue(attrName, attrValue);
4291
4292 admin.modifyTable(tableAname, desc);
4293
4294 admin.enableTable(tableAname);
4295
4296
4297 desc = a.getTableDescriptor();
4298 assertTrue("wrong table descriptor returned",
4299 Bytes.compareTo(desc.getTableName().getName(), tableAname) == 0);
4300
4301 value = desc.getValue(attrName);
4302 assertFalse("missing HTD attribute value", value == null);
4303 assertFalse("HTD attribute value is incorrect",
4304 Bytes.compareTo(value, attrValue) != 0);
4305
4306 for (HColumnDescriptor c : desc.getFamilies()) {
4307 value = c.getValue(attrName);
4308 assertFalse("missing HCD attribute value", value == null);
4309 assertFalse("HCD attribute value is incorrect",
4310 Bytes.compareTo(value, attrValue) != 0);
4311 }
4312 }
4313
4314 @Test
4315 public void testGetClosestRowBefore() throws IOException, InterruptedException {
4316 final byte[] tableAname = Bytes.toBytes("testGetClosestRowBefore");
4317 final byte[] firstRow = Bytes.toBytes("row111");
4318 final byte[] secondRow = Bytes.toBytes("row222");
4319 final byte[] thirdRow = Bytes.toBytes("row333");
4320 final byte[] forthRow = Bytes.toBytes("row444");
4321 final byte[] beforeFirstRow = Bytes.toBytes("row");
4322 final byte[] beforeSecondRow = Bytes.toBytes("row22");
4323 final byte[] beforeThirdRow = Bytes.toBytes("row33");
4324 final byte[] beforeForthRow = Bytes.toBytes("row44");
4325
4326 HTable table =
4327 TEST_UTIL.createTable(tableAname,
4328 new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 64);
4329
4330
4331 table.setAutoFlush(true);
4332 String regionName = table.getRegionLocations().firstKey().getEncodedName();
4333 HRegion region =
4334 TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName);
4335 Put put1 = new Put(firstRow);
4336 Put put2 = new Put(secondRow);
4337 Put put3 = new Put(thirdRow);
4338 Put put4 = new Put(forthRow);
4339 byte[] one = new byte[] { 1 };
4340 byte[] two = new byte[] { 2 };
4341 byte[] three = new byte[] { 3 };
4342 byte[] four = new byte[] { 4 };
4343
4344 put1.add(HConstants.CATALOG_FAMILY, null, one);
4345 put2.add(HConstants.CATALOG_FAMILY, null, two);
4346 put3.add(HConstants.CATALOG_FAMILY, null, three);
4347 put4.add(HConstants.CATALOG_FAMILY, null, four);
4348 table.put(put1);
4349 table.put(put2);
4350 table.put(put3);
4351 table.put(put4);
4352 region.flushcache();
4353 Result result = null;
4354
4355
4356 result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY);
4357 assertTrue(result == null);
4358
4359
4360 result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY);
4361 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4362 assertTrue(Bytes.equals(result.getRow(), firstRow));
4363 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
4364
4365
4366 result = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY);
4367 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4368 assertTrue(Bytes.equals(result.getRow(), firstRow));
4369 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
4370
4371
4372 result = table.getRowOrBefore(secondRow, HConstants.CATALOG_FAMILY);
4373 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4374 assertTrue(Bytes.equals(result.getRow(), secondRow));
4375 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
4376
4377
4378 result = table.getRowOrBefore(beforeThirdRow, HConstants.CATALOG_FAMILY);
4379 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4380 assertTrue(Bytes.equals(result.getRow(), secondRow));
4381 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
4382
4383
4384 result = table.getRowOrBefore(thirdRow, HConstants.CATALOG_FAMILY);
4385 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4386 assertTrue(Bytes.equals(result.getRow(), thirdRow));
4387 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
4388
4389
4390 result = table.getRowOrBefore(beforeForthRow, HConstants.CATALOG_FAMILY);
4391 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4392 assertTrue(Bytes.equals(result.getRow(), thirdRow));
4393 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
4394
4395
4396 result = table.getRowOrBefore(forthRow, HConstants.CATALOG_FAMILY);
4397 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4398 assertTrue(Bytes.equals(result.getRow(), forthRow));
4399 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
4400
4401
4402 result = table.getRowOrBefore(Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY);
4403 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4404 assertTrue(Bytes.equals(result.getRow(), forthRow));
4405 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
4406 }
4407
4408
4409
4410
4411
4412 @Test
4413 public void testScanVariableReuse() throws Exception {
4414 Scan scan = new Scan();
4415 scan.addFamily(FAMILY);
4416 scan.addColumn(FAMILY, ROW);
4417
4418 assertTrue(scan.getFamilyMap().get(FAMILY).size() == 1);
4419
4420 scan = new Scan();
4421 scan.addFamily(FAMILY);
4422
4423 assertTrue(scan.getFamilyMap().get(FAMILY) == null);
4424 assertTrue(scan.getFamilyMap().containsKey(FAMILY));
4425 }
4426
4427 @Test
4428 public void testMultiRowMutation() throws Exception {
4429 LOG.info("Starting testMultiRowMutation");
4430 final byte [] TABLENAME = Bytes.toBytes("testMultiRowMutation");
4431 final byte [] ROW1 = Bytes.toBytes("testRow1");
4432
4433 HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4434 Put p = new Put(ROW);
4435 p.add(FAMILY, QUALIFIER, VALUE);
4436 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p);
4437
4438 p = new Put(ROW1);
4439 p.add(FAMILY, QUALIFIER, VALUE);
4440 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, p);
4441
4442 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder();
4443 mrmBuilder.addMutationRequest(m1);
4444 mrmBuilder.addMutationRequest(m2);
4445 MutateRowsRequest mrm = mrmBuilder.build();
4446 CoprocessorRpcChannel channel = t.coprocessorService(ROW);
4447 MultiRowMutationService.BlockingInterface service =
4448 MultiRowMutationService.newBlockingStub(channel);
4449 service.mutateRows(null, mrm);
4450 Get g = new Get(ROW);
4451 Result r = t.get(g);
4452 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)));
4453 g = new Get(ROW1);
4454 r = t.get(g);
4455 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)));
4456 }
4457
4458 @Test
4459 public void testRowMutation() throws Exception {
4460 LOG.info("Starting testRowMutation");
4461 final byte [] TABLENAME = Bytes.toBytes("testRowMutation");
4462 HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4463 byte [][] QUALIFIERS = new byte [][] {
4464 Bytes.toBytes("a"), Bytes.toBytes("b")
4465 };
4466 RowMutations arm = new RowMutations(ROW);
4467 Put p = new Put(ROW);
4468 p.add(FAMILY, QUALIFIERS[0], VALUE);
4469 arm.add(p);
4470 t.mutateRow(arm);
4471
4472 Get g = new Get(ROW);
4473 Result r = t.get(g);
4474 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0])));
4475
4476 arm = new RowMutations(ROW);
4477 p = new Put(ROW);
4478 p.add(FAMILY, QUALIFIERS[1], VALUE);
4479 arm.add(p);
4480 Delete d = new Delete(ROW);
4481 d.deleteColumns(FAMILY, QUALIFIERS[0]);
4482 arm.add(d);
4483
4484 t.mutateRow(arm);
4485 r = t.get(g);
4486 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1])));
4487 assertNull(r.getValue(FAMILY, QUALIFIERS[0]));
4488 }
4489
4490 @Test
4491 public void testAppend() throws Exception {
4492 LOG.info("Starting testAppend");
4493 final byte [] TABLENAME = Bytes.toBytes("testAppend");
4494 HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4495 byte[] v1 = Bytes.toBytes("42");
4496 byte[] v2 = Bytes.toBytes("23");
4497 byte [][] QUALIFIERS = new byte [][] {
4498 Bytes.toBytes("b"), Bytes.toBytes("a"), Bytes.toBytes("c")
4499 };
4500 Append a = new Append(ROW);
4501 a.add(FAMILY, QUALIFIERS[0], v1);
4502 a.add(FAMILY, QUALIFIERS[1], v2);
4503 a.setReturnResults(false);
4504 assertNullResult(t.append(a));
4505
4506 a = new Append(ROW);
4507 a.add(FAMILY, QUALIFIERS[0], v2);
4508 a.add(FAMILY, QUALIFIERS[1], v1);
4509 a.add(FAMILY, QUALIFIERS[2], v2);
4510 Result r = t.append(a);
4511 assertEquals(0, Bytes.compareTo(Bytes.add(v1,v2), r.getValue(FAMILY, QUALIFIERS[0])));
4512 assertEquals(0, Bytes.compareTo(Bytes.add(v2,v1), r.getValue(FAMILY, QUALIFIERS[1])));
4513
4514 assertEquals(0, Bytes.compareTo(v2, r.getValue(FAMILY, QUALIFIERS[2])));
4515 assertEquals(r.getColumnLatest(FAMILY, QUALIFIERS[0]).getTimestamp(),
4516 r.getColumnLatest(FAMILY, QUALIFIERS[2]).getTimestamp());
4517 }
4518
4519 @Test
4520 public void testIncrementWithDeletes() throws Exception {
4521 LOG.info("Starting testIncrementWithDeletes");
4522 final TableName TABLENAME =
4523 TableName.valueOf("testIncrementWithDeletes");
4524 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4525 final byte[] COLUMN = Bytes.toBytes("column");
4526
4527 ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
4528 TEST_UTIL.flush(TABLENAME);
4529
4530 Delete del = new Delete(ROW);
4531 ht.delete(del);
4532
4533 ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
4534
4535 Get get = new Get(ROW);
4536 Result r = ht.get(get);
4537 assertEquals(1, r.size());
4538 assertEquals(5, Bytes.toLong(r.getValue(FAMILY, COLUMN)));
4539 }
4540
4541 @Test
4542 public void testIncrementingInvalidValue() throws Exception {
4543 LOG.info("Starting testIncrementingInvalidValue");
4544 final byte [] TABLENAME = Bytes.toBytes("testIncrementingInvalidValue");
4545 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4546 final byte[] COLUMN = Bytes.toBytes("column");
4547 Put p = new Put(ROW);
4548
4549 p.add(FAMILY, COLUMN, Bytes.toBytes(5));
4550 ht.put(p);
4551 try {
4552 ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
4553 fail("Should have thrown DoNotRetryIOException");
4554 } catch (DoNotRetryIOException iox) {
4555
4556 }
4557 Increment inc = new Increment(ROW);
4558 inc.addColumn(FAMILY, COLUMN, 5);
4559 try {
4560 ht.increment(inc);
4561 fail("Should have thrown DoNotRetryIOException");
4562 } catch (DoNotRetryIOException iox) {
4563
4564 }
4565 }
4566
4567 @Test
4568 public void testIncrementInvalidArguments() throws Exception {
4569 LOG.info("Starting testIncrementInvalidArguments");
4570 final byte[] TABLENAME = Bytes.toBytes("testIncrementInvalidArguments");
4571 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4572 final byte[] COLUMN = Bytes.toBytes("column");
4573 try {
4574
4575 ht.incrementColumnValue(null, FAMILY, COLUMN, 5);
4576 fail("Should have thrown IOException");
4577 } catch (IOException iox) {
4578
4579 }
4580 try {
4581
4582 ht.incrementColumnValue(ROW, null, COLUMN, 5);
4583 fail("Should have thrown IOException");
4584 } catch (IOException iox) {
4585
4586 }
4587 try {
4588
4589 ht.incrementColumnValue(ROW, FAMILY, null, 5);
4590 fail("Should have thrown IOException");
4591 } catch (IOException iox) {
4592
4593 }
4594
4595 try {
4596 Increment incNoRow = new Increment((byte [])null);
4597 incNoRow.addColumn(FAMILY, COLUMN, 5);
4598 fail("Should have thrown IllegalArgumentException");
4599 } catch (IllegalArgumentException iax) {
4600
4601 } catch (NullPointerException npe) {
4602
4603 }
4604
4605 try {
4606 Increment incNoFamily = new Increment(ROW);
4607 incNoFamily.addColumn(null, COLUMN, 5);
4608 fail("Should have thrown IllegalArgumentException");
4609 } catch (IllegalArgumentException iax) {
4610
4611 }
4612
4613 try {
4614 Increment incNoQualifier = new Increment(ROW);
4615 incNoQualifier.addColumn(FAMILY, null, 5);
4616 fail("Should have thrown IllegalArgumentException");
4617 } catch (IllegalArgumentException iax) {
4618
4619 }
4620 }
4621
4622 @Test
4623 public void testIncrementOutOfOrder() throws Exception {
4624 LOG.info("Starting testIncrementOutOfOrder");
4625 final byte [] TABLENAME = Bytes.toBytes("testIncrementOutOfOrder");
4626 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4627
4628 byte [][] QUALIFIERS = new byte [][] {
4629 Bytes.toBytes("B"), Bytes.toBytes("A"), Bytes.toBytes("C")
4630 };
4631
4632 Increment inc = new Increment(ROW);
4633 for (int i=0; i<QUALIFIERS.length; i++) {
4634 inc.addColumn(FAMILY, QUALIFIERS[i], 1);
4635 }
4636 ht.increment(inc);
4637
4638
4639 Result r = ht.get(new Get(ROW));
4640 Cell [] kvs = r.rawCells();
4641 assertEquals(3, kvs.length);
4642 assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[1], 1);
4643 assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[0], 1);
4644 assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 1);
4645
4646
4647 inc = new Increment(ROW);
4648 for (int i=0; i<QUALIFIERS.length; i++) {
4649 inc.addColumn(FAMILY, QUALIFIERS[i], 1);
4650 }
4651 ht.increment(inc);
4652
4653
4654 r = ht.get(new Get(ROW));
4655 kvs = r.rawCells();
4656 assertEquals(3, kvs.length);
4657 assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[1], 2);
4658 assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[0], 2);
4659 assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 2);
4660 }
4661
4662 @Test
4663 public void testIncrement() throws Exception {
4664 LOG.info("Starting testIncrement");
4665 final byte [] TABLENAME = Bytes.toBytes("testIncrement");
4666 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4667
4668 byte [][] ROWS = new byte [][] {
4669 Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
4670 Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
4671 Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i")
4672 };
4673 byte [][] QUALIFIERS = new byte [][] {
4674 Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
4675 Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
4676 Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i")
4677 };
4678
4679
4680
4681
4682 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[0], 1);
4683 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[1], 2);
4684 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[2], 3);
4685 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[3], 4);
4686
4687
4688 Increment inc = new Increment(ROW);
4689 inc.addColumn(FAMILY, QUALIFIERS[1], 1);
4690 inc.addColumn(FAMILY, QUALIFIERS[3], 1);
4691 inc.addColumn(FAMILY, QUALIFIERS[4], 1);
4692 ht.increment(inc);
4693
4694
4695 Result r = ht.get(new Get(ROW));
4696 Cell [] kvs = r.rawCells();
4697 assertEquals(5, kvs.length);
4698 assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 1);
4699 assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 3);
4700 assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 3);
4701 assertIncrementKey(kvs[3], ROW, FAMILY, QUALIFIERS[3], 5);
4702 assertIncrementKey(kvs[4], ROW, FAMILY, QUALIFIERS[4], 1);
4703
4704
4705 inc = new Increment(ROWS[0]);
4706 for (int i=0;i<QUALIFIERS.length;i++) {
4707 inc.addColumn(FAMILY, QUALIFIERS[i], i+1);
4708 }
4709 ht.increment(inc);
4710
4711 r = ht.get(new Get(ROWS[0]));
4712 kvs = r.rawCells();
4713 assertEquals(QUALIFIERS.length, kvs.length);
4714 for (int i=0;i<QUALIFIERS.length;i++) {
4715 assertIncrementKey(kvs[i], ROWS[0], FAMILY, QUALIFIERS[i], i+1);
4716 }
4717
4718
4719 inc = new Increment(ROWS[0]);
4720 for (int i=0;i<QUALIFIERS.length;i++) {
4721 inc.addColumn(FAMILY, QUALIFIERS[i], i+1);
4722 }
4723 ht.increment(inc);
4724
4725 r = ht.get(new Get(ROWS[0]));
4726 kvs = r.rawCells();
4727 assertEquals(QUALIFIERS.length, kvs.length);
4728 for (int i=0;i<QUALIFIERS.length;i++) {
4729 assertIncrementKey(kvs[i], ROWS[0], FAMILY, QUALIFIERS[i], 2*(i+1));
4730 }
4731 }
4732
4733
4734 @Test
4735 public void testClientPoolRoundRobin() throws IOException {
4736 final byte[] tableName = Bytes.toBytes("testClientPoolRoundRobin");
4737
4738 int poolSize = 3;
4739 int numVersions = poolSize * 2;
4740 Configuration conf = TEST_UTIL.getConfiguration();
4741 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin");
4742 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4743
4744 HTable table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY },
4745 conf, Integer.MAX_VALUE);
4746
4747 final long ts = EnvironmentEdgeManager.currentTimeMillis();
4748 Get get = new Get(ROW);
4749 get.addColumn(FAMILY, QUALIFIER);
4750 get.setMaxVersions();
4751
4752 for (int versions = 1; versions <= numVersions; versions++) {
4753 Put put = new Put(ROW);
4754 put.add(FAMILY, QUALIFIER, ts + versions, VALUE);
4755 table.put(put);
4756
4757 Result result = table.get(get);
4758 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4759 .get(QUALIFIER);
4760
4761 assertEquals("The number of versions of '" + FAMILY + ":" + QUALIFIER
4762 + " did not match " + versions, versions, navigableMap.size());
4763 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4764 assertTrue("The value at time " + entry.getKey()
4765 + " did not match what was put",
4766 Bytes.equals(VALUE, entry.getValue()));
4767 }
4768 }
4769 }
4770
4771 @Ignore ("Flakey: HBASE-8989") @Test
4772 public void testClientPoolThreadLocal() throws IOException {
4773 final byte[] tableName = Bytes.toBytes("testClientPoolThreadLocal");
4774
4775 int poolSize = Integer.MAX_VALUE;
4776 int numVersions = 3;
4777 Configuration conf = TEST_UTIL.getConfiguration();
4778 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local");
4779 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4780
4781 final HTable table = TEST_UTIL.createTable(tableName,
4782 new byte[][] { FAMILY }, conf, 3);
4783
4784 final long ts = EnvironmentEdgeManager.currentTimeMillis();
4785 final Get get = new Get(ROW);
4786 get.addColumn(FAMILY, QUALIFIER);
4787 get.setMaxVersions();
4788
4789 for (int versions = 1; versions <= numVersions; versions++) {
4790 Put put = new Put(ROW);
4791 put.add(FAMILY, QUALIFIER, ts + versions, VALUE);
4792 table.put(put);
4793
4794 Result result = table.get(get);
4795 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4796 .get(QUALIFIER);
4797
4798 assertEquals("The number of versions of '" + FAMILY + ":" + QUALIFIER + " did not match " +
4799 versions + "; " + put.toString() + ", " + get.toString(), versions, navigableMap.size());
4800 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4801 assertTrue("The value at time " + entry.getKey()
4802 + " did not match what was put",
4803 Bytes.equals(VALUE, entry.getValue()));
4804 }
4805 }
4806
4807 final Object waitLock = new Object();
4808 ExecutorService executorService = Executors.newFixedThreadPool(numVersions);
4809 final AtomicReference<AssertionError> error = new AtomicReference<AssertionError>(null);
4810 for (int versions = numVersions; versions < numVersions * 2; versions++) {
4811 final int versionsCopy = versions;
4812 executorService.submit(new Callable<Void>() {
4813 @Override
4814 public Void call() {
4815 try {
4816 Put put = new Put(ROW);
4817 put.add(FAMILY, QUALIFIER, ts + versionsCopy, VALUE);
4818 table.put(put);
4819
4820 Result result = table.get(get);
4821 NavigableMap<Long, byte[]> navigableMap = result.getMap()
4822 .get(FAMILY).get(QUALIFIER);
4823
4824 assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":"
4825 + Bytes.toString(QUALIFIER) + " did not match " + versionsCopy, versionsCopy,
4826 navigableMap.size());
4827 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4828 assertTrue("The value at time " + entry.getKey()
4829 + " did not match what was put",
4830 Bytes.equals(VALUE, entry.getValue()));
4831 }
4832 synchronized (waitLock) {
4833 waitLock.wait();
4834 }
4835 } catch (Exception e) {
4836 } catch (AssertionError e) {
4837
4838
4839 error.set(e);
4840 LOG.error(e);
4841 }
4842
4843 return null;
4844 }
4845 });
4846 }
4847 synchronized (waitLock) {
4848 waitLock.notifyAll();
4849 }
4850 executorService.shutdownNow();
4851 assertNull(error.get());
4852 }
4853
4854 @Test
4855 public void testCheckAndPut() throws IOException {
4856 final byte [] anotherrow = Bytes.toBytes("anotherrow");
4857 final byte [] value2 = Bytes.toBytes("abcd");
4858
4859 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"),
4860 new byte [][] {FAMILY});
4861 Put put1 = new Put(ROW);
4862 put1.add(FAMILY, QUALIFIER, VALUE);
4863
4864
4865 boolean ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put1);
4866 assertEquals(ok, false);
4867
4868
4869 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4870 assertEquals(ok, true);
4871
4872
4873 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4874 assertEquals(ok, false);
4875
4876 Put put2 = new Put(ROW);
4877 put2.add(FAMILY, QUALIFIER, value2);
4878
4879
4880 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put2);
4881 assertEquals(ok, true);
4882
4883 Put put3 = new Put(anotherrow);
4884 put3.add(FAMILY, QUALIFIER, VALUE);
4885
4886
4887 try {
4888 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, value2, put3);
4889 fail("trying to check and modify different rows should have failed.");
4890 } catch(Exception e) {}
4891
4892 }
4893
4894
4895
4896
4897
4898 @Test
4899 @SuppressWarnings ("unused")
4900 public void testScanMetrics() throws Exception {
4901 byte [] TABLENAME = Bytes.toBytes("testScanMetrics");
4902
4903 Configuration conf = TEST_UTIL.getConfiguration();
4904 TEST_UTIL.createTable(TABLENAME, FAMILY);
4905
4906
4907
4908 HTable ht = new HTable(conf, TABLENAME);
4909
4910
4911 int numOfRegions = TEST_UTIL.createMultiRegions(ht, FAMILY);
4912
4913
4914 Put put1 = new Put(Bytes.toBytes("z1"));
4915 put1.add(FAMILY, QUALIFIER, VALUE);
4916 Put put2 = new Put(Bytes.toBytes("z2"));
4917 put2.add(FAMILY, QUALIFIER, VALUE);
4918 Put put3 = new Put(Bytes.toBytes("z3"));
4919 put3.add(FAMILY, QUALIFIER, VALUE);
4920 ht.put(Arrays.asList(put1, put2, put3));
4921
4922 Scan scan1 = new Scan();
4923 int numRecords = 0;
4924 for(Result result : ht.getScanner(scan1)) {
4925 numRecords++;
4926 }
4927 LOG.info("test data has " + numRecords + " records.");
4928
4929
4930 assertEquals(null, scan1.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA));
4931
4932
4933 Scan scan = new Scan();
4934 scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4935 scan.setCaching(numRecords+1);
4936 ResultScanner scanner = ht.getScanner(scan);
4937 for (Result result : scanner.next(numRecords - 1)) {
4938 }
4939 scanner.close();
4940
4941 assertNotNull(scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA));
4942
4943
4944 scan = new Scan();
4945 scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4946 scan.setCaching(1);
4947 scanner = ht.getScanner(scan);
4948
4949
4950 for (Result result : scanner.next(numRecords - 1)) {
4951 }
4952 scanner.close();
4953
4954 ScanMetrics scanMetrics = getScanMetrics(scan);
4955 assertEquals("Did not access all the regions in the table", numOfRegions,
4956 scanMetrics.countOfRegions.get());
4957
4958
4959
4960 Scan scanWithoutClose = new Scan();
4961 scanWithoutClose.setCaching(1);
4962 scanWithoutClose.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4963 ResultScanner scannerWithoutClose = ht.getScanner(scanWithoutClose);
4964 for (Result result : scannerWithoutClose.next(numRecords + 1)) {
4965 }
4966 ScanMetrics scanMetricsWithoutClose = getScanMetrics(scanWithoutClose);
4967 assertEquals("Did not access all the regions in the table", numOfRegions,
4968 scanMetricsWithoutClose.countOfRegions.get());
4969
4970
4971
4972 Scan scanWithClose = new Scan();
4973
4974 scanWithClose.setCaching(numRecords);
4975 scanWithClose.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4976 ResultScanner scannerWithClose = ht.getScanner(scanWithClose);
4977 for (Result result : scannerWithClose.next(numRecords + 1)) {
4978 }
4979 scannerWithClose.close();
4980 ScanMetrics scanMetricsWithClose = getScanMetrics(scanWithClose);
4981 assertEquals("Did not access all the regions in the table", numOfRegions,
4982 scanMetricsWithClose.countOfRegions.get());
4983 }
4984
4985 private ScanMetrics getScanMetrics(Scan scan) throws Exception {
4986 byte[] serializedMetrics = scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA);
4987 assertTrue("Serialized metrics were not found.", serializedMetrics != null);
4988
4989
4990 ScanMetrics scanMetrics = ProtobufUtil.toScanMetrics(serializedMetrics);
4991
4992 return scanMetrics;
4993 }
4994
4995
4996
4997
4998
4999
5000
5001
5002
5003 @Test
5004 public void testCacheOnWriteEvictOnClose() throws Exception {
5005 byte [] tableName = Bytes.toBytes("testCOWEOCfromClient");
5006 byte [] data = Bytes.toBytes("data");
5007 HTable table = TEST_UTIL.createTable(tableName, new byte [][] {FAMILY});
5008
5009 String regionName = table.getRegionLocations().firstKey().getEncodedName();
5010 HRegion region = TEST_UTIL.getRSForFirstRegionInTable(
5011 tableName).getFromOnlineRegions(regionName);
5012 Store store = region.getStores().values().iterator().next();
5013 CacheConfig cacheConf = store.getCacheConfig();
5014 cacheConf.setCacheDataOnWrite(true);
5015 cacheConf.setEvictOnClose(true);
5016 BlockCache cache = cacheConf.getBlockCache();
5017
5018
5019 long startBlockCount = cache.getBlockCount();
5020 long startBlockHits = cache.getStats().getHitCount();
5021 long startBlockMiss = cache.getStats().getMissCount();
5022
5023
5024 for (int i = 0; i < 5; i++) {
5025 Thread.sleep(100);
5026 if (startBlockCount != cache.getBlockCount()
5027 || startBlockHits != cache.getStats().getHitCount()
5028 || startBlockMiss != cache.getStats().getMissCount()) {
5029 startBlockCount = cache.getBlockCount();
5030 startBlockHits = cache.getStats().getHitCount();
5031 startBlockMiss = cache.getStats().getMissCount();
5032 i = -1;
5033 }
5034 }
5035
5036
5037 Put put = new Put(ROW);
5038 put.add(FAMILY, QUALIFIER, data);
5039 table.put(put);
5040 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
5041
5042 assertEquals(startBlockCount, cache.getBlockCount());
5043 assertEquals(startBlockHits, cache.getStats().getHitCount());
5044 assertEquals(startBlockMiss, cache.getStats().getMissCount());
5045
5046 System.out.println("Flushing cache");
5047 region.flushcache();
5048
5049 long expectedBlockCount = startBlockCount + 1;
5050 long expectedBlockHits = startBlockHits;
5051 long expectedBlockMiss = startBlockMiss;
5052 assertEquals(expectedBlockCount, cache.getBlockCount());
5053 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5054 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5055
5056 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
5057 assertEquals(expectedBlockCount, cache.getBlockCount());
5058 assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
5059 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5060
5061 byte [] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
5062 byte [] data2 = Bytes.add(data, data);
5063 put = new Put(ROW);
5064 put.add(FAMILY, QUALIFIER2, data2);
5065 table.put(put);
5066 Result r = table.get(new Get(ROW));
5067 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
5068 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
5069 assertEquals(expectedBlockCount, cache.getBlockCount());
5070 assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
5071 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5072
5073 System.out.println("Flushing cache");
5074 region.flushcache();
5075 assertEquals(++expectedBlockCount, cache.getBlockCount());
5076 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5077 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5078
5079 System.out.println("Compacting");
5080 assertEquals(2, store.getStorefilesCount());
5081 store.triggerMajorCompaction();
5082 region.compactStores();
5083 waitForStoreFileCount(store, 1, 10000);
5084 assertEquals(1, store.getStorefilesCount());
5085 expectedBlockCount -= 2;
5086 assertEquals(expectedBlockCount, cache.getBlockCount());
5087 expectedBlockHits += 2;
5088 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5089 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5090
5091
5092 r = table.get(new Get(ROW));
5093 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
5094 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
5095 expectedBlockCount += 1;
5096 assertEquals(expectedBlockCount, cache.getBlockCount());
5097 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5098 assertEquals(++expectedBlockMiss, cache.getStats().getMissCount());
5099 }
5100
5101 private void waitForStoreFileCount(Store store, int count, int timeout)
5102 throws InterruptedException {
5103 long start = System.currentTimeMillis();
5104 while (start + timeout > System.currentTimeMillis() &&
5105 store.getStorefilesCount() != count) {
5106 Thread.sleep(100);
5107 }
5108 System.out.println("start=" + start + ", now=" +
5109 System.currentTimeMillis() + ", cur=" + store.getStorefilesCount());
5110 assertEquals(count, store.getStorefilesCount());
5111 }
5112
5113 @Test
5114
5115
5116
5117 public void testNonCachedGetRegionLocation() throws Exception {
5118
5119 String tableName = "testNonCachedGetRegionLocation";
5120 byte [] TABLE = Bytes.toBytes(tableName);
5121 byte [] family1 = Bytes.toBytes("f1");
5122 byte [] family2 = Bytes.toBytes("f2");
5123 HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
5124 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
5125 Map <HRegionInfo, ServerName> regionsMap = table.getRegionLocations();
5126 assertEquals(1, regionsMap.size());
5127 HRegionInfo regionInfo = regionsMap.keySet().iterator().next();
5128 ServerName addrBefore = regionsMap.get(regionInfo);
5129
5130 HRegionLocation addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
5131 HRegionLocation addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true);
5132
5133 assertEquals(addrBefore.getPort(), addrCache.getPort());
5134 assertEquals(addrBefore.getPort(), addrNoCache.getPort());
5135
5136 ServerName addrAfter = null;
5137
5138 for (int i = 0; i < SLAVES; i++) {
5139 HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(i);
5140 ServerName addr = regionServer.getServerName();
5141 if (addr.getPort() != addrBefore.getPort()) {
5142 admin.move(regionInfo.getEncodedNameAsBytes(),
5143 Bytes.toBytes(addr.toString()));
5144
5145 Thread.sleep(5000);
5146 addrAfter = addr;
5147 break;
5148 }
5149 }
5150
5151
5152 addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
5153 addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true);
5154 assertNotNull(addrAfter);
5155 assertTrue(addrAfter.getPort() != addrCache.getPort());
5156 assertEquals(addrAfter.getPort(), addrNoCache.getPort());
5157 }
5158
5159 @Test
5160
5161
5162
5163
5164 public void testGetRegionsInRange() throws Exception {
5165
5166 byte [] startKey = Bytes.toBytes("ddc");
5167 byte [] endKey = Bytes.toBytes("mmm");
5168 byte [] TABLE = Bytes.toBytes("testGetRegionsInRange");
5169 HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
5170 int numOfRegions = TEST_UTIL.createMultiRegions(table, FAMILY);
5171 assertEquals(25, numOfRegions);
5172
5173
5174 List<HRegionLocation> regionsList = table.getRegionsInRange(startKey,
5175 endKey);
5176 assertEquals(10, regionsList.size());
5177
5178
5179 startKey = Bytes.toBytes("fff");
5180 regionsList = table.getRegionsInRange(startKey, endKey);
5181 assertEquals(7, regionsList.size());
5182
5183
5184 endKey = Bytes.toBytes("nnn");
5185 regionsList = table.getRegionsInRange(startKey, endKey);
5186 assertEquals(8, regionsList.size());
5187
5188
5189 regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, endKey);
5190 assertEquals(13, regionsList.size());
5191
5192
5193 regionsList = table.getRegionsInRange(startKey, HConstants.EMPTY_END_ROW);
5194 assertEquals(20, regionsList.size());
5195
5196
5197 regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW,
5198 HConstants.EMPTY_END_ROW);
5199 assertEquals(25, regionsList.size());
5200
5201
5202 endKey = Bytes.toBytes("yyz");
5203 regionsList = table.getRegionsInRange(startKey, endKey);
5204 assertEquals(20, regionsList.size());
5205
5206
5207 startKey = Bytes.toBytes("aac");
5208 regionsList = table.getRegionsInRange(startKey, endKey);
5209 assertEquals(25, regionsList.size());
5210
5211
5212 startKey = endKey = Bytes.toBytes("ccc");
5213 regionsList = table.getRegionsInRange(startKey, endKey);
5214 assertEquals(1, regionsList.size());
5215 }
5216
5217 @Test
5218 public void testJira6912() throws Exception {
5219 byte [] TABLE = Bytes.toBytes("testJira6912");
5220 HTable foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
5221
5222 List<Put> puts = new ArrayList<Put>();
5223 for (int i=0;i !=100; i++){
5224 Put put = new Put(Bytes.toBytes(i));
5225 put.add(FAMILY, FAMILY, Bytes.toBytes(i));
5226 puts.add(put);
5227 }
5228 foo.put(puts);
5229
5230 TEST_UTIL.flush();
5231
5232 Scan scan = new Scan();
5233 scan.setStartRow(Bytes.toBytes(1));
5234 scan.setStopRow(Bytes.toBytes(3));
5235 scan.addColumn(FAMILY, FAMILY);
5236 scan.setFilter(new RowFilter(CompareFilter.CompareOp.NOT_EQUAL, new BinaryComparator(Bytes.toBytes(1))));
5237
5238 ResultScanner scanner = foo.getScanner(scan);
5239 Result[] bar = scanner.next(100);
5240 assertEquals(1, bar.length);
5241 }
5242
5243 @Test
5244 public void testScan_NullQualifier() throws IOException {
5245 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testScan_NullQualifier"), FAMILY);
5246 Put put = new Put(ROW);
5247 put.add(FAMILY, QUALIFIER, VALUE);
5248 table.put(put);
5249
5250 put = new Put(ROW);
5251 put.add(FAMILY, null, VALUE);
5252 table.put(put);
5253 LOG.info("Row put");
5254
5255 Scan scan = new Scan();
5256 scan.addColumn(FAMILY, null);
5257
5258 ResultScanner scanner = table.getScanner(scan);
5259 Result[] bar = scanner.next(100);
5260 assertEquals(1, bar.length);
5261 assertEquals(1, bar[0].size());
5262
5263 scan = new Scan();
5264 scan.addFamily(FAMILY);
5265
5266 scanner = table.getScanner(scan);
5267 bar = scanner.next(100);
5268 assertEquals(1, bar.length);
5269 assertEquals(2, bar[0].size());
5270 }
5271
5272 @Test
5273 public void testNegativeTimestamp() throws IOException {
5274 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY);
5275
5276 try {
5277 Put put = new Put(ROW, -1);
5278 put.add(FAMILY, QUALIFIER, VALUE);
5279 table.put(put);
5280 fail("Negative timestamps should not have been allowed");
5281 } catch (IllegalArgumentException ex) {
5282 assertTrue(ex.getMessage().contains("negative"));
5283 }
5284
5285 try {
5286 Put put = new Put(ROW);
5287 put.add(FAMILY, QUALIFIER, -1, VALUE);
5288 table.put(put);
5289 fail("Negative timestamps should not have been allowed");
5290 } catch (IllegalArgumentException ex) {
5291 assertTrue(ex.getMessage().contains("negative"));
5292 }
5293
5294 try {
5295 Delete delete = new Delete(ROW, -1);
5296 table.delete(delete);
5297 fail("Negative timestamps should not have been allowed");
5298 } catch (IllegalArgumentException ex) {
5299 assertTrue(ex.getMessage().contains("negative"));
5300 }
5301
5302 try {
5303 Delete delete = new Delete(ROW);
5304 delete.deleteFamily(FAMILY, -1);
5305 table.delete(delete);
5306 fail("Negative timestamps should not have been allowed");
5307 } catch (IllegalArgumentException ex) {
5308 assertTrue(ex.getMessage().contains("negative"));
5309 }
5310
5311 try {
5312 Scan scan = new Scan();
5313 scan.setTimeRange(-1, 1);
5314 table.getScanner(scan);
5315 fail("Negative timestamps should not have been allowed");
5316 } catch (IllegalArgumentException ex) {
5317 assertTrue(ex.getMessage().contains("negative"));
5318 }
5319
5320
5321
5322 try {
5323 new KeyValue(Bytes.toBytes(42), Bytes.toBytes(42), Bytes.toBytes(42), -1, Bytes.toBytes(42));
5324 } catch (IllegalArgumentException ex) {
5325 fail("KeyValue SHOULD allow negative timestamps");
5326 }
5327
5328 table.close();
5329 }
5330
5331 @Test
5332 public void testRawScanRespectsVersions() throws Exception {
5333 byte[] TABLE = Bytes.toBytes("testRawScan");
5334 HTable table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY });
5335 byte[] row = Bytes.toBytes("row");
5336
5337
5338 Put p = new Put(row);
5339 p.add(FAMILY, QUALIFIER, 10, VALUE);
5340 table.put(p);
5341 table.flushCommits();
5342
5343 p = new Put(row);
5344 p.add(FAMILY, QUALIFIER, 11, ArrayUtils.add(VALUE, (byte) 2));
5345 table.put(p);
5346 table.flushCommits();
5347
5348 p = new Put(row);
5349 p.add(FAMILY, QUALIFIER, 12, ArrayUtils.add(VALUE, (byte) 3));
5350 table.put(p);
5351 table.flushCommits();
5352
5353 p = new Put(row);
5354 p.add(FAMILY, QUALIFIER, 13, ArrayUtils.add(VALUE, (byte) 4));
5355 table.put(p);
5356 table.flushCommits();
5357
5358 int versions = 4;
5359 Scan s = new Scan(row);
5360
5361 s.setMaxVersions();
5362 s.setRaw(true);
5363
5364 ResultScanner scanner = table.getScanner(s);
5365 int count = 0;
5366 for (Result r : scanner) {
5367 assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5368 count++;
5369 }
5370 assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5371 count);
5372 scanner.close();
5373
5374
5375
5376 versions = 2;
5377 s.setMaxVersions(versions);
5378 scanner = table.getScanner(s);
5379 count = 0;
5380 for (Result r : scanner) {
5381 assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5382 count++;
5383 }
5384 assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5385 count);
5386 scanner.close();
5387
5388
5389
5390 versions = 3;
5391 s.setMaxVersions(versions);
5392 scanner = table.getScanner(s);
5393 count = 0;
5394 for (Result r : scanner) {
5395 assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5396 count++;
5397 }
5398 assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5399 count);
5400 scanner.close();
5401
5402 table.close();
5403 TEST_UTIL.deleteTable(TABLE);
5404 }
5405
5406 @Test
5407 public void testSmallScan() throws Exception {
5408
5409 byte[] TABLE = Bytes.toBytes("testSmallScan");
5410 HTable table = TEST_UTIL.createTable(TABLE, FAMILY);
5411
5412
5413 int insertNum = 10;
5414 for (int i = 0; i < 10; i++) {
5415 Put put = new Put(Bytes.toBytes("row" + String.format("%03d", i)));
5416 put.add(FAMILY, QUALIFIER, VALUE);
5417 table.put(put);
5418 }
5419
5420
5421 ResultScanner scanner = table.getScanner(new Scan());
5422 int count = 0;
5423 for (Result r : scanner) {
5424 assertTrue(!r.isEmpty());
5425 count++;
5426 }
5427 assertEquals(insertNum, count);
5428
5429
5430 Scan scan = new Scan(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
5431 scan.setSmall(true);
5432 scan.setCaching(2);
5433 scanner = table.getScanner(scan);
5434 count = 0;
5435 for (Result r : scanner) {
5436 assertTrue(!r.isEmpty());
5437 count++;
5438 }
5439 assertEquals(insertNum, count);
5440
5441 }
5442
5443 @Test
5444 public void testSuperSimpleWithReverseScan() throws Exception {
5445 byte[] TABLE = Bytes.toBytes("testSuperSimpleWithReverseScan");
5446 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
5447 Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000"));
5448 put.add(FAMILY, QUALIFIER, VALUE);
5449 ht.put(put);
5450 put = new Put(Bytes.toBytes("0-b11111-0000000000000000002"));
5451 put.add(FAMILY, QUALIFIER, VALUE);
5452 ht.put(put);
5453 put = new Put(Bytes.toBytes("0-b11111-0000000000000000004"));
5454 put.add(FAMILY, QUALIFIER, VALUE);
5455 ht.put(put);
5456 put = new Put(Bytes.toBytes("0-b11111-0000000000000000006"));
5457 put.add(FAMILY, QUALIFIER, VALUE);
5458 ht.put(put);
5459 put = new Put(Bytes.toBytes("0-b11111-0000000000000000008"));
5460 put.add(FAMILY, QUALIFIER, VALUE);
5461 ht.put(put);
5462 put = new Put(Bytes.toBytes("0-b22222-0000000000000000001"));
5463 put.add(FAMILY, QUALIFIER, VALUE);
5464 ht.put(put);
5465 put = new Put(Bytes.toBytes("0-b22222-0000000000000000003"));
5466 put.add(FAMILY, QUALIFIER, VALUE);
5467 ht.put(put);
5468 put = new Put(Bytes.toBytes("0-b22222-0000000000000000005"));
5469 put.add(FAMILY, QUALIFIER, VALUE);
5470 ht.put(put);
5471 put = new Put(Bytes.toBytes("0-b22222-0000000000000000007"));
5472 put.add(FAMILY, QUALIFIER, VALUE);
5473 ht.put(put);
5474 put = new Put(Bytes.toBytes("0-b22222-0000000000000000009"));
5475 put.add(FAMILY, QUALIFIER, VALUE);
5476 ht.put(put);
5477 ht.flushCommits();
5478 Scan scan = new Scan(Bytes.toBytes("0-b11111-9223372036854775807"),
5479 Bytes.toBytes("0-b11111-0000000000000000000"));
5480 scan.setReversed(true);
5481 ResultScanner scanner = ht.getScanner(scan);
5482 Result result = scanner.next();
5483 assertTrue(Bytes.equals(result.getRow(),
5484 Bytes.toBytes("0-b11111-0000000000000000008")));
5485 scanner.close();
5486 ht.close();
5487 }
5488
5489 @Test
5490 public void testFiltersWithReverseScan() throws Exception {
5491 byte[] TABLE = Bytes.toBytes("testFiltersWithReverseScan");
5492 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
5493 byte[][] ROWS = makeN(ROW, 10);
5494 byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
5495 Bytes.toBytes("col1-<d2v1>-<d3v2>"),
5496 Bytes.toBytes("col2-<d2v1>-<d3v2>"),
5497 Bytes.toBytes("col3-<d2v1>-<d3v2>"),
5498 Bytes.toBytes("col4-<d2v1>-<d3v2>"),
5499 Bytes.toBytes("col5-<d2v1>-<d3v2>"),
5500 Bytes.toBytes("col6-<d2v1>-<d3v2>"),
5501 Bytes.toBytes("col7-<d2v1>-<d3v2>"),
5502 Bytes.toBytes("col8-<d2v1>-<d3v2>"),
5503 Bytes.toBytes("col9-<d2v1>-<d3v2>") };
5504 for (int i = 0; i < 10; i++) {
5505 Put put = new Put(ROWS[i]);
5506 put.add(FAMILY, QUALIFIERS[i], VALUE);
5507 ht.put(put);
5508 }
5509 Scan scan = new Scan();
5510 scan.setReversed(true);
5511 scan.addFamily(FAMILY);
5512 Filter filter = new QualifierFilter(CompareOp.EQUAL,
5513 new RegexStringComparator("col[1-5]"));
5514 scan.setFilter(filter);
5515 ResultScanner scanner = ht.getScanner(scan);
5516 int expectedIndex = 5;
5517 for (Result result : scanner) {
5518 assertEquals(result.size(), 1);
5519 assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[expectedIndex]));
5520 assertTrue(Bytes.equals(result.raw()[0].getQualifier(),
5521 QUALIFIERS[expectedIndex]));
5522 expectedIndex--;
5523 }
5524 assertEquals(expectedIndex, 0);
5525 scanner.close();
5526 ht.close();
5527 }
5528
5529 @Test
5530 public void testKeyOnlyFilterWithReverseScan() throws Exception {
5531 byte[] TABLE = Bytes.toBytes("testKeyOnlyFilterWithReverseScan");
5532 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
5533 byte[][] ROWS = makeN(ROW, 10);
5534 byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
5535 Bytes.toBytes("col1-<d2v1>-<d3v2>"),
5536 Bytes.toBytes("col2-<d2v1>-<d3v2>"),
5537 Bytes.toBytes("col3-<d2v1>-<d3v2>"),
5538 Bytes.toBytes("col4-<d2v1>-<d3v2>"),
5539 Bytes.toBytes("col5-<d2v1>-<d3v2>"),
5540 Bytes.toBytes("col6-<d2v1>-<d3v2>"),
5541 Bytes.toBytes("col7-<d2v1>-<d3v2>"),
5542 Bytes.toBytes("col8-<d2v1>-<d3v2>"),
5543 Bytes.toBytes("col9-<d2v1>-<d3v2>") };
5544 for (int i = 0; i < 10; i++) {
5545 Put put = new Put(ROWS[i]);
5546 put.add(FAMILY, QUALIFIERS[i], VALUE);
5547 ht.put(put);
5548 }
5549 Scan scan = new Scan();
5550 scan.setReversed(true);
5551 scan.addFamily(FAMILY);
5552 Filter filter = new KeyOnlyFilter(true);
5553 scan.setFilter(filter);
5554 ResultScanner scanner = ht.getScanner(scan);
5555 int count = 0;
5556 for (Result result : ht.getScanner(scan)) {
5557 assertEquals(result.size(), 1);
5558 assertEquals(result.raw()[0].getValueLength(), Bytes.SIZEOF_INT);
5559 assertEquals(Bytes.toInt(result.raw()[0].getValue()), VALUE.length);
5560 count++;
5561 }
5562 assertEquals(count, 10);
5563 scanner.close();
5564 ht.close();
5565 }
5566
5567
5568
5569
5570 @Test
5571 public void testSimpleMissingWithReverseScan() throws Exception {
5572 byte[] TABLE = Bytes.toBytes("testSimpleMissingWithReverseScan");
5573 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
5574 byte[][] ROWS = makeN(ROW, 4);
5575
5576
5577 Scan scan = new Scan();
5578 scan.setReversed(true);
5579 Result result = getSingleScanResult(ht, scan);
5580 assertNullResult(result);
5581
5582 scan = new Scan(ROWS[0]);
5583 scan.setReversed(true);
5584 result = getSingleScanResult(ht, scan);
5585 assertNullResult(result);
5586
5587 scan = new Scan(ROWS[0], ROWS[1]);
5588 scan.setReversed(true);
5589 result = getSingleScanResult(ht, scan);
5590 assertNullResult(result);
5591
5592 scan = new Scan();
5593 scan.setReversed(true);
5594 scan.addFamily(FAMILY);
5595 result = getSingleScanResult(ht, scan);
5596 assertNullResult(result);
5597
5598 scan = new Scan();
5599 scan.setReversed(true);
5600 scan.addColumn(FAMILY, QUALIFIER);
5601 result = getSingleScanResult(ht, scan);
5602 assertNullResult(result);
5603
5604
5605
5606 Put put = new Put(ROWS[2]);
5607 put.add(FAMILY, QUALIFIER, VALUE);
5608 ht.put(put);
5609
5610
5611 scan = new Scan();
5612 scan.setReversed(true);
5613 result = getSingleScanResult(ht, scan);
5614 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
5615
5616 scan = new Scan(ROWS[3], ROWS[0]);
5617 scan.setReversed(true);
5618 result = getSingleScanResult(ht, scan);
5619 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
5620
5621 scan = new Scan(ROWS[2], ROWS[1]);
5622 scan.setReversed(true);
5623 result = getSingleScanResult(ht, scan);
5624 assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
5625
5626
5627
5628 scan = new Scan(ROWS[1]);
5629 scan.setReversed(true);
5630 result = getSingleScanResult(ht, scan);
5631 assertNullResult(result);
5632 ht.close();
5633 }
5634
5635 @Test
5636 public void testNullWithReverseScan() throws Exception {
5637 byte[] TABLE = Bytes.toBytes("testNullWithReverseScan");
5638 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
5639
5640 Put put = new Put(ROW);
5641 put.add(FAMILY, null, VALUE);
5642 ht.put(put);
5643 scanTestNull(ht, ROW, FAMILY, VALUE, true);
5644 Delete delete = new Delete(ROW);
5645 delete.deleteColumns(FAMILY, null);
5646 ht.delete(delete);
5647
5648 byte[] TABLE2 = Bytes.toBytes("testNull2WithReverseScan");
5649 ht = TEST_UTIL.createTable(TABLE2, FAMILY);
5650
5651 put = new Put(ROW);
5652 put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
5653 ht.put(put);
5654 scanTestNull(ht, ROW, FAMILY, VALUE, true);
5655 TEST_UTIL.flush();
5656 scanTestNull(ht, ROW, FAMILY, VALUE, true);
5657 delete = new Delete(ROW);
5658 delete.deleteColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY);
5659 ht.delete(delete);
5660
5661 put = new Put(ROW);
5662 put.add(FAMILY, QUALIFIER, null);
5663 ht.put(put);
5664 Scan scan = new Scan();
5665 scan.setReversed(true);
5666 scan.addColumn(FAMILY, QUALIFIER);
5667 Result result = getSingleScanResult(ht, scan);
5668 assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
5669 ht.close();
5670 }
5671
5672 @Test
5673 public void testDeletesWithReverseScan() throws Exception {
5674 byte[] TABLE = Bytes.toBytes("testDeletesWithReverseScan");
5675 byte[][] ROWS = makeNAscii(ROW, 6);
5676 byte[][] FAMILIES = makeNAscii(FAMILY, 3);
5677 byte[][] VALUES = makeN(VALUE, 5);
5678 long[] ts = { 1000, 2000, 3000, 4000, 5000 };
5679 HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES,
5680 TEST_UTIL.getConfiguration(), 3);
5681
5682 Put put = new Put(ROW);
5683 put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
5684 put.add(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]);
5685 ht.put(put);
5686
5687 Delete delete = new Delete(ROW);
5688 delete.deleteFamily(FAMILIES[0], ts[0]);
5689 ht.delete(delete);
5690
5691 Scan scan = new Scan(ROW);
5692 scan.setReversed(true);
5693 scan.addFamily(FAMILIES[0]);
5694 scan.setMaxVersions(Integer.MAX_VALUE);
5695 Result result = getSingleScanResult(ht, scan);
5696 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1] },
5697 new byte[][] { VALUES[1] }, 0, 0);
5698
5699
5700 put = new Put(ROW);
5701 put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]);
5702 put.add(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]);
5703 put.add(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]);
5704 put.add(FAMILIES[0], null, ts[4], VALUES[4]);
5705 put.add(FAMILIES[0], null, ts[2], VALUES[2]);
5706 put.add(FAMILIES[0], null, ts[3], VALUES[3]);
5707 ht.put(put);
5708
5709 delete = new Delete(ROW);
5710 delete.deleteColumn(FAMILIES[0], QUALIFIER);
5711 ht.delete(delete);
5712
5713 scan = new Scan(ROW);
5714 scan.setReversed(true);
5715 scan.addColumn(FAMILIES[0], QUALIFIER);
5716 scan.setMaxVersions(Integer.MAX_VALUE);
5717 result = getSingleScanResult(ht, scan);
5718 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1],
5719 ts[2], ts[3] }, new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2);
5720
5721
5722 delete = new Delete(ROW);
5723 delete.deleteColumn(FAMILIES[0], null);
5724 ht.delete(delete);
5725
5726
5727 delete = new Delete(ROW);
5728 delete.deleteColumns(FAMILIES[0], null);
5729 ht.delete(delete);
5730
5731
5732
5733
5734 put = new Put(ROW);
5735 put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
5736 put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]);
5737 ht.put(put);
5738
5739
5740
5741
5742 scan = new Scan(ROW);
5743 scan.setReversed(true);
5744 scan.addFamily(FAMILIES[0]);
5745 scan.setMaxVersions(Integer.MAX_VALUE);
5746 result = getSingleScanResult(ht, scan);
5747 assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1],
5748 ts[2], ts[3] }, new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2);
5749
5750
5751
5752
5753 put = new Put(ROWS[0]);
5754 put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
5755 put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
5756 put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
5757 put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
5758 ht.put(put);
5759
5760 put = new Put(ROWS[1]);
5761 put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
5762 put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
5763 put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
5764 put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
5765 ht.put(put);
5766
5767 put = new Put(ROWS[2]);
5768 put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
5769 put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
5770 put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
5771 put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
5772 ht.put(put);
5773
5774 delete = new Delete(ROWS[0]);
5775 delete.deleteFamily(FAMILIES[2]);
5776 ht.delete(delete);
5777
5778 delete = new Delete(ROWS[1]);
5779 delete.deleteColumns(FAMILIES[1], QUALIFIER);
5780 ht.delete(delete);
5781
5782 delete = new Delete(ROWS[2]);
5783 delete.deleteColumn(FAMILIES[1], QUALIFIER);
5784 delete.deleteColumn(FAMILIES[1], QUALIFIER);
5785 delete.deleteColumn(FAMILIES[2], QUALIFIER);
5786 ht.delete(delete);
5787
5788 scan = new Scan(ROWS[0]);
5789 scan.setReversed(true);
5790 scan.addFamily(FAMILIES[1]);
5791 scan.addFamily(FAMILIES[2]);
5792 scan.setMaxVersions(Integer.MAX_VALUE);
5793 result = getSingleScanResult(ht, scan);
5794 assertTrue("Expected 2 keys but received " + result.size(),
5795 result.size() == 2);
5796 assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, new long[] { ts[0],
5797 ts[1] }, new byte[][] { VALUES[0], VALUES[1] }, 0, 1);
5798
5799 scan = new Scan(ROWS[1]);
5800 scan.setReversed(true);
5801 scan.addFamily(FAMILIES[1]);
5802 scan.addFamily(FAMILIES[2]);
5803 scan.setMaxVersions(Integer.MAX_VALUE);
5804 result = getSingleScanResult(ht, scan);
5805 assertTrue("Expected 2 keys but received " + result.size(),
5806 result.size() == 2);
5807
5808 scan = new Scan(ROWS[2]);
5809 scan.setReversed(true);
5810 scan.addFamily(FAMILIES[1]);
5811 scan.addFamily(FAMILIES[2]);
5812 scan.setMaxVersions(Integer.MAX_VALUE);
5813 result = getSingleScanResult(ht, scan);
5814 assertEquals(1, result.size());
5815 assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
5816 new long[] { ts[2] }, new byte[][] { VALUES[2] }, 0, 0);
5817
5818
5819
5820 delete = new Delete(ROWS[3]);
5821 delete.deleteFamily(FAMILIES[1]);
5822 ht.delete(delete);
5823
5824 put = new Put(ROWS[3]);
5825 put.add(FAMILIES[2], QUALIFIER, VALUES[0]);
5826 ht.put(put);
5827
5828 put = new Put(ROWS[4]);
5829 put.add(FAMILIES[1], QUALIFIER, VALUES[1]);
5830 put.add(FAMILIES[2], QUALIFIER, VALUES[2]);
5831 ht.put(put);
5832
5833 scan = new Scan(ROWS[4]);
5834 scan.setReversed(true);
5835 scan.addFamily(FAMILIES[1]);
5836 scan.addFamily(FAMILIES[2]);
5837 scan.setMaxVersions(Integer.MAX_VALUE);
5838 ResultScanner scanner = ht.getScanner(scan);
5839 result = scanner.next();
5840 assertTrue("Expected 2 keys but received " + result.size(),
5841 result.size() == 2);
5842 assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[4]));
5843 assertTrue(Bytes.equals(result.raw()[1].getRow(), ROWS[4]));
5844 assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[1]));
5845 assertTrue(Bytes.equals(result.raw()[1].getValue(), VALUES[2]));
5846 result = scanner.next();
5847 assertTrue("Expected 1 key but received " + result.size(),
5848 result.size() == 1);
5849 assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[3]));
5850 assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[0]));
5851 scanner.close();
5852 ht.close();
5853 }
5854
5855
5856
5857
5858 @Test
5859 public void testReversedScanUnderMultiRegions() throws Exception {
5860
5861 byte[] TABLE = Bytes.toBytes("testReversedScanUnderMultiRegions");
5862 byte[] maxByteArray = ReversedClientScanner.MAX_BYTE_ARRAY;
5863 byte[][] splitRows = new byte[][] { Bytes.toBytes("005"),
5864 Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)),
5865 Bytes.toBytes("006"),
5866 Bytes.add(Bytes.toBytes("006"), Bytes.multiple(maxByteArray, 8)),
5867 Bytes.toBytes("007"),
5868 Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)),
5869 Bytes.toBytes("008"), Bytes.multiple(maxByteArray, 2) };
5870 HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
5871 TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
5872
5873 assertEquals(splitRows.length + 1, table.getRegionLocations().size());
5874
5875 int insertNum = splitRows.length;
5876 for (int i = 0; i < insertNum; i++) {
5877 Put put = new Put(splitRows[i]);
5878 put.add(FAMILY, QUALIFIER, VALUE);
5879 table.put(put);
5880 }
5881
5882
5883 ResultScanner scanner = table.getScanner(new Scan());
5884 int count = 0;
5885 for (Result r : scanner) {
5886 assertTrue(!r.isEmpty());
5887 count++;
5888 }
5889 assertEquals(insertNum, count);
5890
5891
5892 Scan scan = new Scan();
5893 scan.setReversed(true);
5894 scanner = table.getScanner(scan);
5895 count = 0;
5896 byte[] lastRow = null;
5897 for (Result r : scanner) {
5898 assertTrue(!r.isEmpty());
5899 count++;
5900 byte[] thisRow = r.getRow();
5901 if (lastRow != null) {
5902 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
5903 + ",this row=" + Bytes.toString(thisRow),
5904 Bytes.compareTo(thisRow, lastRow) < 0);
5905 }
5906 lastRow = thisRow;
5907 }
5908 assertEquals(insertNum, count);
5909 table.close();
5910 }
5911
5912
5913
5914
5915
5916 @Test
5917 public void testSmallReversedScanUnderMultiRegions() throws Exception {
5918
5919 byte[] TABLE = Bytes.toBytes("testSmallReversedScanUnderMultiRegions");
5920 byte[][] splitRows = new byte[][]{
5921 Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"),
5922 Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")};
5923 HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
5924 TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
5925
5926 assertEquals(splitRows.length + 1, table.getRegionLocations().size());
5927 for (byte[] splitRow : splitRows) {
5928 Put put = new Put(splitRow);
5929 put.add(FAMILY, QUALIFIER, VALUE);
5930 table.put(put);
5931
5932 byte[] nextRow = Bytes.copy(splitRow);
5933 nextRow[nextRow.length - 1]++;
5934
5935 put = new Put(nextRow);
5936 put.add(FAMILY, QUALIFIER, VALUE);
5937 table.put(put);
5938 }
5939
5940
5941 ResultScanner scanner = table.getScanner(new Scan());
5942 int count = 0;
5943 for (Result r : scanner) {
5944 assertTrue(!r.isEmpty());
5945 count++;
5946 }
5947 assertEquals(12, count);
5948
5949 reverseScanTest(table, false);
5950 reverseScanTest(table, true);
5951
5952 table.close();
5953 }
5954
5955 private void reverseScanTest(HTable table, boolean small) throws IOException {
5956
5957 Scan scan = new Scan();
5958 scan.setReversed(true);
5959 ResultScanner scanner = table.getScanner(scan);
5960 int count = 0;
5961 byte[] lastRow = null;
5962 for (Result r : scanner) {
5963 assertTrue(!r.isEmpty());
5964 count++;
5965 byte[] thisRow = r.getRow();
5966 if (lastRow != null) {
5967 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
5968 + ",this row=" + Bytes.toString(thisRow),
5969 Bytes.compareTo(thisRow, lastRow) < 0);
5970 }
5971 lastRow = thisRow;
5972 }
5973 assertEquals(12, count);
5974
5975 scan = new Scan();
5976 scan.setSmall(small);
5977 scan.setReversed(true);
5978 scan.setStartRow(Bytes.toBytes("002"));
5979 scanner = table.getScanner(scan);
5980 count = 0;
5981 lastRow = null;
5982 for (Result r : scanner) {
5983 assertTrue(!r.isEmpty());
5984 count++;
5985 byte[] thisRow = r.getRow();
5986 if (lastRow != null) {
5987 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
5988 + ",this row=" + Bytes.toString(thisRow),
5989 Bytes.compareTo(thisRow, lastRow) < 0);
5990 }
5991 lastRow = thisRow;
5992 }
5993 assertEquals(3, count);
5994
5995 scan = new Scan();
5996 scan.setSmall(small);
5997 scan.setReversed(true);
5998 scan.setStartRow(Bytes.toBytes("002"));
5999 scan.setStopRow(Bytes.toBytes("000"));
6000 scanner = table.getScanner(scan);
6001 count = 0;
6002 lastRow = null;
6003 for (Result r : scanner) {
6004 assertTrue(!r.isEmpty());
6005 count++;
6006 byte[] thisRow = r.getRow();
6007 if (lastRow != null) {
6008 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6009 + ",this row=" + Bytes.toString(thisRow),
6010 Bytes.compareTo(thisRow, lastRow) < 0);
6011 }
6012 lastRow = thisRow;
6013 }
6014 assertEquals(2, count);
6015
6016 scan = new Scan();
6017 scan.setSmall(small);
6018 scan.setReversed(true);
6019 scan.setStartRow(Bytes.toBytes("001"));
6020 scanner = table.getScanner(scan);
6021 count = 0;
6022 lastRow = null;
6023 for (Result r : scanner) {
6024 assertTrue(!r.isEmpty());
6025 count++;
6026 byte[] thisRow = r.getRow();
6027 if (lastRow != null) {
6028 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6029 + ",this row=" + Bytes.toString(thisRow),
6030 Bytes.compareTo(thisRow, lastRow) < 0);
6031 }
6032 lastRow = thisRow;
6033 }
6034 assertEquals(2, count);
6035
6036 scan = new Scan();
6037 scan.setSmall(small);
6038 scan.setReversed(true);
6039 scan.setStartRow(Bytes.toBytes("000"));
6040 scanner = table.getScanner(scan);
6041 count = 0;
6042 lastRow = null;
6043 for (Result r : scanner) {
6044 assertTrue(!r.isEmpty());
6045 count++;
6046 byte[] thisRow = r.getRow();
6047 if (lastRow != null) {
6048 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6049 + ",this row=" + Bytes.toString(thisRow),
6050 Bytes.compareTo(thisRow, lastRow) < 0);
6051 }
6052 lastRow = thisRow;
6053 }
6054 assertEquals(1, count);
6055
6056 scan = new Scan();
6057 scan.setSmall(small);
6058 scan.setReversed(true);
6059 scan.setStartRow(Bytes.toBytes("006"));
6060 scan.setStopRow(Bytes.toBytes("002"));
6061 scanner = table.getScanner(scan);
6062 count = 0;
6063 lastRow = null;
6064 for (Result r : scanner) {
6065 assertTrue(!r.isEmpty());
6066 count++;
6067 byte[] thisRow = r.getRow();
6068 if (lastRow != null) {
6069 assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6070 + ",this row=" + Bytes.toString(thisRow),
6071 Bytes.compareTo(thisRow, lastRow) < 0);
6072 }
6073 lastRow = thisRow;
6074 }
6075 assertEquals(4, count);
6076 }
6077 }