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