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);
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,
2684 byte [] value)
2685 throws Exception {
2686
2687 Scan scan = new Scan();
2688 scan.addColumn(family, null);
2689 Result result = getSingleScanResult(ht, scan);
2690 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2691
2692 scan = new Scan();
2693 scan.addColumn(family, HConstants.EMPTY_BYTE_ARRAY);
2694 result = getSingleScanResult(ht, scan);
2695 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2696
2697 scan = new Scan();
2698 scan.addFamily(family);
2699 result = getSingleScanResult(ht, scan);
2700 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2701
2702 scan = new Scan();
2703 result = getSingleScanResult(ht, scan);
2704 assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2705
2706 }
2707
2708 private void singleRowGetTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
2709 byte [][] QUALIFIERS, byte [][] VALUES)
2710 throws Exception {
2711
2712
2713 Get get = new Get(ROWS[0]);
2714 get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2715 Result result = ht.get(get);
2716 assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
2717
2718
2719 get = new Get(ROWS[0]);
2720 get.addColumn(FAMILIES[2], QUALIFIERS[2]);
2721 result = ht.get(get);
2722 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
2723
2724
2725 get = new Get(ROWS[0]);
2726 get.addFamily(FAMILIES[7]);
2727 result = ht.get(get);
2728 assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
2729
2730
2731
2732 get = new Get(ROWS[0]);
2733 get.addFamily(FAMILIES[4]);
2734 result = ht.get(get);
2735 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2736 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2737
2738
2739
2740 get = new Get(ROWS[0]);
2741 get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2742 get.addColumn(FAMILIES[4], QUALIFIERS[4]);
2743 result = ht.get(get);
2744 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2745 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2746
2747
2748
2749 get = new Get(ROWS[0]);
2750 get.addFamily(FAMILIES[4]);
2751 get.addFamily(FAMILIES[7]);
2752 result = ht.get(get);
2753 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2754 new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
2755
2756
2757 get = new Get(ROWS[0]);
2758 get.addFamily(FAMILIES[2]);
2759 get.addFamily(FAMILIES[4]);
2760 get.addFamily(FAMILIES[6]);
2761 get.addFamily(FAMILIES[7]);
2762 result = ht.get(get);
2763 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2764 new int [][] {
2765 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2766 });
2767
2768
2769 get = new Get(ROWS[0]);
2770 get.addColumn(FAMILIES[2], QUALIFIERS[2]);
2771 get.addColumn(FAMILIES[2], QUALIFIERS[4]);
2772 get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2773 get.addColumn(FAMILIES[4], QUALIFIERS[4]);
2774 get.addColumn(FAMILIES[6], QUALIFIERS[6]);
2775 get.addColumn(FAMILIES[6], QUALIFIERS[7]);
2776 get.addColumn(FAMILIES[7], QUALIFIERS[7]);
2777 get.addColumn(FAMILIES[7], QUALIFIERS[8]);
2778 result = ht.get(get);
2779 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2780 new int [][] {
2781 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2782 });
2783
2784
2785 get = new Get(ROWS[0]);
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}, {9, 0, 0}
2790 });
2791
2792
2793
2794 get = new Get(ROWS[1]);
2795 result = ht.get(get);
2796 assertEmptyResult(result);
2797
2798 get = new Get(ROWS[0]);
2799 get.addColumn(FAMILIES[4], QUALIFIERS[3]);
2800 get.addColumn(FAMILIES[2], QUALIFIERS[3]);
2801 result = ht.get(get);
2802 assertEmptyResult(result);
2803
2804 }
2805
2806 private void singleRowScanTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
2807 byte [][] QUALIFIERS, byte [][] VALUES)
2808 throws Exception {
2809
2810
2811 Scan scan = new Scan();
2812 scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2813 Result result = getSingleScanResult(ht, scan);
2814 assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
2815
2816
2817 scan = new Scan();
2818 scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
2819 result = getSingleScanResult(ht, scan);
2820 assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
2821
2822
2823 scan = new Scan();
2824 scan.addFamily(FAMILIES[7]);
2825 result = getSingleScanResult(ht, scan);
2826 assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
2827
2828
2829
2830 scan = new Scan();
2831 scan.addFamily(FAMILIES[4]);
2832 result = getSingleScanResult(ht, scan);
2833 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2834 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2835
2836
2837
2838 scan = new Scan();
2839 scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2840 scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
2841 result = getSingleScanResult(ht, scan);
2842 assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2843 FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2844
2845
2846
2847 scan = new Scan();
2848 scan.addFamily(FAMILIES[4]);
2849 scan.addFamily(FAMILIES[7]);
2850 result = getSingleScanResult(ht, scan);
2851 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2852 new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
2853
2854
2855 scan = new Scan();
2856 scan.addFamily(FAMILIES[2]);
2857 scan.addFamily(FAMILIES[4]);
2858 scan.addFamily(FAMILIES[6]);
2859 scan.addFamily(FAMILIES[7]);
2860 result = getSingleScanResult(ht, scan);
2861 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2862 new int [][] {
2863 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2864 });
2865
2866
2867 scan = new Scan();
2868 scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
2869 scan.addColumn(FAMILIES[2], QUALIFIERS[4]);
2870 scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2871 scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
2872 scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
2873 scan.addColumn(FAMILIES[6], QUALIFIERS[7]);
2874 scan.addColumn(FAMILIES[7], QUALIFIERS[7]);
2875 scan.addColumn(FAMILIES[7], QUALIFIERS[8]);
2876 result = getSingleScanResult(ht, scan);
2877 assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2878 new int [][] {
2879 {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2880 });
2881
2882
2883 scan = new Scan();
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}, {9, 0, 0}
2888 });
2889
2890
2891
2892 scan = new Scan(ROWS[1]);
2893 result = getSingleScanResult(ht, scan);
2894 assertNullResult(result);
2895
2896 scan = new Scan();
2897 scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
2898 scan.addColumn(FAMILIES[2], QUALIFIERS[3]);
2899 result = getSingleScanResult(ht, scan);
2900 assertNullResult(result);
2901 }
2902
2903
2904
2905
2906
2907
2908 private void getVerifySingleColumn(HTable ht,
2909 byte [][] ROWS, int ROWIDX,
2910 byte [][] FAMILIES, int FAMILYIDX,
2911 byte [][] QUALIFIERS, int QUALIFIERIDX,
2912 byte [][] VALUES, int VALUEIDX)
2913 throws Exception {
2914
2915 Get get = new Get(ROWS[ROWIDX]);
2916 Result result = ht.get(get);
2917 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2918 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2919
2920 get = new Get(ROWS[ROWIDX]);
2921 get.addFamily(FAMILIES[FAMILYIDX]);
2922 result = ht.get(get);
2923 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2924 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2925
2926 get = new Get(ROWS[ROWIDX]);
2927 get.addFamily(FAMILIES[FAMILYIDX-2]);
2928 get.addFamily(FAMILIES[FAMILYIDX]);
2929 get.addFamily(FAMILIES[FAMILYIDX+2]);
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.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[0]);
2936 result = ht.get(get);
2937 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2938 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2939
2940 get = new Get(ROWS[ROWIDX]);
2941 get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[1]);
2942 get.addFamily(FAMILIES[FAMILYIDX]);
2943 result = ht.get(get);
2944 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2945 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2946
2947 get = new Get(ROWS[ROWIDX]);
2948 get.addFamily(FAMILIES[FAMILYIDX]);
2949 get.addColumn(FAMILIES[FAMILYIDX+1], QUALIFIERS[1]);
2950 get.addColumn(FAMILIES[FAMILYIDX-2], QUALIFIERS[1]);
2951 get.addFamily(FAMILIES[FAMILYIDX-1]);
2952 get.addFamily(FAMILIES[FAMILYIDX+2]);
2953 result = ht.get(get);
2954 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2955 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2956
2957 }
2958
2959
2960
2961
2962
2963
2964
2965
2966 private void scanVerifySingleColumn(HTable ht,
2967 byte [][] ROWS, int ROWIDX,
2968 byte [][] FAMILIES, int FAMILYIDX,
2969 byte [][] QUALIFIERS, int QUALIFIERIDX,
2970 byte [][] VALUES, int VALUEIDX)
2971 throws Exception {
2972
2973 Scan scan = new Scan();
2974 Result result = getSingleScanResult(ht, scan);
2975 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2976 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2977
2978 scan = new Scan(ROWS[ROWIDX]);
2979 result = getSingleScanResult(ht, scan);
2980 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2981 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2982
2983 scan = new Scan(ROWS[ROWIDX], ROWS[ROWIDX+1]);
2984 result = getSingleScanResult(ht, scan);
2985 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2986 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2987
2988 scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX+1]);
2989 result = getSingleScanResult(ht, scan);
2990 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2991 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2992
2993 scan = new Scan();
2994 scan.addFamily(FAMILIES[FAMILYIDX]);
2995 result = getSingleScanResult(ht, scan);
2996 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2997 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2998
2999 scan = new Scan();
3000 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
3001 result = getSingleScanResult(ht, scan);
3002 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3003 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3004
3005 scan = new Scan();
3006 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
3007 scan.addFamily(FAMILIES[FAMILYIDX]);
3008 result = getSingleScanResult(ht, scan);
3009 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3010 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3011
3012 scan = new Scan();
3013 scan.addColumn(FAMILIES[FAMILYIDX-1], QUALIFIERS[QUALIFIERIDX+1]);
3014 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
3015 scan.addFamily(FAMILIES[FAMILYIDX+1]);
3016 result = getSingleScanResult(ht, scan);
3017 assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3018 QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3019
3020 }
3021
3022
3023
3024
3025
3026 private void getVerifySingleEmpty(HTable ht,
3027 byte [][] ROWS, int ROWIDX,
3028 byte [][] FAMILIES, int FAMILYIDX,
3029 byte [][] QUALIFIERS, int QUALIFIERIDX)
3030 throws Exception {
3031
3032 Get get = new Get(ROWS[ROWIDX]);
3033 get.addFamily(FAMILIES[4]);
3034 get.addColumn(FAMILIES[4], QUALIFIERS[1]);
3035 Result result = ht.get(get);
3036 assertEmptyResult(result);
3037
3038 get = new Get(ROWS[ROWIDX]);
3039 get.addFamily(FAMILIES[4]);
3040 get.addColumn(FAMILIES[4], QUALIFIERS[2]);
3041 result = ht.get(get);
3042 assertEmptyResult(result);
3043
3044 get = new Get(ROWS[ROWIDX]);
3045 get.addFamily(FAMILIES[3]);
3046 get.addColumn(FAMILIES[4], QUALIFIERS[2]);
3047 get.addFamily(FAMILIES[5]);
3048 result = ht.get(get);
3049 assertEmptyResult(result);
3050
3051 get = new Get(ROWS[ROWIDX+1]);
3052 result = ht.get(get);
3053 assertEmptyResult(result);
3054
3055 }
3056
3057 private void scanVerifySingleEmpty(HTable ht,
3058 byte [][] ROWS, int ROWIDX,
3059 byte [][] FAMILIES, int FAMILYIDX,
3060 byte [][] QUALIFIERS, int QUALIFIERIDX)
3061 throws Exception {
3062
3063 Scan scan = new Scan(ROWS[ROWIDX+1]);
3064 Result result = getSingleScanResult(ht, scan);
3065 assertNullResult(result);
3066
3067 scan = new Scan(ROWS[ROWIDX+1],ROWS[ROWIDX+2]);
3068 result = getSingleScanResult(ht, scan);
3069 assertNullResult(result);
3070
3071 scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX]);
3072 result = getSingleScanResult(ht, scan);
3073 assertNullResult(result);
3074
3075 scan = new Scan();
3076 scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
3077 scan.addFamily(FAMILIES[FAMILYIDX-1]);
3078 result = getSingleScanResult(ht, scan);
3079 assertNullResult(result);
3080
3081 }
3082
3083
3084
3085
3086
3087 private void assertKey(Cell key, byte [] row, byte [] family,
3088 byte [] qualifier, byte [] value)
3089 throws Exception {
3090 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3091 "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
3092 equals(row, CellUtil.cloneRow(key)));
3093 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3094 "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3095 equals(family, CellUtil.cloneFamily(key)));
3096 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3097 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]",
3098 equals(qualifier, CellUtil.cloneQualifier(key)));
3099 assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3100 "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]",
3101 equals(value, CellUtil.cloneValue(key)));
3102 }
3103
3104 private void assertIncrementKey(Cell key, byte [] row, byte [] family,
3105 byte [] qualifier, long value)
3106 throws Exception {
3107 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3108 "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
3109 equals(row, CellUtil.cloneRow(key)));
3110 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3111 "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3112 equals(family, CellUtil.cloneFamily(key)));
3113 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3114 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]",
3115 equals(qualifier, CellUtil.cloneQualifier(key)));
3116 assertTrue("Expected value [" + value + "] " +
3117 "Got value [" + Bytes.toLong(CellUtil.cloneValue(key)) + "]",
3118 Bytes.toLong(CellUtil.cloneValue(key)) == value);
3119 }
3120
3121 private void assertNumKeys(Result result, int n) throws Exception {
3122 assertTrue("Expected " + n + " keys but got " + result.size(),
3123 result.size() == n);
3124 }
3125
3126 private void assertNResult(Result result, byte [] row,
3127 byte [][] families, byte [][] qualifiers, byte [][] values,
3128 int [][] idxs)
3129 throws Exception {
3130 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3131 "Got row [" + Bytes.toString(result.getRow()) +"]",
3132 equals(row, result.getRow()));
3133 assertTrue("Expected " + idxs.length + " keys but result contains "
3134 + result.size(), result.size() == idxs.length);
3135
3136 Cell [] keys = result.rawCells();
3137
3138 for(int i=0;i<keys.length;i++) {
3139 byte [] family = families[idxs[i][0]];
3140 byte [] qualifier = qualifiers[idxs[i][1]];
3141 byte [] value = values[idxs[i][2]];
3142 Cell key = keys[i];
3143
3144 byte[] famb = CellUtil.cloneFamily(key);
3145 byte[] qualb = CellUtil.cloneQualifier(key);
3146 byte[] valb = CellUtil.cloneValue(key);
3147 assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
3148 + "] " + "Got family [" + Bytes.toString(famb) + "]",
3149 equals(family, famb));
3150 assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
3151 + "] " + "Got qualifier [" + Bytes.toString(qualb) + "]",
3152 equals(qualifier, qualb));
3153 assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
3154 + "Got value [" + Bytes.toString(valb) + "]",
3155 equals(value, valb));
3156 }
3157 }
3158
3159 private void assertNResult(Result result, byte [] row,
3160 byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
3161 int start, int end)
3162 throws IOException {
3163 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3164 "Got row [" + Bytes.toString(result.getRow()) +"]",
3165 equals(row, result.getRow()));
3166 int expectedResults = end - start + 1;
3167 assertEquals(expectedResults, result.size());
3168
3169 Cell[] keys = result.rawCells();
3170
3171 for (int i=0; i<keys.length; i++) {
3172 byte [] value = values[end-i];
3173 long ts = stamps[end-i];
3174 Cell key = keys[i];
3175
3176 assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
3177 + "] " + "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3178 CellUtil.matchingFamily(key, family));
3179 assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
3180 + "] " + "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key))+ "]",
3181 CellUtil.matchingQualifier(key, qualifier));
3182 assertTrue("Expected ts [" + ts + "] " +
3183 "Got ts [" + key.getTimestamp() + "]", ts == key.getTimestamp());
3184 assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
3185 + "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]",
3186 CellUtil.matchingValue(key, value));
3187 }
3188 }
3189
3190
3191
3192
3193
3194 private void assertDoubleResult(Result result, byte [] row,
3195 byte [] familyA, byte [] qualifierA, byte [] valueA,
3196 byte [] familyB, byte [] qualifierB, byte [] valueB)
3197 throws Exception {
3198 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3199 "Got row [" + Bytes.toString(result.getRow()) +"]",
3200 equals(row, result.getRow()));
3201 assertTrue("Expected two keys but result contains " + result.size(),
3202 result.size() == 2);
3203 Cell [] kv = result.rawCells();
3204 Cell kvA = kv[0];
3205 assertTrue("(A) Expected family [" + Bytes.toString(familyA) + "] " +
3206 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvA)) + "]",
3207 equals(familyA, CellUtil.cloneFamily(kvA)));
3208 assertTrue("(A) Expected qualifier [" + Bytes.toString(qualifierA) + "] " +
3209 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvA)) + "]",
3210 equals(qualifierA, CellUtil.cloneQualifier(kvA)));
3211 assertTrue("(A) Expected value [" + Bytes.toString(valueA) + "] " +
3212 "Got value [" + Bytes.toString(CellUtil.cloneValue(kvA)) + "]",
3213 equals(valueA, CellUtil.cloneValue(kvA)));
3214 Cell kvB = kv[1];
3215 assertTrue("(B) Expected family [" + Bytes.toString(familyB) + "] " +
3216 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvB)) + "]",
3217 equals(familyB, CellUtil.cloneFamily(kvB)));
3218 assertTrue("(B) Expected qualifier [" + Bytes.toString(qualifierB) + "] " +
3219 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvB)) + "]",
3220 equals(qualifierB, CellUtil.cloneQualifier(kvB)));
3221 assertTrue("(B) Expected value [" + Bytes.toString(valueB) + "] " +
3222 "Got value [" + Bytes.toString(CellUtil.cloneValue(kvB)) + "]",
3223 equals(valueB, CellUtil.cloneValue(kvB)));
3224 }
3225
3226 private void assertSingleResult(Result result, byte [] row, byte [] family,
3227 byte [] qualifier, byte [] value)
3228 throws Exception {
3229 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3230 "Got row [" + Bytes.toString(result.getRow()) +"]",
3231 equals(row, result.getRow()));
3232 assertTrue("Expected a single key but result contains " + result.size(),
3233 result.size() == 1);
3234 Cell kv = result.rawCells()[0];
3235 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3236 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]",
3237 equals(family, CellUtil.cloneFamily(kv)));
3238 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3239 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]",
3240 equals(qualifier, CellUtil.cloneQualifier(kv)));
3241 assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3242 "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]",
3243 equals(value, CellUtil.cloneValue(kv)));
3244 }
3245
3246 private void assertSingleResult(Result result, byte [] row, byte [] family,
3247 byte [] qualifier, long ts, byte [] value)
3248 throws Exception {
3249 assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3250 "Got row [" + Bytes.toString(result.getRow()) +"]",
3251 equals(row, result.getRow()));
3252 assertTrue("Expected a single key but result contains " + result.size(),
3253 result.size() == 1);
3254 Cell kv = result.rawCells()[0];
3255 assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3256 "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]",
3257 equals(family, CellUtil.cloneFamily(kv)));
3258 assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3259 "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]",
3260 equals(qualifier, CellUtil.cloneQualifier(kv)));
3261 assertTrue("Expected ts [" + ts + "] " +
3262 "Got ts [" + kv.getTimestamp() + "]", ts == kv.getTimestamp());
3263 assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3264 "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]",
3265 equals(value, CellUtil.cloneValue(kv)));
3266 }
3267
3268 private void assertEmptyResult(Result result) throws Exception {
3269 assertTrue("expected an empty result but result contains " +
3270 result.size() + " keys", result.isEmpty());
3271 }
3272
3273 private void assertNullResult(Result result) throws Exception {
3274 assertTrue("expected null result but received a non-null result",
3275 result == null);
3276 }
3277
3278
3279
3280
3281
3282 private Result getSingleScanResult(HTable ht, Scan scan) throws IOException {
3283 ResultScanner scanner = ht.getScanner(scan);
3284 Result result = scanner.next();
3285 scanner.close();
3286 return result;
3287 }
3288
3289 private byte [][] makeNAscii(byte [] base, int n) {
3290 if(n > 256) {
3291 return makeNBig(base, n);
3292 }
3293 byte [][] ret = new byte[n][];
3294 for(int i=0;i<n;i++) {
3295 byte [] tail = Bytes.toBytes(Integer.toString(i));
3296 ret[i] = Bytes.add(base, tail);
3297 }
3298 return ret;
3299 }
3300
3301 private byte [][] makeN(byte [] base, int n) {
3302 if (n > 256) {
3303 return makeNBig(base, n);
3304 }
3305 byte [][] ret = new byte[n][];
3306 for(int i=0;i<n;i++) {
3307 ret[i] = Bytes.add(base, new byte[]{(byte)i});
3308 }
3309 return ret;
3310 }
3311
3312 private byte [][] makeNBig(byte [] base, int n) {
3313 byte [][] ret = new byte[n][];
3314 for(int i=0;i<n;i++) {
3315 int byteA = (i % 256);
3316 int byteB = (i >> 8);
3317 ret[i] = Bytes.add(base, new byte[]{(byte)byteB,(byte)byteA});
3318 }
3319 return ret;
3320 }
3321
3322 private long [] makeStamps(int n) {
3323 long [] stamps = new long[n];
3324 for(int i=0;i<n;i++) stamps[i] = i+1;
3325 return stamps;
3326 }
3327
3328 private boolean equals(byte [] left, byte [] right) {
3329 if (left == null && right == null) return true;
3330 if (left == null && right.length == 0) return true;
3331 if (right == null && left.length == 0) return true;
3332 return Bytes.equals(left, right);
3333 }
3334
3335 @Test
3336 public void testDuplicateVersions() throws Exception {
3337 byte [] TABLE = Bytes.toBytes("testDuplicateVersions");
3338
3339 long [] STAMPS = makeStamps(20);
3340 byte [][] VALUES = makeNAscii(VALUE, 20);
3341
3342 HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3343
3344
3345 Put put = new Put(ROW);
3346 put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3347 put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3348 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3349 put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3350 ht.put(put);
3351
3352
3353 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3354 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3355 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3356 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3357 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3358 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3359 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3360 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3361
3362
3363 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3364 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3365 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3366 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3367 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3368 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3369
3370
3371 Get get = new Get(ROW);
3372 get.addColumn(FAMILY, QUALIFIER);
3373 get.setMaxVersions(2);
3374 Result result = ht.get(get);
3375 assertNResult(result, ROW, FAMILY, QUALIFIER,
3376 new long [] {STAMPS[4], STAMPS[5]},
3377 new byte[][] {VALUES[4], VALUES[5]},
3378 0, 1);
3379
3380 Scan scan = new Scan(ROW);
3381 scan.addColumn(FAMILY, QUALIFIER);
3382 scan.setMaxVersions(2);
3383 result = getSingleScanResult(ht, scan);
3384 assertNResult(result, ROW, FAMILY, QUALIFIER,
3385 new long [] {STAMPS[4], STAMPS[5]},
3386 new byte[][] {VALUES[4], VALUES[5]},
3387 0, 1);
3388
3389
3390
3391 TEST_UTIL.flush();
3392
3393
3394 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3395 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3396 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3397 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3398 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3399 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3400 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3401 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3402
3403
3404 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3405 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3406 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3407 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3408 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3409 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3410
3411
3412 get = new Get(ROW);
3413 get.addColumn(FAMILY, QUALIFIER);
3414 get.setMaxVersions(2);
3415 result = ht.get(get);
3416 assertNResult(result, ROW, FAMILY, QUALIFIER,
3417 new long [] {STAMPS[4], STAMPS[5]},
3418 new byte[][] {VALUES[4], VALUES[5]},
3419 0, 1);
3420
3421 scan = new Scan(ROW);
3422 scan.addColumn(FAMILY, QUALIFIER);
3423 scan.setMaxVersions(2);
3424 result = getSingleScanResult(ht, scan);
3425 assertNResult(result, ROW, FAMILY, QUALIFIER,
3426 new long [] {STAMPS[4], STAMPS[5]},
3427 new byte[][] {VALUES[4], VALUES[5]},
3428 0, 1);
3429
3430
3431
3432
3433
3434 put = new Put(ROW);
3435 put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
3436 put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3437 put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
3438 put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3439 put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]);
3440 ht.put(put);
3441
3442
3443 get = new Get(ROW);
3444 get.addColumn(FAMILY, QUALIFIER);
3445 get.setMaxVersions(7);
3446 result = ht.get(get);
3447 assertNResult(result, ROW, FAMILY, QUALIFIER,
3448 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3449 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3450 0, 6);
3451
3452 scan = new Scan(ROW);
3453 scan.addColumn(FAMILY, QUALIFIER);
3454 scan.setMaxVersions(7);
3455 result = getSingleScanResult(ht, scan);
3456 assertNResult(result, ROW, FAMILY, QUALIFIER,
3457 new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3458 new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3459 0, 6);
3460
3461 get = new Get(ROW);
3462 get.setMaxVersions(7);
3463 result = ht.get(get);
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 scan = new Scan(ROW);
3470 scan.setMaxVersions(7);
3471 result = getSingleScanResult(ht, scan);
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
3478 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3479 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3480 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3481 getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3482 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3483 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3484 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3485 scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3486
3487
3488 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3489 getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3490 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3491 scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3492
3493
3494
3495 TEST_UTIL.flush();
3496
3497
3498 put = new Put(ROW);
3499 put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]);
3500 put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]);
3501 put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]);
3502 put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
3503 ht.put(put);
3504
3505 get = new Get(ROW);
3506 get.addColumn(FAMILY, QUALIFIER);
3507 get.setMaxVersions(Integer.MAX_VALUE);
3508 result = ht.get(get);
3509 assertNResult(result, ROW, FAMILY, QUALIFIER,
3510 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3511 new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3512 0, 9);
3513
3514 scan = new Scan(ROW);
3515 scan.addColumn(FAMILY, QUALIFIER);
3516 scan.setMaxVersions(Integer.MAX_VALUE);
3517 result = getSingleScanResult(ht, scan);
3518 assertNResult(result, ROW, FAMILY, QUALIFIER,
3519 new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3520 new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3521 0, 9);
3522
3523
3524 Delete delete = new Delete(ROW);
3525 delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
3526 delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
3527 ht.delete(delete);
3528
3529
3530 get = new Get(ROW);
3531 get.addColumn(FAMILY, QUALIFIER);
3532 get.setMaxVersions(Integer.MAX_VALUE);
3533 result = ht.get(get);
3534 assertNResult(result, ROW, FAMILY, QUALIFIER,
3535 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3536 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3537 0, 9);
3538
3539 scan = new Scan(ROW);
3540 scan.addColumn(FAMILY, QUALIFIER);
3541 scan.setMaxVersions(Integer.MAX_VALUE);
3542 result = getSingleScanResult(ht, scan);
3543 assertNResult(result, ROW, FAMILY, QUALIFIER,
3544 new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3545 new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3546 0, 9);
3547 }
3548
3549 @Test
3550 public void testUpdates() throws Exception {
3551
3552 byte [] TABLE = Bytes.toBytes("testUpdates");
3553 HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3554
3555
3556 byte[] row = Bytes.toBytes("row1");
3557 byte[] qualifier = Bytes.toBytes("myCol");
3558 Put put = new Put(row);
3559 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3560 hTable.put(put);
3561
3562 put = new Put(row);
3563 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3564 hTable.put(put);
3565
3566 put = new Put(row);
3567 put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3568 hTable.put(put);
3569
3570 Get get = new Get(row);
3571 get.addColumn(FAMILY, qualifier);
3572 get.setMaxVersions();
3573
3574
3575
3576 Result result = hTable.get(get);
3577 NavigableMap<Long, byte[]> navigableMap =
3578 result.getMap().get(FAMILY).get(qualifier);
3579 assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3580 assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3581
3582
3583 put = new Put(row);
3584 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3585 hTable.put(put);
3586
3587
3588 put = new Put(row);
3589 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3590 hTable.put(put);
3591
3592
3593 result = hTable.get(get);
3594 navigableMap = result.getMap().get(FAMILY).get(qualifier);
3595 assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3596 assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3597 }
3598
3599 @Test
3600 public void testUpdatesWithMajorCompaction() throws Exception {
3601
3602 String tableName = "testUpdatesWithMajorCompaction";
3603 byte [] TABLE = Bytes.toBytes(tableName);
3604 HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3605 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3606
3607
3608 byte[] row = Bytes.toBytes("row2");
3609 byte[] qualifier = Bytes.toBytes("myCol");
3610 Put put = new Put(row);
3611 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3612 hTable.put(put);
3613
3614 put = new Put(row);
3615 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3616 hTable.put(put);
3617
3618 put = new Put(row);
3619 put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3620 hTable.put(put);
3621
3622 Get get = new Get(row);
3623 get.addColumn(FAMILY, qualifier);
3624 get.setMaxVersions();
3625
3626
3627
3628 Result result = hTable.get(get);
3629 NavigableMap<Long, byte[]> navigableMap =
3630 result.getMap().get(FAMILY).get(qualifier);
3631 assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3632 assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3633
3634
3635 admin.flush(tableName);
3636 admin.majorCompact(tableName);
3637 Thread.sleep(6000);
3638
3639
3640 put = new Put(row);
3641 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3642 hTable.put(put);
3643
3644
3645 put = new Put(row);
3646 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3647 hTable.put(put);
3648
3649
3650 admin.flush(tableName);
3651 admin.majorCompact(tableName);
3652 Thread.sleep(6000);
3653
3654
3655 result = hTable.get(get);
3656 navigableMap = result.getMap().get(FAMILY).get(qualifier);
3657 assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3658 assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3659 }
3660
3661 @Test
3662 public void testMajorCompactionBetweenTwoUpdates() throws Exception {
3663
3664 String tableName = "testMajorCompactionBetweenTwoUpdates";
3665 byte [] TABLE = Bytes.toBytes(tableName);
3666 HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3667 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3668
3669
3670 byte[] row = Bytes.toBytes("row3");
3671 byte[] qualifier = Bytes.toBytes("myCol");
3672 Put put = new Put(row);
3673 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3674 hTable.put(put);
3675
3676 put = new Put(row);
3677 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3678 hTable.put(put);
3679
3680 put = new Put(row);
3681 put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3682 hTable.put(put);
3683
3684 Get get = new Get(row);
3685 get.addColumn(FAMILY, qualifier);
3686 get.setMaxVersions();
3687
3688
3689
3690 Result result = hTable.get(get);
3691 NavigableMap<Long, byte[]> navigableMap =
3692 result.getMap().get(FAMILY).get(qualifier);
3693 assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3694 assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3695
3696
3697 admin.flush(tableName);
3698 admin.majorCompact(tableName);
3699 Thread.sleep(6000);
3700
3701
3702 put = new Put(row);
3703 put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3704 hTable.put(put);
3705
3706
3707 admin.flush(tableName);
3708 admin.majorCompact(tableName);
3709 Thread.sleep(6000);
3710
3711
3712 put = new Put(row);
3713 put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3714 hTable.put(put);
3715
3716
3717 admin.flush(tableName);
3718 admin.majorCompact(tableName);
3719 Thread.sleep(6000);
3720
3721
3722 result = hTable.get(get);
3723 navigableMap = result.getMap().get(FAMILY).get(qualifier);
3724
3725 assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3726 assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3727 }
3728
3729 @Test
3730 public void testGet_EmptyTable() throws IOException {
3731 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY);
3732 Get get = new Get(ROW);
3733 get.addFamily(FAMILY);
3734 Result r = table.get(get);
3735 assertTrue(r.isEmpty());
3736 }
3737
3738 @Test
3739 public void testGet_NullQualifier() throws IOException {
3740 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NullQualifier"), FAMILY);
3741 Put put = new Put(ROW);
3742 put.add(FAMILY, QUALIFIER, VALUE);
3743 table.put(put);
3744
3745 put = new Put(ROW);
3746 put.add(FAMILY, null, VALUE);
3747 table.put(put);
3748 LOG.info("Row put");
3749
3750 Get get = new Get(ROW);
3751 get.addColumn(FAMILY, null);
3752 Result r = table.get(get);
3753 assertEquals(1, r.size());
3754
3755 get = new Get(ROW);
3756 get.addFamily(FAMILY);
3757 r = table.get(get);
3758 assertEquals(2, r.size());
3759 }
3760
3761 @Test
3762 public void testGet_NonExistentRow() throws IOException {
3763 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY);
3764 Put put = new Put(ROW);
3765 put.add(FAMILY, QUALIFIER, VALUE);
3766 table.put(put);
3767 LOG.info("Row put");
3768
3769 Get get = new Get(ROW);
3770 get.addFamily(FAMILY);
3771 Result r = table.get(get);
3772 assertFalse(r.isEmpty());
3773 System.out.println("Row retrieved successfully");
3774
3775 byte [] missingrow = Bytes.toBytes("missingrow");
3776 get = new Get(missingrow);
3777 get.addFamily(FAMILY);
3778 r = table.get(get);
3779 assertTrue(r.isEmpty());
3780 LOG.info("Row missing as it should be");
3781 }
3782
3783 @Test
3784 public void testPut() throws IOException {
3785 final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
3786 final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
3787 final byte [] row1 = Bytes.toBytes("row1");
3788 final byte [] row2 = Bytes.toBytes("row2");
3789 final byte [] value = Bytes.toBytes("abcd");
3790 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
3791 new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
3792 Put put = new Put(row1);
3793 put.add(CONTENTS_FAMILY, null, value);
3794 table.put(put);
3795
3796 put = new Put(row2);
3797 put.add(CONTENTS_FAMILY, null, value);
3798
3799 assertEquals(put.size(), 1);
3800 assertEquals(put.getFamilyCellMap().get(CONTENTS_FAMILY).size(), 1);
3801
3802
3803 KeyValue kv = (KeyValue)put.getFamilyCellMap().get(CONTENTS_FAMILY).get(0);
3804
3805 assertTrue(Bytes.equals(kv.getFamily(), CONTENTS_FAMILY));
3806
3807 assertTrue(Bytes.equals(kv.getQualifier(), new byte[0]));
3808
3809 assertTrue(Bytes.equals(kv.getValue(), value));
3810
3811 table.put(put);
3812
3813 Scan scan = new Scan();
3814 scan.addColumn(CONTENTS_FAMILY, null);
3815 ResultScanner scanner = table.getScanner(scan);
3816 for (Result r : scanner) {
3817 for(Cell key : r.rawCells()) {
3818 System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString());
3819 }
3820 }
3821 }
3822
3823 @Test
3824 public void testPutNoCF() throws IOException {
3825 final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
3826 final byte[] VAL = Bytes.toBytes(100);
3827 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY});
3828
3829 boolean caughtNSCFE = false;
3830
3831 try {
3832 Put p = new Put(ROW);
3833 p.add(BAD_FAM, QUALIFIER, VAL);
3834 table.put(p);
3835 } catch (RetriesExhaustedWithDetailsException e) {
3836 caughtNSCFE = e.getCause(0) instanceof NoSuchColumnFamilyException;
3837 }
3838 assertTrue("Should throw NoSuchColumnFamilyException", caughtNSCFE);
3839
3840 }
3841
3842 @Test
3843 public void testRowsPut() throws IOException {
3844 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3845 final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
3846 final int NB_BATCH_ROWS = 10;
3847 final byte[] value = Bytes.toBytes("abcd");
3848 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
3849 new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
3850 ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3851 for (int i = 0; i < NB_BATCH_ROWS; i++) {
3852 byte[] row = Bytes.toBytes("row" + i);
3853 Put put = new Put(row);
3854 put.setDurability(Durability.SKIP_WAL);
3855 put.add(CONTENTS_FAMILY, null, value);
3856 rowsUpdate.add(put);
3857 }
3858 table.put(rowsUpdate);
3859 Scan scan = new Scan();
3860 scan.addFamily(CONTENTS_FAMILY);
3861 ResultScanner scanner = table.getScanner(scan);
3862 int nbRows = 0;
3863 for (@SuppressWarnings("unused")
3864 Result row : scanner)
3865 nbRows++;
3866 assertEquals(NB_BATCH_ROWS, nbRows);
3867 }
3868
3869 @Test
3870 public void testRowsPutBufferedOneFlush() throws IOException {
3871 final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
3872 final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
3873 final byte [] value = Bytes.toBytes("abcd");
3874 final int NB_BATCH_ROWS = 10;
3875 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
3876 new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
3877 table.setAutoFlush(false, true);
3878 ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3879 for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
3880 byte[] row = Bytes.toBytes("row" + i);
3881 Put put = new Put(row);
3882 put.setDurability(Durability.SKIP_WAL);
3883 put.add(CONTENTS_FAMILY, null, value);
3884 rowsUpdate.add(put);
3885 }
3886 table.put(rowsUpdate);
3887
3888 Scan scan = new Scan();
3889 scan.addFamily(CONTENTS_FAMILY);
3890 ResultScanner scanner = table.getScanner(scan);
3891 int nbRows = 0;
3892 for (@SuppressWarnings("unused")
3893 Result row : scanner)
3894 nbRows++;
3895 assertEquals(0, nbRows);
3896 scanner.close();
3897
3898 table.flushCommits();
3899
3900 scan = new Scan();
3901 scan.addFamily(CONTENTS_FAMILY);
3902 scanner = table.getScanner(scan);
3903 nbRows = 0;
3904 for (@SuppressWarnings("unused")
3905 Result row : scanner)
3906 nbRows++;
3907 assertEquals(NB_BATCH_ROWS * 10, nbRows);
3908 }
3909
3910 @Test
3911 public void testRowsPutBufferedManyManyFlushes() throws IOException {
3912 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3913 final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
3914 final byte[] value = Bytes.toBytes("abcd");
3915 final int NB_BATCH_ROWS = 10;
3916 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
3917 new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
3918 table.setAutoFlush(false, true);
3919 table.setWriteBufferSize(10);
3920 ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3921 for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
3922 byte[] row = Bytes.toBytes("row" + i);
3923 Put put = new Put(row);
3924 put.setDurability(Durability.SKIP_WAL);
3925 put.add(CONTENTS_FAMILY, null, value);
3926 rowsUpdate.add(put);
3927 }
3928 table.put(rowsUpdate);
3929
3930 table.flushCommits();
3931
3932 Scan scan = new Scan();
3933 scan.addFamily(CONTENTS_FAMILY);
3934 ResultScanner scanner = table.getScanner(scan);
3935 int nbRows = 0;
3936 for (@SuppressWarnings("unused")
3937 Result row : scanner)
3938 nbRows++;
3939 assertEquals(NB_BATCH_ROWS * 10, nbRows);
3940 }
3941
3942 @Test
3943 public void testAddKeyValue() throws IOException {
3944 final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3945 final byte[] value = Bytes.toBytes("abcd");
3946 final byte[] row1 = Bytes.toBytes("row1");
3947 final byte[] row2 = Bytes.toBytes("row2");
3948 byte[] qualifier = Bytes.toBytes("qf1");
3949 Put put = new Put(row1);
3950
3951
3952 KeyValue kv = new KeyValue(row1, CONTENTS_FAMILY, qualifier, value);
3953 boolean ok = true;
3954 try {
3955 put.add(kv);
3956 } catch (IOException e) {
3957 ok = false;
3958 }
3959 assertEquals(true, ok);
3960
3961
3962 kv = new KeyValue(row2, CONTENTS_FAMILY, qualifier, value);
3963 ok = false;
3964 try {
3965 put.add(kv);
3966 } catch (IOException e) {
3967 ok = true;
3968 }
3969 assertEquals(true, ok);
3970 }
3971
3972
3973
3974
3975
3976 @Test
3977 public void testHBase737 () throws IOException {
3978 final byte [] FAM1 = Bytes.toBytes("fam1");
3979 final byte [] FAM2 = Bytes.toBytes("fam2");
3980
3981 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
3982 new byte [][] {FAM1, FAM2});
3983
3984 Put put = new Put(ROW);
3985 put.add(FAM1, Bytes.toBytes("letters"), Bytes.toBytes("abcdefg"));
3986 table.put(put);
3987 try {
3988 Thread.sleep(1000);
3989 } catch (InterruptedException i) {
3990
3991 }
3992
3993 put = new Put(ROW);
3994 put.add(FAM1, Bytes.toBytes("numbers"), Bytes.toBytes("123456"));
3995 table.put(put);
3996
3997 try {
3998 Thread.sleep(1000);
3999 } catch (InterruptedException i) {
4000
4001 }
4002
4003 put = new Put(ROW);
4004 put.add(FAM2, Bytes.toBytes("letters"), Bytes.toBytes("hijklmnop"));
4005 table.put(put);
4006
4007 long times[] = new long[3];
4008
4009
4010
4011 Scan scan = new Scan();
4012 scan.addFamily(FAM1);
4013 scan.addFamily(FAM2);
4014 ResultScanner s = table.getScanner(scan);
4015 try {
4016 int index = 0;
4017 Result r = null;
4018 while ((r = s.next()) != null) {
4019 for(Cell key : r.rawCells()) {
4020 times[index++] = key.getTimestamp();
4021 }
4022 }
4023 } finally {
4024 s.close();
4025 }
4026 for (int i = 0; i < times.length - 1; i++) {
4027 for (int j = i + 1; j < times.length; j++) {
4028 assertTrue(times[j] > times[i]);
4029 }
4030 }
4031
4032
4033 TEST_UTIL.flush();
4034
4035
4036 for(int i=0;i<times.length;i++) {
4037 times[i] = 0;
4038 }
4039
4040 try {
4041 Thread.sleep(1000);
4042 } catch (InterruptedException i) {
4043
4044 }
4045 scan = new Scan();
4046 scan.addFamily(FAM1);
4047 scan.addFamily(FAM2);
4048 s = table.getScanner(scan);
4049 try {
4050 int index = 0;
4051 Result r = null;
4052 while ((r = s.next()) != null) {
4053 for(Cell key : r.rawCells()) {
4054 times[index++] = key.getTimestamp();
4055 }
4056 }
4057 } finally {
4058 s.close();
4059 }
4060 for (int i = 0; i < times.length - 1; i++) {
4061 for (int j = i + 1; j < times.length; j++) {
4062 assertTrue(times[j] > times[i]);
4063 }
4064 }
4065 }
4066
4067 @Test
4068 public void testListTables() throws IOException, InterruptedException {
4069 byte [] t1 = Bytes.toBytes("testListTables1");
4070 byte [] t2 = Bytes.toBytes("testListTables2");
4071 byte [] t3 = Bytes.toBytes("testListTables3");
4072 byte [][] tables = new byte[][] { t1, t2, t3 };
4073 for (int i = 0; i < tables.length; i++) {
4074 TEST_UTIL.createTable(tables[i], FAMILY);
4075 }
4076 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
4077 HTableDescriptor[] ts = admin.listTables();
4078 HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
4079 for (int i = 0; i < ts.length; i++) {
4080 result.add(ts[i]);
4081 }
4082 int size = result.size();
4083 assertTrue(size >= tables.length);
4084 for (int i = 0; i < tables.length && i < size; i++) {
4085 boolean found = false;
4086 for (int j = 0; j < ts.length; j++) {
4087 if (Bytes.equals(ts[j].getTableName().getName(), tables[i])) {
4088 found = true;
4089 break;
4090 }
4091 }
4092 assertTrue("Not found: " + Bytes.toString(tables[i]), found);
4093 }
4094 }
4095
4096
4097
4098
4099
4100
4101
4102
4103 HTable createUnmangedHConnectionHTable(final byte [] tableName) throws IOException {
4104 TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
4105 HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
4106 return (HTable)conn.getTable(tableName);
4107 }
4108
4109
4110
4111
4112
4113
4114
4115 @Test
4116 public void testUnmanagedHConnection() throws IOException {
4117 final byte[] tableName = Bytes.toBytes("testUnmanagedHConnection");
4118 HTable t = createUnmangedHConnectionHTable(tableName);
4119 HBaseAdmin ha = new HBaseAdmin(t.getConnection());
4120 assertTrue(ha.tableExists(tableName));
4121 assertTrue(t.get(new Get(ROW)).isEmpty());
4122 }
4123
4124
4125
4126
4127
4128
4129
4130 @Test
4131 public void testUnmanagedHConnectionReconnect() throws Exception {
4132 final byte[] tableName = Bytes.toBytes("testUnmanagedHConnectionReconnect");
4133 HTable t = createUnmangedHConnectionHTable(tableName);
4134 HConnection conn = t.getConnection();
4135 HBaseAdmin ha = new HBaseAdmin(conn);
4136 assertTrue(ha.tableExists(tableName));
4137 assertTrue(t.get(new Get(ROW)).isEmpty());
4138
4139
4140 MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
4141 cluster.stopMaster(0, false);
4142 cluster.waitOnMaster(0);
4143
4144
4145 cluster.startMaster();
4146 assertTrue(cluster.waitForActiveAndReadyMaster());
4147
4148
4149
4150 HBaseAdmin newAdmin = new HBaseAdmin(conn);
4151 assertTrue(newAdmin.tableExists(tableName));
4152 assert(newAdmin.getClusterStatus().getServersSize() == SLAVES);
4153 }
4154
4155 @Test
4156 public void testMiscHTableStuff() throws IOException {
4157 final byte[] tableAname = Bytes.toBytes("testMiscHTableStuffA");
4158 final byte[] tableBname = Bytes.toBytes("testMiscHTableStuffB");
4159 final byte[] attrName = Bytes.toBytes("TESTATTR");
4160 final byte[] attrValue = Bytes.toBytes("somevalue");
4161 byte[] value = Bytes.toBytes("value");
4162
4163 HTable a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY);
4164 HTable b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY);
4165 Put put = new Put(ROW);
4166 put.add(HConstants.CATALOG_FAMILY, null, value);
4167 a.put(put);
4168
4169
4170 HTable newA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
4171
4172
4173 Scan scan = new Scan();
4174 scan.addFamily(HConstants.CATALOG_FAMILY);
4175 ResultScanner s = newA.getScanner(scan);
4176 try {
4177 for (Result r : s) {
4178 put = new Put(r.getRow());
4179 put.setDurability(Durability.SKIP_WAL);
4180 for (Cell kv : r.rawCells()) {
4181 put.add(kv);
4182 }
4183 b.put(put);
4184 }
4185 } finally {
4186 s.close();
4187 }
4188
4189
4190 HTable anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
4191 Get get = new Get(ROW);
4192 get.addFamily(HConstants.CATALOG_FAMILY);
4193 anotherA.get(get);
4194
4195
4196
4197
4198
4199
4200 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
4201
4202 HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor());
4203
4204 admin.disableTable(tableAname);
4205
4206 desc.setValue(attrName, attrValue);
4207
4208 for (HColumnDescriptor c : desc.getFamilies())
4209 c.setValue(attrName, attrValue);
4210
4211 admin.modifyTable(tableAname, desc);
4212
4213 admin.enableTable(tableAname);
4214
4215
4216 desc = a.getTableDescriptor();
4217 assertTrue("wrong table descriptor returned",
4218 Bytes.compareTo(desc.getTableName().getName(), tableAname) == 0);
4219
4220 value = desc.getValue(attrName);
4221 assertFalse("missing HTD attribute value", value == null);
4222 assertFalse("HTD attribute value is incorrect",
4223 Bytes.compareTo(value, attrValue) != 0);
4224
4225 for (HColumnDescriptor c : desc.getFamilies()) {
4226 value = c.getValue(attrName);
4227 assertFalse("missing HCD attribute value", value == null);
4228 assertFalse("HCD attribute value is incorrect",
4229 Bytes.compareTo(value, attrValue) != 0);
4230 }
4231 }
4232
4233 @Test
4234 public void testGetClosestRowBefore() throws IOException {
4235 final byte [] tableAname = Bytes.toBytes("testGetClosestRowBefore");
4236 final byte [] row = Bytes.toBytes("row");
4237
4238
4239 byte[] firstRow = Bytes.toBytes("ro");
4240 byte[] beforeFirstRow = Bytes.toBytes("rn");
4241 byte[] beforeSecondRow = Bytes.toBytes("rov");
4242
4243 HTable table = TEST_UTIL.createTable(tableAname,
4244 new byte [][] {HConstants.CATALOG_FAMILY, Bytes.toBytes("info2")});
4245 Put put = new Put(firstRow);
4246 Put put2 = new Put(row);
4247 byte[] zero = new byte[]{0};
4248 byte[] one = new byte[]{1};
4249
4250 put.add(HConstants.CATALOG_FAMILY, null, zero);
4251 put2.add(HConstants.CATALOG_FAMILY, null, one);
4252
4253 table.put(put);
4254 table.put(put2);
4255
4256 Result result = null;
4257
4258
4259 result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY);
4260 assertTrue(result == null);
4261
4262
4263 result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY);
4264 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4265 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), zero));
4266
4267
4268 result = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY);
4269 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4270 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), zero));
4271
4272
4273 result = table.getRowOrBefore(row, HConstants.CATALOG_FAMILY);
4274 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4275 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
4276
4277
4278 result = table.getRowOrBefore(Bytes.add(row,one), HConstants.CATALOG_FAMILY);
4279 assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4280 assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
4281 }
4282
4283
4284
4285
4286
4287 @Test
4288 public void testScanVariableReuse() throws Exception {
4289 Scan scan = new Scan();
4290 scan.addFamily(FAMILY);
4291 scan.addColumn(FAMILY, ROW);
4292
4293 assertTrue(scan.getFamilyMap().get(FAMILY).size() == 1);
4294
4295 scan = new Scan();
4296 scan.addFamily(FAMILY);
4297
4298 assertTrue(scan.getFamilyMap().get(FAMILY) == null);
4299 assertTrue(scan.getFamilyMap().containsKey(FAMILY));
4300 }
4301
4302 @Test
4303 public void testMultiRowMutation() throws Exception {
4304 LOG.info("Starting testMultiRowMutation");
4305 final byte [] TABLENAME = Bytes.toBytes("testMultiRowMutation");
4306 final byte [] ROW1 = Bytes.toBytes("testRow1");
4307
4308 HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4309 Put p = new Put(ROW);
4310 p.add(FAMILY, QUALIFIER, VALUE);
4311 MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p);
4312
4313 p = new Put(ROW1);
4314 p.add(FAMILY, QUALIFIER, VALUE);
4315 MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, p);
4316
4317 MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder();
4318 mrmBuilder.addMutationRequest(m1);
4319 mrmBuilder.addMutationRequest(m2);
4320 MutateRowsRequest mrm = mrmBuilder.build();
4321 CoprocessorRpcChannel channel = t.coprocessorService(ROW);
4322 MultiRowMutationService.BlockingInterface service =
4323 MultiRowMutationService.newBlockingStub(channel);
4324 service.mutateRows(null, mrm);
4325 Get g = new Get(ROW);
4326 Result r = t.get(g);
4327 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)));
4328 g = new Get(ROW1);
4329 r = t.get(g);
4330 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)));
4331 }
4332
4333 @Test
4334 public void testRowMutation() throws Exception {
4335 LOG.info("Starting testRowMutation");
4336 final byte [] TABLENAME = Bytes.toBytes("testRowMutation");
4337 HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4338 byte [][] QUALIFIERS = new byte [][] {
4339 Bytes.toBytes("a"), Bytes.toBytes("b")
4340 };
4341 RowMutations arm = new RowMutations(ROW);
4342 Put p = new Put(ROW);
4343 p.add(FAMILY, QUALIFIERS[0], VALUE);
4344 arm.add(p);
4345 t.mutateRow(arm);
4346
4347 Get g = new Get(ROW);
4348 Result r = t.get(g);
4349 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0])));
4350
4351 arm = new RowMutations(ROW);
4352 p = new Put(ROW);
4353 p.add(FAMILY, QUALIFIERS[1], VALUE);
4354 arm.add(p);
4355 Delete d = new Delete(ROW);
4356 d.deleteColumns(FAMILY, QUALIFIERS[0]);
4357 arm.add(d);
4358
4359 t.mutateRow(arm);
4360 r = t.get(g);
4361 assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1])));
4362 assertNull(r.getValue(FAMILY, QUALIFIERS[0]));
4363 }
4364
4365 @Test
4366 public void testAppend() throws Exception {
4367 LOG.info("Starting testAppend");
4368 final byte [] TABLENAME = Bytes.toBytes("testAppend");
4369 HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4370 byte[] v1 = Bytes.toBytes("42");
4371 byte[] v2 = Bytes.toBytes("23");
4372 byte [][] QUALIFIERS = new byte [][] {
4373 Bytes.toBytes("b"), Bytes.toBytes("a")
4374 };
4375 Append a = new Append(ROW);
4376 a.add(FAMILY, QUALIFIERS[0], v1);
4377 a.add(FAMILY, QUALIFIERS[1], v2);
4378 a.setReturnResults(false);
4379 assertNullResult(t.append(a));
4380
4381 a = new Append(ROW);
4382 a.add(FAMILY, QUALIFIERS[0], v2);
4383 a.add(FAMILY, QUALIFIERS[1], v1);
4384 Result r = t.append(a);
4385 assertEquals(0, Bytes.compareTo(Bytes.add(v1,v2), r.getValue(FAMILY, QUALIFIERS[0])));
4386 assertEquals(0, Bytes.compareTo(Bytes.add(v2,v1), r.getValue(FAMILY, QUALIFIERS[1])));
4387 }
4388
4389 @Test
4390 public void testIncrementWithDeletes() throws Exception {
4391 LOG.info("Starting testIncrementWithDeletes");
4392 final TableName TABLENAME =
4393 TableName.valueOf("testIncrementWithDeletes");
4394 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4395 final byte[] COLUMN = Bytes.toBytes("column");
4396
4397 ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
4398 TEST_UTIL.flush(TABLENAME);
4399
4400 Delete del = new Delete(ROW);
4401 ht.delete(del);
4402
4403 ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
4404
4405 Get get = new Get(ROW);
4406 Result r = ht.get(get);
4407 assertEquals(1, r.size());
4408 assertEquals(5, Bytes.toLong(r.getValue(FAMILY, COLUMN)));
4409 }
4410
4411 @Test
4412 public void testIncrementingInvalidValue() throws Exception {
4413 LOG.info("Starting testIncrementingInvalidValue");
4414 final byte [] TABLENAME = Bytes.toBytes("testIncrementingInvalidValue");
4415 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4416 final byte[] COLUMN = Bytes.toBytes("column");
4417 Put p = new Put(ROW);
4418
4419 p.add(FAMILY, COLUMN, Bytes.toBytes(5));
4420 ht.put(p);
4421 try {
4422 ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
4423 fail("Should have thrown DoNotRetryIOException");
4424 } catch (DoNotRetryIOException iox) {
4425
4426 }
4427 Increment inc = new Increment(ROW);
4428 inc.addColumn(FAMILY, COLUMN, 5);
4429 try {
4430 ht.increment(inc);
4431 fail("Should have thrown DoNotRetryIOException");
4432 } catch (DoNotRetryIOException iox) {
4433
4434 }
4435 }
4436
4437 @Test
4438 public void testIncrementInvalidArguments() throws Exception {
4439 LOG.info("Starting testIncrementInvalidArguments");
4440 final byte[] TABLENAME = Bytes.toBytes("testIncrementInvalidArguments");
4441 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4442 final byte[] COLUMN = Bytes.toBytes("column");
4443 try {
4444
4445 ht.incrementColumnValue(null, FAMILY, COLUMN, 5);
4446 fail("Should have thrown IOException");
4447 } catch (IOException iox) {
4448
4449 }
4450 try {
4451
4452 ht.incrementColumnValue(ROW, null, COLUMN, 5);
4453 fail("Should have thrown IOException");
4454 } catch (IOException iox) {
4455
4456 }
4457 try {
4458
4459 ht.incrementColumnValue(ROW, FAMILY, null, 5);
4460 fail("Should have thrown IOException");
4461 } catch (IOException iox) {
4462
4463 }
4464
4465 try {
4466 Increment incNoRow = new Increment(null);
4467 incNoRow.addColumn(FAMILY, COLUMN, 5);
4468 fail("Should have thrown IllegalArgumentException");
4469 } catch (IllegalArgumentException iax) {
4470
4471 } catch (NullPointerException npe) {
4472
4473 }
4474
4475 try {
4476 Increment incNoFamily = new Increment(ROW);
4477 incNoFamily.addColumn(null, COLUMN, 5);
4478 fail("Should have thrown IllegalArgumentException");
4479 } catch (IllegalArgumentException iax) {
4480
4481 }
4482
4483 try {
4484 Increment incNoQualifier = new Increment(ROW);
4485 incNoQualifier.addColumn(FAMILY, null, 5);
4486 fail("Should have thrown IllegalArgumentException");
4487 } catch (IllegalArgumentException iax) {
4488
4489 }
4490 }
4491
4492
4493
4494 @Test
4495 public void testIncrement() throws Exception {
4496 LOG.info("Starting testIncrement");
4497 final byte [] TABLENAME = Bytes.toBytes("testIncrement");
4498 HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
4499
4500 byte [][] ROWS = new byte [][] {
4501 Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
4502 Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
4503 Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i")
4504 };
4505 byte [][] QUALIFIERS = new byte [][] {
4506 Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
4507 Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
4508 Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i")
4509 };
4510
4511
4512
4513
4514 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[0], 1);
4515 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[1], 2);
4516 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[2], 3);
4517 ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[3], 4);
4518
4519
4520 Increment inc = new Increment(ROW);
4521 inc.addColumn(FAMILY, QUALIFIERS[1], 1);
4522 inc.addColumn(FAMILY, QUALIFIERS[3], 1);
4523 inc.addColumn(FAMILY, QUALIFIERS[4], 1);
4524 ht.increment(inc);
4525
4526
4527 Result r = ht.get(new Get(ROW));
4528 Cell [] kvs = r.rawCells();
4529 assertEquals(5, kvs.length);
4530 assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 1);
4531 assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 3);
4532 assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 3);
4533 assertIncrementKey(kvs[3], ROW, FAMILY, QUALIFIERS[3], 5);
4534 assertIncrementKey(kvs[4], ROW, FAMILY, QUALIFIERS[4], 1);
4535
4536
4537 inc = new Increment(ROWS[0]);
4538 for (int i=0;i<QUALIFIERS.length;i++) {
4539 inc.addColumn(FAMILY, QUALIFIERS[i], i+1);
4540 }
4541 ht.increment(inc);
4542
4543 r = ht.get(new Get(ROWS[0]));
4544 kvs = r.rawCells();
4545 assertEquals(QUALIFIERS.length, kvs.length);
4546 for (int i=0;i<QUALIFIERS.length;i++) {
4547 assertIncrementKey(kvs[i], ROWS[0], FAMILY, QUALIFIERS[i], i+1);
4548 }
4549
4550
4551 inc = new Increment(ROWS[0]);
4552 for (int i=0;i<QUALIFIERS.length;i++) {
4553 inc.addColumn(FAMILY, QUALIFIERS[i], i+1);
4554 }
4555 ht.increment(inc);
4556
4557 r = ht.get(new Get(ROWS[0]));
4558 kvs = r.rawCells();
4559 assertEquals(QUALIFIERS.length, kvs.length);
4560 for (int i=0;i<QUALIFIERS.length;i++) {
4561 assertIncrementKey(kvs[i], ROWS[0], FAMILY, QUALIFIERS[i], 2*(i+1));
4562 }
4563 }
4564
4565
4566 @Test
4567 public void testClientPoolRoundRobin() throws IOException {
4568 final byte[] tableName = Bytes.toBytes("testClientPoolRoundRobin");
4569
4570 int poolSize = 3;
4571 int numVersions = poolSize * 2;
4572 Configuration conf = TEST_UTIL.getConfiguration();
4573 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin");
4574 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4575
4576 HTable table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY },
4577 conf, Integer.MAX_VALUE);
4578
4579 final long ts = EnvironmentEdgeManager.currentTimeMillis();
4580 Get get = new Get(ROW);
4581 get.addColumn(FAMILY, QUALIFIER);
4582 get.setMaxVersions();
4583
4584 for (int versions = 1; versions <= numVersions; versions++) {
4585 Put put = new Put(ROW);
4586 put.add(FAMILY, QUALIFIER, ts + versions, VALUE);
4587 table.put(put);
4588
4589 Result result = table.get(get);
4590 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4591 .get(QUALIFIER);
4592
4593 assertEquals("The number of versions of '" + FAMILY + ":" + QUALIFIER
4594 + " did not match " + versions, versions, navigableMap.size());
4595 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4596 assertTrue("The value at time " + entry.getKey()
4597 + " did not match what was put",
4598 Bytes.equals(VALUE, entry.getValue()));
4599 }
4600 }
4601 }
4602
4603 @Ignore ("Flakey: HBASE-8989") @Test
4604 public void testClientPoolThreadLocal() throws IOException {
4605 final byte[] tableName = Bytes.toBytes("testClientPoolThreadLocal");
4606
4607 int poolSize = Integer.MAX_VALUE;
4608 int numVersions = 3;
4609 Configuration conf = TEST_UTIL.getConfiguration();
4610 conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local");
4611 conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4612
4613 final HTable table = TEST_UTIL.createTable(tableName,
4614 new byte[][] { FAMILY }, conf, 3);
4615
4616 final long ts = EnvironmentEdgeManager.currentTimeMillis();
4617 final Get get = new Get(ROW);
4618 get.addColumn(FAMILY, QUALIFIER);
4619 get.setMaxVersions();
4620
4621 for (int versions = 1; versions <= numVersions; versions++) {
4622 Put put = new Put(ROW);
4623 put.add(FAMILY, QUALIFIER, ts + versions, VALUE);
4624 table.put(put);
4625
4626 Result result = table.get(get);
4627 NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4628 .get(QUALIFIER);
4629
4630 assertEquals("The number of versions of '" + FAMILY + ":" + QUALIFIER + " did not match " +
4631 versions + "; " + put.toString() + ", " + get.toString(), versions, navigableMap.size());
4632 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4633 assertTrue("The value at time " + entry.getKey()
4634 + " did not match what was put",
4635 Bytes.equals(VALUE, entry.getValue()));
4636 }
4637 }
4638
4639 final Object waitLock = new Object();
4640 ExecutorService executorService = Executors.newFixedThreadPool(numVersions);
4641 final AtomicReference<AssertionError> error = new AtomicReference<AssertionError>(null);
4642 for (int versions = numVersions; versions < numVersions * 2; versions++) {
4643 final int versionsCopy = versions;
4644 executorService.submit(new Callable<Void>() {
4645 @Override
4646 public Void call() {
4647 try {
4648 Put put = new Put(ROW);
4649 put.add(FAMILY, QUALIFIER, ts + versionsCopy, VALUE);
4650 table.put(put);
4651
4652 Result result = table.get(get);
4653 NavigableMap<Long, byte[]> navigableMap = result.getMap()
4654 .get(FAMILY).get(QUALIFIER);
4655
4656 assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":"
4657 + Bytes.toString(QUALIFIER) + " did not match " + versionsCopy, versionsCopy,
4658 navigableMap.size());
4659 for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4660 assertTrue("The value at time " + entry.getKey()
4661 + " did not match what was put",
4662 Bytes.equals(VALUE, entry.getValue()));
4663 }
4664 synchronized (waitLock) {
4665 waitLock.wait();
4666 }
4667 } catch (Exception e) {
4668 } catch (AssertionError e) {
4669
4670
4671 error.set(e);
4672 LOG.error(e);
4673 }
4674
4675 return null;
4676 }
4677 });
4678 }
4679 synchronized (waitLock) {
4680 waitLock.notifyAll();
4681 }
4682 executorService.shutdownNow();
4683 assertNull(error.get());
4684 }
4685
4686 @Test
4687 public void testCheckAndPut() throws IOException {
4688 final byte [] anotherrow = Bytes.toBytes("anotherrow");
4689 final byte [] value2 = Bytes.toBytes("abcd");
4690
4691 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"),
4692 new byte [][] {FAMILY});
4693 Put put1 = new Put(ROW);
4694 put1.add(FAMILY, QUALIFIER, VALUE);
4695
4696
4697 boolean ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put1);
4698 assertEquals(ok, false);
4699
4700
4701 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4702 assertEquals(ok, true);
4703
4704
4705 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4706 assertEquals(ok, false);
4707
4708 Put put2 = new Put(ROW);
4709 put2.add(FAMILY, QUALIFIER, value2);
4710
4711
4712 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put2);
4713 assertEquals(ok, true);
4714
4715 Put put3 = new Put(anotherrow);
4716 put3.add(FAMILY, QUALIFIER, VALUE);
4717
4718
4719 try {
4720 ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, value2, put3);
4721 fail("trying to check and modify different rows should have failed.");
4722 } catch(Exception e) {}
4723
4724 }
4725
4726
4727
4728
4729
4730 @Test
4731 @SuppressWarnings ("unused")
4732 public void testScanMetrics() throws Exception {
4733 byte [] TABLENAME = Bytes.toBytes("testScanMetrics");
4734
4735 Configuration conf = TEST_UTIL.getConfiguration();
4736 TEST_UTIL.createTable(TABLENAME, FAMILY);
4737
4738
4739
4740 HTable ht = new HTable(conf, TABLENAME);
4741
4742
4743 int numOfRegions = TEST_UTIL.createMultiRegions(ht, FAMILY);
4744
4745
4746 Put put1 = new Put(Bytes.toBytes("z1"));
4747 put1.add(FAMILY, QUALIFIER, VALUE);
4748 Put put2 = new Put(Bytes.toBytes("z2"));
4749 put2.add(FAMILY, QUALIFIER, VALUE);
4750 Put put3 = new Put(Bytes.toBytes("z3"));
4751 put3.add(FAMILY, QUALIFIER, VALUE);
4752 ht.put(Arrays.asList(put1, put2, put3));
4753
4754 Scan scan1 = new Scan();
4755 int numRecords = 0;
4756 for(Result result : ht.getScanner(scan1)) {
4757 numRecords++;
4758 }
4759 LOG.info("test data has " + numRecords + " records.");
4760
4761
4762 assertEquals(null, scan1.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA));
4763
4764
4765 Scan scan = new Scan();
4766 scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4767 scan.setCaching(numRecords+1);
4768 ResultScanner scanner = ht.getScanner(scan);
4769 for (Result result : scanner.next(numRecords - 1)) {
4770 }
4771 scanner.close();
4772
4773 assertNotNull(scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA));
4774
4775
4776 scan = new Scan();
4777 scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4778 scan.setCaching(1);
4779 scanner = ht.getScanner(scan);
4780
4781
4782 for (Result result : scanner.next(numRecords - 1)) {
4783 }
4784 scanner.close();
4785
4786 ScanMetrics scanMetrics = getScanMetrics(scan);
4787 assertEquals("Did not access all the regions in the table", numOfRegions,
4788 scanMetrics.countOfRegions.get());
4789
4790
4791
4792 Scan scanWithoutClose = new Scan();
4793 scanWithoutClose.setCaching(1);
4794 scanWithoutClose.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4795 ResultScanner scannerWithoutClose = ht.getScanner(scanWithoutClose);
4796 for (Result result : scannerWithoutClose.next(numRecords + 1)) {
4797 }
4798 ScanMetrics scanMetricsWithoutClose = getScanMetrics(scanWithoutClose);
4799 assertEquals("Did not access all the regions in the table", numOfRegions,
4800 scanMetricsWithoutClose.countOfRegions.get());
4801
4802
4803
4804 Scan scanWithClose = new Scan();
4805
4806 scanWithClose.setCaching(numRecords);
4807 scanWithClose.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
4808 ResultScanner scannerWithClose = ht.getScanner(scanWithClose);
4809 for (Result result : scannerWithClose.next(numRecords + 1)) {
4810 }
4811 scannerWithClose.close();
4812 ScanMetrics scanMetricsWithClose = getScanMetrics(scanWithClose);
4813 assertEquals("Did not access all the regions in the table", numOfRegions,
4814 scanMetricsWithClose.countOfRegions.get());
4815 }
4816
4817 private ScanMetrics getScanMetrics(Scan scan) throws Exception {
4818 byte[] serializedMetrics = scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA);
4819 assertTrue("Serialized metrics were not found.", serializedMetrics != null);
4820
4821
4822 ScanMetrics scanMetrics = ProtobufUtil.toScanMetrics(serializedMetrics);
4823
4824 return scanMetrics;
4825 }
4826
4827
4828
4829
4830
4831
4832
4833
4834
4835 @Test
4836 public void testCacheOnWriteEvictOnClose() throws Exception {
4837 byte [] tableName = Bytes.toBytes("testCOWEOCfromClient");
4838 byte [] data = Bytes.toBytes("data");
4839 HTable table = TEST_UTIL.createTable(tableName, new byte [][] {FAMILY});
4840
4841 String regionName = table.getRegionLocations().firstKey().getEncodedName();
4842 HRegion region = TEST_UTIL.getRSForFirstRegionInTable(
4843 tableName).getFromOnlineRegions(regionName);
4844 Store store = region.getStores().values().iterator().next();
4845 CacheConfig cacheConf = store.getCacheConfig();
4846 cacheConf.setCacheDataOnWrite(true);
4847 cacheConf.setEvictOnClose(true);
4848 BlockCache cache = cacheConf.getBlockCache();
4849
4850
4851 long startBlockCount = cache.getBlockCount();
4852 long startBlockHits = cache.getStats().getHitCount();
4853 long startBlockMiss = cache.getStats().getMissCount();
4854
4855
4856 for (int i = 0; i < 5; i++) {
4857 Thread.sleep(100);
4858 if (startBlockCount != cache.getBlockCount()
4859 || startBlockHits != cache.getStats().getHitCount()
4860 || startBlockMiss != cache.getStats().getMissCount()) {
4861 startBlockCount = cache.getBlockCount();
4862 startBlockHits = cache.getStats().getHitCount();
4863 startBlockMiss = cache.getStats().getMissCount();
4864 i = -1;
4865 }
4866 }
4867
4868
4869 Put put = new Put(ROW);
4870 put.add(FAMILY, QUALIFIER, data);
4871 table.put(put);
4872 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
4873
4874 assertEquals(startBlockCount, cache.getBlockCount());
4875 assertEquals(startBlockHits, cache.getStats().getHitCount());
4876 assertEquals(startBlockMiss, cache.getStats().getMissCount());
4877
4878 System.out.println("Flushing cache");
4879 region.flushcache();
4880
4881 long expectedBlockCount = startBlockCount + 1;
4882 long expectedBlockHits = startBlockHits;
4883 long expectedBlockMiss = startBlockMiss;
4884 assertEquals(expectedBlockCount, cache.getBlockCount());
4885 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
4886 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4887
4888 assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
4889 assertEquals(expectedBlockCount, cache.getBlockCount());
4890 assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
4891 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4892
4893 byte [] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
4894 byte [] data2 = Bytes.add(data, data);
4895 put = new Put(ROW);
4896 put.add(FAMILY, QUALIFIER2, data2);
4897 table.put(put);
4898 Result r = table.get(new Get(ROW));
4899 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
4900 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
4901 assertEquals(expectedBlockCount, cache.getBlockCount());
4902 assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
4903 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4904
4905 System.out.println("Flushing cache");
4906 region.flushcache();
4907 assertEquals(++expectedBlockCount, cache.getBlockCount());
4908 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
4909 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4910
4911 System.out.println("Compacting");
4912 assertEquals(2, store.getStorefilesCount());
4913 store.triggerMajorCompaction();
4914 region.compactStores();
4915 waitForStoreFileCount(store, 1, 10000);
4916 assertEquals(1, store.getStorefilesCount());
4917 expectedBlockCount -= 2;
4918 assertEquals(expectedBlockCount, cache.getBlockCount());
4919 expectedBlockHits += 2;
4920 assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4921 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
4922
4923
4924 r = table.get(new Get(ROW));
4925 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
4926 assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
4927 expectedBlockCount += 1;
4928 assertEquals(expectedBlockCount, cache.getBlockCount());
4929 assertEquals(expectedBlockHits, cache.getStats().getHitCount());
4930 assertEquals(++expectedBlockMiss, cache.getStats().getMissCount());
4931 }
4932
4933 private void waitForStoreFileCount(Store store, int count, int timeout)
4934 throws InterruptedException {
4935 long start = System.currentTimeMillis();
4936 while (start + timeout > System.currentTimeMillis() &&
4937 store.getStorefilesCount() != count) {
4938 Thread.sleep(100);
4939 }
4940 System.out.println("start=" + start + ", now=" +
4941 System.currentTimeMillis() + ", cur=" + store.getStorefilesCount());
4942 assertEquals(count, store.getStorefilesCount());
4943 }
4944
4945 @Test
4946
4947
4948
4949 public void testNonCachedGetRegionLocation() throws Exception {
4950
4951 String tableName = "testNonCachedGetRegionLocation";
4952 byte [] TABLE = Bytes.toBytes(tableName);
4953 byte [] family1 = Bytes.toBytes("f1");
4954 byte [] family2 = Bytes.toBytes("f2");
4955 HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
4956 HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
4957 Map <HRegionInfo, ServerName> regionsMap = table.getRegionLocations();
4958 assertEquals(1, regionsMap.size());
4959 HRegionInfo regionInfo = regionsMap.keySet().iterator().next();
4960 ServerName addrBefore = regionsMap.get(regionInfo);
4961
4962 HRegionLocation addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
4963 HRegionLocation addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true);
4964
4965 assertEquals(addrBefore.getPort(), addrCache.getPort());
4966 assertEquals(addrBefore.getPort(), addrNoCache.getPort());
4967
4968 ServerName addrAfter = null;
4969
4970 for (int i = 0; i < SLAVES; i++) {
4971 HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(i);
4972 ServerName addr = regionServer.getServerName();
4973 if (addr.getPort() != addrBefore.getPort()) {
4974 admin.move(regionInfo.getEncodedNameAsBytes(),
4975 Bytes.toBytes(addr.toString()));
4976
4977 Thread.sleep(5000);
4978 addrAfter = addr;
4979 break;
4980 }
4981 }
4982
4983
4984 addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
4985 addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true);
4986 assertNotNull(addrAfter);
4987 assertTrue(addrAfter.getPort() != addrCache.getPort());
4988 assertEquals(addrAfter.getPort(), addrNoCache.getPort());
4989 }
4990
4991 @Test
4992
4993
4994
4995
4996 public void testGetRegionsInRange() throws Exception {
4997
4998 byte [] startKey = Bytes.toBytes("ddc");
4999 byte [] endKey = Bytes.toBytes("mmm");
5000 byte [] TABLE = Bytes.toBytes("testGetRegionsInRange");
5001 HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
5002 int numOfRegions = TEST_UTIL.createMultiRegions(table, FAMILY);
5003 assertEquals(25, numOfRegions);
5004
5005
5006 List<HRegionLocation> regionsList = table.getRegionsInRange(startKey,
5007 endKey);
5008 assertEquals(10, regionsList.size());
5009
5010
5011 startKey = Bytes.toBytes("fff");
5012 regionsList = table.getRegionsInRange(startKey, endKey);
5013 assertEquals(7, regionsList.size());
5014
5015
5016 endKey = Bytes.toBytes("nnn");
5017 regionsList = table.getRegionsInRange(startKey, endKey);
5018 assertEquals(8, regionsList.size());
5019
5020
5021 regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, endKey);
5022 assertEquals(13, regionsList.size());
5023
5024
5025 regionsList = table.getRegionsInRange(startKey, HConstants.EMPTY_END_ROW);
5026 assertEquals(20, regionsList.size());
5027
5028
5029 regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW,
5030 HConstants.EMPTY_END_ROW);
5031 assertEquals(25, regionsList.size());
5032
5033
5034 endKey = Bytes.toBytes("yyz");
5035 regionsList = table.getRegionsInRange(startKey, endKey);
5036 assertEquals(20, regionsList.size());
5037
5038
5039 startKey = Bytes.toBytes("aac");
5040 regionsList = table.getRegionsInRange(startKey, endKey);
5041 assertEquals(25, regionsList.size());
5042
5043
5044 startKey = endKey = Bytes.toBytes("ccc");
5045 regionsList = table.getRegionsInRange(startKey, endKey);
5046 assertEquals(1, regionsList.size());
5047 }
5048
5049 @Test
5050 public void testJira6912() throws Exception {
5051 byte [] TABLE = Bytes.toBytes("testJira6912");
5052 HTable foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
5053
5054 List<Put> puts = new ArrayList<Put>();
5055 for (int i=0;i !=100; i++){
5056 Put put = new Put(Bytes.toBytes(i));
5057 put.add(FAMILY, FAMILY, Bytes.toBytes(i));
5058 puts.add(put);
5059 }
5060 foo.put(puts);
5061
5062 TEST_UTIL.flush();
5063
5064 Scan scan = new Scan();
5065 scan.setStartRow(Bytes.toBytes(1));
5066 scan.setStopRow(Bytes.toBytes(3));
5067 scan.addColumn(FAMILY, FAMILY);
5068 scan.setFilter(new RowFilter(CompareFilter.CompareOp.NOT_EQUAL, new BinaryComparator(Bytes.toBytes(1))));
5069
5070 ResultScanner scanner = foo.getScanner(scan);
5071 Result[] bar = scanner.next(100);
5072 assertEquals(1, bar.length);
5073 }
5074
5075 @Test
5076 public void testScan_NullQualifier() throws IOException {
5077 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testScan_NullQualifier"), FAMILY);
5078 Put put = new Put(ROW);
5079 put.add(FAMILY, QUALIFIER, VALUE);
5080 table.put(put);
5081
5082 put = new Put(ROW);
5083 put.add(FAMILY, null, VALUE);
5084 table.put(put);
5085 LOG.info("Row put");
5086
5087 Scan scan = new Scan();
5088 scan.addColumn(FAMILY, null);
5089
5090 ResultScanner scanner = table.getScanner(scan);
5091 Result[] bar = scanner.next(100);
5092 assertEquals(1, bar.length);
5093 assertEquals(1, bar[0].size());
5094
5095 scan = new Scan();
5096 scan.addFamily(FAMILY);
5097
5098 scanner = table.getScanner(scan);
5099 bar = scanner.next(100);
5100 assertEquals(1, bar.length);
5101 assertEquals(2, bar[0].size());
5102 }
5103
5104 @Test
5105 public void testNegativeTimestamp() throws IOException {
5106 HTable table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY);
5107
5108 try {
5109 Put put = new Put(ROW, -1);
5110 put.add(FAMILY, QUALIFIER, VALUE);
5111 table.put(put);
5112 fail("Negative timestamps should not have been allowed");
5113 } catch (IllegalArgumentException ex) {
5114 assertTrue(ex.getMessage().contains("negative"));
5115 }
5116
5117 try {
5118 Put put = new Put(ROW);
5119 put.add(FAMILY, QUALIFIER, -1, VALUE);
5120 table.put(put);
5121 fail("Negative timestamps should not have been allowed");
5122 } catch (IllegalArgumentException ex) {
5123 assertTrue(ex.getMessage().contains("negative"));
5124 }
5125
5126 try {
5127 Delete delete = new Delete(ROW, -1);
5128 table.delete(delete);
5129 fail("Negative timestamps should not have been allowed");
5130 } catch (IllegalArgumentException ex) {
5131 assertTrue(ex.getMessage().contains("negative"));
5132 }
5133
5134 try {
5135 Delete delete = new Delete(ROW);
5136 delete.deleteFamily(FAMILY, -1);
5137 table.delete(delete);
5138 fail("Negative timestamps should not have been allowed");
5139 } catch (IllegalArgumentException ex) {
5140 assertTrue(ex.getMessage().contains("negative"));
5141 }
5142
5143 try {
5144 Scan scan = new Scan();
5145 scan.setTimeRange(-1, 1);
5146 table.getScanner(scan);
5147 fail("Negative timestamps should not have been allowed");
5148 } catch (IllegalArgumentException ex) {
5149 assertTrue(ex.getMessage().contains("negative"));
5150 }
5151
5152
5153
5154 try {
5155 new KeyValue(Bytes.toBytes(42), Bytes.toBytes(42), Bytes.toBytes(42), -1, Bytes.toBytes(42));
5156 } catch (IllegalArgumentException ex) {
5157 fail("KeyValue SHOULD allow negative timestamps");
5158 }
5159
5160 table.close();
5161 }
5162
5163 @Test
5164 public void testRawScanRespectsVersions() throws Exception {
5165 byte[] TABLE = Bytes.toBytes("testRawScan");
5166 HTable table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY });
5167 byte[] row = Bytes.toBytes("row");
5168
5169
5170 Put p = new Put(row);
5171 p.add(FAMILY, QUALIFIER, 10, VALUE);
5172 table.put(p);
5173 table.flushCommits();
5174
5175 p = new Put(row);
5176 p.add(FAMILY, QUALIFIER, 11, ArrayUtils.add(VALUE, (byte) 2));
5177 table.put(p);
5178 table.flushCommits();
5179
5180 p = new Put(row);
5181 p.add(FAMILY, QUALIFIER, 12, ArrayUtils.add(VALUE, (byte) 3));
5182 table.put(p);
5183 table.flushCommits();
5184
5185 p = new Put(row);
5186 p.add(FAMILY, QUALIFIER, 13, ArrayUtils.add(VALUE, (byte) 4));
5187 table.put(p);
5188 table.flushCommits();
5189
5190 int versions = 4;
5191 Scan s = new Scan(row);
5192
5193 s.setMaxVersions();
5194 s.setRaw(true);
5195
5196 ResultScanner scanner = table.getScanner(s);
5197 int count = 0;
5198 for (Result r : scanner) {
5199 assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5200 count++;
5201 }
5202 assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5203 count);
5204 scanner.close();
5205
5206
5207
5208 versions = 2;
5209 s.setMaxVersions(versions);
5210 scanner = table.getScanner(s);
5211 count = 0;
5212 for (Result r : scanner) {
5213 assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5214 count++;
5215 }
5216 assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5217 count);
5218 scanner.close();
5219
5220
5221
5222 versions = 3;
5223 s.setMaxVersions(versions);
5224 scanner = table.getScanner(s);
5225 count = 0;
5226 for (Result r : scanner) {
5227 assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5228 count++;
5229 }
5230 assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5231 count);
5232 scanner.close();
5233
5234 table.close();
5235 TEST_UTIL.deleteTable(TABLE);
5236 }
5237
5238 @Test
5239 public void testSmallScan() throws Exception {
5240
5241 byte[] TABLE = Bytes.toBytes("testSmallScan");
5242 HTable table = TEST_UTIL.createTable(TABLE, FAMILY);
5243
5244
5245 int insertNum = 10;
5246 for (int i = 0; i < 10; i++) {
5247 Put put = new Put(Bytes.toBytes("row" + String.format("%03d", i)));
5248 put.add(FAMILY, QUALIFIER, VALUE);
5249 table.put(put);
5250 }
5251
5252
5253 ResultScanner scanner = table.getScanner(new Scan());
5254 int count = 0;
5255 for (Result r : scanner) {
5256 assertTrue(!r.isEmpty());
5257 count++;
5258 }
5259 assertEquals(insertNum, count);
5260
5261
5262 Scan scan = new Scan(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
5263 scan.setSmall(true);
5264 scan.setCaching(2);
5265 scanner = table.getScanner(scan);
5266 count = 0;
5267 for (Result r : scanner) {
5268 assertTrue(!r.isEmpty());
5269 count++;
5270 }
5271 assertEquals(insertNum, count);
5272
5273 }
5274 }