1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client;
21  
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertFalse;
24  import static org.junit.Assert.assertSame;
25  import static org.junit.Assert.assertTrue;
26  import static org.junit.Assert.fail;
27  
28  import java.io.DataInputStream;
29  import java.io.DataOutputStream;
30  import java.io.File;
31  import java.io.FileInputStream;
32  import java.io.FileOutputStream;
33  import java.io.IOException;
34  import java.util.ArrayList;
35  import java.util.HashSet;
36  import java.util.Iterator;
37  import java.util.List;
38  import java.util.Map;
39  import java.util.NavigableMap;
40  import java.util.UUID;
41  import java.util.concurrent.Callable;
42  import java.util.concurrent.ExecutorService;
43  import java.util.concurrent.Executors;
44  import java.util.concurrent.SynchronousQueue;
45  import java.util.concurrent.ThreadPoolExecutor;
46  import org.apache.commons.logging.Log;
47  import org.apache.commons.logging.LogFactory;
48  import org.apache.hadoop.conf.Configuration;
49  import org.apache.hadoop.fs.Path;
50  import org.apache.hadoop.hbase.HBaseTestingUtility;
51  import org.apache.hadoop.hbase.HColumnDescriptor;
52  import org.apache.hadoop.hbase.HConstants;
53  import org.apache.hadoop.hbase.HRegionInfo;
54  import org.apache.hadoop.hbase.HServerAddress;
55  import org.apache.hadoop.hbase.HTableDescriptor;
56  import org.apache.hadoop.hbase.KeyValue;
57  import org.apache.hadoop.hbase.filter.BinaryComparator;
58  import org.apache.hadoop.hbase.filter.CompareFilter;
59  import org.apache.hadoop.hbase.filter.Filter;
60  import org.apache.hadoop.hbase.filter.FilterList;
61  import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
62  import org.apache.hadoop.hbase.filter.PrefixFilter;
63  import org.apache.hadoop.hbase.filter.QualifierFilter;
64  import org.apache.hadoop.hbase.filter.RegexStringComparator;
65  import org.apache.hadoop.hbase.filter.RowFilter;
66  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
67  import org.apache.hadoop.hbase.filter.WhileMatchFilter;
68  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
69  import org.apache.hadoop.hbase.io.hfile.BlockCache;
70  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
71  import org.apache.hadoop.hbase.regionserver.HRegion;
72  import org.apache.hadoop.hbase.regionserver.Store;
73  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
74  import org.apache.hadoop.hbase.util.Bytes;
75  import org.junit.After;
76  import org.junit.AfterClass;
77  import org.junit.Before;
78  import org.junit.BeforeClass;
79  import org.junit.Ignore;
80  import org.junit.Test;
81  
82  /**
83   * Run tests that use the HBase clients; {@link HTable} and {@link HTablePool}.
84   * Sets up the HBase mini cluster once at start and runs through all client tests.
85   * Each creates a table named for the method and does its stuff against that.
86   */
87  public class TestFromClientSide {
88    final Log LOG = LogFactory.getLog(getClass());
89    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
90    private static byte [] ROW = Bytes.toBytes("testRow");
91    private static byte [] FAMILY = Bytes.toBytes("testFamily");
92    private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
93    private static byte [] VALUE = Bytes.toBytes("testValue");
94  
95    /**
96     * @throws java.lang.Exception
97     */
98    @BeforeClass
99    public static void setUpBeforeClass() throws Exception {
100     // We need more than one region server in this test
101     TEST_UTIL.startMiniCluster(3);
102   }
103 
104   /**
105    * @throws java.lang.Exception
106    */
107   @AfterClass
108   public static void tearDownAfterClass() throws Exception {
109     TEST_UTIL.shutdownMiniCluster();
110   }
111 
112   /**
113    * @throws java.lang.Exception
114    */
115   @Before
116   public void setUp() throws Exception {
117     // Nothing to do.
118   }
119 
120   /**
121    * @throws java.lang.Exception
122    */
123   @After
124   public void tearDown() throws Exception {
125     // Nothing to do.
126   }
127 
128   /**
129    * HBASE-2468 use case 1 and 2: region info de/serialization
130    */
131    @Test
132    public void testRegionCacheDeSerialization() throws Exception {
133      // 1. test serialization.
134      LOG.info("Starting testRegionCacheDeSerialization");
135      final byte[] TABLENAME = Bytes.toBytes("testCachePrewarm2");
136      final byte[] FAMILY = Bytes.toBytes("family");
137      Configuration conf = TEST_UTIL.getConfiguration();
138      TEST_UTIL.createTable(TABLENAME, FAMILY);
139 
140      // Set up test table:
141      // Create table:
142      HTable table = new HTable(conf, TABLENAME);
143 
144      // Create multiple regions for this table
145      TEST_UTIL.createMultiRegions(table, FAMILY);
146      Scan s = new Scan();
147      ResultScanner scanner = table.getScanner(s);
148      while (scanner.next() != null) continue;
149 
150      Path tempPath = new Path(TEST_UTIL.getDataTestDir(), "regions.dat");
151 
152      final String tempFileName = tempPath.toString();
153 
154      FileOutputStream fos = new FileOutputStream(tempFileName);
155      DataOutputStream dos = new DataOutputStream(fos);
156 
157      // serialize the region info and output to a local file.
158      table.serializeRegionInfo(dos);
159      dos.flush();
160      dos.close();
161 
162      // read a local file and deserialize the region info from it.
163      FileInputStream fis = new FileInputStream(tempFileName);
164      DataInputStream dis = new DataInputStream(fis);
165 
166      Map<HRegionInfo, HServerAddress> deserRegions =
167        table.deserializeRegionInfo(dis);
168      dis.close();
169 
170      // regions obtained from meta scanner.
171      Map<HRegionInfo, HServerAddress> loadedRegions =
172        table.getRegionsInfo();
173 
174      // set the deserialized regions to the global cache.
175      table.getConnection().clearRegionCache();
176 
177      table.getConnection().prewarmRegionCache(table.getTableName(),
178          deserRegions);
179 
180      // verify whether the 2 maps are identical or not.
181      assertEquals("Number of cached region is incorrect",
182          HConnectionManager.getCachedRegionCount(conf, TABLENAME),
183          loadedRegions.size());
184 
185      // verify each region is prefetched or not.
186      for (Map.Entry<HRegionInfo, HServerAddress> e: loadedRegions.entrySet()) {
187        HRegionInfo hri = e.getKey();
188        assertTrue(HConnectionManager.isRegionCached(conf,
189            hri.getTableName(), hri.getStartKey()));
190      }
191 
192      // delete the temp file
193      File f = new java.io.File(tempFileName);
194      f.delete();
195      LOG.info("Finishing testRegionCacheDeSerialization");
196    }
197 
198   /**
199    * HBASE-2468 use case 3:
200    */
201   @Test
202   public void testRegionCachePreWarm() throws Exception {
203     LOG.info("Starting testRegionCachePreWarm");
204     final byte [] TABLENAME = Bytes.toBytes("testCachePrewarm");
205     Configuration conf = TEST_UTIL.getConfiguration();
206 
207     // Set up test table:
208     // Create table:
209     TEST_UTIL.createTable(TABLENAME, FAMILY);
210 
211     // disable region cache for the table.
212     HTable.setRegionCachePrefetch(conf, TABLENAME, false);
213     assertFalse("The table is disabled for region cache prefetch",
214         HTable.getRegionCachePrefetch(conf, TABLENAME));
215 
216     HTable table = new HTable(conf, TABLENAME);
217 
218     // create many regions for the table.
219     TEST_UTIL.createMultiRegions(table, FAMILY);
220     // This count effectively waits until the regions have been
221     // fully assigned
222     TEST_UTIL.countRows(table);
223     table.getConnection().clearRegionCache();
224     assertEquals("Clearing cache should have 0 cached ", 0,
225         HConnectionManager.getCachedRegionCount(conf, TABLENAME));
226 
227     // A Get is suppose to do a region lookup request
228     Get g = new Get(Bytes.toBytes("aaa"));
229     table.get(g);
230 
231     // only one region should be cached if the cache prefetch is disabled.
232     assertEquals("Number of cached region is incorrect ", 1,
233         HConnectionManager.getCachedRegionCount(conf, TABLENAME));
234 
235     // now we enable cached prefetch.
236     HTable.setRegionCachePrefetch(conf, TABLENAME, true);
237     assertTrue("The table is enabled for region cache prefetch",
238         HTable.getRegionCachePrefetch(conf, TABLENAME));
239 
240     HTable.setRegionCachePrefetch(conf, TABLENAME, false);
241     assertFalse("The table is disabled for region cache prefetch",
242         HTable.getRegionCachePrefetch(conf, TABLENAME));
243 
244     HTable.setRegionCachePrefetch(conf, TABLENAME, true);
245     assertTrue("The table is enabled for region cache prefetch",
246         HTable.getRegionCachePrefetch(conf, TABLENAME));
247 
248     table.getConnection().clearRegionCache();
249 
250     assertEquals("Number of cached region is incorrect ", 0,
251         HConnectionManager.getCachedRegionCount(conf, TABLENAME));
252 
253     // if there is a cache miss, some additional regions should be prefetched.
254     Get g2 = new Get(Bytes.toBytes("bbb"));
255     table.get(g2);
256 
257     // Get the configured number of cache read-ahead regions.
258     int prefetchRegionNumber = conf.getInt("hbase.client.prefetch.limit", 10);
259 
260     // the total number of cached regions == region('aaa") + prefeched regions.
261     LOG.info("Testing how many regions cached");
262     assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
263         HConnectionManager.getCachedRegionCount(conf, TABLENAME));
264 
265     table.getConnection().clearRegionCache();
266 
267     Get g3 = new Get(Bytes.toBytes("abc"));
268     table.get(g3);
269     assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
270         HConnectionManager.getCachedRegionCount(conf, TABLENAME));
271 
272     LOG.info("Finishing testRegionCachePreWarm");
273   }
274 
275 
276   /**
277    * Verifies that getConfiguration returns the same Configuration object used
278    * to create the HTable instance.
279    */
280   @Test
281   public void testGetConfiguration() throws Exception {
282     byte[] TABLE = Bytes.toBytes("testGetConfiguration");
283     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
284     Configuration conf = TEST_UTIL.getConfiguration();
285     HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
286     assertSame(conf, table.getConfiguration());
287   }
288 
289   /**
290    * Test from client side of an involved filter against a multi family that
291    * involves deletes.
292    *
293    * @throws Exception
294    */
295   @Test
296   public void testWeirdCacheBehaviour() throws Exception {
297     byte [] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
298     byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
299         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
300         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
301     HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
302     String value = "this is the value";
303     String value2 = "this is some other value";
304     String keyPrefix1 = UUID.randomUUID().toString();
305     String keyPrefix2 = UUID.randomUUID().toString();
306     String keyPrefix3 = UUID.randomUUID().toString();
307     putRows(ht, 3, value, keyPrefix1);
308     putRows(ht, 3, value, keyPrefix2);
309     putRows(ht, 3, value, keyPrefix3);
310     ht.flushCommits();
311     putRows(ht, 3, value2, keyPrefix1);
312     putRows(ht, 3, value2, keyPrefix2);
313     putRows(ht, 3, value2, keyPrefix3);
314     HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
315     System.out.println("Checking values for key: " + keyPrefix1);
316     assertEquals("Got back incorrect number of rows from scan", 3,
317         getNumberOfRows(keyPrefix1, value2, table));
318     System.out.println("Checking values for key: " + keyPrefix2);
319     assertEquals("Got back incorrect number of rows from scan", 3,
320         getNumberOfRows(keyPrefix2, value2, table));
321     System.out.println("Checking values for key: " + keyPrefix3);
322     assertEquals("Got back incorrect number of rows from scan", 3,
323         getNumberOfRows(keyPrefix3, value2, table));
324     deleteColumns(ht, value2, keyPrefix1);
325     deleteColumns(ht, value2, keyPrefix2);
326     deleteColumns(ht, value2, keyPrefix3);
327     System.out.println("Starting important checks.....");
328     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
329       0, getNumberOfRows(keyPrefix1, value2, table));
330     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
331       0, getNumberOfRows(keyPrefix2, value2, table));
332     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
333       0, getNumberOfRows(keyPrefix3, value2, table));
334     ht.setScannerCaching(0);
335     assertEquals("Got back incorrect number of rows from scan", 0,
336       getNumberOfRows(keyPrefix1, value2, table)); ht.setScannerCaching(100);
337     assertEquals("Got back incorrect number of rows from scan", 0,
338       getNumberOfRows(keyPrefix2, value2, table));
339   }
340 
341   private void deleteColumns(HTable ht, String value, String keyPrefix)
342   throws IOException {
343     ResultScanner scanner = buildScanner(keyPrefix, value, ht);
344     Iterator<Result> it = scanner.iterator();
345     int count = 0;
346     while (it.hasNext()) {
347       Result result = it.next();
348       Delete delete = new Delete(result.getRow());
349       delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
350       ht.delete(delete);
351       count++;
352     }
353     assertEquals("Did not perform correct number of deletes", 3, count);
354   }
355 
356   private int getNumberOfRows(String keyPrefix, String value, HTable ht)
357       throws Exception {
358     ResultScanner resultScanner = buildScanner(keyPrefix, value, ht);
359     Iterator<Result> scanner = resultScanner.iterator();
360     int numberOfResults = 0;
361     while (scanner.hasNext()) {
362       Result result = scanner.next();
363       System.out.println("Got back key: " + Bytes.toString(result.getRow()));
364       for (KeyValue kv : result.raw()) {
365         System.out.println("kv=" + kv.toString() + ", "
366             + Bytes.toString(kv.getValue()));
367       }
368       numberOfResults++;
369     }
370     return numberOfResults;
371   }
372 
373   private ResultScanner buildScanner(String keyPrefix, String value, HTable ht)
374       throws IOException {
375     // OurFilterList allFilters = new OurFilterList();
376     FilterList allFilters = new FilterList(/* FilterList.Operator.MUST_PASS_ALL */);
377     allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
378     SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes
379         .toBytes("trans-tags"), Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes
380         .toBytes(value));
381     filter.setFilterIfMissing(true);
382     allFilters.addFilter(filter);
383 
384     // allFilters.addFilter(new
385     // RowExcludingSingleColumnValueFilter(Bytes.toBytes("trans-tags"),
386     // Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value)));
387 
388     Scan scan = new Scan();
389     scan.addFamily(Bytes.toBytes("trans-blob"));
390     scan.addFamily(Bytes.toBytes("trans-type"));
391     scan.addFamily(Bytes.toBytes("trans-date"));
392     scan.addFamily(Bytes.toBytes("trans-tags"));
393     scan.addFamily(Bytes.toBytes("trans-group"));
394     scan.setFilter(allFilters);
395 
396     return ht.getScanner(scan);
397   }
398 
399   private void putRows(HTable ht, int numRows, String value, String key)
400       throws IOException {
401     for (int i = 0; i < numRows; i++) {
402       String row = key + "_" + UUID.randomUUID().toString();
403       System.out.println(String.format("Saving row: %s, with value %s", row,
404           value));
405       Put put = new Put(Bytes.toBytes(row));
406       put.setWriteToWAL(false);
407       put.add(Bytes.toBytes("trans-blob"), null, Bytes
408           .toBytes("value for blob"));
409       put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
410       put.add(Bytes.toBytes("trans-date"), null, Bytes
411           .toBytes("20090921010101999"));
412       put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"), Bytes
413           .toBytes(value));
414       put.add(Bytes.toBytes("trans-group"), null, Bytes
415           .toBytes("adhocTransactionGroupId"));
416       ht.put(put);
417     }
418   }
419 
420   /**
421    * Test filters when multiple regions.  It does counts.  Needs eye-balling of
422    * logs to ensure that we're not scanning more regions that we're supposed to.
423    * Related to the TestFilterAcrossRegions over in the o.a.h.h.filter package.
424    * @throws IOException
425    * @throws InterruptedException
426    */
427   @Test
428   public void testFilterAcrossMultipleRegions()
429   throws IOException, InterruptedException {
430     byte [] name = Bytes.toBytes("testFilterAcrossMutlipleRegions");
431     HTable t = TEST_UTIL.createTable(name, FAMILY);
432     int rowCount = TEST_UTIL.loadTable(t, FAMILY);
433     assertRowCount(t, rowCount);
434     // Split the table.  Should split on a reasonable key; 'lqj'
435     Map<HRegionInfo, HServerAddress> regions  = splitTable(t);
436     assertRowCount(t, rowCount);
437     // Get end key of first region.
438     byte [] endKey = regions.keySet().iterator().next().getEndKey();
439     // Count rows with a filter that stops us before passed 'endKey'.
440     // Should be count of rows in first region.
441     int endKeyCount = countRows(t, createScanWithRowFilter(endKey));
442     assertTrue(endKeyCount < rowCount);
443 
444     // How do I know I did not got to second region?  Thats tough.  Can't really
445     // do that in client-side region test.  I verified by tracing in debugger.
446     // I changed the messages that come out when set to DEBUG so should see
447     // when scanner is done. Says "Finished with scanning..." with region name.
448     // Check that its finished in right region.
449 
450     // New test.  Make it so scan goes into next region by one and then two.
451     // Make sure count comes out right.
452     byte [] key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] + 1)};
453     int plusOneCount = countRows(t, createScanWithRowFilter(key));
454     assertEquals(endKeyCount + 1, plusOneCount);
455     key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] + 2)};
456     int plusTwoCount = countRows(t, createScanWithRowFilter(key));
457     assertEquals(endKeyCount + 2, plusTwoCount);
458 
459     // New test.  Make it so I scan one less than endkey.
460     key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] - 1)};
461     int minusOneCount = countRows(t, createScanWithRowFilter(key));
462     assertEquals(endKeyCount - 1, minusOneCount);
463     // For above test... study logs.  Make sure we do "Finished with scanning.."
464     // in first region and that we do not fall into the next region.
465 
466     key = new byte [] {'a', 'a', 'a'};
467     int countBBB = countRows(t,
468       createScanWithRowFilter(key, null, CompareFilter.CompareOp.EQUAL));
469     assertEquals(1, countBBB);
470 
471     int countGreater = countRows(t, createScanWithRowFilter(endKey, null,
472       CompareFilter.CompareOp.GREATER_OR_EQUAL));
473     // Because started at start of table.
474     assertEquals(0, countGreater);
475     countGreater = countRows(t, createScanWithRowFilter(endKey, endKey,
476       CompareFilter.CompareOp.GREATER_OR_EQUAL));
477     assertEquals(rowCount - endKeyCount, countGreater);
478   }
479 
480   /*
481    * @param key
482    * @return Scan with RowFilter that does LESS than passed key.
483    */
484   private Scan createScanWithRowFilter(final byte [] key) {
485     return createScanWithRowFilter(key, null, CompareFilter.CompareOp.LESS);
486   }
487 
488   /*
489    * @param key
490    * @param op
491    * @param startRow
492    * @return Scan with RowFilter that does CompareOp op on passed key.
493    */
494   private Scan createScanWithRowFilter(final byte [] key,
495       final byte [] startRow, CompareFilter.CompareOp op) {
496     // Make sure key is of some substance... non-null and > than first key.
497     assertTrue(key != null && key.length > 0 &&
498       Bytes.BYTES_COMPARATOR.compare(key, new byte [] {'a', 'a', 'a'}) >= 0);
499     LOG.info("Key=" + Bytes.toString(key));
500     Scan s = startRow == null? new Scan(): new Scan(startRow);
501     Filter f = new RowFilter(op, new BinaryComparator(key));
502     f = new WhileMatchFilter(f);
503     s.setFilter(f);
504     return s;
505   }
506 
507   /*
508    * @param t
509    * @param s
510    * @return Count of rows in table.
511    * @throws IOException
512    */
513   private int countRows(final HTable t, final Scan s)
514   throws IOException {
515     // Assert all rows in table.
516     ResultScanner scanner = t.getScanner(s);
517     int count = 0;
518     for (Result result: scanner) {
519       count++;
520       assertTrue(result.size() > 0);
521       // LOG.info("Count=" + count + ", row=" + Bytes.toString(result.getRow()));
522     }
523     return count;
524   }
525 
526   private void assertRowCount(final HTable t, final int expected)
527   throws IOException {
528     assertEquals(expected, countRows(t, new Scan()));
529   }
530 
531   /*
532    * Split table into multiple regions.
533    * @param t Table to split.
534    * @return Map of regions to servers.
535    * @throws IOException
536    */
537   private Map<HRegionInfo, HServerAddress> splitTable(final HTable t)
538   throws IOException, InterruptedException {
539     // Split this table in two.
540     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
541     admin.split(t.getTableName());
542     Map<HRegionInfo, HServerAddress> regions = waitOnSplit(t);
543     assertTrue(regions.size() > 1);
544     return regions;
545   }
546 
547   /*
548    * Wait on table split.  May return because we waited long enough on the split
549    * and it didn't happen.  Caller should check.
550    * @param t
551    * @return Map of table regions; caller needs to check table actually split.
552    */
553   private Map<HRegionInfo, HServerAddress> waitOnSplit(final HTable t)
554   throws IOException {
555     Map<HRegionInfo, HServerAddress> regions = t.getRegionsInfo();
556     int originalCount = regions.size();
557     for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) {
558       Thread.currentThread();
559       try {
560         Thread.sleep(1000);
561       } catch (InterruptedException e) {
562         e.printStackTrace();
563       }
564       regions = t.getRegionsInfo();
565       if (regions.size() > originalCount) break;
566     }
567     return regions;
568   }
569 
570   @Test
571   public void testSuperSimple() throws Exception {
572     byte [] TABLE = Bytes.toBytes("testSuperSimple");
573     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
574     Put put = new Put(ROW);
575     put.add(FAMILY, QUALIFIER, VALUE);
576     ht.put(put);
577     Scan scan = new Scan();
578     scan.addColumn(FAMILY, TABLE);
579     ResultScanner scanner = ht.getScanner(scan);
580     Result result = scanner.next();
581     assertTrue("Expected null result", result == null);
582     scanner.close();
583   }
584 
585   @Test
586   public void testMaxKeyValueSize() throws Exception {
587     byte [] TABLE = Bytes.toBytes("testMaxKeyValueSize");
588     Configuration conf = TEST_UTIL.getConfiguration();
589     String oldMaxSize = conf.get("hbase.client.keyvalue.maxsize");
590     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
591     byte[] value = new byte[4 * 1024 * 1024];
592     Put put = new Put(ROW);
593     put.add(FAMILY, QUALIFIER, value);
594     ht.put(put);
595     try {
596       conf.setInt("hbase.client.keyvalue.maxsize", 2 * 1024 * 1024);
597       TABLE = Bytes.toBytes("testMaxKeyValueSize2");
598       ht = TEST_UTIL.createTable(TABLE, FAMILY);
599       put = new Put(ROW);
600       put.add(FAMILY, QUALIFIER, value);
601       ht.put(put);
602       fail("Inserting a too large KeyValue worked, should throw exception");
603     } catch(Exception e) {}
604     conf.set("hbase.client.keyvalue.maxsize", oldMaxSize);
605   }
606 
607   @Test
608   public void testFilters() throws Exception {
609     byte [] TABLE = Bytes.toBytes("testFilters");
610     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
611     byte [][] ROWS = makeN(ROW, 10);
612     byte [][] QUALIFIERS = {
613         Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
614         Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"),
615         Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"),
616         Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"),
617         Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>")
618     };
619     for(int i=0;i<10;i++) {
620       Put put = new Put(ROWS[i]);
621       put.setWriteToWAL(false);
622       put.add(FAMILY, QUALIFIERS[i], VALUE);
623       ht.put(put);
624     }
625     Scan scan = new Scan();
626     scan.addFamily(FAMILY);
627     Filter filter = new QualifierFilter(CompareOp.EQUAL,
628       new RegexStringComparator("col[1-5]"));
629     scan.setFilter(filter);
630     ResultScanner scanner = ht.getScanner(scan);
631     int expectedIndex = 1;
632     for(Result result : ht.getScanner(scan)) {
633       assertEquals(result.size(), 1);
634       assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[expectedIndex]));
635       assertTrue(Bytes.equals(result.raw()[0].getQualifier(),
636           QUALIFIERS[expectedIndex]));
637       expectedIndex++;
638     }
639     assertEquals(expectedIndex, 6);
640     scanner.close();
641   }
642 
643   @Test
644   public void testKeyOnlyFilter() throws Exception {
645     byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
646     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
647     byte [][] ROWS = makeN(ROW, 10);
648     byte [][] QUALIFIERS = {
649         Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
650         Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"),
651         Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"),
652         Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"),
653         Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>")
654     };
655     for(int i=0;i<10;i++) {
656       Put put = new Put(ROWS[i]);
657       put.setWriteToWAL(false);
658       put.add(FAMILY, QUALIFIERS[i], VALUE);
659       ht.put(put);
660     }
661     Scan scan = new Scan();
662     scan.addFamily(FAMILY);
663     Filter filter = new KeyOnlyFilter(true);
664     scan.setFilter(filter);
665     ResultScanner scanner = ht.getScanner(scan);
666     int count = 0;
667     for(Result result : ht.getScanner(scan)) {
668       assertEquals(result.size(), 1);
669       assertEquals(result.raw()[0].getValueLength(), Bytes.SIZEOF_INT);
670       assertEquals(Bytes.toInt(result.raw()[0].getValue()), VALUE.length);
671       count++;
672     }
673     assertEquals(count, 10);
674     scanner.close();
675   }
676 
677   /**
678    * Test simple table and non-existent row cases.
679    */
680   @Test
681   public void testSimpleMissing() throws Exception {
682     byte [] TABLE = Bytes.toBytes("testSimpleMissing");
683     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
684     byte [][] ROWS = makeN(ROW, 4);
685 
686     // Try to get a row on an empty table
687     Get get = new Get(ROWS[0]);
688     Result result = ht.get(get);
689     assertEmptyResult(result);
690 
691     get = new Get(ROWS[0]);
692     get.addFamily(FAMILY);
693     result = ht.get(get);
694     assertEmptyResult(result);
695 
696     get = new Get(ROWS[0]);
697     get.addColumn(FAMILY, QUALIFIER);
698     result = ht.get(get);
699     assertEmptyResult(result);
700 
701     Scan scan = new Scan();
702     result = getSingleScanResult(ht, scan);
703     assertNullResult(result);
704 
705 
706     scan = new Scan(ROWS[0]);
707     result = getSingleScanResult(ht, scan);
708     assertNullResult(result);
709 
710     scan = new Scan(ROWS[0],ROWS[1]);
711     result = getSingleScanResult(ht, scan);
712     assertNullResult(result);
713 
714     scan = new Scan();
715     scan.addFamily(FAMILY);
716     result = getSingleScanResult(ht, scan);
717     assertNullResult(result);
718 
719     scan = new Scan();
720     scan.addColumn(FAMILY, QUALIFIER);
721     result = getSingleScanResult(ht, scan);
722     assertNullResult(result);
723 
724     // Insert a row
725 
726     Put put = new Put(ROWS[2]);
727     put.add(FAMILY, QUALIFIER, VALUE);
728     ht.put(put);
729 
730     // Try to get empty rows around it
731 
732     get = new Get(ROWS[1]);
733     result = ht.get(get);
734     assertEmptyResult(result);
735 
736     get = new Get(ROWS[0]);
737     get.addFamily(FAMILY);
738     result = ht.get(get);
739     assertEmptyResult(result);
740 
741     get = new Get(ROWS[3]);
742     get.addColumn(FAMILY, QUALIFIER);
743     result = ht.get(get);
744     assertEmptyResult(result);
745 
746     // Try to scan empty rows around it
747 
748     scan = new Scan(ROWS[3]);
749     result = getSingleScanResult(ht, scan);
750     assertNullResult(result);
751 
752     scan = new Scan(ROWS[0],ROWS[2]);
753     result = getSingleScanResult(ht, scan);
754     assertNullResult(result);
755 
756     // Make sure we can actually get the row
757 
758     get = new Get(ROWS[2]);
759     result = ht.get(get);
760     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
761 
762     get = new Get(ROWS[2]);
763     get.addFamily(FAMILY);
764     result = ht.get(get);
765     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
766 
767     get = new Get(ROWS[2]);
768     get.addColumn(FAMILY, QUALIFIER);
769     result = ht.get(get);
770     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
771 
772     // Make sure we can scan the row
773 
774     scan = new Scan();
775     result = getSingleScanResult(ht, scan);
776     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
777 
778     scan = new Scan(ROWS[0],ROWS[3]);
779     result = getSingleScanResult(ht, scan);
780     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
781 
782     scan = new Scan(ROWS[2],ROWS[3]);
783     result = getSingleScanResult(ht, scan);
784     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
785   }
786 
787   /**
788    * Test basic puts, gets, scans, and deletes for a single row
789    * in a multiple family table.
790    */
791   @Test
792   public void testSingleRowMultipleFamily() throws Exception {
793     byte [] TABLE = Bytes.toBytes("testSingleRowMultipleFamily");
794     byte [][] ROWS = makeN(ROW, 3);
795     byte [][] FAMILIES = makeNAscii(FAMILY, 10);
796     byte [][] QUALIFIERS = makeN(QUALIFIER, 10);
797     byte [][] VALUES = makeN(VALUE, 10);
798 
799     HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
800 
801     Get get;
802     Scan scan;
803     Delete delete;
804     Put put;
805     Result result;
806 
807     ////////////////////////////////////////////////////////////////////////////
808     // Insert one column to one family
809     ////////////////////////////////////////////////////////////////////////////
810 
811     put = new Put(ROWS[0]);
812     put.add(FAMILIES[4], QUALIFIERS[0], VALUES[0]);
813     ht.put(put);
814 
815     // Get the single column
816     getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
817 
818     // Scan the single column
819     scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
820 
821     // Get empty results around inserted column
822     getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
823 
824     // Scan empty results around inserted column
825     scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
826 
827     ////////////////////////////////////////////////////////////////////////////
828     // Flush memstore and run same tests from storefiles
829     ////////////////////////////////////////////////////////////////////////////
830 
831     TEST_UTIL.flush();
832 
833     // Redo get and scan tests from storefile
834     getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
835     scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
836     getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
837     scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
838 
839     ////////////////////////////////////////////////////////////////////////////
840     // Now, Test reading from memstore and storefiles at once
841     ////////////////////////////////////////////////////////////////////////////
842 
843     // Insert multiple columns to two other families
844     put = new Put(ROWS[0]);
845     put.add(FAMILIES[2], QUALIFIERS[2], VALUES[2]);
846     put.add(FAMILIES[2], QUALIFIERS[4], VALUES[4]);
847     put.add(FAMILIES[4], QUALIFIERS[4], VALUES[4]);
848     put.add(FAMILIES[6], QUALIFIERS[6], VALUES[6]);
849     put.add(FAMILIES[6], QUALIFIERS[7], VALUES[7]);
850     put.add(FAMILIES[7], QUALIFIERS[7], VALUES[7]);
851     put.add(FAMILIES[9], QUALIFIERS[0], VALUES[0]);
852     ht.put(put);
853 
854     // Get multiple columns across multiple families and get empties around it
855     singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
856 
857     // Scan multiple columns across multiple families and scan empties around it
858     singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
859 
860     ////////////////////////////////////////////////////////////////////////////
861     // Flush the table again
862     ////////////////////////////////////////////////////////////////////////////
863 
864     TEST_UTIL.flush();
865 
866     // Redo tests again
867     singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
868     singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
869 
870     // Insert more data to memstore
871     put = new Put(ROWS[0]);
872     put.add(FAMILIES[6], QUALIFIERS[5], VALUES[5]);
873     put.add(FAMILIES[6], QUALIFIERS[8], VALUES[8]);
874     put.add(FAMILIES[6], QUALIFIERS[9], VALUES[9]);
875     put.add(FAMILIES[4], QUALIFIERS[3], VALUES[3]);
876     ht.put(put);
877 
878     ////////////////////////////////////////////////////////////////////////////
879     // Delete a storefile column
880     ////////////////////////////////////////////////////////////////////////////
881     delete = new Delete(ROWS[0]);
882     delete.deleteColumns(FAMILIES[6], QUALIFIERS[7]);
883     ht.delete(delete);
884 
885     // Try to get deleted column
886     get = new Get(ROWS[0]);
887     get.addColumn(FAMILIES[6], QUALIFIERS[7]);
888     result = ht.get(get);
889     assertEmptyResult(result);
890 
891     // Try to scan deleted column
892     scan = new Scan();
893     scan.addColumn(FAMILIES[6], QUALIFIERS[7]);
894     result = getSingleScanResult(ht, scan);
895     assertNullResult(result);
896 
897     // Make sure we can still get a column before it and after it
898     get = new Get(ROWS[0]);
899     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
900     result = ht.get(get);
901     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
902 
903     get = new Get(ROWS[0]);
904     get.addColumn(FAMILIES[6], QUALIFIERS[8]);
905     result = ht.get(get);
906     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
907 
908     // Make sure we can still scan a column before it and after it
909     scan = new Scan();
910     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
911     result = getSingleScanResult(ht, scan);
912     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
913 
914     scan = new Scan();
915     scan.addColumn(FAMILIES[6], QUALIFIERS[8]);
916     result = getSingleScanResult(ht, scan);
917     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
918 
919     ////////////////////////////////////////////////////////////////////////////
920     // Delete a memstore column
921     ////////////////////////////////////////////////////////////////////////////
922     delete = new Delete(ROWS[0]);
923     delete.deleteColumns(FAMILIES[6], QUALIFIERS[8]);
924     ht.delete(delete);
925 
926     // Try to get deleted column
927     get = new Get(ROWS[0]);
928     get.addColumn(FAMILIES[6], QUALIFIERS[8]);
929     result = ht.get(get);
930     assertEmptyResult(result);
931 
932     // Try to scan deleted column
933     scan = new Scan();
934     scan.addColumn(FAMILIES[6], QUALIFIERS[8]);
935     result = getSingleScanResult(ht, scan);
936     assertNullResult(result);
937 
938     // Make sure we can still get a column before it and after it
939     get = new Get(ROWS[0]);
940     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
941     result = ht.get(get);
942     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
943 
944     get = new Get(ROWS[0]);
945     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
946     result = ht.get(get);
947     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
948 
949     // Make sure we can still scan a column before it and after it
950     scan = new Scan();
951     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
952     result = getSingleScanResult(ht, scan);
953     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
954 
955     scan = new Scan();
956     scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
957     result = getSingleScanResult(ht, scan);
958     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
959 
960     ////////////////////////////////////////////////////////////////////////////
961     // Delete joint storefile/memstore family
962     ////////////////////////////////////////////////////////////////////////////
963 
964     delete = new Delete(ROWS[0]);
965     delete.deleteFamily(FAMILIES[4]);
966     ht.delete(delete);
967 
968     // Try to get storefile column in deleted family
969     get = new Get(ROWS[0]);
970     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
971     result = ht.get(get);
972     assertEmptyResult(result);
973 
974     // Try to get memstore column in deleted family
975     get = new Get(ROWS[0]);
976     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
977     result = ht.get(get);
978     assertEmptyResult(result);
979 
980     // Try to get deleted family
981     get = new Get(ROWS[0]);
982     get.addFamily(FAMILIES[4]);
983     result = ht.get(get);
984     assertEmptyResult(result);
985 
986     // Try to scan storefile column in deleted family
987     scan = new Scan();
988     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
989     result = getSingleScanResult(ht, scan);
990     assertNullResult(result);
991 
992     // Try to scan memstore column in deleted family
993     scan = new Scan();
994     scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
995     result = getSingleScanResult(ht, scan);
996     assertNullResult(result);
997 
998     // Try to scan deleted family
999     scan = new Scan();
1000     scan.addFamily(FAMILIES[4]);
1001     result = getSingleScanResult(ht, scan);
1002     assertNullResult(result);
1003 
1004     // Make sure we can still get another family
1005     get = new Get(ROWS[0]);
1006     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
1007     result = ht.get(get);
1008     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1009 
1010     get = new Get(ROWS[0]);
1011     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1012     result = ht.get(get);
1013     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1014 
1015     // Make sure we can still scan another family
1016     scan = new Scan();
1017     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1018     result = getSingleScanResult(ht, scan);
1019     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1020 
1021     scan = new Scan();
1022     scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1023     result = getSingleScanResult(ht, scan);
1024     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1025 
1026     ////////////////////////////////////////////////////////////////////////////
1027     // Flush everything and rerun delete tests
1028     ////////////////////////////////////////////////////////////////////////////
1029 
1030     TEST_UTIL.flush();
1031 
1032     // Try to get storefile column in deleted family
1033     get = new Get(ROWS[0]);
1034     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
1035     result = ht.get(get);
1036     assertEmptyResult(result);
1037 
1038     // Try to get memstore column in deleted family
1039     get = new Get(ROWS[0]);
1040     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
1041     result = ht.get(get);
1042     assertEmptyResult(result);
1043 
1044     // Try to get deleted family
1045     get = new Get(ROWS[0]);
1046     get.addFamily(FAMILIES[4]);
1047     result = ht.get(get);
1048     assertEmptyResult(result);
1049 
1050     // Try to scan storefile column in deleted family
1051     scan = new Scan();
1052     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
1053     result = getSingleScanResult(ht, scan);
1054     assertNullResult(result);
1055 
1056     // Try to scan memstore column in deleted family
1057     scan = new Scan();
1058     scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
1059     result = getSingleScanResult(ht, scan);
1060     assertNullResult(result);
1061 
1062     // Try to scan deleted family
1063     scan = new Scan();
1064     scan.addFamily(FAMILIES[4]);
1065     result = getSingleScanResult(ht, scan);
1066     assertNullResult(result);
1067 
1068     // Make sure we can still get another family
1069     get = new Get(ROWS[0]);
1070     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
1071     result = ht.get(get);
1072     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1073 
1074     get = new Get(ROWS[0]);
1075     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1076     result = ht.get(get);
1077     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1078 
1079     // Make sure we can still scan another family
1080     scan = new Scan();
1081     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1082     result = getSingleScanResult(ht, scan);
1083     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1084 
1085     scan = new Scan();
1086     scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1087     result = getSingleScanResult(ht, scan);
1088     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1089 
1090   }
1091 
1092   @Test
1093   public void testNull() throws Exception {
1094     byte [] TABLE = Bytes.toBytes("testNull");
1095 
1096     // Null table name (should NOT work)
1097     try {
1098       TEST_UTIL.createTable(null, FAMILY);
1099       fail("Creating a table with null name passed, should have failed");
1100     } catch(Exception e) {}
1101 
1102     // Null family (should NOT work)
1103     try {
1104       TEST_UTIL.createTable(TABLE, (byte[])null);
1105       fail("Creating a table with a null family passed, should fail");
1106     } catch(Exception e) {}
1107 
1108     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
1109 
1110     // Null row (should NOT work)
1111     try {
1112       Put put = new Put((byte[])null);
1113       put.add(FAMILY, QUALIFIER, VALUE);
1114       ht.put(put);
1115       fail("Inserting a null row worked, should throw exception");
1116     } catch(Exception e) {}
1117 
1118     // Null qualifier (should work)
1119     {
1120       Put put = new Put(ROW);
1121       put.add(FAMILY, null, VALUE);
1122       ht.put(put);
1123 
1124       getTestNull(ht, ROW, FAMILY, VALUE);
1125 
1126       scanTestNull(ht, ROW, FAMILY, VALUE);
1127 
1128       Delete delete = new Delete(ROW);
1129       delete.deleteColumns(FAMILY, null);
1130       ht.delete(delete);
1131 
1132       Get get = new Get(ROW);
1133       Result result = ht.get(get);
1134       assertEmptyResult(result);
1135     }
1136 
1137     // Use a new table
1138     byte [] TABLE2 = Bytes.toBytes("testNull2");
1139     ht = TEST_UTIL.createTable(TABLE2, FAMILY);
1140 
1141     // Empty qualifier, byte[0] instead of null (should work)
1142     try {
1143       Put put = new Put(ROW);
1144       put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
1145       ht.put(put);
1146 
1147       getTestNull(ht, ROW, FAMILY, VALUE);
1148 
1149       scanTestNull(ht, ROW, FAMILY, VALUE);
1150 
1151       // Flush and try again
1152 
1153       TEST_UTIL.flush();
1154 
1155       getTestNull(ht, ROW, FAMILY, VALUE);
1156 
1157       scanTestNull(ht, ROW, FAMILY, VALUE);
1158 
1159       Delete delete = new Delete(ROW);
1160       delete.deleteColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY);
1161       ht.delete(delete);
1162 
1163       Get get = new Get(ROW);
1164       Result result = ht.get(get);
1165       assertEmptyResult(result);
1166 
1167     } catch(Exception e) {
1168       throw new IOException("Using a row with null qualifier threw exception, should ");
1169     }
1170 
1171     // Null value
1172     try {
1173       Put put = new Put(ROW);
1174       put.add(FAMILY, QUALIFIER, null);
1175       ht.put(put);
1176 
1177       Get get = new Get(ROW);
1178       get.addColumn(FAMILY, QUALIFIER);
1179       Result result = ht.get(get);
1180       assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
1181 
1182       Scan scan = new Scan();
1183       scan.addColumn(FAMILY, QUALIFIER);
1184       result = getSingleScanResult(ht, scan);
1185       assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
1186 
1187       Delete delete = new Delete(ROW);
1188       delete.deleteColumns(FAMILY, QUALIFIER);
1189       ht.delete(delete);
1190 
1191       get = new Get(ROW);
1192       result = ht.get(get);
1193       assertEmptyResult(result);
1194 
1195     } catch(Exception e) {
1196       throw new IOException("Null values should be allowed, but threw exception");
1197     }
1198   }
1199 
1200   @Test
1201   public void testVersions() throws Exception {
1202     byte [] TABLE = Bytes.toBytes("testVersions");
1203 
1204     long [] STAMPS = makeStamps(20);
1205     byte [][] VALUES = makeNAscii(VALUE, 20);
1206 
1207     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
1208 
1209     // Insert 4 versions of same column
1210     Put put = new Put(ROW);
1211     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1212     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1213     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1214     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1215     ht.put(put);
1216 
1217     // Verify we can get each one properly
1218     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1219     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1220     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1221     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1222     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1223     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1224     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1225     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1226 
1227     // Verify we don't accidentally get others
1228     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1229     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1230     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1231     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1232     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1233     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1234 
1235     // Ensure maxVersions in query is respected
1236     Get get = new Get(ROW);
1237     get.addColumn(FAMILY, QUALIFIER);
1238     get.setMaxVersions(2);
1239     Result result = ht.get(get);
1240     assertNResult(result, ROW, FAMILY, QUALIFIER,
1241         new long [] {STAMPS[4], STAMPS[5]},
1242         new byte[][] {VALUES[4], VALUES[5]},
1243         0, 1);
1244 
1245     Scan scan = new Scan(ROW);
1246     scan.addColumn(FAMILY, QUALIFIER);
1247     scan.setMaxVersions(2);
1248     result = getSingleScanResult(ht, scan);
1249     assertNResult(result, ROW, FAMILY, QUALIFIER,
1250         new long [] {STAMPS[4], STAMPS[5]},
1251         new byte[][] {VALUES[4], VALUES[5]},
1252         0, 1);
1253 
1254     // Flush and redo
1255 
1256     TEST_UTIL.flush();
1257 
1258     // Verify we can get each one properly
1259     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1260     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1261     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1262     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1263     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1264     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1265     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1266     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1267 
1268     // Verify we don't accidentally get others
1269     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1270     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1271     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1272     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1273     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1274     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1275 
1276     // Ensure maxVersions in query is respected
1277     get = new Get(ROW);
1278     get.addColumn(FAMILY, QUALIFIER);
1279     get.setMaxVersions(2);
1280     result = ht.get(get);
1281     assertNResult(result, ROW, FAMILY, QUALIFIER,
1282         new long [] {STAMPS[4], STAMPS[5]},
1283         new byte[][] {VALUES[4], VALUES[5]},
1284         0, 1);
1285 
1286     scan = new Scan(ROW);
1287     scan.addColumn(FAMILY, QUALIFIER);
1288     scan.setMaxVersions(2);
1289     result = getSingleScanResult(ht, scan);
1290     assertNResult(result, ROW, FAMILY, QUALIFIER,
1291         new long [] {STAMPS[4], STAMPS[5]},
1292         new byte[][] {VALUES[4], VALUES[5]},
1293         0, 1);
1294 
1295 
1296     // Add some memstore and retest
1297 
1298     // Insert 4 more versions of same column and a dupe
1299     put = new Put(ROW);
1300     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
1301     put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
1302     put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1303     put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]);
1304     ht.put(put);
1305 
1306     // Ensure maxVersions in query is respected
1307     get = new Get(ROW);
1308     get.addColumn(FAMILY, QUALIFIER);
1309     get.setMaxVersions();
1310     result = ht.get(get);
1311     assertNResult(result, ROW, FAMILY, QUALIFIER,
1312         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1313         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1314         0, 7);
1315 
1316     scan = new Scan(ROW);
1317     scan.addColumn(FAMILY, QUALIFIER);
1318     scan.setMaxVersions();
1319     result = getSingleScanResult(ht, scan);
1320     assertNResult(result, ROW, FAMILY, QUALIFIER,
1321         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1322         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1323         0, 7);
1324 
1325     get = new Get(ROW);
1326     get.setMaxVersions();
1327     result = ht.get(get);
1328     assertNResult(result, ROW, FAMILY, QUALIFIER,
1329         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1330         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1331         0, 7);
1332 
1333     scan = new Scan(ROW);
1334     scan.setMaxVersions();
1335     result = getSingleScanResult(ht, scan);
1336     assertNResult(result, ROW, FAMILY, QUALIFIER,
1337         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1338         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1339         0, 7);
1340 
1341     // Verify we can get each one properly
1342     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1343     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1344     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1345     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1346     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1347     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1348     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1349     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1350 
1351     // Verify we don't accidentally get others
1352     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1353     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
1354     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1355     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
1356 
1357     // Ensure maxVersions of table is respected
1358 
1359     TEST_UTIL.flush();
1360 
1361     // Insert 4 more versions of same column and a dupe
1362     put = new Put(ROW);
1363     put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]);
1364     put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]);
1365     put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]);
1366     put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
1367     ht.put(put);
1368 
1369     get = new Get(ROW);
1370     get.addColumn(FAMILY, QUALIFIER);
1371     get.setMaxVersions(Integer.MAX_VALUE);
1372     result = ht.get(get);
1373     assertNResult(result, ROW, FAMILY, QUALIFIER,
1374         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
1375         new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
1376         0, 9);
1377 
1378     scan = new Scan(ROW);
1379     scan.addColumn(FAMILY, QUALIFIER);
1380     scan.setMaxVersions(Integer.MAX_VALUE);
1381     result = getSingleScanResult(ht, scan);
1382     assertNResult(result, ROW, FAMILY, QUALIFIER,
1383         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
1384         new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
1385         0, 9);
1386 
1387     // Delete a version in the memstore and a version in a storefile
1388     Delete delete = new Delete(ROW);
1389     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
1390     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
1391     ht.delete(delete);
1392 
1393     // Test that it's gone
1394     get = new Get(ROW);
1395     get.addColumn(FAMILY, QUALIFIER);
1396     get.setMaxVersions(Integer.MAX_VALUE);
1397     result = ht.get(get);
1398     assertNResult(result, ROW, FAMILY, QUALIFIER,
1399         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
1400         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
1401         0, 9);
1402 
1403     scan = new Scan(ROW);
1404     scan.addColumn(FAMILY, QUALIFIER);
1405     scan.setMaxVersions(Integer.MAX_VALUE);
1406     result = getSingleScanResult(ht, scan);
1407     assertNResult(result, ROW, FAMILY, QUALIFIER,
1408         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
1409         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
1410         0, 9);
1411 
1412   }
1413 
1414   @Test
1415   public void testVersionLimits() throws Exception {
1416     byte [] TABLE = Bytes.toBytes("testVersionLimits");
1417     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
1418     int [] LIMITS = {1,3,5};
1419     long [] STAMPS = makeStamps(10);
1420     byte [][] VALUES = makeNAscii(VALUE, 10);
1421     HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS);
1422 
1423     // Insert limit + 1 on each family
1424     Put put = new Put(ROW);
1425     put.add(FAMILIES[0], QUALIFIER, STAMPS[0], VALUES[0]);
1426     put.add(FAMILIES[0], QUALIFIER, STAMPS[1], VALUES[1]);
1427     put.add(FAMILIES[1], QUALIFIER, STAMPS[0], VALUES[0]);
1428     put.add(FAMILIES[1], QUALIFIER, STAMPS[1], VALUES[1]);
1429     put.add(FAMILIES[1], QUALIFIER, STAMPS[2], VALUES[2]);
1430     put.add(FAMILIES[1], QUALIFIER, STAMPS[3], VALUES[3]);
1431     put.add(FAMILIES[2], QUALIFIER, STAMPS[0], VALUES[0]);
1432     put.add(FAMILIES[2], QUALIFIER, STAMPS[1], VALUES[1]);
1433     put.add(FAMILIES[2], QUALIFIER, STAMPS[2], VALUES[2]);
1434     put.add(FAMILIES[2], QUALIFIER, STAMPS[3], VALUES[3]);
1435     put.add(FAMILIES[2], QUALIFIER, STAMPS[4], VALUES[4]);
1436     put.add(FAMILIES[2], QUALIFIER, STAMPS[5], VALUES[5]);
1437     put.add(FAMILIES[2], QUALIFIER, STAMPS[6], VALUES[6]);
1438     ht.put(put);
1439 
1440     // Verify we only get the right number out of each
1441 
1442     // Family0
1443 
1444     Get get = new Get(ROW);
1445     get.addColumn(FAMILIES[0], QUALIFIER);
1446     get.setMaxVersions(Integer.MAX_VALUE);
1447     Result result = ht.get(get);
1448     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1449         new long [] {STAMPS[1]},
1450         new byte[][] {VALUES[1]},
1451         0, 0);
1452 
1453     get = new Get(ROW);
1454     get.addFamily(FAMILIES[0]);
1455     get.setMaxVersions(Integer.MAX_VALUE);
1456     result = ht.get(get);
1457     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1458         new long [] {STAMPS[1]},
1459         new byte[][] {VALUES[1]},
1460         0, 0);
1461 
1462     Scan scan = new Scan(ROW);
1463     scan.addColumn(FAMILIES[0], QUALIFIER);
1464     scan.setMaxVersions(Integer.MAX_VALUE);
1465     result = getSingleScanResult(ht, scan);
1466     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1467         new long [] {STAMPS[1]},
1468         new byte[][] {VALUES[1]},
1469         0, 0);
1470 
1471     scan = new Scan(ROW);
1472     scan.addFamily(FAMILIES[0]);
1473     scan.setMaxVersions(Integer.MAX_VALUE);
1474     result = getSingleScanResult(ht, scan);
1475     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1476         new long [] {STAMPS[1]},
1477         new byte[][] {VALUES[1]},
1478         0, 0);
1479 
1480     // Family1
1481 
1482     get = new Get(ROW);
1483     get.addColumn(FAMILIES[1], QUALIFIER);
1484     get.setMaxVersions(Integer.MAX_VALUE);
1485     result = ht.get(get);
1486     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1487         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1488         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1489         0, 2);
1490 
1491     get = new Get(ROW);
1492     get.addFamily(FAMILIES[1]);
1493     get.setMaxVersions(Integer.MAX_VALUE);
1494     result = ht.get(get);
1495     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1496         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1497         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1498         0, 2);
1499 
1500     scan = new Scan(ROW);
1501     scan.addColumn(FAMILIES[1], QUALIFIER);
1502     scan.setMaxVersions(Integer.MAX_VALUE);
1503     result = getSingleScanResult(ht, scan);
1504     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1505         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1506         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1507         0, 2);
1508 
1509     scan = new Scan(ROW);
1510     scan.addFamily(FAMILIES[1]);
1511     scan.setMaxVersions(Integer.MAX_VALUE);
1512     result = getSingleScanResult(ht, scan);
1513     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1514         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1515         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1516         0, 2);
1517 
1518     // Family2
1519 
1520     get = new Get(ROW);
1521     get.addColumn(FAMILIES[2], QUALIFIER);
1522     get.setMaxVersions(Integer.MAX_VALUE);
1523     result = ht.get(get);
1524     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1525         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1526         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1527         0, 4);
1528 
1529     get = new Get(ROW);
1530     get.addFamily(FAMILIES[2]);
1531     get.setMaxVersions(Integer.MAX_VALUE);
1532     result = ht.get(get);
1533     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1534         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1535         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1536         0, 4);
1537 
1538     scan = new Scan(ROW);
1539     scan.addColumn(FAMILIES[2], QUALIFIER);
1540     scan.setMaxVersions(Integer.MAX_VALUE);
1541     result = getSingleScanResult(ht, scan);
1542     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1543         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1544         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1545         0, 4);
1546 
1547     scan = new Scan(ROW);
1548     scan.addFamily(FAMILIES[2]);
1549     scan.setMaxVersions(Integer.MAX_VALUE);
1550     result = getSingleScanResult(ht, scan);
1551     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1552         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1553         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1554         0, 4);
1555 
1556     // Try all families
1557 
1558     get = new Get(ROW);
1559     get.setMaxVersions(Integer.MAX_VALUE);
1560     result = ht.get(get);
1561     assertTrue("Expected 9 keys but received " + result.size(),
1562         result.size() == 9);
1563 
1564     get = new Get(ROW);
1565     get.addFamily(FAMILIES[0]);
1566     get.addFamily(FAMILIES[1]);
1567     get.addFamily(FAMILIES[2]);
1568     get.setMaxVersions(Integer.MAX_VALUE);
1569     result = ht.get(get);
1570     assertTrue("Expected 9 keys but received " + result.size(),
1571         result.size() == 9);
1572 
1573     get = new Get(ROW);
1574     get.addColumn(FAMILIES[0], QUALIFIER);
1575     get.addColumn(FAMILIES[1], QUALIFIER);
1576     get.addColumn(FAMILIES[2], QUALIFIER);
1577     get.setMaxVersions(Integer.MAX_VALUE);
1578     result = ht.get(get);
1579     assertTrue("Expected 9 keys but received " + result.size(),
1580         result.size() == 9);
1581 
1582     scan = new Scan(ROW);
1583     scan.setMaxVersions(Integer.MAX_VALUE);
1584     result = getSingleScanResult(ht, scan);
1585     assertTrue("Expected 9 keys but received " + result.size(),
1586         result.size() == 9);
1587 
1588     scan = new Scan(ROW);
1589     scan.setMaxVersions(Integer.MAX_VALUE);
1590     scan.addFamily(FAMILIES[0]);
1591     scan.addFamily(FAMILIES[1]);
1592     scan.addFamily(FAMILIES[2]);
1593     result = getSingleScanResult(ht, scan);
1594     assertTrue("Expected 9 keys but received " + result.size(),
1595         result.size() == 9);
1596 
1597     scan = new Scan(ROW);
1598     scan.setMaxVersions(Integer.MAX_VALUE);
1599     scan.addColumn(FAMILIES[0], QUALIFIER);
1600     scan.addColumn(FAMILIES[1], QUALIFIER);
1601     scan.addColumn(FAMILIES[2], QUALIFIER);
1602     result = getSingleScanResult(ht, scan);
1603     assertTrue("Expected 9 keys but received " + result.size(),
1604         result.size() == 9);
1605 
1606   }
1607 
1608   @Test
1609   public void testDeletes() throws Exception {
1610     byte [] TABLE = Bytes.toBytes("testDeletes");
1611 
1612     byte [][] ROWS = makeNAscii(ROW, 6);
1613     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
1614     byte [][] VALUES = makeN(VALUE, 5);
1615     long [] ts = {1000, 2000, 3000, 4000, 5000};
1616 
1617     HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
1618 
1619     Put put = new Put(ROW);
1620     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
1621     put.add(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]);
1622     ht.put(put);
1623 
1624     Delete delete = new Delete(ROW);
1625     delete.deleteFamily(FAMILIES[0], ts[0]);
1626     ht.delete(delete);
1627 
1628     Get get = new Get(ROW);
1629     get.addFamily(FAMILIES[0]);
1630     get.setMaxVersions(Integer.MAX_VALUE);
1631     Result result = ht.get(get);
1632     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1633         new long [] {ts[1]},
1634         new byte[][] {VALUES[1]},
1635         0, 0);
1636 
1637     Scan scan = new Scan(ROW);
1638     scan.addFamily(FAMILIES[0]);
1639     scan.setMaxVersions(Integer.MAX_VALUE);
1640     result = getSingleScanResult(ht, scan);
1641     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1642         new long [] {ts[1]},
1643         new byte[][] {VALUES[1]},
1644         0, 0);
1645 
1646     // Test delete latest version
1647     put = new Put(ROW);
1648     put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]);
1649     put.add(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]);
1650     put.add(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]);
1651     put.add(FAMILIES[0], null, ts[4], VALUES[4]);
1652     put.add(FAMILIES[0], null, ts[2], VALUES[2]);
1653     put.add(FAMILIES[0], null, ts[3], VALUES[3]);
1654     ht.put(put);
1655 
1656     delete = new Delete(ROW);
1657     delete.deleteColumn(FAMILIES[0], QUALIFIER); // ts[4]
1658     ht.delete(delete);
1659 
1660     get = new Get(ROW);
1661     get.addColumn(FAMILIES[0], QUALIFIER);
1662     get.setMaxVersions(Integer.MAX_VALUE);
1663     result = ht.get(get);
1664     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1665         new long [] {ts[1], ts[2], ts[3]},
1666         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1667         0, 2);
1668 
1669     scan = new Scan(ROW);
1670     scan.addColumn(FAMILIES[0], QUALIFIER);
1671     scan.setMaxVersions(Integer.MAX_VALUE);
1672     result = getSingleScanResult(ht, scan);
1673     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1674         new long [] {ts[1], ts[2], ts[3]},
1675         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1676         0, 2);
1677 
1678     // Test for HBASE-1847
1679     delete = new Delete(ROW);
1680     delete.deleteColumn(FAMILIES[0], null);
1681     ht.delete(delete);
1682 
1683     // Cleanup null qualifier
1684     delete = new Delete(ROW);
1685     delete.deleteColumns(FAMILIES[0], null);
1686     ht.delete(delete);
1687 
1688     // Expected client behavior might be that you can re-put deleted values
1689     // But alas, this is not to be.  We can't put them back in either case.
1690 
1691     put = new Put(ROW);
1692     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); // 1000
1693     put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); // 5000
1694     ht.put(put);
1695 
1696 
1697     // It used to be due to the internal implementation of Get, that
1698     // the Get() call would return ts[4] UNLIKE the Scan below. With
1699     // the switch to using Scan for Get this is no longer the case.
1700     get = new Get(ROW);
1701     get.addFamily(FAMILIES[0]);
1702     get.setMaxVersions(Integer.MAX_VALUE);
1703     result = ht.get(get);
1704     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1705         new long [] {ts[1], ts[2], ts[3]},
1706         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1707         0, 2);
1708 
1709     // The Scanner returns the previous values, the expected-naive-unexpected behavior
1710 
1711     scan = new Scan(ROW);
1712     scan.addFamily(FAMILIES[0]);
1713     scan.setMaxVersions(Integer.MAX_VALUE);
1714     result = getSingleScanResult(ht, scan);
1715     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1716         new long [] {ts[1], ts[2], ts[3]},
1717         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1718         0, 2);
1719 
1720     // Test deleting an entire family from one row but not the other various ways
1721 
1722     put = new Put(ROWS[0]);
1723     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
1724     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
1725     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
1726     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
1727     ht.put(put);
1728 
1729     put = new Put(ROWS[1]);
1730     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
1731     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
1732     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
1733     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
1734     ht.put(put);
1735 
1736     put = new Put(ROWS[2]);
1737     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
1738     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
1739     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
1740     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
1741     ht.put(put);
1742 
1743     // Assert that above went in.
1744     get = new Get(ROWS[2]);
1745     get.addFamily(FAMILIES[1]);
1746     get.addFamily(FAMILIES[2]);
1747     get.setMaxVersions(Integer.MAX_VALUE);
1748     result = ht.get(get);
1749     assertTrue("Expected 4 key but received " + result.size() + ": " + result,
1750         result.size() == 4);
1751 
1752     delete = new Delete(ROWS[0]);
1753     delete.deleteFamily(FAMILIES[2]);
1754     ht.delete(delete);
1755 
1756     delete = new Delete(ROWS[1]);
1757     delete.deleteColumns(FAMILIES[1], QUALIFIER);
1758     ht.delete(delete);
1759 
1760     delete = new Delete(ROWS[2]);
1761     delete.deleteColumn(FAMILIES[1], QUALIFIER);
1762     delete.deleteColumn(FAMILIES[1], QUALIFIER);
1763     delete.deleteColumn(FAMILIES[2], QUALIFIER);
1764     ht.delete(delete);
1765 
1766     get = new Get(ROWS[0]);
1767     get.addFamily(FAMILIES[1]);
1768     get.addFamily(FAMILIES[2]);
1769     get.setMaxVersions(Integer.MAX_VALUE);
1770     result = ht.get(get);
1771     assertTrue("Expected 2 keys but received " + result.size(),
1772         result.size() == 2);
1773     assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER,
1774         new long [] {ts[0], ts[1]},
1775         new byte[][] {VALUES[0], VALUES[1]},
1776         0, 1);
1777 
1778     scan = new Scan(ROWS[0]);
1779     scan.addFamily(FAMILIES[1]);
1780     scan.addFamily(FAMILIES[2]);
1781     scan.setMaxVersions(Integer.MAX_VALUE);
1782     result = getSingleScanResult(ht, scan);
1783     assertTrue("Expected 2 keys but received " + result.size(),
1784         result.size() == 2);
1785     assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER,
1786         new long [] {ts[0], ts[1]},
1787         new byte[][] {VALUES[0], VALUES[1]},
1788         0, 1);
1789 
1790     get = new Get(ROWS[1]);
1791     get.addFamily(FAMILIES[1]);
1792     get.addFamily(FAMILIES[2]);
1793     get.setMaxVersions(Integer.MAX_VALUE);
1794     result = ht.get(get);
1795     assertTrue("Expected 2 keys but received " + result.size(),
1796         result.size() == 2);
1797 
1798     scan = new Scan(ROWS[1]);
1799     scan.addFamily(FAMILIES[1]);
1800     scan.addFamily(FAMILIES[2]);
1801     scan.setMaxVersions(Integer.MAX_VALUE);
1802     result = getSingleScanResult(ht, scan);
1803     assertTrue("Expected 2 keys but received " + result.size(),
1804         result.size() == 2);
1805 
1806     get = new Get(ROWS[2]);
1807     get.addFamily(FAMILIES[1]);
1808     get.addFamily(FAMILIES[2]);
1809     get.setMaxVersions(Integer.MAX_VALUE);
1810     result = ht.get(get);
1811     assertEquals(1, result.size());
1812     assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
1813         new long [] {ts[2]},
1814         new byte[][] {VALUES[2]},
1815         0, 0);
1816 
1817     scan = new Scan(ROWS[2]);
1818     scan.addFamily(FAMILIES[1]);
1819     scan.addFamily(FAMILIES[2]);
1820     scan.setMaxVersions(Integer.MAX_VALUE);
1821     result = getSingleScanResult(ht, scan);
1822     assertEquals(1, result.size());
1823     assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
1824         new long [] {ts[2]},
1825         new byte[][] {VALUES[2]},
1826         0, 0);
1827 
1828     // Test if we delete the family first in one row (HBASE-1541)
1829 
1830     delete = new Delete(ROWS[3]);
1831     delete.deleteFamily(FAMILIES[1]);
1832     ht.delete(delete);
1833 
1834     put = new Put(ROWS[3]);
1835     put.add(FAMILIES[2], QUALIFIER, VALUES[0]);
1836     ht.put(put);
1837 
1838     put = new Put(ROWS[4]);
1839     put.add(FAMILIES[1], QUALIFIER, VALUES[1]);
1840     put.add(FAMILIES[2], QUALIFIER, VALUES[2]);
1841     ht.put(put);
1842 
1843     get = new Get(ROWS[3]);
1844     get.addFamily(FAMILIES[1]);
1845     get.addFamily(FAMILIES[2]);
1846     get.setMaxVersions(Integer.MAX_VALUE);
1847     result = ht.get(get);
1848     assertTrue("Expected 1 key but received " + result.size(),
1849         result.size() == 1);
1850 
1851     get = new Get(ROWS[4]);
1852     get.addFamily(FAMILIES[1]);
1853     get.addFamily(FAMILIES[2]);
1854     get.setMaxVersions(Integer.MAX_VALUE);
1855     result = ht.get(get);
1856     assertTrue("Expected 2 keys but received " + result.size(),
1857         result.size() == 2);
1858 
1859     scan = new Scan(ROWS[3]);
1860     scan.addFamily(FAMILIES[1]);
1861     scan.addFamily(FAMILIES[2]);
1862     scan.setMaxVersions(Integer.MAX_VALUE);
1863     ResultScanner scanner = ht.getScanner(scan);
1864     result = scanner.next();
1865     assertTrue("Expected 1 key but received " + result.size(),
1866         result.size() == 1);
1867     assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[3]));
1868     assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[0]));
1869     result = scanner.next();
1870     assertTrue("Expected 2 keys but received " + result.size(),
1871         result.size() == 2);
1872     assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[4]));
1873     assertTrue(Bytes.equals(result.raw()[1].getRow(), ROWS[4]));
1874     assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[1]));
1875     assertTrue(Bytes.equals(result.raw()[1].getValue(), VALUES[2]));
1876     scanner.close();
1877 
1878     // Add test of bulk deleting.
1879     for (int i = 0; i < 10; i++) {
1880       byte [] bytes = Bytes.toBytes(i);
1881       put = new Put(bytes);
1882       put.setWriteToWAL(false);
1883       put.add(FAMILIES[0], QUALIFIER, bytes);
1884       ht.put(put);
1885     }
1886     for (int i = 0; i < 10; i++) {
1887       byte [] bytes = Bytes.toBytes(i);
1888       get = new Get(bytes);
1889       get.addFamily(FAMILIES[0]);
1890       result = ht.get(get);
1891       assertTrue(result.size() == 1);
1892     }
1893     ArrayList<Delete> deletes = new ArrayList<Delete>();
1894     for (int i = 0; i < 10; i++) {
1895       byte [] bytes = Bytes.toBytes(i);
1896       delete = new Delete(bytes);
1897       delete.deleteFamily(FAMILIES[0]);
1898       deletes.add(delete);
1899     }
1900     ht.delete(deletes);
1901     for (int i = 0; i < 10; i++) {
1902       byte [] bytes = Bytes.toBytes(i);
1903       get = new Get(bytes);
1904       get.addFamily(FAMILIES[0]);
1905       result = ht.get(get);
1906       assertTrue(result.size() == 0);
1907     }
1908   }
1909 
1910   /*
1911    * Baseline "scalability" test.
1912    *
1913    * Tests one hundred families, one million columns, one million versions
1914    */
1915   @Ignore @Test
1916   public void testMillions() throws Exception {
1917 
1918     // 100 families
1919 
1920     // millions of columns
1921 
1922     // millions of versions
1923 
1924   }
1925 
1926   @Ignore @Test
1927   public void testMultipleRegionsAndBatchPuts() throws Exception {
1928     // Two family table
1929 
1930     // Insert lots of rows
1931 
1932     // Insert to the same row with batched puts
1933 
1934     // Insert to multiple rows with batched puts
1935 
1936     // Split the table
1937 
1938     // Get row from first region
1939 
1940     // Get row from second region
1941 
1942     // Scan all rows
1943 
1944     // Insert to multiple regions with batched puts
1945 
1946     // Get row from first region
1947 
1948     // Get row from second region
1949 
1950     // Scan all rows
1951 
1952 
1953   }
1954 
1955   @Ignore @Test
1956   public void testMultipleRowMultipleFamily() throws Exception {
1957 
1958   }
1959 
1960   //
1961   // JIRA Testers
1962   //
1963 
1964   /**
1965    * HBASE-867
1966    *    If millions of columns in a column family, hbase scanner won't come up
1967    *
1968    *    Test will create numRows rows, each with numColsPerRow columns
1969    *    (1 version each), and attempt to scan them all.
1970    *
1971    *    To test at scale, up numColsPerRow to the millions
1972    *    (have not gotten that to work running as junit though)
1973    */
1974   @Test
1975   public void testJiraTest867() throws Exception {
1976     int numRows = 10;
1977     int numColsPerRow = 2000;
1978 
1979     byte [] TABLE = Bytes.toBytes("testJiraTest867");
1980 
1981     byte [][] ROWS = makeN(ROW, numRows);
1982     byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
1983 
1984     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
1985 
1986     // Insert rows
1987 
1988     for(int i=0;i<numRows;i++) {
1989       Put put = new Put(ROWS[i]);
1990       put.setWriteToWAL(false);
1991       for(int j=0;j<numColsPerRow;j++) {
1992         put.add(FAMILY, QUALIFIERS[j], QUALIFIERS[j]);
1993       }
1994       assertTrue("Put expected to contain " + numColsPerRow + " columns but " +
1995           "only contains " + put.size(), put.size() == numColsPerRow);
1996       ht.put(put);
1997     }
1998 
1999     // Get a row
2000     Get get = new Get(ROWS[numRows-1]);
2001     Result result = ht.get(get);
2002     assertNumKeys(result, numColsPerRow);
2003     KeyValue [] keys = result.raw();
2004     for(int i=0;i<result.size();i++) {
2005       assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2006     }
2007 
2008     // Scan the rows
2009     Scan scan = new Scan();
2010     ResultScanner scanner = ht.getScanner(scan);
2011     int rowCount = 0;
2012     while((result = scanner.next()) != null) {
2013       assertNumKeys(result, numColsPerRow);
2014       KeyValue [] kvs = result.raw();
2015       for(int i=0;i<numColsPerRow;i++) {
2016         assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2017       }
2018       rowCount++;
2019     }
2020     scanner.close();
2021     assertTrue("Expected to scan " + numRows + " rows but actually scanned "
2022         + rowCount + " rows", rowCount == numRows);
2023 
2024     // flush and try again
2025 
2026     TEST_UTIL.flush();
2027 
2028     // Get a row
2029     get = new Get(ROWS[numRows-1]);
2030     result = ht.get(get);
2031     assertNumKeys(result, numColsPerRow);
2032     keys = result.raw();
2033     for(int i=0;i<result.size();i++) {
2034       assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2035     }
2036 
2037     // Scan the rows
2038     scan = new Scan();
2039     scanner = ht.getScanner(scan);
2040     rowCount = 0;
2041     while((result = scanner.next()) != null) {
2042       assertNumKeys(result, numColsPerRow);
2043       KeyValue [] kvs = result.raw();
2044       for(int i=0;i<numColsPerRow;i++) {
2045         assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2046       }
2047       rowCount++;
2048     }
2049     scanner.close();
2050     assertTrue("Expected to scan " + numRows + " rows but actually scanned "
2051         + rowCount + " rows", rowCount == numRows);
2052 
2053   }
2054 
2055   /**
2056    * HBASE-861
2057    *    get with timestamp will return a value if there is a version with an
2058    *    earlier timestamp
2059    */
2060   @Test
2061   public void testJiraTest861() throws Exception {
2062 
2063     byte [] TABLE = Bytes.toBytes("testJiraTest861");
2064     byte [][] VALUES = makeNAscii(VALUE, 7);
2065     long [] STAMPS = makeStamps(7);
2066 
2067     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2068 
2069     // Insert three versions
2070 
2071     Put put = new Put(ROW);
2072     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2073     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2074     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2075     ht.put(put);
2076 
2077     // Get the middle value
2078     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2079 
2080     // Try to get one version before (expect fail)
2081     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2082 
2083     // Try to get one version after (expect fail)
2084     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2085 
2086     // Try same from storefile
2087     TEST_UTIL.flush();
2088     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2089     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2090     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2091 
2092     // Insert two more versions surrounding others, into memstore
2093     put = new Put(ROW);
2094     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2095     put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2096     ht.put(put);
2097 
2098     // Check we can get everything we should and can't get what we shouldn't
2099     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2100     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2101     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2102     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2103     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2104     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2105     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2106 
2107     // Try same from two storefiles
2108     TEST_UTIL.flush();
2109     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2110     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2111     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2112     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2113     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2114     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2115     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2116 
2117   }
2118 
2119   /**
2120    * HBASE-33
2121    *    Add a HTable get/obtainScanner method that retrieves all versions of a
2122    *    particular column and row between two timestamps
2123    */
2124   @Test
2125   public void testJiraTest33() throws Exception {
2126 
2127     byte [] TABLE = Bytes.toBytes("testJiraTest33");
2128     byte [][] VALUES = makeNAscii(VALUE, 7);
2129     long [] STAMPS = makeStamps(7);
2130 
2131     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2132 
2133     // Insert lots versions
2134 
2135     Put put = new Put(ROW);
2136     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2137     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2138     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2139     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2140     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2141     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2142     ht.put(put);
2143 
2144     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2145     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2146     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2147     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2148 
2149     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2150     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2151     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2152     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2153 
2154     // Try same from storefile
2155     TEST_UTIL.flush();
2156 
2157     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2158     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2159     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2160     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2161 
2162     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2163     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2164     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2165     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2166 
2167   }
2168 
2169   /**
2170    * HBASE-1014
2171    *    commit(BatchUpdate) method should return timestamp
2172    */
2173   @Test
2174   public void testJiraTest1014() throws Exception {
2175 
2176     byte [] TABLE = Bytes.toBytes("testJiraTest1014");
2177 
2178     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2179 
2180     long manualStamp = 12345;
2181 
2182     // Insert lots versions
2183 
2184     Put put = new Put(ROW);
2185     put.add(FAMILY, QUALIFIER, manualStamp, VALUE);
2186     ht.put(put);
2187 
2188     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, manualStamp, VALUE);
2189     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp-1);
2190     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp+1);
2191 
2192   }
2193 
2194   /**
2195    * HBASE-1182
2196    *    Scan for columns > some timestamp
2197    */
2198   @Test
2199   public void testJiraTest1182() throws Exception {
2200 
2201     byte [] TABLE = Bytes.toBytes("testJiraTest1182");
2202     byte [][] VALUES = makeNAscii(VALUE, 7);
2203     long [] STAMPS = makeStamps(7);
2204 
2205     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2206 
2207     // Insert lots versions
2208 
2209     Put put = new Put(ROW);
2210     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2211     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2212     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2213     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2214     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2215     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2216     ht.put(put);
2217 
2218     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2219     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2220     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2221 
2222     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2223     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2224     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2225 
2226     // Try same from storefile
2227     TEST_UTIL.flush();
2228 
2229     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2230     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2231     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2232 
2233     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2234     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2235     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2236   }
2237 
2238   /**
2239    * HBASE-52
2240    *    Add a means of scanning over all versions
2241    */
2242   @Test
2243   public void testJiraTest52() throws Exception {
2244     byte [] TABLE = Bytes.toBytes("testJiraTest52");
2245     byte [][] VALUES = makeNAscii(VALUE, 7);
2246     long [] STAMPS = makeStamps(7);
2247 
2248     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2249 
2250     // Insert lots versions
2251 
2252     Put put = new Put(ROW);
2253     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2254     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2255     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2256     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2257     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2258     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2259     ht.put(put);
2260 
2261     getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2262 
2263     scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2264 
2265     // Try same from storefile
2266     TEST_UTIL.flush();
2267 
2268     getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2269 
2270     scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2271   }
2272 
2273   //
2274   // Bulk Testers
2275   //
2276 
2277   private void getVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row,
2278       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
2279       int start, int end)
2280   throws IOException {
2281     Get get = new Get(row);
2282     get.addColumn(family, qualifier);
2283     get.setMaxVersions(Integer.MAX_VALUE);
2284     get.setTimeRange(stamps[start+1], Long.MAX_VALUE);
2285     Result result = ht.get(get);
2286     assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
2287   }
2288 
2289   private void getVersionRangeAndVerify(HTable ht, byte [] row, byte [] family,
2290       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2291   throws IOException {
2292     Get get = new Get(row);
2293     get.addColumn(family, qualifier);
2294     get.setMaxVersions(Integer.MAX_VALUE);
2295     get.setTimeRange(stamps[start], stamps[end]+1);
2296     Result result = ht.get(get);
2297     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2298   }
2299 
2300   private void getAllVersionsAndVerify(HTable ht, byte [] row, byte [] family,
2301       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2302   throws IOException {
2303     Get get = new Get(row);
2304     get.addColumn(family, qualifier);
2305     get.setMaxVersions(Integer.MAX_VALUE);
2306     Result result = ht.get(get);
2307     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2308   }
2309 
2310   private void scanVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row,
2311       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
2312       int start, int end)
2313   throws IOException {
2314     Scan scan = new Scan(row);
2315     scan.addColumn(family, qualifier);
2316     scan.setMaxVersions(Integer.MAX_VALUE);
2317     scan.setTimeRange(stamps[start+1], Long.MAX_VALUE);
2318     Result result = getSingleScanResult(ht, scan);
2319     assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
2320   }
2321 
2322   private void scanVersionRangeAndVerify(HTable ht, byte [] row, byte [] family,
2323       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2324   throws IOException {
2325     Scan scan = new Scan(row);
2326     scan.addColumn(family, qualifier);
2327     scan.setMaxVersions(Integer.MAX_VALUE);
2328     scan.setTimeRange(stamps[start], stamps[end]+1);
2329     Result result = getSingleScanResult(ht, scan);
2330     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2331   }
2332 
2333   private void scanAllVersionsAndVerify(HTable ht, byte [] row, byte [] family,
2334       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2335   throws IOException {
2336     Scan scan = new Scan(row);
2337     scan.addColumn(family, qualifier);
2338     scan.setMaxVersions(Integer.MAX_VALUE);
2339     Result result = getSingleScanResult(ht, scan);
2340     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2341   }
2342 
2343   private void getVersionAndVerify(HTable ht, byte [] row, byte [] family,
2344       byte [] qualifier, long stamp, byte [] value)
2345   throws Exception {
2346     Get get = new Get(row);
2347     get.addColumn(family, qualifier);
2348     get.setTimeStamp(stamp);
2349     get.setMaxVersions(Integer.MAX_VALUE);
2350     Result result = ht.get(get);
2351     assertSingleResult(result, row, family, qualifier, stamp, value);
2352   }
2353 
2354   private void getVersionAndVerifyMissing(HTable ht, byte [] row, byte [] family,
2355       byte [] qualifier, long stamp)
2356   throws Exception {
2357     Get get = new Get(row);
2358     get.addColumn(family, qualifier);
2359     get.setTimeStamp(stamp);
2360     get.setMaxVersions(Integer.MAX_VALUE);
2361     Result result = ht.get(get);
2362     assertEmptyResult(result);
2363   }
2364 
2365   private void scanVersionAndVerify(HTable ht, byte [] row, byte [] family,
2366       byte [] qualifier, long stamp, byte [] value)
2367   throws Exception {
2368     Scan scan = new Scan(row);
2369     scan.addColumn(family, qualifier);
2370     scan.setTimeStamp(stamp);
2371     scan.setMaxVersions(Integer.MAX_VALUE);
2372     Result result = getSingleScanResult(ht, scan);
2373     assertSingleResult(result, row, family, qualifier, stamp, value);
2374   }
2375 
2376   private void scanVersionAndVerifyMissing(HTable ht, byte [] row,
2377       byte [] family, byte [] qualifier, long stamp)
2378   throws Exception {
2379     Scan scan = new Scan(row);
2380     scan.addColumn(family, qualifier);
2381     scan.setTimeStamp(stamp);
2382     scan.setMaxVersions(Integer.MAX_VALUE);
2383     Result result = getSingleScanResult(ht, scan);
2384     assertNullResult(result);
2385   }
2386 
2387   private void getTestNull(HTable ht, byte [] row, byte [] family,
2388       byte [] value)
2389   throws Exception {
2390 
2391     Get get = new Get(row);
2392     get.addColumn(family, null);
2393     Result result = ht.get(get);
2394     assertSingleResult(result, row, family, null, value);
2395 
2396     get = new Get(row);
2397     get.addColumn(family, HConstants.EMPTY_BYTE_ARRAY);
2398     result = ht.get(get);
2399     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2400 
2401     get = new Get(row);
2402     get.addFamily(family);
2403     result = ht.get(get);
2404     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2405 
2406     get = new Get(row);
2407     result = ht.get(get);
2408     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2409 
2410   }
2411 
2412   private void scanTestNull(HTable ht, byte [] row, byte [] family,
2413       byte [] value)
2414   throws Exception {
2415 
2416     Scan scan = new Scan();
2417     scan.addColumn(family, null);
2418     Result result = getSingleScanResult(ht, scan);
2419     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2420 
2421     scan = new Scan();
2422     scan.addColumn(family, HConstants.EMPTY_BYTE_ARRAY);
2423     result = getSingleScanResult(ht, scan);
2424     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2425 
2426     scan = new Scan();
2427     scan.addFamily(family);
2428     result = getSingleScanResult(ht, scan);
2429     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2430 
2431     scan = new Scan();
2432     result = getSingleScanResult(ht, scan);
2433     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2434 
2435   }
2436 
2437   private void singleRowGetTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
2438       byte [][] QUALIFIERS, byte [][] VALUES)
2439   throws Exception {
2440 
2441     // Single column from memstore
2442     Get get = new Get(ROWS[0]);
2443     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2444     Result result = ht.get(get);
2445     assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
2446 
2447     // Single column from storefile
2448     get = new Get(ROWS[0]);
2449     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
2450     result = ht.get(get);
2451     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
2452 
2453     // Single column from storefile, family match
2454     get = new Get(ROWS[0]);
2455     get.addFamily(FAMILIES[7]);
2456     result = ht.get(get);
2457     assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
2458 
2459     // Two columns, one from memstore one from storefile, same family,
2460     // wildcard match
2461     get = new Get(ROWS[0]);
2462     get.addFamily(FAMILIES[4]);
2463     result = ht.get(get);
2464     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2465         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2466 
2467     // Two columns, one from memstore one from storefile, same family,
2468     // explicit match
2469     get = new Get(ROWS[0]);
2470     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2471     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
2472     result = ht.get(get);
2473     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2474         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2475 
2476     // Three column, one from memstore two from storefile, different families,
2477     // wildcard match
2478     get = new Get(ROWS[0]);
2479     get.addFamily(FAMILIES[4]);
2480     get.addFamily(FAMILIES[7]);
2481     result = ht.get(get);
2482     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2483         new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
2484 
2485     // Multiple columns from everywhere storefile, many family, wildcard
2486     get = new Get(ROWS[0]);
2487     get.addFamily(FAMILIES[2]);
2488     get.addFamily(FAMILIES[4]);
2489     get.addFamily(FAMILIES[6]);
2490     get.addFamily(FAMILIES[7]);
2491     result = ht.get(get);
2492     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2493         new int [][] {
2494           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2495     });
2496 
2497     // Multiple columns from everywhere storefile, many family, wildcard
2498     get = new Get(ROWS[0]);
2499     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
2500     get.addColumn(FAMILIES[2], QUALIFIERS[4]);
2501     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
2502     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
2503     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
2504     get.addColumn(FAMILIES[6], QUALIFIERS[7]);
2505     get.addColumn(FAMILIES[7], QUALIFIERS[7]);
2506     get.addColumn(FAMILIES[7], QUALIFIERS[8]);
2507     result = ht.get(get);
2508     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2509         new int [][] {
2510           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2511     });
2512 
2513     // Everything
2514     get = new Get(ROWS[0]);
2515     result = ht.get(get);
2516     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2517         new int [][] {
2518           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
2519     });
2520 
2521     // Get around inserted columns
2522 
2523     get = new Get(ROWS[1]);
2524     result = ht.get(get);
2525     assertEmptyResult(result);
2526 
2527     get = new Get(ROWS[0]);
2528     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
2529     get.addColumn(FAMILIES[2], QUALIFIERS[3]);
2530     result = ht.get(get);
2531     assertEmptyResult(result);
2532 
2533   }
2534 
2535   private void singleRowScanTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
2536       byte [][] QUALIFIERS, byte [][] VALUES)
2537   throws Exception {
2538 
2539     // Single column from memstore
2540     Scan scan = new Scan();
2541     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2542     Result result = getSingleScanResult(ht, scan);
2543     assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
2544 
2545     // Single column from storefile
2546     scan = new Scan();
2547     scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
2548     result = getSingleScanResult(ht, scan);
2549     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
2550 
2551     // Single column from storefile, family match
2552     scan = new Scan();
2553     scan.addFamily(FAMILIES[7]);
2554     result = getSingleScanResult(ht, scan);
2555     assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
2556 
2557     // Two columns, one from memstore one from storefile, same family,
2558     // wildcard match
2559     scan = new Scan();
2560     scan.addFamily(FAMILIES[4]);
2561     result = getSingleScanResult(ht, scan);
2562     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2563         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2564 
2565     // Two columns, one from memstore one from storefile, same family,
2566     // explicit match
2567     scan = new Scan();
2568     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2569     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
2570     result = getSingleScanResult(ht, scan);
2571     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
2572         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
2573 
2574     // Three column, one from memstore two from storefile, different families,
2575     // wildcard match
2576     scan = new Scan();
2577     scan.addFamily(FAMILIES[4]);
2578     scan.addFamily(FAMILIES[7]);
2579     result = getSingleScanResult(ht, scan);
2580     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2581         new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
2582 
2583     // Multiple columns from everywhere storefile, many family, wildcard
2584     scan = new Scan();
2585     scan.addFamily(FAMILIES[2]);
2586     scan.addFamily(FAMILIES[4]);
2587     scan.addFamily(FAMILIES[6]);
2588     scan.addFamily(FAMILIES[7]);
2589     result = getSingleScanResult(ht, scan);
2590     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2591         new int [][] {
2592           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2593     });
2594 
2595     // Multiple columns from everywhere storefile, many family, wildcard
2596     scan = new Scan();
2597     scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
2598     scan.addColumn(FAMILIES[2], QUALIFIERS[4]);
2599     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
2600     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
2601     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
2602     scan.addColumn(FAMILIES[6], QUALIFIERS[7]);
2603     scan.addColumn(FAMILIES[7], QUALIFIERS[7]);
2604     scan.addColumn(FAMILIES[7], QUALIFIERS[8]);
2605     result = getSingleScanResult(ht, scan);
2606     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2607         new int [][] {
2608           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
2609     });
2610 
2611     // Everything
2612     scan = new Scan();
2613     result = getSingleScanResult(ht, scan);
2614     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
2615         new int [][] {
2616           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
2617     });
2618 
2619     // Scan around inserted columns
2620 
2621     scan = new Scan(ROWS[1]);
2622     result = getSingleScanResult(ht, scan);
2623     assertNullResult(result);
2624 
2625     scan = new Scan();
2626     scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
2627     scan.addColumn(FAMILIES[2], QUALIFIERS[3]);
2628     result = getSingleScanResult(ht, scan);
2629     assertNullResult(result);
2630   }
2631 
2632   /**
2633    * Verify a single column using gets.
2634    * Expects family and qualifier arrays to be valid for at least
2635    * the range:  idx-2 < idx < idx+2
2636    */
2637   private void getVerifySingleColumn(HTable ht,
2638       byte [][] ROWS, int ROWIDX,
2639       byte [][] FAMILIES, int FAMILYIDX,
2640       byte [][] QUALIFIERS, int QUALIFIERIDX,
2641       byte [][] VALUES, int VALUEIDX)
2642   throws Exception {
2643 
2644     Get get = new Get(ROWS[ROWIDX]);
2645     Result result = ht.get(get);
2646     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2647         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2648 
2649     get = new Get(ROWS[ROWIDX]);
2650     get.addFamily(FAMILIES[FAMILYIDX]);
2651     result = ht.get(get);
2652     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2653         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2654 
2655     get = new Get(ROWS[ROWIDX]);
2656     get.addFamily(FAMILIES[FAMILYIDX-2]);
2657     get.addFamily(FAMILIES[FAMILYIDX]);
2658     get.addFamily(FAMILIES[FAMILYIDX+2]);
2659     result = ht.get(get);
2660     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2661         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2662 
2663     get = new Get(ROWS[ROWIDX]);
2664     get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[0]);
2665     result = ht.get(get);
2666     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2667         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2668 
2669     get = new Get(ROWS[ROWIDX]);
2670     get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[1]);
2671     get.addFamily(FAMILIES[FAMILYIDX]);
2672     result = ht.get(get);
2673     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2674         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2675 
2676     get = new Get(ROWS[ROWIDX]);
2677     get.addFamily(FAMILIES[FAMILYIDX]);
2678     get.addColumn(FAMILIES[FAMILYIDX+1], QUALIFIERS[1]);
2679     get.addColumn(FAMILIES[FAMILYIDX-2], QUALIFIERS[1]);
2680     get.addFamily(FAMILIES[FAMILYIDX-1]);
2681     get.addFamily(FAMILIES[FAMILYIDX+2]);
2682     result = ht.get(get);
2683     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2684         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2685 
2686   }
2687 
2688 
2689   /**
2690    * Verify a single column using scanners.
2691    * Expects family and qualifier arrays to be valid for at least
2692    * the range:  idx-2 to idx+2
2693    * Expects row array to be valid for at least idx to idx+2
2694    */
2695   private void scanVerifySingleColumn(HTable ht,
2696       byte [][] ROWS, int ROWIDX,
2697       byte [][] FAMILIES, int FAMILYIDX,
2698       byte [][] QUALIFIERS, int QUALIFIERIDX,
2699       byte [][] VALUES, int VALUEIDX)
2700   throws Exception {
2701 
2702     Scan scan = new Scan();
2703     Result result = getSingleScanResult(ht, scan);
2704     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2705         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2706 
2707     scan = new Scan(ROWS[ROWIDX]);
2708     result = getSingleScanResult(ht, scan);
2709     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2710         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2711 
2712     scan = new Scan(ROWS[ROWIDX], ROWS[ROWIDX+1]);
2713     result = getSingleScanResult(ht, scan);
2714     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2715         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2716 
2717     scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX+1]);
2718     result = getSingleScanResult(ht, scan);
2719     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2720         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2721 
2722     scan = new Scan();
2723     scan.addFamily(FAMILIES[FAMILYIDX]);
2724     result = getSingleScanResult(ht, scan);
2725     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2726         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2727 
2728     scan = new Scan();
2729     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
2730     result = getSingleScanResult(ht, scan);
2731     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2732         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2733 
2734     scan = new Scan();
2735     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
2736     scan.addFamily(FAMILIES[FAMILYIDX]);
2737     result = getSingleScanResult(ht, scan);
2738     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2739         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2740 
2741     scan = new Scan();
2742     scan.addColumn(FAMILIES[FAMILYIDX-1], QUALIFIERS[QUALIFIERIDX+1]);
2743     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
2744     scan.addFamily(FAMILIES[FAMILYIDX+1]);
2745     result = getSingleScanResult(ht, scan);
2746     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
2747         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
2748 
2749   }
2750 
2751   /**
2752    * Verify we do not read any values by accident around a single column
2753    * Same requirements as getVerifySingleColumn
2754    */
2755   private void getVerifySingleEmpty(HTable ht,
2756       byte [][] ROWS, int ROWIDX,
2757       byte [][] FAMILIES, int FAMILYIDX,
2758       byte [][] QUALIFIERS, int QUALIFIERIDX)
2759   throws Exception {
2760 
2761     Get get = new Get(ROWS[ROWIDX]);
2762     get.addFamily(FAMILIES[4]);
2763     get.addColumn(FAMILIES[4], QUALIFIERS[1]);
2764     Result result = ht.get(get);
2765     assertEmptyResult(result);
2766 
2767     get = new Get(ROWS[ROWIDX]);
2768     get.addFamily(FAMILIES[4]);
2769     get.addColumn(FAMILIES[4], QUALIFIERS[2]);
2770     result = ht.get(get);
2771     assertEmptyResult(result);
2772 
2773     get = new Get(ROWS[ROWIDX]);
2774     get.addFamily(FAMILIES[3]);
2775     get.addColumn(FAMILIES[4], QUALIFIERS[2]);
2776     get.addFamily(FAMILIES[5]);
2777     result = ht.get(get);
2778     assertEmptyResult(result);
2779 
2780     get = new Get(ROWS[ROWIDX+1]);
2781     result = ht.get(get);
2782     assertEmptyResult(result);
2783 
2784   }
2785 
2786   private void scanVerifySingleEmpty(HTable ht,
2787       byte [][] ROWS, int ROWIDX,
2788       byte [][] FAMILIES, int FAMILYIDX,
2789       byte [][] QUALIFIERS, int QUALIFIERIDX)
2790   throws Exception {
2791 
2792     Scan scan = new Scan(ROWS[ROWIDX+1]);
2793     Result result = getSingleScanResult(ht, scan);
2794     assertNullResult(result);
2795 
2796     scan = new Scan(ROWS[ROWIDX+1],ROWS[ROWIDX+2]);
2797     result = getSingleScanResult(ht, scan);
2798     assertNullResult(result);
2799 
2800     scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX]);
2801     result = getSingleScanResult(ht, scan);
2802     assertNullResult(result);
2803 
2804     scan = new Scan();
2805     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
2806     scan.addFamily(FAMILIES[FAMILYIDX-1]);
2807     result = getSingleScanResult(ht, scan);
2808     assertNullResult(result);
2809 
2810   }
2811 
2812   //
2813   // Verifiers
2814   //
2815 
2816   private void assertKey(KeyValue key, byte [] row, byte [] family,
2817       byte [] qualifier, byte [] value)
2818   throws Exception {
2819     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
2820         "Got row [" + Bytes.toString(key.getRow()) +"]",
2821         equals(row, key.getRow()));
2822     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
2823         "Got family [" + Bytes.toString(key.getFamily()) + "]",
2824         equals(family, key.getFamily()));
2825     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
2826         "Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
2827         equals(qualifier, key.getQualifier()));
2828     assertTrue("Expected value [" + Bytes.toString(value) + "] " +
2829         "Got value [" + Bytes.toString(key.getValue()) + "]",
2830         equals(value, key.getValue()));
2831   }
2832 
2833   private void assertIncrementKey(KeyValue key, byte [] row, byte [] family,
2834       byte [] qualifier, long value)
2835   throws Exception {
2836     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
2837         "Got row [" + Bytes.toString(key.getRow()) +"]",
2838         equals(row, key.getRow()));
2839     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
2840         "Got family [" + Bytes.toString(key.getFamily()) + "]",
2841         equals(family, key.getFamily()));
2842     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
2843         "Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
2844         equals(qualifier, key.getQualifier()));
2845     assertTrue("Expected value [" + value + "] " +
2846         "Got value [" + Bytes.toLong(key.getValue()) + "]",
2847         Bytes.toLong(key.getValue()) == value);
2848   }
2849 
2850   private void assertNumKeys(Result result, int n) throws Exception {
2851     assertTrue("Expected " + n + " keys but got " + result.size(),
2852         result.size() == n);
2853   }
2854 
2855   private void assertNResult(Result result, byte [] row,
2856       byte [][] families, byte [][] qualifiers, byte [][] values,
2857       int [][] idxs)
2858   throws Exception {
2859     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
2860         "Got row [" + Bytes.toString(result.getRow()) +"]",
2861         equals(row, result.getRow()));
2862     assertTrue("Expected " + idxs.length + " keys but result contains "
2863         + result.size(), result.size() == idxs.length);
2864 
2865     KeyValue [] keys = result.raw();
2866 
2867     for(int i=0;i<keys.length;i++) {
2868       byte [] family = families[idxs[i][0]];
2869       byte [] qualifier = qualifiers[idxs[i][1]];
2870       byte [] value = values[idxs[i][2]];
2871       KeyValue key = keys[i];
2872 
2873       assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
2874           + "] " + "Got family [" + Bytes.toString(key.getFamily()) + "]",
2875           equals(family, key.getFamily()));
2876       assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
2877           + "] " + "Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
2878           equals(qualifier, key.getQualifier()));
2879       assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
2880           + "Got value [" + Bytes.toString(key.getValue()) + "]",
2881           equals(value, key.getValue()));
2882     }
2883   }
2884 
2885   private void assertNResult(Result result, byte [] row,
2886       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
2887       int start, int end)
2888   throws IOException {
2889     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
2890         "Got row [" + Bytes.toString(result.getRow()) +"]",
2891         equals(row, result.getRow()));
2892     int expectedResults = end - start + 1;
2893     assertEquals(expectedResults, result.size());
2894 
2895     KeyValue [] keys = result.raw();
2896 
2897     for (int i=0; i<keys.length; i++) {
2898       byte [] value = values[end-i];
2899       long ts = stamps[end-i];
2900       KeyValue key = keys[i];
2901 
2902       assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
2903           + "] " + "Got family [" + Bytes.toString(key.getFamily()) + "]",
2904           equals(family, key.getFamily()));
2905       assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
2906           + "] " + "Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
2907           equals(qualifier, key.getQualifier()));
2908       assertTrue("Expected ts [" + ts + "] " +
2909           "Got ts [" + key.getTimestamp() + "]", ts == key.getTimestamp());
2910       assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
2911           + "Got value [" + Bytes.toString(key.getValue()) + "]",
2912           equals(value, key.getValue()));
2913     }
2914   }
2915 
2916   /**
2917    * Validate that result contains two specified keys, exactly.
2918    * It is assumed key A sorts before key B.
2919    */
2920   private void assertDoubleResult(Result result, byte [] row,
2921       byte [] familyA, byte [] qualifierA, byte [] valueA,
2922       byte [] familyB, byte [] qualifierB, byte [] valueB)
2923   throws Exception {
2924     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
2925         "Got row [" + Bytes.toString(result.getRow()) +"]",
2926         equals(row, result.getRow()));
2927     assertTrue("Expected two keys but result contains " + result.size(),
2928         result.size() == 2);
2929     KeyValue [] kv = result.raw();
2930     KeyValue kvA = kv[0];
2931     assertTrue("(A) Expected family [" + Bytes.toString(familyA) + "] " +
2932         "Got family [" + Bytes.toString(kvA.getFamily()) + "]",
2933         equals(familyA, kvA.getFamily()));
2934     assertTrue("(A) Expected qualifier [" + Bytes.toString(qualifierA) + "] " +
2935         "Got qualifier [" + Bytes.toString(kvA.getQualifier()) + "]",
2936         equals(qualifierA, kvA.getQualifier()));
2937     assertTrue("(A) Expected value [" + Bytes.toString(valueA) + "] " +
2938         "Got value [" + Bytes.toString(kvA.getValue()) + "]",
2939         equals(valueA, kvA.getValue()));
2940     KeyValue kvB = kv[1];
2941     assertTrue("(B) Expected family [" + Bytes.toString(familyB) + "] " +
2942         "Got family [" + Bytes.toString(kvB.getFamily()) + "]",
2943         equals(familyB, kvB.getFamily()));
2944     assertTrue("(B) Expected qualifier [" + Bytes.toString(qualifierB) + "] " +
2945         "Got qualifier [" + Bytes.toString(kvB.getQualifier()) + "]",
2946         equals(qualifierB, kvB.getQualifier()));
2947     assertTrue("(B) Expected value [" + Bytes.toString(valueB) + "] " +
2948         "Got value [" + Bytes.toString(kvB.getValue()) + "]",
2949         equals(valueB, kvB.getValue()));
2950   }
2951 
2952   private void assertSingleResult(Result result, byte [] row, byte [] family,
2953       byte [] qualifier, byte [] value)
2954   throws Exception {
2955     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
2956         "Got row [" + Bytes.toString(result.getRow()) +"]",
2957         equals(row, result.getRow()));
2958     assertTrue("Expected a single key but result contains " + result.size(),
2959         result.size() == 1);
2960     KeyValue kv = result.raw()[0];
2961     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
2962         "Got family [" + Bytes.toString(kv.getFamily()) + "]",
2963         equals(family, kv.getFamily()));
2964     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
2965         "Got qualifier [" + Bytes.toString(kv.getQualifier()) + "]",
2966         equals(qualifier, kv.getQualifier()));
2967     assertTrue("Expected value [" + Bytes.toString(value) + "] " +
2968         "Got value [" + Bytes.toString(kv.getValue()) + "]",
2969         equals(value, kv.getValue()));
2970   }
2971 
2972   private void assertSingleResult(Result result, byte [] row, byte [] family,
2973       byte [] qualifier, long ts, byte [] value)
2974   throws Exception {
2975     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
2976         "Got row [" + Bytes.toString(result.getRow()) +"]",
2977         equals(row, result.getRow()));
2978     assertTrue("Expected a single key but result contains " + result.size(),
2979         result.size() == 1);
2980     KeyValue kv = result.raw()[0];
2981     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
2982         "Got family [" + Bytes.toString(kv.getFamily()) + "]",
2983         equals(family, kv.getFamily()));
2984     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
2985         "Got qualifier [" + Bytes.toString(kv.getQualifier()) + "]",
2986         equals(qualifier, kv.getQualifier()));
2987     assertTrue("Expected ts [" + ts + "] " +
2988         "Got ts [" + kv.getTimestamp() + "]", ts == kv.getTimestamp());
2989     assertTrue("Expected value [" + Bytes.toString(value) + "] " +
2990         "Got value [" + Bytes.toString(kv.getValue()) + "]",
2991         equals(value, kv.getValue()));
2992   }
2993 
2994   private void assertEmptyResult(Result result) throws Exception {
2995     assertTrue("expected an empty result but result contains " +
2996         result.size() + " keys", result.isEmpty());
2997   }
2998 
2999   private void assertNullResult(Result result) throws Exception {
3000     assertTrue("expected null result but received a non-null result",
3001         result == null);
3002   }
3003 
3004   //
3005   // Helpers
3006   //
3007 
3008   private Result getSingleScanResult(HTable ht, Scan scan) throws IOException {
3009     ResultScanner scanner = ht.getScanner(scan);
3010     Result result = scanner.next();
3011     scanner.close();
3012     return result;
3013   }
3014 
3015   private byte [][] makeNAscii(byte [] base, int n) {
3016     if(n > 256) {
3017       return makeNBig(base, n);
3018     }
3019     byte [][] ret = new byte[n][];
3020     for(int i=0;i<n;i++) {
3021       byte [] tail = Bytes.toBytes(Integer.toString(i));
3022       ret[i] = Bytes.add(base, tail);
3023     }
3024     return ret;
3025   }
3026 
3027   private byte [][] makeN(byte [] base, int n) {
3028     if (n > 256) {
3029       return makeNBig(base, n);
3030     }
3031     byte [][] ret = new byte[n][];
3032     for(int i=0;i<n;i++) {
3033       ret[i] = Bytes.add(base, new byte[]{(byte)i});
3034     }
3035     return ret;
3036   }
3037 
3038   private byte [][] makeNBig(byte [] base, int n) {
3039     byte [][] ret = new byte[n][];
3040     for(int i=0;i<n;i++) {
3041       int byteA = (i % 256);
3042       int byteB = (i >> 8);
3043       ret[i] = Bytes.add(base, new byte[]{(byte)byteB,(byte)byteA});
3044     }
3045     return ret;
3046   }
3047 
3048   private long [] makeStamps(int n) {
3049     long [] stamps = new long[n];
3050     for(int i=0;i<n;i++) stamps[i] = i+1;
3051     return stamps;
3052   }
3053 
3054   private boolean equals(byte [] left, byte [] right) {
3055     if (left == null && right == null) return true;
3056     if (left == null && right.length == 0) return true;
3057     if (right == null && left.length == 0) return true;
3058     return Bytes.equals(left, right);
3059   }
3060 
3061   @Test
3062   public void testDuplicateVersions() throws Exception {
3063     byte [] TABLE = Bytes.toBytes("testDuplicateVersions");
3064 
3065     long [] STAMPS = makeStamps(20);
3066     byte [][] VALUES = makeNAscii(VALUE, 20);
3067 
3068     HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3069 
3070     // Insert 4 versions of same column
3071     Put put = new Put(ROW);
3072     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3073     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3074     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3075     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3076     ht.put(put);
3077 
3078     // Verify we can get each one properly
3079     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3080     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3081     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3082     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3083     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3084     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3085     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3086     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3087 
3088     // Verify we don't accidentally get others
3089     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3090     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3091     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3092     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3093     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3094     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3095 
3096     // Ensure maxVersions in query is respected
3097     Get get = new Get(ROW);
3098     get.addColumn(FAMILY, QUALIFIER);
3099     get.setMaxVersions(2);
3100     Result result = ht.get(get);
3101     assertNResult(result, ROW, FAMILY, QUALIFIER,
3102         new long [] {STAMPS[4], STAMPS[5]},
3103         new byte[][] {VALUES[4], VALUES[5]},
3104         0, 1);
3105 
3106     Scan scan = new Scan(ROW);
3107     scan.addColumn(FAMILY, QUALIFIER);
3108     scan.setMaxVersions(2);
3109     result = getSingleScanResult(ht, scan);
3110     assertNResult(result, ROW, FAMILY, QUALIFIER,
3111         new long [] {STAMPS[4], STAMPS[5]},
3112         new byte[][] {VALUES[4], VALUES[5]},
3113         0, 1);
3114 
3115     // Flush and redo
3116 
3117     TEST_UTIL.flush();
3118 
3119     // Verify we can get each one properly
3120     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3121     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3122     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3123     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3124     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3125     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3126     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3127     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3128 
3129     // Verify we don't accidentally get others
3130     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3131     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3132     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3133     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3134     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3135     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3136 
3137     // Ensure maxVersions in query is respected
3138     get = new Get(ROW);
3139     get.addColumn(FAMILY, QUALIFIER);
3140     get.setMaxVersions(2);
3141     result = ht.get(get);
3142     assertNResult(result, ROW, FAMILY, QUALIFIER,
3143         new long [] {STAMPS[4], STAMPS[5]},
3144         new byte[][] {VALUES[4], VALUES[5]},
3145         0, 1);
3146 
3147     scan = new Scan(ROW);
3148     scan.addColumn(FAMILY, QUALIFIER);
3149     scan.setMaxVersions(2);
3150     result = getSingleScanResult(ht, scan);
3151     assertNResult(result, ROW, FAMILY, QUALIFIER,
3152         new long [] {STAMPS[4], STAMPS[5]},
3153         new byte[][] {VALUES[4], VALUES[5]},
3154         0, 1);
3155 
3156 
3157     // Add some memstore and retest
3158 
3159     // Insert 4 more versions of same column and a dupe
3160     put = new Put(ROW);
3161     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
3162     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3163     put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
3164     put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3165     put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]);
3166     ht.put(put);
3167 
3168     // Ensure maxVersions in query is respected
3169     get = new Get(ROW);
3170     get.addColumn(FAMILY, QUALIFIER);
3171     get.setMaxVersions(7);
3172     result = ht.get(get);
3173     assertNResult(result, ROW, FAMILY, QUALIFIER,
3174         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3175         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3176         0, 6);
3177 
3178     scan = new Scan(ROW);
3179     scan.addColumn(FAMILY, QUALIFIER);
3180     scan.setMaxVersions(7);
3181     result = getSingleScanResult(ht, scan);
3182     assertNResult(result, ROW, FAMILY, QUALIFIER,
3183         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3184         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3185         0, 6);
3186 
3187     get = new Get(ROW);
3188     get.setMaxVersions(7);
3189     result = ht.get(get);
3190     assertNResult(result, ROW, FAMILY, QUALIFIER,
3191         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3192         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3193         0, 6);
3194 
3195     scan = new Scan(ROW);
3196     scan.setMaxVersions(7);
3197     result = getSingleScanResult(ht, scan);
3198     assertNResult(result, ROW, FAMILY, QUALIFIER,
3199         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3200         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3201         0, 6);
3202 
3203     // Verify we can get each one properly
3204     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3205     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3206     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3207     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3208     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3209     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3210     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3211     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3212 
3213     // Verify we don't accidentally get others
3214     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3215     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3216     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3217     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3218 
3219     // Ensure maxVersions of table is respected
3220 
3221     TEST_UTIL.flush();
3222 
3223     // Insert 4 more versions of same column and a dupe
3224     put = new Put(ROW);
3225     put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]);
3226     put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]);
3227     put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]);
3228     put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
3229     ht.put(put);
3230 
3231     get = new Get(ROW);
3232     get.addColumn(FAMILY, QUALIFIER);
3233     get.setMaxVersions(Integer.MAX_VALUE);
3234     result = ht.get(get);
3235     assertNResult(result, ROW, FAMILY, QUALIFIER,
3236         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3237         new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3238         0, 9);
3239 
3240     scan = new Scan(ROW);
3241     scan.addColumn(FAMILY, QUALIFIER);
3242     scan.setMaxVersions(Integer.MAX_VALUE);
3243     result = getSingleScanResult(ht, scan);
3244     assertNResult(result, ROW, FAMILY, QUALIFIER,
3245         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3246         new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3247         0, 9);
3248 
3249     // Delete a version in the memstore and a version in a storefile
3250     Delete delete = new Delete(ROW);
3251     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
3252     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
3253     ht.delete(delete);
3254 
3255     // Test that it's gone
3256     get = new Get(ROW);
3257     get.addColumn(FAMILY, QUALIFIER);
3258     get.setMaxVersions(Integer.MAX_VALUE);
3259     result = ht.get(get);
3260     assertNResult(result, ROW, FAMILY, QUALIFIER,
3261         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3262         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3263         0, 9);
3264 
3265     scan = new Scan(ROW);
3266     scan.addColumn(FAMILY, QUALIFIER);
3267     scan.setMaxVersions(Integer.MAX_VALUE);
3268     result = getSingleScanResult(ht, scan);
3269     assertNResult(result, ROW, FAMILY, QUALIFIER,
3270         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3271         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3272         0, 9);
3273   }
3274 
3275   @Test
3276   public void testUpdates() throws Exception {
3277 
3278     byte [] TABLE = Bytes.toBytes("testUpdates");
3279     HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3280 
3281     // Write a column with values at timestamp 1, 2 and 3
3282     byte[] row = Bytes.toBytes("row1");
3283     byte[] qualifier = Bytes.toBytes("myCol");
3284     Put put = new Put(row);
3285     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3286     hTable.put(put);
3287 
3288     put = new Put(row);
3289     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3290     hTable.put(put);
3291 
3292     put = new Put(row);
3293     put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3294     hTable.put(put);
3295 
3296     Get get = new Get(row);
3297     get.addColumn(FAMILY, qualifier);
3298     get.setMaxVersions();
3299 
3300     // Check that the column indeed has the right values at timestamps 1 and
3301     // 2
3302     Result result = hTable.get(get);
3303     NavigableMap<Long, byte[]> navigableMap =
3304         result.getMap().get(FAMILY).get(qualifier);
3305     assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3306     assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3307 
3308     // Update the value at timestamp 1
3309     put = new Put(row);
3310     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3311     hTable.put(put);
3312 
3313     // Update the value at timestamp 2
3314     put = new Put(row);
3315     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3316     hTable.put(put);
3317 
3318     // Check that the values at timestamp 2 and 1 got updated
3319     result = hTable.get(get);
3320     navigableMap = result.getMap().get(FAMILY).get(qualifier);
3321     assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3322     assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3323   }
3324 
3325   @Test
3326   public void testUpdatesWithMajorCompaction() throws Exception {
3327 
3328     String tableName = "testUpdatesWithMajorCompaction";
3329     byte [] TABLE = Bytes.toBytes(tableName);
3330     HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3331     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3332 
3333     // Write a column with values at timestamp 1, 2 and 3
3334     byte[] row = Bytes.toBytes("row2");
3335     byte[] qualifier = Bytes.toBytes("myCol");
3336     Put put = new Put(row);
3337     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3338     hTable.put(put);
3339 
3340     put = new Put(row);
3341     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3342     hTable.put(put);
3343 
3344     put = new Put(row);
3345     put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3346     hTable.put(put);
3347 
3348     Get get = new Get(row);
3349     get.addColumn(FAMILY, qualifier);
3350     get.setMaxVersions();
3351 
3352     // Check that the column indeed has the right values at timestamps 1 and
3353     // 2
3354     Result result = hTable.get(get);
3355     NavigableMap<Long, byte[]> navigableMap =
3356         result.getMap().get(FAMILY).get(qualifier);
3357     assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3358     assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3359 
3360     // Trigger a major compaction
3361     admin.flush(tableName);
3362     admin.majorCompact(tableName);
3363     Thread.sleep(6000);
3364 
3365     // Update the value at timestamp 1
3366     put = new Put(row);
3367     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3368     hTable.put(put);
3369 
3370     // Update the value at timestamp 2
3371     put = new Put(row);
3372     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3373     hTable.put(put);
3374 
3375     // Trigger a major compaction
3376     admin.flush(tableName);
3377     admin.majorCompact(tableName);
3378     Thread.sleep(6000);
3379 
3380     // Check that the values at timestamp 2 and 1 got updated
3381     result = hTable.get(get);
3382     navigableMap = result.getMap().get(FAMILY).get(qualifier);
3383     assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3384     assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3385   }
3386 
3387   @Test
3388   public void testMajorCompactionBetweenTwoUpdates() throws Exception {
3389 
3390     String tableName = "testMajorCompactionBetweenTwoUpdates";
3391     byte [] TABLE = Bytes.toBytes(tableName);
3392     HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3393     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3394 
3395     // Write a column with values at timestamp 1, 2 and 3
3396     byte[] row = Bytes.toBytes("row3");
3397     byte[] qualifier = Bytes.toBytes("myCol");
3398     Put put = new Put(row);
3399     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3400     hTable.put(put);
3401 
3402     put = new Put(row);
3403     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3404     hTable.put(put);
3405 
3406     put = new Put(row);
3407     put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3408     hTable.put(put);
3409 
3410     Get get = new Get(row);
3411     get.addColumn(FAMILY, qualifier);
3412     get.setMaxVersions();
3413 
3414     // Check that the column indeed has the right values at timestamps 1 and
3415     // 2
3416     Result result = hTable.get(get);
3417     NavigableMap<Long, byte[]> navigableMap =
3418         result.getMap().get(FAMILY).get(qualifier);
3419     assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3420     assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3421 
3422     // Trigger a major compaction
3423     admin.flush(tableName);
3424     admin.majorCompact(tableName);
3425     Thread.sleep(6000);
3426 
3427     // Update the value at timestamp 1
3428     put = new Put(row);
3429     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3430     hTable.put(put);
3431 
3432     // Trigger a major compaction
3433     admin.flush(tableName);
3434     admin.majorCompact(tableName);
3435     Thread.sleep(6000);
3436 
3437     // Update the value at timestamp 2
3438     put = new Put(row);
3439     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3440     hTable.put(put);
3441 
3442     // Trigger a major compaction
3443     admin.flush(tableName);
3444     admin.majorCompact(tableName);
3445     Thread.sleep(6000);
3446 
3447     // Check that the values at timestamp 2 and 1 got updated
3448     result = hTable.get(get);
3449     navigableMap = result.getMap().get(FAMILY).get(qualifier);
3450 
3451     assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3452     assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3453   }
3454 
3455   @Test
3456   public void testGet_EmptyTable() throws IOException {
3457     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY);
3458     Get get = new Get(ROW);
3459     get.addFamily(FAMILY);
3460     Result r = table.get(get);
3461     assertTrue(r.isEmpty());
3462   }
3463 
3464   @Test
3465   public void testGet_NonExistentRow() throws IOException {
3466     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY);
3467     Put put = new Put(ROW);
3468     put.add(FAMILY, QUALIFIER, VALUE);
3469     table.put(put);
3470     LOG.info("Row put");
3471 
3472     Get get = new Get(ROW);
3473     get.addFamily(FAMILY);
3474     Result r = table.get(get);
3475     assertFalse(r.isEmpty());
3476     System.out.println("Row retrieved successfully");
3477 
3478     byte [] missingrow = Bytes.toBytes("missingrow");
3479     get = new Get(missingrow);
3480     get.addFamily(FAMILY);
3481     r = table.get(get);
3482     assertTrue(r.isEmpty());
3483     LOG.info("Row missing as it should be");
3484   }
3485 
3486   @Test
3487   public void testPut() throws IOException {
3488     final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
3489     final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
3490     final byte [] row1 = Bytes.toBytes("row1");
3491     final byte [] row2 = Bytes.toBytes("row2");
3492     final byte [] value = Bytes.toBytes("abcd");
3493     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
3494       new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
3495     Put put = new Put(row1);
3496     put.add(CONTENTS_FAMILY, null, value);
3497     table.put(put);
3498 
3499     put = new Put(row2);
3500     put.add(CONTENTS_FAMILY, null, value);
3501 
3502     assertEquals(put.size(), 1);
3503     assertEquals(put.getFamilyMap().get(CONTENTS_FAMILY).size(), 1);
3504 
3505     KeyValue kv = put.getFamilyMap().get(CONTENTS_FAMILY).get(0);
3506 
3507     assertTrue(Bytes.equals(kv.getFamily(), CONTENTS_FAMILY));
3508     // will it return null or an empty byte array?
3509     assertTrue(Bytes.equals(kv.getQualifier(), new byte[0]));
3510 
3511     assertTrue(Bytes.equals(kv.getValue(), value));
3512 
3513     table.put(put);
3514 
3515     Scan scan = new Scan();
3516     scan.addColumn(CONTENTS_FAMILY, null);
3517     ResultScanner scanner = table.getScanner(scan);
3518     for (Result r : scanner) {
3519       for(KeyValue key : r.raw()) {
3520         System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString());
3521       }
3522     }
3523   }
3524 
3525   @Test
3526   public void testRowsPut() throws IOException {
3527     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3528     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
3529     final int NB_BATCH_ROWS = 10;
3530     final byte[] value = Bytes.toBytes("abcd");
3531     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
3532       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
3533     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3534     for (int i = 0; i < NB_BATCH_ROWS; i++) {
3535       byte[] row = Bytes.toBytes("row" + i);
3536       Put put = new Put(row);
3537       put.setWriteToWAL(false);
3538       put.add(CONTENTS_FAMILY, null, value);
3539       rowsUpdate.add(put);
3540     }
3541     table.put(rowsUpdate);
3542     Scan scan = new Scan();
3543     scan.addFamily(CONTENTS_FAMILY);
3544     ResultScanner scanner = table.getScanner(scan);
3545     int nbRows = 0;
3546     for (@SuppressWarnings("unused")
3547     Result row : scanner)
3548       nbRows++;
3549     assertEquals(NB_BATCH_ROWS, nbRows);
3550   }
3551 
3552   @Test
3553   public void testRowsPutBufferedOneFlush() throws IOException {
3554     final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
3555     final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
3556     final byte [] value = Bytes.toBytes("abcd");
3557     final int NB_BATCH_ROWS = 10;
3558     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
3559       new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
3560     table.setAutoFlush(false);
3561     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3562     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
3563       byte[] row = Bytes.toBytes("row" + i);
3564       Put put = new Put(row);
3565       put.setWriteToWAL(false);
3566       put.add(CONTENTS_FAMILY, null, value);
3567       rowsUpdate.add(put);
3568     }
3569     table.put(rowsUpdate);
3570 
3571     Scan scan = new Scan();
3572     scan.addFamily(CONTENTS_FAMILY);
3573     ResultScanner scanner = table.getScanner(scan);
3574     int nbRows = 0;
3575     for (@SuppressWarnings("unused")
3576     Result row : scanner)
3577       nbRows++;
3578     assertEquals(0, nbRows);
3579     scanner.close();
3580 
3581     table.flushCommits();
3582 
3583     scan = new Scan();
3584     scan.addFamily(CONTENTS_FAMILY);
3585     scanner = table.getScanner(scan);
3586     nbRows = 0;
3587     for (@SuppressWarnings("unused")
3588     Result row : scanner)
3589       nbRows++;
3590     assertEquals(NB_BATCH_ROWS * 10, nbRows);
3591   }
3592 
3593   @Test
3594   public void testRowsPutBufferedManyManyFlushes() throws IOException {
3595     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3596     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
3597     final byte[] value = Bytes.toBytes("abcd");
3598     final int NB_BATCH_ROWS = 10;
3599     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
3600       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
3601     table.setAutoFlush(false);
3602     table.setWriteBufferSize(10);
3603     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
3604     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
3605       byte[] row = Bytes.toBytes("row" + i);
3606       Put put = new Put(row);
3607       put.setWriteToWAL(false);
3608       put.add(CONTENTS_FAMILY, null, value);
3609       rowsUpdate.add(put);
3610     }
3611     table.put(rowsUpdate);
3612 
3613     table.flushCommits();
3614 
3615     Scan scan = new Scan();
3616     scan.addFamily(CONTENTS_FAMILY);
3617     ResultScanner scanner = table.getScanner(scan);
3618     int nbRows = 0;
3619     for (@SuppressWarnings("unused")
3620     Result row : scanner)
3621       nbRows++;
3622     assertEquals(NB_BATCH_ROWS * 10, nbRows);
3623   }
3624 
3625   @Test
3626   public void testAddKeyValue() throws IOException {
3627     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
3628     final byte[] value = Bytes.toBytes("abcd");
3629     final byte[] row1 = Bytes.toBytes("row1");
3630     final byte[] row2 = Bytes.toBytes("row2");
3631     byte[] qualifier = Bytes.toBytes("qf1");
3632     Put put = new Put(row1);
3633 
3634     // Adding KeyValue with the same row
3635     KeyValue kv = new KeyValue(row1, CONTENTS_FAMILY, qualifier, value);
3636     boolean ok = true;
3637     try {
3638       put.add(kv);
3639     } catch (IOException e) {
3640       ok = false;
3641     }
3642     assertEquals(true, ok);
3643 
3644     // Adding KeyValue with the different row
3645     kv = new KeyValue(row2, CONTENTS_FAMILY, qualifier, value);
3646     ok = false;
3647     try {
3648       put.add(kv);
3649     } catch (IOException e) {
3650       ok = true;
3651     }
3652     assertEquals(true, ok);
3653   }
3654 
3655   /**
3656    * test for HBASE-737
3657    * @throws IOException
3658    */
3659   @Test
3660   public void testHBase737 () throws IOException {
3661     final byte [] FAM1 = Bytes.toBytes("fam1");
3662     final byte [] FAM2 = Bytes.toBytes("fam2");
3663     // Open table
3664     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
3665       new byte [][] {FAM1, FAM2});
3666     // Insert some values
3667     Put put = new Put(ROW);
3668     put.add(FAM1, Bytes.toBytes("letters"), Bytes.toBytes("abcdefg"));
3669     table.put(put);
3670     try {
3671       Thread.sleep(1000);
3672     } catch (InterruptedException i) {
3673       //ignore
3674     }
3675 
3676     put = new Put(ROW);
3677     put.add(FAM1, Bytes.toBytes("numbers"), Bytes.toBytes("123456"));
3678     table.put(put);
3679 
3680     try {
3681       Thread.sleep(1000);
3682     } catch (InterruptedException i) {
3683       //ignore
3684     }
3685 
3686     put = new Put(ROW);
3687     put.add(FAM2, Bytes.toBytes("letters"), Bytes.toBytes("hijklmnop"));
3688     table.put(put);
3689 
3690     long times[] = new long[3];
3691 
3692     // First scan the memstore
3693 
3694     Scan scan = new Scan();
3695     scan.addFamily(FAM1);
3696     scan.addFamily(FAM2);
3697     ResultScanner s = table.getScanner(scan);
3698     try {
3699       int index = 0;
3700       Result r = null;
3701       while ((r = s.next()) != null) {
3702         for(KeyValue key : r.raw()) {
3703           times[index++] = key.getTimestamp();
3704         }
3705       }
3706     } finally {
3707       s.close();
3708     }
3709     for (int i = 0; i < times.length - 1; i++) {
3710       for (int j = i + 1; j < times.length; j++) {
3711         assertTrue(times[j] > times[i]);
3712       }
3713     }
3714 
3715     // Flush data to disk and try again
3716     TEST_UTIL.flush();
3717 
3718     // Reset times
3719     for(int i=0;i<times.length;i++) {
3720       times[i] = 0;
3721     }
3722 
3723     try {
3724       Thread.sleep(1000);
3725     } catch (InterruptedException i) {
3726       //ignore
3727     }
3728     scan = new Scan();
3729     scan.addFamily(FAM1);
3730     scan.addFamily(FAM2);
3731     s = table.getScanner(scan);
3732     try {
3733       int index = 0;
3734       Result r = null;
3735       while ((r = s.next()) != null) {
3736         for(KeyValue key : r.raw()) {
3737           times[index++] = key.getTimestamp();
3738         }
3739       }
3740     } finally {
3741       s.close();
3742     }
3743     for (int i = 0; i < times.length - 1; i++) {
3744       for (int j = i + 1; j < times.length; j++) {
3745         assertTrue(times[j] > times[i]);
3746       }
3747     }
3748   }
3749 
3750   @Test
3751   public void testListTables() throws IOException, InterruptedException {
3752     byte [] t1 = Bytes.toBytes("testListTables1");
3753     byte [] t2 = Bytes.toBytes("testListTables2");
3754     byte [] t3 = Bytes.toBytes("testListTables3");
3755     byte [][] tables = new byte[][] { t1, t2, t3 };
3756     for (int i = 0; i < tables.length; i++) {
3757       TEST_UTIL.createTable(tables[i], FAMILY);
3758     }
3759     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3760     HTableDescriptor[] ts = admin.listTables();
3761     HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
3762     for (int i = 0; i < ts.length; i++) {
3763       result.add(ts[i]);
3764     }
3765     int size = result.size();
3766     assertTrue(size >= tables.length);
3767     for (int i = 0; i < tables.length && i < size; i++) {
3768       boolean found = false;
3769       for (int j = 0; j < ts.length; j++) {
3770         if (Bytes.equals(ts[j].getName(), tables[i])) {
3771           found = true;
3772           break;
3773         }
3774       }
3775       assertTrue("Not found: " + Bytes.toString(tables[i]), found);
3776     }
3777   }
3778 
3779   @Test
3780   public void testMiscHTableStuff() throws IOException {
3781     final byte[] tableAname = Bytes.toBytes("testMiscHTableStuffA");
3782     final byte[] tableBname = Bytes.toBytes("testMiscHTableStuffB");
3783     final byte[] attrName = Bytes.toBytes("TESTATTR");
3784     final byte[] attrValue = Bytes.toBytes("somevalue");
3785     byte[] value = Bytes.toBytes("value");
3786 
3787     HTable a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY);
3788     HTable b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY);
3789     Put put = new Put(ROW);
3790     put.add(HConstants.CATALOG_FAMILY, null, value);
3791     a.put(put);
3792 
3793     // open a new connection to A and a connection to b
3794     HTable newA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
3795 
3796     // copy data from A to B
3797     Scan scan = new Scan();
3798     scan.addFamily(HConstants.CATALOG_FAMILY);
3799     ResultScanner s = newA.getScanner(scan);
3800     try {
3801       for (Result r : s) {
3802         put = new Put(r.getRow());
3803         put.setWriteToWAL(false);
3804         for (KeyValue kv : r.raw()) {
3805           put.add(kv);
3806         }
3807         b.put(put);
3808       }
3809     } finally {
3810       s.close();
3811     }
3812 
3813     // Opening a new connection to A will cause the tables to be reloaded
3814     HTable anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
3815     Get get = new Get(ROW);
3816     get.addFamily(HConstants.CATALOG_FAMILY);
3817     anotherA.get(get);
3818 
3819     // We can still access A through newA because it has the table information
3820     // cached. And if it needs to recalibrate, that will cause the information
3821     // to be reloaded.
3822 
3823     // Test user metadata
3824     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3825     // make a modifiable descriptor
3826     HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor());
3827     // offline the table
3828     admin.disableTable(tableAname);
3829     // add a user attribute to HTD
3830     desc.setValue(attrName, attrValue);
3831     // add a user attribute to HCD
3832     for (HColumnDescriptor c : desc.getFamilies())
3833       c.setValue(attrName, attrValue);
3834     // update metadata for all regions of this table
3835     admin.modifyTable(tableAname, desc);
3836     // enable the table
3837     admin.enableTable(tableAname);
3838 
3839     // Test that attribute changes were applied
3840     desc = a.getTableDescriptor();
3841     assertTrue("wrong table descriptor returned",
3842       Bytes.compareTo(desc.getName(), tableAname) == 0);
3843     // check HTD attribute
3844     value = desc.getValue(attrName);
3845     assertFalse("missing HTD attribute value", value == null);
3846     assertFalse("HTD attribute value is incorrect",
3847       Bytes.compareTo(value, attrValue) != 0);
3848     // check HCD attribute
3849     for (HColumnDescriptor c : desc.getFamilies()) {
3850       value = c.getValue(attrName);
3851       assertFalse("missing HCD attribute value", value == null);
3852       assertFalse("HCD attribute value is incorrect",
3853         Bytes.compareTo(value, attrValue) != 0);
3854     }
3855   }
3856 
3857   @Test
3858   public void testGetClosestRowBefore() throws IOException {
3859     final byte [] tableAname = Bytes.toBytes("testGetClosestRowBefore");
3860     final byte [] row = Bytes.toBytes("row");
3861 
3862 
3863     byte[] firstRow = Bytes.toBytes("ro");
3864     byte[] beforeFirstRow = Bytes.toBytes("rn");
3865     byte[] beforeSecondRow = Bytes.toBytes("rov");
3866 
3867     HTable table = TEST_UTIL.createTable(tableAname,
3868       new byte [][] {HConstants.CATALOG_FAMILY, Bytes.toBytes("info2")});
3869     Put put = new Put(firstRow);
3870     Put put2 = new Put(row);
3871     byte[] zero = new byte[]{0};
3872     byte[] one = new byte[]{1};
3873 
3874     put.add(HConstants.CATALOG_FAMILY, null, zero);
3875     put2.add(HConstants.CATALOG_FAMILY, null, one);
3876 
3877     table.put(put);
3878     table.put(put2);
3879 
3880     Result result = null;
3881 
3882     // Test before first that null is returned
3883     result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY);
3884     assertTrue(result == null);
3885 
3886     // Test at first that first is returned
3887     result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY);
3888     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
3889     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), zero));
3890 
3891     // Test in between first and second that first is returned
3892     result = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY);
3893     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
3894     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), zero));
3895 
3896     // Test at second make sure second is returned
3897     result = table.getRowOrBefore(row, HConstants.CATALOG_FAMILY);
3898     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
3899     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
3900 
3901     // Test after second, make sure second is returned
3902     result = table.getRowOrBefore(Bytes.add(row,one), HConstants.CATALOG_FAMILY);
3903     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
3904     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
3905   }
3906 
3907   /**
3908    * For HBASE-2156
3909    * @throws Exception
3910    */
3911   @Test
3912   public void testScanVariableReuse() throws Exception {
3913     Scan scan = new Scan();
3914     scan.addFamily(FAMILY);
3915     scan.addColumn(FAMILY, ROW);
3916 
3917     assertTrue(scan.getFamilyMap().get(FAMILY).size() == 1);
3918 
3919     scan = new Scan();
3920     scan.addFamily(FAMILY);
3921 
3922     assertTrue(scan.getFamilyMap().get(FAMILY) == null);
3923     assertTrue(scan.getFamilyMap().containsKey(FAMILY));
3924   }
3925 
3926  
3927   @Test
3928   public void testIncrement() throws Exception {
3929     LOG.info("Starting testIncrement");
3930     final byte [] TABLENAME = Bytes.toBytes("testIncrement");
3931     HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
3932 
3933     byte [][] ROWS = new byte [][] {
3934         Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
3935         Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
3936         Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i")
3937     };
3938     byte [][] QUALIFIERS = new byte [][] {
3939         Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
3940         Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
3941         Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i")
3942     };
3943 
3944     // Do some simple single-column increments
3945 
3946     // First with old API
3947     ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[0], 1);
3948     ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[1], 2);
3949     ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[2], 3);
3950     ht.incrementColumnValue(ROW, FAMILY, QUALIFIERS[3], 4);
3951 
3952     // Now increment things incremented with old and do some new
3953     Increment inc = new Increment(ROW);
3954     inc.addColumn(FAMILY, QUALIFIERS[1], 1);
3955     inc.addColumn(FAMILY, QUALIFIERS[3], 1);
3956     inc.addColumn(FAMILY, QUALIFIERS[4], 1);
3957     ht.increment(inc);
3958 
3959     // Verify expected results
3960     Result r = ht.get(new Get(ROW));
3961     KeyValue [] kvs = r.raw();
3962     assertEquals(5, kvs.length);
3963     assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 1);
3964     assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 3);
3965     assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 3);
3966     assertIncrementKey(kvs[3], ROW, FAMILY, QUALIFIERS[3], 5);
3967     assertIncrementKey(kvs[4], ROW, FAMILY, QUALIFIERS[4], 1);
3968 
3969     // Now try multiple columns by different amounts
3970     inc = new Increment(ROWS[0]);
3971     for (int i=0;i<QUALIFIERS.length;i++) {
3972       inc.addColumn(FAMILY, QUALIFIERS[i], i+1);
3973     }
3974     ht.increment(inc);
3975     // Verify
3976     r = ht.get(new Get(ROWS[0]));
3977     kvs = r.raw();
3978     assertEquals(QUALIFIERS.length, kvs.length);
3979     for (int i=0;i<QUALIFIERS.length;i++) {
3980       assertIncrementKey(kvs[i], ROWS[0], FAMILY, QUALIFIERS[i], i+1);
3981     }
3982 
3983     // Re-increment them
3984     inc = new Increment(ROWS[0]);
3985     for (int i=0;i<QUALIFIERS.length;i++) {
3986       inc.addColumn(FAMILY, QUALIFIERS[i], i+1);
3987     }
3988     ht.increment(inc);
3989     // Verify
3990     r = ht.get(new Get(ROWS[0]));
3991     kvs = r.raw();
3992     assertEquals(QUALIFIERS.length, kvs.length);
3993     for (int i=0;i<QUALIFIERS.length;i++) {
3994       assertIncrementKey(kvs[i], ROWS[0], FAMILY, QUALIFIERS[i], 2*(i+1));
3995     }
3996   }
3997 
3998   /**
3999    * This test demonstrates how we use ThreadPoolExecutor.
4000    * It needs to show that we only use as many threads in the pool as we have
4001    * region servers. To do this, instead of doing real requests, we use a
4002    * SynchronousQueue where each put must wait for a take (and vice versa)
4003    * so that way we have full control of the number of active threads.
4004    * @throws IOException
4005    * @throws InterruptedException
4006    */
4007   @Test
4008   public void testPoolBehavior() throws IOException, InterruptedException {
4009     byte[] someBytes = Bytes.toBytes("pool");
4010     HTable table = TEST_UTIL.createTable(someBytes, someBytes);
4011     ThreadPoolExecutor pool = (ThreadPoolExecutor)table.getPool();
4012 
4013     // Make sure that the TPE stars with a core pool size of one and 0
4014     // initialized worker threads
4015     assertEquals(1, pool.getCorePoolSize());
4016     assertEquals(0, pool.getPoolSize());
4017 
4018     // Build a SynchronousQueue that we use for thread coordination
4019     final SynchronousQueue<Object> queue = new SynchronousQueue<Object>();
4020     List<Thread> threads = new ArrayList<Thread>(5);
4021     for (int i = 0; i < 5; i++) {
4022       threads.add(new Thread() {
4023         public void run() {
4024           try {
4025             // The thread blocks here until we decide to let it go
4026             queue.take();
4027           } catch (InterruptedException ie) { }
4028         }
4029       });
4030     }
4031     // First, add two threads and make sure the pool size follows
4032     pool.submit(threads.get(0));
4033     assertEquals(1, pool.getPoolSize());
4034     pool.submit(threads.get(1));
4035     assertEquals(2, pool.getPoolSize());
4036 
4037     // Next, terminate those threads and then make sure the pool is still the
4038     // same size
4039     queue.put(new Object());
4040     threads.get(0).join();
4041     queue.put(new Object());
4042     threads.get(1).join();
4043     assertEquals(2, pool.getPoolSize());
4044 
4045     // Now let's simulate adding a RS meaning that we'll go up to three
4046     // concurrent threads. The pool should not grow larger than three.
4047     pool.submit(threads.get(2));
4048     pool.submit(threads.get(3));
4049     pool.submit(threads.get(4));
4050     assertEquals(3, pool.getPoolSize());
4051     queue.put(new Object());
4052     queue.put(new Object());
4053     queue.put(new Object());
4054   }
4055 
4056   @Test
4057   public void testClientPoolRoundRobin() throws IOException {
4058     final byte[] tableName = Bytes.toBytes("testClientPoolRoundRobin");
4059 
4060     int poolSize = 3;
4061     int numVersions = poolSize * 2;
4062     Configuration conf = TEST_UTIL.getConfiguration();
4063     conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin");
4064     conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4065 
4066     HTable table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY },
4067         conf, Integer.MAX_VALUE);
4068     table.setAutoFlush(true);
4069     Put put = new Put(ROW);
4070     put.add(FAMILY, QUALIFIER, VALUE);
4071 
4072     Get get = new Get(ROW);
4073     get.addColumn(FAMILY, QUALIFIER);
4074     get.setMaxVersions();
4075 
4076     for (int versions = 1; versions <= numVersions; versions++) {
4077       table.put(put);
4078 
4079       Result result = table.get(get);
4080       NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4081           .get(QUALIFIER);
4082 
4083       assertEquals("The number of versions of '" + FAMILY + ":" + QUALIFIER
4084           + " did not match " + versions, versions, navigableMap.size());
4085       for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4086         assertTrue("The value at time " + entry.getKey()
4087             + " did not match what was put",
4088             Bytes.equals(VALUE, entry.getValue()));
4089       }
4090     }
4091   }
4092 
4093   @Test
4094   public void testClientPoolThreadLocal() throws IOException {
4095     final byte[] tableName = Bytes.toBytes("testClientPoolThreadLocal");
4096 
4097     int poolSize = Integer.MAX_VALUE;
4098     int numVersions = 3;
4099     Configuration conf = TEST_UTIL.getConfiguration();
4100     conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local");
4101     conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4102 
4103     final HTable table = TEST_UTIL.createTable(tableName,
4104         new byte[][] { FAMILY }, conf);
4105     table.setAutoFlush(true);
4106     final Put put = new Put(ROW);
4107     put.add(FAMILY, QUALIFIER, VALUE);
4108 
4109     final Get get = new Get(ROW);
4110     get.addColumn(FAMILY, QUALIFIER);
4111     get.setMaxVersions();
4112 
4113     for (int versions = 1; versions <= numVersions; versions++) {
4114       table.put(put);
4115 
4116       Result result = table.get(get);
4117       NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4118           .get(QUALIFIER);
4119 
4120       assertEquals("The number of versions of '" + FAMILY + ":" + QUALIFIER
4121           + " did not match " + versions, versions, navigableMap.size());
4122       for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4123         assertTrue("The value at time " + entry.getKey()
4124             + " did not match what was put",
4125             Bytes.equals(VALUE, entry.getValue()));
4126       }
4127     }
4128 
4129     final Object waitLock = new Object();
4130 
4131     ExecutorService executorService = Executors.newFixedThreadPool(numVersions);
4132     for (int versions = numVersions; versions < numVersions * 2; versions++) {
4133       final int versionsCopy = versions;
4134       executorService.submit(new Callable<Void>() {
4135         @Override
4136         public Void call() {
4137           try {
4138             table.put(put);
4139 
4140             Result result = table.get(get);
4141             NavigableMap<Long, byte[]> navigableMap = result.getMap()
4142                 .get(FAMILY).get(QUALIFIER);
4143 
4144             assertEquals("The number of versions of '" + FAMILY + ":"
4145                 + QUALIFIER + " did not match " + versionsCopy, versionsCopy,
4146                 navigableMap.size());
4147             for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4148               assertTrue("The value at time " + entry.getKey()
4149                   + " did not match what was put",
4150                   Bytes.equals(VALUE, entry.getValue()));
4151             }
4152             synchronized (waitLock) {
4153               waitLock.wait();
4154             }
4155           } catch (Exception e) {
4156           }
4157 
4158           return null;
4159         }
4160       });
4161     }
4162     synchronized (waitLock) {
4163       waitLock.notifyAll();
4164     }
4165     executorService.shutdownNow();
4166   }
4167   
4168   
4169   @Test
4170   public void testCheckAndPut() throws IOException {
4171     final byte [] anotherrow = Bytes.toBytes("anotherrow");
4172     final byte [] value2 = Bytes.toBytes("abcd");
4173     
4174     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"),
4175       new byte [][] {FAMILY});
4176     Put put1 = new Put(ROW);
4177     put1.add(FAMILY, QUALIFIER, VALUE);
4178 
4179     // row doesn't exist, so using non-null value should be considered "not match".
4180     boolean ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put1);
4181     assertEquals(ok, false);
4182     
4183     // row doesn't exist, so using "null" to check for existence should be considered "match".
4184     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4185     assertEquals(ok, true);
4186 
4187     // row now exists, so using "null" to check for existence should be considered "not match".
4188     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4189     assertEquals(ok, false);
4190     
4191     Put put2 = new Put(ROW);
4192     put2.add(FAMILY, QUALIFIER, value2);
4193     
4194     // row now exists, use the matching value to check
4195     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put2);
4196     assertEquals(ok, true);
4197 
4198     Put put3 = new Put(anotherrow);
4199     put3.add(FAMILY, QUALIFIER, VALUE);
4200     
4201     // try to do CheckAndPut on different rows       
4202     try {
4203         ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, value2, put3);
4204         fail("trying to check and modify different rows should have failed.");
4205       } catch(Exception e) {}      
4206     
4207   }  
4208 
4209   /**
4210    * Tests that cache on write works all the way up from the client-side.
4211    *
4212    * Performs inserts, flushes, and compactions, verifying changes in the block
4213    * cache along the way.
4214    *
4215    * @throws Exception
4216    */
4217   @Test
4218   public void testCacheOnWriteEvictOnClose() throws Exception {
4219     byte [] tableName = Bytes.toBytes("testCOWEOCfromClient");
4220     byte [] data = Bytes.toBytes("data");
4221     HTable table = TEST_UTIL.createTable(tableName, new byte [][] {FAMILY});
4222     // get the block cache and region
4223     String regionName = table.getRegionLocations().firstKey().getEncodedName();
4224     HRegion region = TEST_UTIL.getRSForFirstRegionInTable(
4225         tableName).getFromOnlineRegions(regionName);
4226     Store store = region.getStores().values().iterator().next();
4227     CacheConfig cacheConf = store.getCacheConfig();
4228     cacheConf.forTestsOnly_setCacheDataOnWrite(true);
4229     cacheConf.forTestsOnly_setEvictOnClose(true);
4230     BlockCache cache = cacheConf.getBlockCache();
4231 
4232     // establish baseline stats
4233     long startBlockCount = cache.getBlockCount();
4234     long startBlockHits = cache.getStats().getHitCount();
4235     long startBlockMiss = cache.getStats().getMissCount();
4236     // insert data
4237     Put put = new Put(ROW);
4238     put.add(FAMILY, QUALIFIER, data);
4239     table.put(put);
4240     assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
4241     // data was in memstore so don't expect any changes
4242     assertEquals(startBlockCount, cache.getBlockCount());
4243     assertEquals(startBlockHits, cache.getStats().getHitCount());
4244     assertEquals(startBlockMiss, cache.getStats().getMissCount());
4245     // flush the data
4246     System.out.println("Flushing cache");
4247     region.flushcache();
4248     // expect one more block in cache, no change in hits/misses
4249     long expectedBlockCount = startBlockCount + 1;
4250     long expectedBlockHits = startBlockHits;
4251     long expectedBlockMiss = startBlockMiss;
4252     assertEquals(expectedBlockCount, cache.getBlockCount());
4253     assertEquals(expectedBlockHits, cache.getStats().getHitCount());
4254     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4255     // read the data and expect same blocks, one new hit, no misses
4256     assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
4257     assertEquals(expectedBlockCount, cache.getBlockCount());
4258     assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
4259     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4260     // insert a second column, read the row, no new blocks, one new hit
4261     byte [] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
4262     byte [] data2 = Bytes.add(data, data);
4263     put = new Put(ROW);
4264     put.add(FAMILY, QUALIFIER2, data2);
4265     table.put(put);
4266     Result r = table.get(new Get(ROW));
4267     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
4268     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
4269     assertEquals(expectedBlockCount, cache.getBlockCount());
4270     assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
4271     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4272     // flush, one new block
4273     System.out.println("Flushing cache");
4274     region.flushcache();
4275     assertEquals(++expectedBlockCount, cache.getBlockCount());
4276     assertEquals(expectedBlockHits, cache.getStats().getHitCount());
4277     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4278     // compact, net minus on block, two hits, no misses
4279     System.out.println("Compacting");
4280     assertEquals(2, store.getNumberOfstorefiles());
4281     store.triggerMajorCompaction();
4282     region.compactStores();
4283     waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max
4284     assertEquals(1, store.getNumberOfstorefiles());
4285     assertEquals(--expectedBlockCount, cache.getBlockCount());
4286     expectedBlockHits += 2;
4287     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4288     assertEquals(expectedBlockHits, cache.getStats().getHitCount());
4289     // read the row, same blocks, one hit no miss
4290     r = table.get(new Get(ROW));
4291     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
4292     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
4293     assertEquals(expectedBlockCount, cache.getBlockCount());
4294     assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
4295     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
4296     // no cache misses!
4297     assertEquals(startBlockMiss, cache.getStats().getMissCount());
4298   }
4299 
4300   private void waitForStoreFileCount(Store store, int count, int timeout)
4301   throws InterruptedException {
4302     long start = System.currentTimeMillis();
4303     while (start + timeout > System.currentTimeMillis() &&
4304         store.getNumberOfstorefiles() != count) {
4305       Thread.sleep(100);
4306     }
4307     System.out.println("start=" + start + ", now=" +
4308         System.currentTimeMillis() + ", cur=" + store.getNumberOfstorefiles());
4309     assertEquals(count, store.getNumberOfstorefiles());
4310   }
4311 }