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