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