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