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