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