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