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