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