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