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