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