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