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