View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.client;
20  
21  import static org.junit.Assert.assertArrayEquals;
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertFalse;
24  import static org.junit.Assert.assertNotNull;
25  import static org.junit.Assert.assertNull;
26  import static org.junit.Assert.assertSame;
27  import static org.junit.Assert.assertTrue;
28  import static org.junit.Assert.fail;
29  import java.io.IOException;
30  import java.lang.reflect.Method;
31  import java.util.ArrayList;
32  import java.util.Arrays;
33  import java.util.Collection;
34  import java.util.Collections;
35  import java.util.HashMap;
36  import java.util.HashSet;
37  import java.util.Iterator;
38  import java.util.LinkedList;
39  import java.util.List;
40  import java.util.Map;
41  import java.util.NavigableMap;
42  import java.util.NavigableSet;
43  import java.util.UUID;
44  import java.util.concurrent.Callable;
45  import java.util.concurrent.ExecutorService;
46  import java.util.concurrent.Executors;
47  import java.util.concurrent.atomic.AtomicBoolean;
48  import java.util.concurrent.atomic.AtomicLong;
49  import java.util.concurrent.atomic.AtomicReference;
50  
51  import org.apache.log4j.Level;
52  import org.apache.commons.lang.ArrayUtils;
53  import org.apache.commons.logging.Log;
54  import org.apache.commons.logging.LogFactory;
55  import org.apache.hadoop.conf.Configuration;
56  import org.apache.hadoop.hbase.Abortable;
57  import org.apache.hadoop.hbase.Cell;
58  import org.apache.hadoop.hbase.CellUtil;
59  import org.apache.hadoop.hbase.DoNotRetryIOException;
60  import org.apache.hadoop.hbase.HBaseTestingUtility;
61  import org.apache.hadoop.hbase.HColumnDescriptor;
62  import org.apache.hadoop.hbase.HConstants;
63  import org.apache.hadoop.hbase.HRegionInfo;
64  import org.apache.hadoop.hbase.HRegionLocation;
65  import org.apache.hadoop.hbase.HTableDescriptor;
66  import org.apache.hadoop.hbase.KeyValue;
67  import org.apache.hadoop.hbase.MiniHBaseCluster;
68  import org.apache.hadoop.hbase.RegionLocations;
69  import org.apache.hadoop.hbase.ServerName;
70  import org.apache.hadoop.hbase.TableName;
71  import org.apache.hadoop.hbase.Waiter;
72  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
73  import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
74  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
75  import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
76  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
77  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
78  import org.apache.hadoop.hbase.exceptions.ScannerResetException;
79  import org.apache.hadoop.hbase.filter.BinaryComparator;
80  import org.apache.hadoop.hbase.filter.CompareFilter;
81  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
82  import org.apache.hadoop.hbase.filter.Filter;
83  import org.apache.hadoop.hbase.filter.FilterList;
84  import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
85  import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
86  import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
87  import org.apache.hadoop.hbase.filter.LongComparator;
88  import org.apache.hadoop.hbase.filter.PrefixFilter;
89  import org.apache.hadoop.hbase.filter.QualifierFilter;
90  import org.apache.hadoop.hbase.filter.RegexStringComparator;
91  import org.apache.hadoop.hbase.filter.RowFilter;
92  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
93  import org.apache.hadoop.hbase.filter.WhileMatchFilter;
94  import org.apache.hadoop.hbase.io.hfile.BlockCache;
95  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
96  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
97  import org.apache.hadoop.hbase.master.HMaster;
98  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
99  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
100 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
101 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
102 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
103 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
104 import org.apache.hadoop.hbase.regionserver.DelegatingKeyValueScanner;
105 import org.apache.hadoop.hbase.regionserver.HRegion;
106 import org.apache.hadoop.hbase.regionserver.HRegionServer;
107 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
108 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
109 import org.apache.hadoop.hbase.regionserver.Region;
110 import org.apache.hadoop.hbase.regionserver.ScanInfo;
111 import org.apache.hadoop.hbase.regionserver.Store;
112 import org.apache.hadoop.hbase.regionserver.StoreScanner;
113 import org.apache.hadoop.hbase.testclassification.LargeTests;
114 import org.apache.hadoop.hbase.util.Bytes;
115 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
116 import org.apache.hadoop.hbase.util.Pair;
117 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
118 import org.apache.log4j.AppenderSkeleton;
119 import org.apache.log4j.Logger;
120 import org.apache.log4j.spi.LoggingEvent;
121 import org.junit.AfterClass;
122 import org.junit.Assume;
123 import org.junit.Ignore;
124 import org.junit.Test;
125 import org.junit.experimental.categories.Category;
126 import org.junit.runner.RunWith;
127 import org.junit.runners.Parameterized;
128 
129 /**
130  * Run tests that use the HBase clients; {@link HTable}.
131  * Sets up the HBase mini cluster once at start and runs through all client tests.
132  * Each creates a table named for the method and does its stuff against that.
133  *
134  * Parameterized to run with different registry implementations.
135  */
136 @Category(LargeTests.class)
137 @SuppressWarnings ("deprecation")
138 @RunWith(Parameterized.class)
139 public class TestFromClientSide {
140   private static final Log LOG = LogFactory.getLog(TestFromClientSide.class);
141   protected static HBaseTestingUtility TEST_UTIL;
142   private static byte [] ROW = Bytes.toBytes("testRow");
143   private static byte [] FAMILY = Bytes.toBytes("testFamily");
144   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
145   private static byte [] VALUE = Bytes.toBytes("testValue");
146   protected static int SLAVES = 3;
147 
148   @Parameterized.Parameters
149   public static Collection parameters() {
150     return Arrays.asList(new Object[][] {
151         { MasterRegistry.class },
152         { ZKConnectionRegistry.class }
153     });
154   }
155 
156   // To keep the child classes happy.
157   TestFromClientSide() {}
158 
159   public TestFromClientSide(Class<? extends ConnectionRegistry> registry) throws Exception {
160     initialize(registry);
161   }
162 
163   /**
164    * JUnit does not provide an easy way to run a hook after each parameterized run. Without that
165    * there is no easy way to restart the test cluster after each parameterized run. Annotation
166    * BeforeParam does not work either because it runs before parameterization and hence does not
167    * have access to the test parameters (which is weird).
168    *
169    * This *hack* checks if the current instance of test cluster configuration has the passed
170    * parameterized configs. In such a case, we can just reuse the cluster for test and do not need
171    * to initialize from scratch. While this is a hack, it saves a ton of time for the full
172    * test and de-flakes it.
173    */
174   protected static boolean isSameParameterizedCluster(
175       Class<? extends ConnectionRegistry> registryImpl) {
176     if (TEST_UTIL == null) {
177       return false;
178     }
179     Configuration conf = TEST_UTIL.getConfiguration();
180     Class<? extends ConnectionRegistry> confClass = conf.getClass(HConstants.REGISTRY_IMPL_CONF_KEY,
181         ZKConnectionRegistry.class, ConnectionRegistry.class);
182     return confClass.getName().equals(registryImpl.getName());
183   }
184 
185   public static void initialize(Class<? extends ConnectionRegistry> registry) throws Exception {
186     // initialize() is called for every unit test, however we only want to reset the cluster state
187     // at the end of every parameterized run.
188     if (isSameParameterizedCluster(registry)) {
189       return;
190     }
191     if (TEST_UTIL != null) {
192       // We reached end of a parameterized run, clean up.
193       TEST_UTIL.shutdownMiniCluster();
194     }
195     TEST_UTIL = new HBaseTestingUtility();
196     // Uncomment the following lines if more verbosity is needed for
197     // debugging (see HBASE-12285 for details).
198     //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
199     //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
200     //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
201     Configuration conf = TEST_UTIL.getConfiguration();
202     conf.setClass(HConstants.REGISTRY_IMPL_CONF_KEY, registry, ConnectionRegistry.class);
203     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
204         MultiRowMutationEndpoint.class.getName());
205     conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
206     conf.setLong(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 6000000);
207     // We need more than one region server in this test
208     TEST_UTIL.startMiniCluster(SLAVES);
209   }
210 
211   /**
212    * @throws java.lang.Exception
213    */
214   @AfterClass
215   public static void tearDownAfterClass() throws Exception {
216     TEST_UTIL.shutdownMiniCluster();
217   }
218 
219   /**
220    * Test append result when there are duplicate rpc request.
221    */
222   @Test
223   public void testDuplicateAppend() throws Exception {
224     HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testDuplicateAppend");
225     Map<String, String> kvs = new HashMap<String, String>();
226     kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
227     hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs);
228     TEST_UTIL.createTable(hdt, new byte[][] { ROW }).close();
229     TEST_UTIL.waitTableAvailable(hdt.getTableName(), 10000);
230     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
231     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
232     // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call
233     c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500);
234 
235     Connection connection = ConnectionFactory.createConnection(c);
236     Table t = connection.getTable(TableName.valueOf("HCM-testDuplicateAppend"));
237     if (t instanceof HTable) {
238       HTable table = (HTable) t;
239       table.setOperationTimeout(3 * 1000);
240 
241       try {
242         Append append = new Append(ROW);
243         append.add(TEST_UTIL.fam1, QUALIFIER, VALUE);
244         Result result = table.append(append);
245 
246         // Verify expected result
247         Cell[] cells = result.rawCells();
248         assertEquals(1, cells.length);
249         assertKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, VALUE);
250 
251         // Verify expected result again
252         Result readResult = table.get(new Get(ROW));
253         cells = readResult.rawCells();
254         assertEquals(1, cells.length);
255         assertKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, VALUE);
256       } finally {
257         table.close();
258         connection.close();
259       }
260     }
261   }
262 
263   /**
264    * Basic client side validation of HBASE-4536
265    */
266    @Test
267    public void testKeepDeletedCells() throws Exception {
268      final TableName TABLENAME = TableName.valueOf("testKeepDeletesCells");
269      final byte[] FAMILY = Bytes.toBytes("family");
270      final byte[] C0 = Bytes.toBytes("c0");
271 
272      final byte[] T1 = Bytes.toBytes("T1");
273      final byte[] T2 = Bytes.toBytes("T2");
274      final byte[] T3 = Bytes.toBytes("T3");
275      HColumnDescriptor hcd = new HColumnDescriptor(FAMILY)
276          .setKeepDeletedCells(true).setMaxVersions(3);
277 
278      HTableDescriptor desc = new HTableDescriptor(TABLENAME);
279      desc.addFamily(hcd);
280      TEST_UTIL.getHBaseAdmin().createTable(desc);
281      TEST_UTIL.waitTableAvailable(desc.getTableName(), 10000);
282      Configuration c = TEST_UTIL.getConfiguration();
283      Table h = new HTable(c, TABLENAME);
284 
285      long ts = System.currentTimeMillis();
286      Put p = new Put(T1, ts);
287      p.add(FAMILY, C0, T1);
288      h.put(p);
289      p = new Put(T1, ts+2);
290      p.add(FAMILY, C0, T2);
291      h.put(p);
292      p = new Put(T1, ts+4);
293      p.add(FAMILY, C0, T3);
294      h.put(p);
295 
296      Delete d = new Delete(T1, ts+3);
297      h.delete(d);
298 
299      d = new Delete(T1, ts+3);
300      d.deleteColumns(FAMILY, C0, ts+3);
301      h.delete(d);
302 
303      Get g = new Get(T1);
304      // does *not* include the delete
305      g.setTimeRange(0, ts+3);
306      Result r = h.get(g);
307      assertArrayEquals(T2, r.getValue(FAMILY, C0));
308 
309      Scan s = new Scan(T1);
310      s.setTimeRange(0, ts+3);
311      s.setMaxVersions();
312      ResultScanner scanner = h.getScanner(s);
313      Cell[] kvs = scanner.next().rawCells();
314      assertArrayEquals(T2, CellUtil.cloneValue(kvs[0]));
315      assertArrayEquals(T1, CellUtil.cloneValue(kvs[1]));
316      scanner.close();
317 
318      s = new Scan(T1);
319      s.setRaw(true);
320      s.setMaxVersions();
321      scanner = h.getScanner(s);
322      kvs = scanner.next().rawCells();
323      assertTrue(CellUtil.isDeleteFamily(kvs[0]));
324      assertArrayEquals(T3, CellUtil.cloneValue(kvs[1]));
325      assertTrue(CellUtil.isDelete(kvs[2]));
326      assertArrayEquals(T2, CellUtil.cloneValue(kvs[3]));
327      assertArrayEquals(T1, CellUtil.cloneValue(kvs[4]));
328      scanner.close();
329      h.close();
330    }
331 
332     /**
333     * Basic client side validation of HBASE-10118
334     */
335    @Test
336    public void testPurgeFutureDeletes() throws Exception {
337      final TableName TABLENAME = TableName.valueOf("testPurgeFutureDeletes");
338      final byte[] ROW = Bytes.toBytes("row");
339      final byte[] FAMILY = Bytes.toBytes("family");
340      final byte[] COLUMN = Bytes.toBytes("column");
341      final byte[] VALUE = Bytes.toBytes("value");
342 
343      Table table = TEST_UTIL.createTable(TABLENAME, FAMILY);
344      TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
345      // future timestamp
346      long ts = System.currentTimeMillis() * 2;
347      Put put = new Put(ROW, ts);
348      put.add(FAMILY, COLUMN, VALUE);
349      table.put(put);
350 
351      Get get = new Get(ROW);
352      Result result = table.get(get);
353      assertArrayEquals(VALUE, result.getValue(FAMILY, COLUMN));
354 
355      Delete del = new Delete(ROW);
356      del.deleteColumn(FAMILY, COLUMN, ts);
357      table.delete(del);
358 
359      get = new Get(ROW);
360      result = table.get(get);
361      assertNull(result.getValue(FAMILY, COLUMN));
362 
363      // major compaction, purged future deletes
364      TEST_UTIL.getHBaseAdmin().flush(TABLENAME);
365      TEST_UTIL.getHBaseAdmin().majorCompact(TABLENAME);
366 
367      // waiting for the major compaction to complete
368      TEST_UTIL.waitFor(6000, new Waiter.Predicate<IOException>() {
369        @Override
370        public boolean evaluate() throws IOException {
371          return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) ==
372              AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
373        }
374      });
375 
376      put = new Put(ROW, ts);
377      put.add(FAMILY, COLUMN, VALUE);
378      table.put(put);
379 
380      get = new Get(ROW);
381      result = table.get(get);
382      assertArrayEquals(VALUE, result.getValue(FAMILY, COLUMN));
383 
384      table.close();
385    }
386 
387    /**
388     * @deprecated Tests deprecated functionality. Remove when we are past 1.0.
389     * @throws Exception
390     */
391    @Deprecated
392    @Test
393    public void testSharedZooKeeper() throws Exception {
394      Configuration newConfig = new Configuration(TEST_UTIL.getConfiguration());
395      newConfig.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "12345");
396 
397      // First with a simple ZKW
398      ZooKeeperWatcher z0 = new ZooKeeperWatcher(
399        newConfig, "hconnection", new Abortable() {
400        @Override public void abort(String why, Throwable e) {}
401        @Override public boolean isAborted() {return false;}
402      });
403      z0.getRecoverableZooKeeper().getZooKeeper().exists("/oldZooKeeperWatcher", false);
404      z0.close();
405 
406      // Then a ZooKeeperKeepAliveConnection
407      ConnectionManager.HConnectionImplementation connection1 =
408        (ConnectionManager.HConnectionImplementation)
409          HConnectionManager.getConnection(newConfig);
410 
411      ZooKeeperKeepAliveConnection z1 = connection1.getKeepAliveZooKeeperWatcher();
412      z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1", false);
413 
414      z1.close();
415 
416      // will still work, because the real connection is not closed yet
417      // Not do be done in real code
418      z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1afterclose", false);
419 
420 
421      ZooKeeperKeepAliveConnection z2 = connection1.getKeepAliveZooKeeperWatcher();
422      assertTrue(
423        "ZooKeeperKeepAliveConnection equals on same connection", z1 == z2);
424 
425 
426 
427      Configuration newConfig2 = new Configuration(TEST_UTIL.getConfiguration());
428      newConfig2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "6789");
429      ConnectionManager.HConnectionImplementation connection2 =
430        (ConnectionManager.HConnectionImplementation)
431          HConnectionManager.getConnection(newConfig2);
432 
433      assertTrue("connections should be different ", connection1 != connection2);
434 
435      ZooKeeperKeepAliveConnection z3 = connection2.getKeepAliveZooKeeperWatcher();
436      assertTrue(
437        "ZooKeeperKeepAliveConnection should be different" +
438          " on different connections", z1 != z3);
439 
440      // Bypass the private access
441      Method m = ConnectionManager.HConnectionImplementation.class.
442        getDeclaredMethod("closeZooKeeperWatcher");
443      m.setAccessible(true);
444      m.invoke(connection2);
445 
446      ZooKeeperKeepAliveConnection z4 = connection2.getKeepAliveZooKeeperWatcher();
447      assertTrue(
448        "ZooKeeperKeepAliveConnection should be recreated" +
449          " when previous connections was closed"
450        , z3 != z4);
451 
452 
453      z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false);
454      z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
455 
456 
457      HConnectionManager.deleteConnection(newConfig);
458      try {
459        z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false);
460        assertTrue("We should not have a valid connection for z2", false);
461      } catch (Exception e){
462      }
463 
464      z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
465      // We expect success here.
466 
467 
468      HConnectionManager.deleteConnection(newConfig2);
469      try {
470        z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
471        assertTrue("We should not have a valid connection for z4", false);
472      } catch (Exception e){
473      }
474    }
475 
476 
477   /**
478    * Verifies that getConfiguration returns the same Configuration object used
479    * to create the HTable instance.
480    */
481   @Test
482   public void testGetConfiguration() throws Exception {
483     TableName TABLE = TableName.valueOf("testGetConfiguration");
484     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
485     Configuration conf = TEST_UTIL.getConfiguration();
486     Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
487     TEST_UTIL.waitTableAvailable(TABLE, 10000);
488     assertSame(conf, table.getConfiguration());
489   }
490 
491   /**
492    * Test from client side of an involved filter against a multi family that
493    * involves deletes.
494    *
495    * @throws Exception
496    */
497   @Test
498   public void testWeirdCacheBehaviour() throws Exception {
499     TableName TABLE = TableName.valueOf("testWeirdCacheBehaviour");
500     byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
501         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
502         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
503     HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
504     TEST_UTIL.waitTableAvailable(TABLE, 10000);
505     String value = "this is the value";
506     String value2 = "this is some other value";
507     String keyPrefix1 = UUID.randomUUID().toString();
508     String keyPrefix2 = UUID.randomUUID().toString();
509     String keyPrefix3 = UUID.randomUUID().toString();
510     putRows(ht, 3, value, keyPrefix1);
511     putRows(ht, 3, value, keyPrefix2);
512     putRows(ht, 3, value, keyPrefix3);
513     ht.flushCommits();
514     putRows(ht, 3, value2, keyPrefix1);
515     putRows(ht, 3, value2, keyPrefix2);
516     putRows(ht, 3, value2, keyPrefix3);
517     Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
518     System.out.println("Checking values for key: " + keyPrefix1);
519     assertEquals("Got back incorrect number of rows from scan", 3,
520         getNumberOfRows(keyPrefix1, value2, table));
521     System.out.println("Checking values for key: " + keyPrefix2);
522     assertEquals("Got back incorrect number of rows from scan", 3,
523         getNumberOfRows(keyPrefix2, value2, table));
524     System.out.println("Checking values for key: " + keyPrefix3);
525     assertEquals("Got back incorrect number of rows from scan", 3,
526         getNumberOfRows(keyPrefix3, value2, table));
527     deleteColumns(ht, value2, keyPrefix1);
528     deleteColumns(ht, value2, keyPrefix2);
529     deleteColumns(ht, value2, keyPrefix3);
530     System.out.println("Starting important checks.....");
531     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
532       0, getNumberOfRows(keyPrefix1, value2, table));
533     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
534       0, getNumberOfRows(keyPrefix2, value2, table));
535     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
536       0, getNumberOfRows(keyPrefix3, value2, table));
537     ht.setScannerCaching(0);
538     assertEquals("Got back incorrect number of rows from scan", 0,
539       getNumberOfRows(keyPrefix1, value2, table)); ht.setScannerCaching(100);
540     assertEquals("Got back incorrect number of rows from scan", 0,
541       getNumberOfRows(keyPrefix2, value2, table));
542   }
543 
544   private void deleteColumns(Table ht, String value, String keyPrefix)
545   throws IOException {
546     ResultScanner scanner = buildScanner(keyPrefix, value, ht);
547     Iterator<Result> it = scanner.iterator();
548     int count = 0;
549     while (it.hasNext()) {
550       Result result = it.next();
551       Delete delete = new Delete(result.getRow());
552       delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
553       ht.delete(delete);
554       count++;
555     }
556     assertEquals("Did not perform correct number of deletes", 3, count);
557   }
558 
559   private int getNumberOfRows(String keyPrefix, String value, Table ht)
560       throws Exception {
561     ResultScanner resultScanner = buildScanner(keyPrefix, value, ht);
562     Iterator<Result> scanner = resultScanner.iterator();
563     int numberOfResults = 0;
564     while (scanner.hasNext()) {
565       Result result = scanner.next();
566       System.out.println("Got back key: " + Bytes.toString(result.getRow()));
567       for (Cell kv : result.rawCells()) {
568         System.out.println("kv=" + kv.toString() + ", "
569             + Bytes.toString(CellUtil.cloneValue(kv)));
570       }
571       numberOfResults++;
572     }
573     return numberOfResults;
574   }
575 
576   private ResultScanner buildScanner(String keyPrefix, String value, Table ht)
577       throws IOException {
578     // OurFilterList allFilters = new OurFilterList();
579     FilterList allFilters = new FilterList(/* FilterList.Operator.MUST_PASS_ALL */);
580     allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
581     SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes
582         .toBytes("trans-tags"), Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes
583         .toBytes(value));
584     filter.setFilterIfMissing(true);
585     allFilters.addFilter(filter);
586 
587     // allFilters.addFilter(new
588     // RowExcludingSingleColumnValueFilter(Bytes.toBytes("trans-tags"),
589     // Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value)));
590 
591     Scan scan = new Scan();
592     scan.addFamily(Bytes.toBytes("trans-blob"));
593     scan.addFamily(Bytes.toBytes("trans-type"));
594     scan.addFamily(Bytes.toBytes("trans-date"));
595     scan.addFamily(Bytes.toBytes("trans-tags"));
596     scan.addFamily(Bytes.toBytes("trans-group"));
597     scan.setFilter(allFilters);
598 
599     return ht.getScanner(scan);
600   }
601 
602   private void putRows(Table ht, int numRows, String value, String key)
603       throws IOException {
604     for (int i = 0; i < numRows; i++) {
605       String row = key + "_" + UUID.randomUUID().toString();
606       System.out.println(String.format("Saving row: %s, with value %s", row,
607           value));
608       Put put = new Put(Bytes.toBytes(row));
609       put.setDurability(Durability.SKIP_WAL);
610       put.add(Bytes.toBytes("trans-blob"), null, Bytes
611           .toBytes("value for blob"));
612       put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
613       put.add(Bytes.toBytes("trans-date"), null, Bytes
614           .toBytes("20090921010101999"));
615       put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"), Bytes
616           .toBytes(value));
617       put.add(Bytes.toBytes("trans-group"), null, Bytes
618           .toBytes("adhocTransactionGroupId"));
619       ht.put(put);
620     }
621   }
622 
623   /**
624    * Test filters when multiple regions.  It does counts.  Needs eye-balling of
625    * logs to ensure that we're not scanning more regions that we're supposed to.
626    * Related to the TestFilterAcrossRegions over in the o.a.h.h.filter package.
627    * @throws IOException
628    * @throws InterruptedException
629    */
630   @Test
631   public void testFilterAcrossMultipleRegions()
632   throws IOException, InterruptedException {
633     TableName name = TableName.valueOf("testFilterAcrossMutlipleRegions");
634     HTable t = TEST_UTIL.createTable(name, FAMILY);
635     TEST_UTIL.waitTableAvailable(name, 10000);
636     int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
637     assertRowCount(t, rowCount);
638     // Split the table.  Should split on a reasonable key; 'lqj'
639     Map<HRegionInfo, ServerName> regions  = splitTable(t);
640     assertRowCount(t, rowCount);
641     // Get end key of first region.
642     byte [] endKey = regions.keySet().iterator().next().getEndKey();
643     // Count rows with a filter that stops us before passed 'endKey'.
644     // Should be count of rows in first region.
645     int endKeyCount = countRows(t, createScanWithRowFilter(endKey));
646     assertTrue(endKeyCount < rowCount);
647 
648     // How do I know I did not got to second region?  Thats tough.  Can't really
649     // do that in client-side region test.  I verified by tracing in debugger.
650     // I changed the messages that come out when set to DEBUG so should see
651     // when scanner is done. Says "Finished with scanning..." with region name.
652     // Check that its finished in right region.
653 
654     // New test.  Make it so scan goes into next region by one and then two.
655     // Make sure count comes out right.
656     byte [] key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] + 1)};
657     int plusOneCount = countRows(t, createScanWithRowFilter(key));
658     assertEquals(endKeyCount + 1, plusOneCount);
659     key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] + 2)};
660     int plusTwoCount = countRows(t, createScanWithRowFilter(key));
661     assertEquals(endKeyCount + 2, plusTwoCount);
662 
663     // New test.  Make it so I scan one less than endkey.
664     key = new byte [] {endKey[0], endKey[1], (byte)(endKey[2] - 1)};
665     int minusOneCount = countRows(t, createScanWithRowFilter(key));
666     assertEquals(endKeyCount - 1, minusOneCount);
667     // For above test... study logs.  Make sure we do "Finished with scanning.."
668     // in first region and that we do not fall into the next region.
669 
670     key = new byte [] {'a', 'a', 'a'};
671     int countBBB = countRows(t,
672       createScanWithRowFilter(key, null, CompareFilter.CompareOp.EQUAL));
673     assertEquals(1, countBBB);
674 
675     int countGreater = countRows(t, createScanWithRowFilter(endKey, null,
676       CompareFilter.CompareOp.GREATER_OR_EQUAL));
677     // Because started at start of table.
678     assertEquals(0, countGreater);
679     countGreater = countRows(t, createScanWithRowFilter(endKey, endKey,
680       CompareFilter.CompareOp.GREATER_OR_EQUAL));
681     assertEquals(rowCount - endKeyCount, countGreater);
682   }
683 
684   /**
685    * This is a coprocessor to inject a test failure so that a store scanner.reseek() call will
686    * fail with an IOException() on the first call.
687    */
688   public static class ExceptionInReseekRegionObserver extends BaseRegionObserver {
689     static AtomicLong reqCount = new AtomicLong(0);
690     static AtomicBoolean isDoNotRetry = new AtomicBoolean(false); // whether to throw DNRIOE
691     static AtomicBoolean throwOnce = new AtomicBoolean(true); // whether to only throw once
692 
693     static void reset() {
694       reqCount.set(0);
695       isDoNotRetry.set(false);
696       throwOnce.set(true);
697     }
698 
699     class MyStoreScanner extends StoreScanner {
700       public MyStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
701           long readPt) throws IOException {
702         super(store, scanInfo, scan, columns, readPt);
703       }
704 
705       @Override
706       protected List<KeyValueScanner> selectScannersFrom(
707           List<? extends KeyValueScanner> allScanners) {
708         List<KeyValueScanner> scanners = super.selectScannersFrom(allScanners);
709         List<KeyValueScanner> newScanners = new ArrayList<>(scanners.size());
710         for (KeyValueScanner scanner : scanners) {
711           newScanners.add(new DelegatingKeyValueScanner(scanner) {
712             @Override
713             public boolean reseek(Cell key) throws IOException {
714               reqCount.incrementAndGet();
715               if (!throwOnce.get()||  reqCount.get() == 1) {
716                 if (isDoNotRetry.get()) {
717                   throw new DoNotRetryIOException("Injected exception");
718                 } else {
719                   throw new IOException("Injected exception");
720                 }
721               }
722               return super.reseek(key);
723             }
724           });
725         }
726         return newScanners;
727       }
728     }
729 
730     @Override
731     public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
732         Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s)
733             throws IOException {
734       return new MyStoreScanner(store, store.getScanInfo(), scan, targetCols, Long.MAX_VALUE);
735     }
736   }
737 
738   /**
739    * Tests the case where a Scan can throw an IOException in the middle of the seek / reseek
740    * leaving the server side RegionScanner to be in dirty state. The client has to ensure that the
741    * ClientScanner does not get an exception and also sees all the data.
742    * @throws IOException
743    * @throws InterruptedException
744    */
745   @Test
746   public void testClientScannerIsResetWhenScanThrowsIOException()
747   throws IOException, InterruptedException {
748     TEST_UTIL.getConfiguration().setBoolean("hbase.client.log.scanner.activity", true);
749     TableName name = TableName.valueOf("testClientScannerIsResetWhenScanThrowsIOException");
750 
751     HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name, FAMILY);
752     htd.addCoprocessor(ExceptionInReseekRegionObserver.class.getName());
753     TEST_UTIL.getHBaseAdmin().createTable(htd);
754     TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
755     ExceptionInReseekRegionObserver.reset();
756     ExceptionInReseekRegionObserver.throwOnce.set(true); // throw exceptions only once
757     try (Table t = TEST_UTIL.getConnection().getTable(name)) {
758       int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
759       TEST_UTIL.getHBaseAdmin().flush(name);
760       int actualRowCount = countRows(t, new Scan().addColumn(FAMILY, FAMILY));
761       assertEquals(rowCount, actualRowCount);
762     }
763     assertTrue(ExceptionInReseekRegionObserver.reqCount.get() > 0);
764   }
765 
766   /**
767    * Tests the case where a coprocessor throws a DoNotRetryIOException in the scan. The expectation
768    * is that the exception will bubble up to the client scanner instead of being retried.
769    */
770   @Test (timeout = 180000)
771   public void testScannerThrowsExceptionWhenCoprocessorThrowsDNRIOE()
772       throws IOException, InterruptedException {
773     TEST_UTIL.getConfiguration().setBoolean("hbase.client.log.scanner.activity", true);
774     TableName name = TableName.valueOf("testClientScannerIsNotRetriedWhenCoprocessorThrowsDNRIOE");
775 
776     HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name, FAMILY);
777     htd.addCoprocessor(ExceptionInReseekRegionObserver.class.getName());
778     TEST_UTIL.getHBaseAdmin().createTable(htd);
779     TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
780     ExceptionInReseekRegionObserver.reset();
781     ExceptionInReseekRegionObserver.isDoNotRetry.set(true);
782     try (Table t = TEST_UTIL.getConnection().getTable(name)) {
783       TEST_UTIL.loadTable(t, FAMILY, false);
784       TEST_UTIL.getHBaseAdmin().flush(name);
785       TEST_UTIL.countRows(t, new Scan().addColumn(FAMILY, FAMILY));
786       fail("Should have thrown an exception");
787     } catch (DoNotRetryIOException expected) {
788       // expected
789     }
790     assertTrue(ExceptionInReseekRegionObserver.reqCount.get() > 0);
791   }
792 
793   /**
794    * Tests the case where a coprocessor throws a regular IOException in the scan. The expectation
795    * is that the we will keep on retrying, but fail after the retries are exhausted instead of
796    * retrying indefinitely.
797    */
798   @Test (timeout = 180000)
799   public void testScannerFailsAfterRetriesWhenCoprocessorThrowsIOE()
800       throws IOException, InterruptedException {
801     TEST_UTIL.getConfiguration().setBoolean("hbase.client.log.scanner.activity", true);
802     TableName name = TableName.valueOf("testScannerFailsAfterRetriesWhenCoprocessorThrowsIOE");
803     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
804     HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name, FAMILY);
805     htd.addCoprocessor(ExceptionInReseekRegionObserver.class.getName());
806     TEST_UTIL.getHBaseAdmin().createTable(htd);
807     TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
808     ExceptionInReseekRegionObserver.reset();
809     ExceptionInReseekRegionObserver.throwOnce.set(false); // throw exceptions in every retry
810     try (Table t = TEST_UTIL.getConnection().getTable(name)) {
811       TEST_UTIL.loadTable(t, FAMILY, false);
812       TEST_UTIL.getHBaseAdmin().flush(name);
813       TEST_UTIL.countRows(t, new Scan().addColumn(FAMILY, FAMILY));
814       fail("Should have thrown an exception");
815     } catch (DoNotRetryIOException expected) {
816       assertTrue(expected instanceof ScannerResetException);
817       // expected
818     }
819     assertTrue(ExceptionInReseekRegionObserver.reqCount.get() >= 3);
820   }
821 
822   /*
823    * @param key
824    * @return Scan with RowFilter that does LESS than passed key.
825    */
826   private Scan createScanWithRowFilter(final byte [] key) {
827     return createScanWithRowFilter(key, null, CompareFilter.CompareOp.LESS);
828   }
829 
830   /*
831    * @param key
832    * @param op
833    * @param startRow
834    * @return Scan with RowFilter that does CompareOp op on passed key.
835    */
836   private Scan createScanWithRowFilter(final byte [] key,
837       final byte [] startRow, CompareFilter.CompareOp op) {
838     // Make sure key is of some substance... non-null and > than first key.
839     assertTrue(key != null && key.length > 0 &&
840       Bytes.BYTES_COMPARATOR.compare(key, new byte [] {'a', 'a', 'a'}) >= 0);
841     LOG.info("Key=" + Bytes.toString(key));
842     Scan s = startRow == null? new Scan(): new Scan(startRow);
843     Filter f = new RowFilter(op, new BinaryComparator(key));
844     f = new WhileMatchFilter(f);
845     s.setFilter(f);
846     return s;
847   }
848 
849   /*
850    * @param t
851    * @param s
852    * @return Count of rows in table.
853    * @throws IOException
854    */
855   private int countRows(final Table t, final Scan s)
856   throws IOException {
857     // Assert all rows in table.
858     ResultScanner scanner = t.getScanner(s);
859     int count = 0;
860     for (Result result: scanner) {
861       count++;
862       assertTrue(result.size() > 0);
863       // LOG.info("Count=" + count + ", row=" + Bytes.toString(result.getRow()));
864     }
865     return count;
866   }
867 
868   private void assertRowCount(final Table t, final int expected)
869   throws IOException {
870     assertEquals(expected, countRows(t, new Scan()));
871   }
872 
873   /*
874    * Split table into multiple regions.
875    * @param t Table to split.
876    * @return Map of regions to servers.
877    * @throws IOException
878    */
879   private Map<HRegionInfo, ServerName> splitTable(final HTable t)
880   throws IOException, InterruptedException {
881     // Split this table in two.
882     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
883     admin.split(t.getTableName());
884     admin.close();
885     Map<HRegionInfo, ServerName> regions = waitOnSplit(t);
886     assertTrue(regions.size() > 1);
887     return regions;
888   }
889 
890   /*
891    * Wait on table split.  May return because we waited long enough on the split
892    * and it didn't happen.  Caller should check.
893    * @param t
894    * @return Map of table regions; caller needs to check table actually split.
895    */
896   private Map<HRegionInfo, ServerName> waitOnSplit(final HTable t)
897   throws IOException {
898     Map<HRegionInfo, ServerName> regions = t.getRegionLocations();
899     int originalCount = regions.size();
900     for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) {
901       Thread.currentThread();
902       try {
903         Thread.sleep(1000);
904       } catch (InterruptedException e) {
905         e.printStackTrace();
906       }
907       regions = t.getRegionLocations();
908       if (regions.size() > originalCount) break;
909     }
910     return regions;
911   }
912 
913   @Test
914   public void testSuperSimple() throws Exception {
915     byte [] TABLE = Bytes.toBytes("testSuperSimple");
916     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
917     TEST_UTIL.waitTableAvailable(TABLE, 10000);
918     Put put = new Put(ROW);
919     put.add(FAMILY, QUALIFIER, VALUE);
920     ht.put(put);
921     Scan scan = new Scan();
922     scan.addColumn(FAMILY, TABLE);
923     ResultScanner scanner = ht.getScanner(scan);
924     Result result = scanner.next();
925     assertTrue("Expected null result", result == null);
926     scanner.close();
927   }
928 
929   @Test
930   public void testMaxKeyValueSize() throws Exception {
931     byte [] TABLE = Bytes.toBytes("testMaxKeyValueSize");
932     Configuration conf = TEST_UTIL.getConfiguration();
933     String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY);
934     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
935     TEST_UTIL.waitTableAvailable(TABLE, 10000);
936     byte[] value = new byte[4 * 1024 * 1024];
937     Put put = new Put(ROW);
938     put.add(FAMILY, QUALIFIER, value);
939     ht.put(put);
940     try {
941       TEST_UTIL.getConfiguration().setInt(
942           ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, 2 * 1024 * 1024);
943       // Create new table so we pick up the change in Configuration.
944       try (Connection connection =
945           ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
946         try (Table t = connection.getTable(TableName.valueOf(FAMILY))) {
947           put = new Put(ROW);
948           put.add(FAMILY, QUALIFIER, value);
949           t.put(put);
950         }
951       }
952       fail("Inserting a too large KeyValue worked, should throw exception");
953     } catch(Exception e) {}
954     conf.set(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, oldMaxSize);
955   }
956 
957   @Test
958   public void testFilters() throws Exception {
959     byte [] TABLE = Bytes.toBytes("testFilters");
960     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
961     TEST_UTIL.waitTableAvailable(TABLE, 10000);
962     byte [][] ROWS = makeN(ROW, 10);
963     byte [][] QUALIFIERS = {
964         Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
965         Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"),
966         Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"),
967         Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"),
968         Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>")
969     };
970     for(int i=0;i<10;i++) {
971       Put put = new Put(ROWS[i]);
972       put.setDurability(Durability.SKIP_WAL);
973       put.add(FAMILY, QUALIFIERS[i], VALUE);
974       ht.put(put);
975     }
976     Scan scan = new Scan();
977     scan.addFamily(FAMILY);
978     Filter filter = new QualifierFilter(CompareOp.EQUAL,
979       new RegexStringComparator("col[1-5]"));
980     scan.setFilter(filter);
981     ResultScanner scanner = ht.getScanner(scan);
982     int expectedIndex = 1;
983     for(Result result : ht.getScanner(scan)) {
984       assertEquals(result.size(), 1);
985       assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[expectedIndex]));
986       assertTrue(Bytes.equals(CellUtil.cloneQualifier(result.rawCells()[0]),
987           QUALIFIERS[expectedIndex]));
988       expectedIndex++;
989     }
990     assertEquals(expectedIndex, 6);
991     scanner.close();
992   }
993 
994   @Test
995   public void testFilterWithLongCompartor() throws Exception {
996     byte [] TABLE = Bytes.toBytes("testFilterWithLongCompartor");
997     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
998     TEST_UTIL.waitTableAvailable(TABLE, 10000);
999     byte [][] ROWS = makeN(ROW, 10);
1000     byte [][] values = new byte[10][];
1001     for (int i = 0; i < 10; i ++) {
1002         values[i] = Bytes.toBytes(100L * i);
1003     }
1004     for(int i = 0; i < 10; i ++) {
1005       Put put = new Put(ROWS[i]);
1006       put.setDurability(Durability.SKIP_WAL);
1007       put.add(FAMILY, QUALIFIER, values[i]);
1008       ht.put(put);
1009     }
1010     Scan scan = new Scan();
1011     scan.addFamily(FAMILY);
1012     Filter filter = new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOp.GREATER,
1013       new LongComparator(500));
1014     scan.setFilter(filter);
1015     ResultScanner scanner = ht.getScanner(scan);
1016     int expectedIndex = 0;
1017     for(Result result : ht.getScanner(scan)) {
1018       assertEquals(result.size(), 1);
1019       assertTrue(Bytes.toLong(result.getValue(FAMILY, QUALIFIER)) > 500);
1020       expectedIndex++;
1021     }
1022     assertEquals(expectedIndex, 4);
1023     scanner.close();
1024 }
1025 
1026   @Test
1027   public void testKeyOnlyFilter() throws Exception {
1028     byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
1029     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
1030     TEST_UTIL.waitTableAvailable(TABLE, 10000);
1031     byte [][] ROWS = makeN(ROW, 10);
1032     byte [][] QUALIFIERS = {
1033         Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
1034         Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"),
1035         Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"),
1036         Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"),
1037         Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>")
1038     };
1039     for(int i=0;i<10;i++) {
1040       Put put = new Put(ROWS[i]);
1041       put.setDurability(Durability.SKIP_WAL);
1042       put.add(FAMILY, QUALIFIERS[i], VALUE);
1043       ht.put(put);
1044     }
1045     Scan scan = new Scan();
1046     scan.addFamily(FAMILY);
1047     Filter filter = new KeyOnlyFilter(true);
1048     scan.setFilter(filter);
1049     ResultScanner scanner = ht.getScanner(scan);
1050     int count = 0;
1051     for(Result result : ht.getScanner(scan)) {
1052       assertEquals(result.size(), 1);
1053       assertEquals(result.rawCells()[0].getValueLength(), Bytes.SIZEOF_INT);
1054       assertEquals(Bytes.toInt(CellUtil.cloneValue(result.rawCells()[0])), VALUE.length);
1055       count++;
1056     }
1057     assertEquals(count, 10);
1058     scanner.close();
1059   }
1060 
1061   /**
1062    * Test simple table and non-existent row cases.
1063    */
1064   @Test
1065   public void testSimpleMissing() throws Exception {
1066     byte [] TABLE = Bytes.toBytes("testSimpleMissing");
1067     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
1068     TEST_UTIL.waitTableAvailable(TABLE, 10000);
1069     byte [][] ROWS = makeN(ROW, 4);
1070 
1071     // Try to get a row on an empty table
1072     Get get = new Get(ROWS[0]);
1073     Result result = ht.get(get);
1074     assertEmptyResult(result);
1075 
1076     get = new Get(ROWS[0]);
1077     get.addFamily(FAMILY);
1078     result = ht.get(get);
1079     assertEmptyResult(result);
1080 
1081     get = new Get(ROWS[0]);
1082     get.addColumn(FAMILY, QUALIFIER);
1083     result = ht.get(get);
1084     assertEmptyResult(result);
1085 
1086     Scan scan = new Scan();
1087     result = getSingleScanResult(ht, scan);
1088     assertNullResult(result);
1089 
1090 
1091     scan = new Scan(ROWS[0]);
1092     result = getSingleScanResult(ht, scan);
1093     assertNullResult(result);
1094 
1095     scan = new Scan(ROWS[0],ROWS[1]);
1096     result = getSingleScanResult(ht, scan);
1097     assertNullResult(result);
1098 
1099     scan = new Scan();
1100     scan.addFamily(FAMILY);
1101     result = getSingleScanResult(ht, scan);
1102     assertNullResult(result);
1103 
1104     scan = new Scan();
1105     scan.addColumn(FAMILY, QUALIFIER);
1106     result = getSingleScanResult(ht, scan);
1107     assertNullResult(result);
1108 
1109     // Insert a row
1110 
1111     Put put = new Put(ROWS[2]);
1112     put.add(FAMILY, QUALIFIER, VALUE);
1113     ht.put(put);
1114 
1115     // Try to get empty rows around it
1116 
1117     get = new Get(ROWS[1]);
1118     result = ht.get(get);
1119     assertEmptyResult(result);
1120 
1121     get = new Get(ROWS[0]);
1122     get.addFamily(FAMILY);
1123     result = ht.get(get);
1124     assertEmptyResult(result);
1125 
1126     get = new Get(ROWS[3]);
1127     get.addColumn(FAMILY, QUALIFIER);
1128     result = ht.get(get);
1129     assertEmptyResult(result);
1130 
1131     // Try to scan empty rows around it
1132 
1133     scan = new Scan(ROWS[3]);
1134     result = getSingleScanResult(ht, scan);
1135     assertNullResult(result);
1136 
1137     scan = new Scan(ROWS[0],ROWS[2]);
1138     result = getSingleScanResult(ht, scan);
1139     assertNullResult(result);
1140 
1141     // Make sure we can actually get the row
1142 
1143     get = new Get(ROWS[2]);
1144     result = ht.get(get);
1145     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
1146 
1147     get = new Get(ROWS[2]);
1148     get.addFamily(FAMILY);
1149     result = ht.get(get);
1150     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
1151 
1152     get = new Get(ROWS[2]);
1153     get.addColumn(FAMILY, QUALIFIER);
1154     result = ht.get(get);
1155     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
1156 
1157     // Make sure we can scan the row
1158 
1159     scan = new Scan();
1160     result = getSingleScanResult(ht, scan);
1161     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
1162 
1163     scan = new Scan(ROWS[0],ROWS[3]);
1164     result = getSingleScanResult(ht, scan);
1165     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
1166 
1167     scan = new Scan(ROWS[2],ROWS[3]);
1168     result = getSingleScanResult(ht, scan);
1169     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
1170   }
1171 
1172   /**
1173    * Test basic puts, gets, scans, and deletes for a single row
1174    * in a multiple family table.
1175    */
1176   @Test
1177   public void testSingleRowMultipleFamily() throws Exception {
1178     byte [] TABLE = Bytes.toBytes("testSingleRowMultipleFamily");
1179     byte [][] ROWS = makeN(ROW, 3);
1180     byte [][] FAMILIES = makeNAscii(FAMILY, 10);
1181     byte [][] QUALIFIERS = makeN(QUALIFIER, 10);
1182     byte [][] VALUES = makeN(VALUE, 10);
1183 
1184     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
1185     TEST_UTIL.waitTableAvailable(TABLE, 10000);
1186 
1187     Get get;
1188     Scan scan;
1189     Delete delete;
1190     Put put;
1191     Result result;
1192 
1193     ////////////////////////////////////////////////////////////////////////////
1194     // Insert one column to one family
1195     ////////////////////////////////////////////////////////////////////////////
1196 
1197     put = new Put(ROWS[0]);
1198     put.add(FAMILIES[4], QUALIFIERS[0], VALUES[0]);
1199     ht.put(put);
1200 
1201     // Get the single column
1202     getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1203 
1204     // Scan the single column
1205     scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1206 
1207     // Get empty results around inserted column
1208     getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1209 
1210     // Scan empty results around inserted column
1211     scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1212 
1213     ////////////////////////////////////////////////////////////////////////////
1214     // Flush memstore and run same tests from storefiles
1215     ////////////////////////////////////////////////////////////////////////////
1216 
1217     TEST_UTIL.flush();
1218 
1219     // Redo get and scan tests from storefile
1220     getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1221     scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0);
1222     getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1223     scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
1224 
1225     ////////////////////////////////////////////////////////////////////////////
1226     // Now, Test reading from memstore and storefiles at once
1227     ////////////////////////////////////////////////////////////////////////////
1228 
1229     // Insert multiple columns to two other families
1230     put = new Put(ROWS[0]);
1231     put.add(FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1232     put.add(FAMILIES[2], QUALIFIERS[4], VALUES[4]);
1233     put.add(FAMILIES[4], QUALIFIERS[4], VALUES[4]);
1234     put.add(FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1235     put.add(FAMILIES[6], QUALIFIERS[7], VALUES[7]);
1236     put.add(FAMILIES[7], QUALIFIERS[7], VALUES[7]);
1237     put.add(FAMILIES[9], QUALIFIERS[0], VALUES[0]);
1238     ht.put(put);
1239 
1240     // Get multiple columns across multiple families and get empties around it
1241     singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1242 
1243     // Scan multiple columns across multiple families and scan empties around it
1244     singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1245 
1246     ////////////////////////////////////////////////////////////////////////////
1247     // Flush the table again
1248     ////////////////////////////////////////////////////////////////////////////
1249 
1250     TEST_UTIL.flush();
1251 
1252     // Redo tests again
1253     singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1254     singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
1255 
1256     // Insert more data to memstore
1257     put = new Put(ROWS[0]);
1258     put.add(FAMILIES[6], QUALIFIERS[5], VALUES[5]);
1259     put.add(FAMILIES[6], QUALIFIERS[8], VALUES[8]);
1260     put.add(FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1261     put.add(FAMILIES[4], QUALIFIERS[3], VALUES[3]);
1262     ht.put(put);
1263 
1264     ////////////////////////////////////////////////////////////////////////////
1265     // Delete a storefile column
1266     ////////////////////////////////////////////////////////////////////////////
1267     delete = new Delete(ROWS[0]);
1268     delete.deleteColumns(FAMILIES[6], QUALIFIERS[7]);
1269     ht.delete(delete);
1270 
1271     // Try to get deleted column
1272     get = new Get(ROWS[0]);
1273     get.addColumn(FAMILIES[6], QUALIFIERS[7]);
1274     result = ht.get(get);
1275     assertEmptyResult(result);
1276 
1277     // Try to scan deleted column
1278     scan = new Scan();
1279     scan.addColumn(FAMILIES[6], QUALIFIERS[7]);
1280     result = getSingleScanResult(ht, scan);
1281     assertNullResult(result);
1282 
1283     // Make sure we can still get a column before it and after it
1284     get = new Get(ROWS[0]);
1285     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
1286     result = ht.get(get);
1287     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1288 
1289     get = new Get(ROWS[0]);
1290     get.addColumn(FAMILIES[6], QUALIFIERS[8]);
1291     result = ht.get(get);
1292     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
1293 
1294     // Make sure we can still scan a column before it and after it
1295     scan = new Scan();
1296     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1297     result = getSingleScanResult(ht, scan);
1298     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1299 
1300     scan = new Scan();
1301     scan.addColumn(FAMILIES[6], QUALIFIERS[8]);
1302     result = getSingleScanResult(ht, scan);
1303     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
1304 
1305     ////////////////////////////////////////////////////////////////////////////
1306     // Delete a memstore column
1307     ////////////////////////////////////////////////////////////////////////////
1308     delete = new Delete(ROWS[0]);
1309     delete.deleteColumns(FAMILIES[6], QUALIFIERS[8]);
1310     ht.delete(delete);
1311 
1312     // Try to get deleted column
1313     get = new Get(ROWS[0]);
1314     get.addColumn(FAMILIES[6], QUALIFIERS[8]);
1315     result = ht.get(get);
1316     assertEmptyResult(result);
1317 
1318     // Try to scan deleted column
1319     scan = new Scan();
1320     scan.addColumn(FAMILIES[6], QUALIFIERS[8]);
1321     result = getSingleScanResult(ht, scan);
1322     assertNullResult(result);
1323 
1324     // Make sure we can still get a column before it and after it
1325     get = new Get(ROWS[0]);
1326     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
1327     result = ht.get(get);
1328     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1329 
1330     get = new Get(ROWS[0]);
1331     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1332     result = ht.get(get);
1333     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1334 
1335     // Make sure we can still scan a column before it and after it
1336     scan = new Scan();
1337     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1338     result = getSingleScanResult(ht, scan);
1339     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1340 
1341     scan = new Scan();
1342     scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1343     result = getSingleScanResult(ht, scan);
1344     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1345 
1346     ////////////////////////////////////////////////////////////////////////////
1347     // Delete joint storefile/memstore family
1348     ////////////////////////////////////////////////////////////////////////////
1349 
1350     delete = new Delete(ROWS[0]);
1351     delete.deleteFamily(FAMILIES[4]);
1352     ht.delete(delete);
1353 
1354     // Try to get storefile column in deleted family
1355     get = new Get(ROWS[0]);
1356     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
1357     result = ht.get(get);
1358     assertEmptyResult(result);
1359 
1360     // Try to get memstore column in deleted family
1361     get = new Get(ROWS[0]);
1362     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
1363     result = ht.get(get);
1364     assertEmptyResult(result);
1365 
1366     // Try to get deleted family
1367     get = new Get(ROWS[0]);
1368     get.addFamily(FAMILIES[4]);
1369     result = ht.get(get);
1370     assertEmptyResult(result);
1371 
1372     // Try to scan storefile column in deleted family
1373     scan = new Scan();
1374     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
1375     result = getSingleScanResult(ht, scan);
1376     assertNullResult(result);
1377 
1378     // Try to scan memstore column in deleted family
1379     scan = new Scan();
1380     scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
1381     result = getSingleScanResult(ht, scan);
1382     assertNullResult(result);
1383 
1384     // Try to scan deleted family
1385     scan = new Scan();
1386     scan.addFamily(FAMILIES[4]);
1387     result = getSingleScanResult(ht, scan);
1388     assertNullResult(result);
1389 
1390     // Make sure we can still get another family
1391     get = new Get(ROWS[0]);
1392     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
1393     result = ht.get(get);
1394     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1395 
1396     get = new Get(ROWS[0]);
1397     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1398     result = ht.get(get);
1399     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1400 
1401     // Make sure we can still scan another family
1402     scan = new Scan();
1403     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1404     result = getSingleScanResult(ht, scan);
1405     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1406 
1407     scan = new Scan();
1408     scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1409     result = getSingleScanResult(ht, scan);
1410     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1411 
1412     ////////////////////////////////////////////////////////////////////////////
1413     // Flush everything and rerun delete tests
1414     ////////////////////////////////////////////////////////////////////////////
1415 
1416     TEST_UTIL.flush();
1417 
1418     // Try to get storefile column in deleted family
1419     get = new Get(ROWS[0]);
1420     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
1421     result = ht.get(get);
1422     assertEmptyResult(result);
1423 
1424     // Try to get memstore column in deleted family
1425     get = new Get(ROWS[0]);
1426     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
1427     result = ht.get(get);
1428     assertEmptyResult(result);
1429 
1430     // Try to get deleted family
1431     get = new Get(ROWS[0]);
1432     get.addFamily(FAMILIES[4]);
1433     result = ht.get(get);
1434     assertEmptyResult(result);
1435 
1436     // Try to scan storefile column in deleted family
1437     scan = new Scan();
1438     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
1439     result = getSingleScanResult(ht, scan);
1440     assertNullResult(result);
1441 
1442     // Try to scan memstore column in deleted family
1443     scan = new Scan();
1444     scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
1445     result = getSingleScanResult(ht, scan);
1446     assertNullResult(result);
1447 
1448     // Try to scan deleted family
1449     scan = new Scan();
1450     scan.addFamily(FAMILIES[4]);
1451     result = getSingleScanResult(ht, scan);
1452     assertNullResult(result);
1453 
1454     // Make sure we can still get another family
1455     get = new Get(ROWS[0]);
1456     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
1457     result = ht.get(get);
1458     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
1459 
1460     get = new Get(ROWS[0]);
1461     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
1462     result = ht.get(get);
1463     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1464 
1465     // Make sure we can still scan another family
1466     scan = new Scan();
1467     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
1468     result = getSingleScanResult(ht, scan);
1469     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
1470 
1471     scan = new Scan();
1472     scan.addColumn(FAMILIES[6], QUALIFIERS[9]);
1473     result = getSingleScanResult(ht, scan);
1474     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
1475 
1476   }
1477 
1478   @Test
1479   public void testNull() throws Exception {
1480     byte [] TABLE = Bytes.toBytes("testNull");
1481 
1482     // Null table name (should NOT work)
1483     try {
1484       TEST_UTIL.createTable((TableName)null, FAMILY);
1485       fail("Creating a table with null name passed, should have failed");
1486     } catch(Exception e) {}
1487 
1488     // Null family (should NOT work)
1489     try {
1490       TEST_UTIL.createTable(TABLE, new byte[][]{(byte[])null});
1491       fail("Creating a table with a null family passed, should fail");
1492     } catch(Exception e) {}
1493 
1494     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
1495     TEST_UTIL.waitTableAvailable(TABLE, 10000);
1496 
1497     // Null row (should NOT work)
1498     try {
1499       Put put = new Put((byte[])null);
1500       put.add(FAMILY, QUALIFIER, VALUE);
1501       ht.put(put);
1502       fail("Inserting a null row worked, should throw exception");
1503     } catch(Exception e) {}
1504 
1505     // Null qualifier (should work)
1506     {
1507       Put put = new Put(ROW);
1508       put.add(FAMILY, null, VALUE);
1509       ht.put(put);
1510 
1511       getTestNull(ht, ROW, FAMILY, VALUE);
1512 
1513       scanTestNull(ht, ROW, FAMILY, VALUE);
1514 
1515       Delete delete = new Delete(ROW);
1516       delete.deleteColumns(FAMILY, null);
1517       ht.delete(delete);
1518 
1519       Get get = new Get(ROW);
1520       Result result = ht.get(get);
1521       assertEmptyResult(result);
1522     }
1523 
1524     // Use a new table
1525     byte [] TABLE2 = Bytes.toBytes("testNull2");
1526     ht = TEST_UTIL.createTable(TableName.valueOf(TABLE2), FAMILY);
1527     TEST_UTIL.waitTableAvailable(TableName.valueOf(TABLE2), 10000);
1528     // Empty qualifier, byte[0] instead of null (should work)
1529     try {
1530       Put put = new Put(ROW);
1531       put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
1532       ht.put(put);
1533 
1534       getTestNull(ht, ROW, FAMILY, VALUE);
1535 
1536       scanTestNull(ht, ROW, FAMILY, VALUE);
1537 
1538       // Flush and try again
1539 
1540       TEST_UTIL.flush();
1541 
1542       getTestNull(ht, ROW, FAMILY, VALUE);
1543 
1544       scanTestNull(ht, ROW, FAMILY, VALUE);
1545 
1546       Delete delete = new Delete(ROW);
1547       delete.deleteColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY);
1548       ht.delete(delete);
1549 
1550       Get get = new Get(ROW);
1551       Result result = ht.get(get);
1552       assertEmptyResult(result);
1553 
1554     } catch(Exception e) {
1555       throw new IOException("Using a row with null qualifier threw exception, should ");
1556     }
1557 
1558     // Null value
1559     try {
1560       Put put = new Put(ROW);
1561       put.add(FAMILY, QUALIFIER, null);
1562       ht.put(put);
1563 
1564       Get get = new Get(ROW);
1565       get.addColumn(FAMILY, QUALIFIER);
1566       Result result = ht.get(get);
1567       assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
1568 
1569       Scan scan = new Scan();
1570       scan.addColumn(FAMILY, QUALIFIER);
1571       result = getSingleScanResult(ht, scan);
1572       assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
1573 
1574       Delete delete = new Delete(ROW);
1575       delete.deleteColumns(FAMILY, QUALIFIER);
1576       ht.delete(delete);
1577 
1578       get = new Get(ROW);
1579       result = ht.get(get);
1580       assertEmptyResult(result);
1581 
1582     } catch(Exception e) {
1583       throw new IOException("Null values should be allowed, but threw exception");
1584     }
1585   }
1586 
1587   @Test
1588   public void testVersions() throws Exception {
1589     byte [] TABLE = Bytes.toBytes("testVersions");
1590 
1591     long [] STAMPS = makeStamps(20);
1592     byte [][] VALUES = makeNAscii(VALUE, 20);
1593 
1594     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
1595     TEST_UTIL.waitTableAvailable(TABLE, 10000);
1596     // Insert 4 versions of same column
1597     Put put = new Put(ROW);
1598     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1599     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1600     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1601     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1602     ht.put(put);
1603 
1604     // Verify we can get each one properly
1605     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1606     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1607     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1608     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1609     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1610     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1611     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1612     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1613 
1614     // Verify we don't accidentally get others
1615     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1616     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1617     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1618     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1619     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1620     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1621 
1622     // Ensure maxVersions in query is respected
1623     Get get = new Get(ROW);
1624     get.addColumn(FAMILY, QUALIFIER);
1625     get.setMaxVersions(2);
1626     Result result = ht.get(get);
1627     assertNResult(result, ROW, FAMILY, QUALIFIER,
1628         new long [] {STAMPS[4], STAMPS[5]},
1629         new byte[][] {VALUES[4], VALUES[5]},
1630         0, 1);
1631 
1632     Scan scan = new Scan(ROW);
1633     scan.addColumn(FAMILY, QUALIFIER);
1634     scan.setMaxVersions(2);
1635     result = getSingleScanResult(ht, scan);
1636     assertNResult(result, ROW, FAMILY, QUALIFIER,
1637         new long [] {STAMPS[4], STAMPS[5]},
1638         new byte[][] {VALUES[4], VALUES[5]},
1639         0, 1);
1640 
1641     // Flush and redo
1642 
1643     TEST_UTIL.flush();
1644 
1645     // Verify we can get each one properly
1646     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1647     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1648     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1649     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1650     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1651     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1652     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1653     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
1654 
1655     // Verify we don't accidentally get others
1656     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1657     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1658     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1659     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1660     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
1661     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
1662 
1663     // Ensure maxVersions in query is respected
1664     get = new Get(ROW);
1665     get.addColumn(FAMILY, QUALIFIER);
1666     get.setMaxVersions(2);
1667     result = ht.get(get);
1668     assertNResult(result, ROW, FAMILY, QUALIFIER,
1669         new long [] {STAMPS[4], STAMPS[5]},
1670         new byte[][] {VALUES[4], VALUES[5]},
1671         0, 1);
1672 
1673     scan = new Scan(ROW);
1674     scan.addColumn(FAMILY, QUALIFIER);
1675     scan.setMaxVersions(2);
1676     result = getSingleScanResult(ht, scan);
1677     assertNResult(result, ROW, FAMILY, QUALIFIER,
1678         new long [] {STAMPS[4], STAMPS[5]},
1679         new byte[][] {VALUES[4], VALUES[5]},
1680         0, 1);
1681 
1682 
1683     // Add some memstore and retest
1684 
1685     // Insert 4 more versions of same column and a dupe
1686     put = new Put(ROW);
1687     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
1688     put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
1689     put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1690     put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]);
1691     ht.put(put);
1692 
1693     // Ensure maxVersions in query is respected
1694     get = new Get(ROW);
1695     get.addColumn(FAMILY, QUALIFIER);
1696     get.setMaxVersions();
1697     result = ht.get(get);
1698     assertNResult(result, ROW, FAMILY, QUALIFIER,
1699         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1700         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1701         0, 7);
1702 
1703     scan = new Scan(ROW);
1704     scan.addColumn(FAMILY, QUALIFIER);
1705     scan.setMaxVersions();
1706     result = getSingleScanResult(ht, scan);
1707     assertNResult(result, ROW, FAMILY, QUALIFIER,
1708         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1709         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1710         0, 7);
1711 
1712     get = new Get(ROW);
1713     get.setMaxVersions();
1714     result = ht.get(get);
1715     assertNResult(result, ROW, FAMILY, QUALIFIER,
1716         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1717         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1718         0, 7);
1719 
1720     scan = new Scan(ROW);
1721     scan.setMaxVersions();
1722     result = getSingleScanResult(ht, scan);
1723     assertNResult(result, ROW, FAMILY, QUALIFIER,
1724         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
1725         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
1726         0, 7);
1727 
1728     // Verify we can get each one properly
1729     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1730     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1731     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1732     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1733     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
1734     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
1735     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
1736     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
1737 
1738     // Verify we don't accidentally get others
1739     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1740     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
1741     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
1742     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
1743 
1744     // Ensure maxVersions of table is respected
1745 
1746     TEST_UTIL.flush();
1747 
1748     // Insert 4 more versions of same column and a dupe
1749     put = new Put(ROW);
1750     put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]);
1751     put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]);
1752     put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]);
1753     put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
1754     ht.put(put);
1755 
1756     get = new Get(ROW);
1757     get.addColumn(FAMILY, QUALIFIER);
1758     get.setMaxVersions(Integer.MAX_VALUE);
1759     result = ht.get(get);
1760     assertNResult(result, ROW, FAMILY, QUALIFIER,
1761         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
1762         new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
1763         0, 9);
1764 
1765     scan = new Scan(ROW);
1766     scan.addColumn(FAMILY, QUALIFIER);
1767     scan.setMaxVersions(Integer.MAX_VALUE);
1768     result = getSingleScanResult(ht, scan);
1769     assertNResult(result, ROW, FAMILY, QUALIFIER,
1770         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
1771         new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
1772         0, 9);
1773 
1774     // Delete a version in the memstore and a version in a storefile
1775     Delete delete = new Delete(ROW);
1776     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
1777     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
1778     ht.delete(delete);
1779 
1780     // Test that it's gone
1781     get = new Get(ROW);
1782     get.addColumn(FAMILY, QUALIFIER);
1783     get.setMaxVersions(Integer.MAX_VALUE);
1784     result = ht.get(get);
1785     assertNResult(result, ROW, FAMILY, QUALIFIER,
1786         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
1787         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
1788         0, 9);
1789 
1790     scan = new Scan(ROW);
1791     scan.addColumn(FAMILY, QUALIFIER);
1792     scan.setMaxVersions(Integer.MAX_VALUE);
1793     result = getSingleScanResult(ht, scan);
1794     assertNResult(result, ROW, FAMILY, QUALIFIER,
1795         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
1796         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
1797         0, 9);
1798 
1799   }
1800 
1801   @Test
1802   public void testVersionLimits() throws Exception {
1803     byte [] TABLE = Bytes.toBytes("testVersionLimits");
1804     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
1805     int [] LIMITS = {1,3,5};
1806     long [] STAMPS = makeStamps(10);
1807     byte [][] VALUES = makeNAscii(VALUE, 10);
1808     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS);
1809     TEST_UTIL.waitTableAvailable(TABLE, 10000);
1810     // Insert limit + 1 on each family
1811     Put put = new Put(ROW);
1812     put.add(FAMILIES[0], QUALIFIER, STAMPS[0], VALUES[0]);
1813     put.add(FAMILIES[0], QUALIFIER, STAMPS[1], VALUES[1]);
1814     put.add(FAMILIES[1], QUALIFIER, STAMPS[0], VALUES[0]);
1815     put.add(FAMILIES[1], QUALIFIER, STAMPS[1], VALUES[1]);
1816     put.add(FAMILIES[1], QUALIFIER, STAMPS[2], VALUES[2]);
1817     put.add(FAMILIES[1], QUALIFIER, STAMPS[3], VALUES[3]);
1818     put.add(FAMILIES[2], QUALIFIER, STAMPS[0], VALUES[0]);
1819     put.add(FAMILIES[2], QUALIFIER, STAMPS[1], VALUES[1]);
1820     put.add(FAMILIES[2], QUALIFIER, STAMPS[2], VALUES[2]);
1821     put.add(FAMILIES[2], QUALIFIER, STAMPS[3], VALUES[3]);
1822     put.add(FAMILIES[2], QUALIFIER, STAMPS[4], VALUES[4]);
1823     put.add(FAMILIES[2], QUALIFIER, STAMPS[5], VALUES[5]);
1824     put.add(FAMILIES[2], QUALIFIER, STAMPS[6], VALUES[6]);
1825     ht.put(put);
1826 
1827     // Verify we only get the right number out of each
1828 
1829     // Family0
1830 
1831     Get get = new Get(ROW);
1832     get.addColumn(FAMILIES[0], QUALIFIER);
1833     get.setMaxVersions(Integer.MAX_VALUE);
1834     Result result = ht.get(get);
1835     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1836         new long [] {STAMPS[1]},
1837         new byte[][] {VALUES[1]},
1838         0, 0);
1839 
1840     get = new Get(ROW);
1841     get.addFamily(FAMILIES[0]);
1842     get.setMaxVersions(Integer.MAX_VALUE);
1843     result = ht.get(get);
1844     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1845         new long [] {STAMPS[1]},
1846         new byte[][] {VALUES[1]},
1847         0, 0);
1848 
1849     Scan scan = new Scan(ROW);
1850     scan.addColumn(FAMILIES[0], QUALIFIER);
1851     scan.setMaxVersions(Integer.MAX_VALUE);
1852     result = getSingleScanResult(ht, scan);
1853     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1854         new long [] {STAMPS[1]},
1855         new byte[][] {VALUES[1]},
1856         0, 0);
1857 
1858     scan = new Scan(ROW);
1859     scan.addFamily(FAMILIES[0]);
1860     scan.setMaxVersions(Integer.MAX_VALUE);
1861     result = getSingleScanResult(ht, scan);
1862     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
1863         new long [] {STAMPS[1]},
1864         new byte[][] {VALUES[1]},
1865         0, 0);
1866 
1867     // Family1
1868 
1869     get = new Get(ROW);
1870     get.addColumn(FAMILIES[1], QUALIFIER);
1871     get.setMaxVersions(Integer.MAX_VALUE);
1872     result = ht.get(get);
1873     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1874         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1875         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1876         0, 2);
1877 
1878     get = new Get(ROW);
1879     get.addFamily(FAMILIES[1]);
1880     get.setMaxVersions(Integer.MAX_VALUE);
1881     result = ht.get(get);
1882     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1883         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1884         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1885         0, 2);
1886 
1887     scan = new Scan(ROW);
1888     scan.addColumn(FAMILIES[1], QUALIFIER);
1889     scan.setMaxVersions(Integer.MAX_VALUE);
1890     result = getSingleScanResult(ht, scan);
1891     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1892         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1893         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1894         0, 2);
1895 
1896     scan = new Scan(ROW);
1897     scan.addFamily(FAMILIES[1]);
1898     scan.setMaxVersions(Integer.MAX_VALUE);
1899     result = getSingleScanResult(ht, scan);
1900     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
1901         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
1902         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
1903         0, 2);
1904 
1905     // Family2
1906 
1907     get = new Get(ROW);
1908     get.addColumn(FAMILIES[2], QUALIFIER);
1909     get.setMaxVersions(Integer.MAX_VALUE);
1910     result = ht.get(get);
1911     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1912         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1913         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1914         0, 4);
1915 
1916     get = new Get(ROW);
1917     get.addFamily(FAMILIES[2]);
1918     get.setMaxVersions(Integer.MAX_VALUE);
1919     result = ht.get(get);
1920     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1921         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1922         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1923         0, 4);
1924 
1925     scan = new Scan(ROW);
1926     scan.addColumn(FAMILIES[2], QUALIFIER);
1927     scan.setMaxVersions(Integer.MAX_VALUE);
1928     result = getSingleScanResult(ht, scan);
1929     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1930         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1931         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1932         0, 4);
1933 
1934     scan = new Scan(ROW);
1935     scan.addFamily(FAMILIES[2]);
1936     scan.setMaxVersions(Integer.MAX_VALUE);
1937     result = getSingleScanResult(ht, scan);
1938     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
1939         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
1940         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
1941         0, 4);
1942 
1943     // Try all families
1944 
1945     get = new Get(ROW);
1946     get.setMaxVersions(Integer.MAX_VALUE);
1947     result = ht.get(get);
1948     assertTrue("Expected 9 keys but received " + result.size(),
1949         result.size() == 9);
1950 
1951     get = new Get(ROW);
1952     get.addFamily(FAMILIES[0]);
1953     get.addFamily(FAMILIES[1]);
1954     get.addFamily(FAMILIES[2]);
1955     get.setMaxVersions(Integer.MAX_VALUE);
1956     result = ht.get(get);
1957     assertTrue("Expected 9 keys but received " + result.size(),
1958         result.size() == 9);
1959 
1960     get = new Get(ROW);
1961     get.addColumn(FAMILIES[0], QUALIFIER);
1962     get.addColumn(FAMILIES[1], QUALIFIER);
1963     get.addColumn(FAMILIES[2], QUALIFIER);
1964     get.setMaxVersions(Integer.MAX_VALUE);
1965     result = ht.get(get);
1966     assertTrue("Expected 9 keys but received " + result.size(),
1967         result.size() == 9);
1968 
1969     scan = new Scan(ROW);
1970     scan.setMaxVersions(Integer.MAX_VALUE);
1971     result = getSingleScanResult(ht, scan);
1972     assertTrue("Expected 9 keys but received " + result.size(),
1973         result.size() == 9);
1974 
1975     scan = new Scan(ROW);
1976     scan.setMaxVersions(Integer.MAX_VALUE);
1977     scan.addFamily(FAMILIES[0]);
1978     scan.addFamily(FAMILIES[1]);
1979     scan.addFamily(FAMILIES[2]);
1980     result = getSingleScanResult(ht, scan);
1981     assertTrue("Expected 9 keys but received " + result.size(),
1982         result.size() == 9);
1983 
1984     scan = new Scan(ROW);
1985     scan.setMaxVersions(Integer.MAX_VALUE);
1986     scan.addColumn(FAMILIES[0], QUALIFIER);
1987     scan.addColumn(FAMILIES[1], QUALIFIER);
1988     scan.addColumn(FAMILIES[2], QUALIFIER);
1989     result = getSingleScanResult(ht, scan);
1990     assertTrue("Expected 9 keys but received " + result.size(),
1991         result.size() == 9);
1992 
1993   }
1994 
1995   @Test
1996   public void testDeleteFamilyVersion() throws Exception {
1997     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
1998     byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersion");
1999 
2000     byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 1);
2001     byte [][] VALUES = makeN(VALUE, 5);
2002     long [] ts = {1000, 2000, 3000, 4000, 5000};
2003 
2004     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
2005     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2006 
2007     Put put = new Put(ROW);
2008     for (int q = 0; q < 1; q++)
2009       for (int t = 0; t < 5; t++)
2010         put.add(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]);
2011     ht.put(put);
2012     admin.flush(TABLE);
2013 
2014     Delete delete = new Delete(ROW);
2015     delete.deleteFamilyVersion(FAMILY, ts[1]);  // delete version '2000'
2016     delete.deleteFamilyVersion(FAMILY, ts[3]);  // delete version '4000'
2017     ht.delete(delete);
2018     admin.flush(TABLE);
2019 
2020     for (int i = 0; i < 1; i++) {
2021       Get get = new Get(ROW);
2022       get.addColumn(FAMILY, QUALIFIERS[i]);
2023       get.setMaxVersions(Integer.MAX_VALUE);
2024       Result result = ht.get(get);
2025       // verify version '1000'/'3000'/'5000' remains for all columns
2026       assertNResult(result, ROW, FAMILY, QUALIFIERS[i],
2027           new long [] {ts[0], ts[2], ts[4]},
2028           new byte[][] {VALUES[0], VALUES[2], VALUES[4]},
2029           0, 2);
2030     }
2031     ht.close();
2032     admin.close();
2033   }
2034 
2035   @Test
2036   public void testDeleteFamilyVersionWithOtherDeletes() throws Exception {
2037     byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersionWithOtherDeletes");
2038 
2039     byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 5);
2040     byte [][] VALUES = makeN(VALUE, 5);
2041     long [] ts = {1000, 2000, 3000, 4000, 5000};
2042 
2043     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
2044     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
2045     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2046 
2047     Put put = null;
2048     Result result = null;
2049     Get get = null;
2050     Delete delete = null;
2051 
2052     // 1. put on ROW
2053     put = new Put(ROW);
2054     for (int q = 0; q < 5; q++)
2055       for (int t = 0; t < 5; t++)
2056         put.add(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]);
2057     ht.put(put);
2058     admin.flush(TABLE);
2059 
2060     // 2. put on ROWS[0]
2061     byte [] ROW2 = Bytes.toBytes("myRowForTest");
2062     put = new Put(ROW2);
2063     for (int q = 0; q < 5; q++)
2064       for (int t = 0; t < 5; t++)
2065         put.add(FAMILY, QUALIFIERS[q], ts[t], VALUES[t]);
2066     ht.put(put);
2067     admin.flush(TABLE);
2068 
2069     // 3. delete on ROW
2070     delete = new Delete(ROW);
2071     // delete version <= 2000 of all columns
2072     // note: deleteFamily must be the first since it will mask
2073     // the subsequent other type deletes!
2074     delete.deleteFamily(FAMILY, ts[1]);
2075     // delete version '4000' of all columns
2076     delete.deleteFamilyVersion(FAMILY, ts[3]);
2077    // delete version <= 3000 of column 0
2078     delete.deleteColumns(FAMILY, QUALIFIERS[0], ts[2]);
2079     // delete version <= 5000 of column 2
2080     delete.deleteColumns(FAMILY, QUALIFIERS[2], ts[4]);
2081     // delete version 5000 of column 4
2082     delete.deleteColumn(FAMILY, QUALIFIERS[4], ts[4]);
2083     ht.delete(delete);
2084     admin.flush(TABLE);
2085 
2086      // 4. delete on ROWS[0]
2087     delete = new Delete(ROW2);
2088     delete.deleteFamilyVersion(FAMILY, ts[1]);  // delete version '2000'
2089     delete.deleteFamilyVersion(FAMILY, ts[3]);  // delete version '4000'
2090     ht.delete(delete);
2091     admin.flush(TABLE);
2092 
2093     // 5. check ROW
2094     get = new Get(ROW);
2095     get.addColumn(FAMILY, QUALIFIERS[0]);
2096     get.setMaxVersions(Integer.MAX_VALUE);
2097     result = ht.get(get);
2098     assertNResult(result, ROW, FAMILY, QUALIFIERS[0],
2099         new long [] {ts[4]},
2100         new byte[][] {VALUES[4]},
2101         0, 0);
2102 
2103     get = new Get(ROW);
2104     get.addColumn(FAMILY, QUALIFIERS[1]);
2105     get.setMaxVersions(Integer.MAX_VALUE);
2106     result = ht.get(get);
2107     assertNResult(result, ROW, FAMILY, QUALIFIERS[1],
2108         new long [] {ts[2], ts[4]},
2109         new byte[][] {VALUES[2], VALUES[4]},
2110         0, 1);
2111 
2112     get = new Get(ROW);
2113     get.addColumn(FAMILY, QUALIFIERS[2]);
2114     get.setMaxVersions(Integer.MAX_VALUE);
2115     result = ht.get(get);
2116     assertEquals(0, result.size());
2117 
2118     get = new Get(ROW);
2119     get.addColumn(FAMILY, QUALIFIERS[3]);
2120     get.setMaxVersions(Integer.MAX_VALUE);
2121     result = ht.get(get);
2122     assertNResult(result, ROW, FAMILY, QUALIFIERS[3],
2123         new long [] {ts[2], ts[4]},
2124         new byte[][] {VALUES[2], VALUES[4]},
2125         0, 1);
2126 
2127     get = new Get(ROW);
2128     get.addColumn(FAMILY, QUALIFIERS[4]);
2129     get.setMaxVersions(Integer.MAX_VALUE);
2130     result = ht.get(get);
2131     assertNResult(result, ROW, FAMILY, QUALIFIERS[4],
2132         new long [] {ts[2]},
2133         new byte[][] {VALUES[2]},
2134         0, 0);
2135 
2136     // 6. check ROWS[0]
2137     for (int i = 0; i < 5; i++) {
2138       get = new Get(ROW2);
2139       get.addColumn(FAMILY, QUALIFIERS[i]);
2140       get.setMaxVersions(Integer.MAX_VALUE);
2141       result = ht.get(get);
2142       // verify version '1000'/'3000'/'5000' remains for all columns
2143       assertNResult(result, ROW2, FAMILY, QUALIFIERS[i],
2144           new long [] {ts[0], ts[2], ts[4]},
2145           new byte[][] {VALUES[0], VALUES[2], VALUES[4]},
2146           0, 2);
2147     }
2148     ht.close();
2149     admin.close();
2150   }
2151 
2152   @Test
2153   public void testDeletes() throws Exception {
2154     byte [] TABLE = Bytes.toBytes("testDeletes");
2155 
2156     byte [][] ROWS = makeNAscii(ROW, 6);
2157     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
2158     byte [][] VALUES = makeN(VALUE, 5);
2159     long [] ts = {1000, 2000, 3000, 4000, 5000};
2160 
2161     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3);
2162     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2163 
2164     Put put = new Put(ROW);
2165     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
2166     put.add(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]);
2167     ht.put(put);
2168 
2169     Delete delete = new Delete(ROW);
2170     delete.deleteFamily(FAMILIES[0], ts[0]);
2171     ht.delete(delete);
2172 
2173     Get get = new Get(ROW);
2174     get.addFamily(FAMILIES[0]);
2175     get.setMaxVersions(Integer.MAX_VALUE);
2176     Result result = ht.get(get);
2177     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2178         new long [] {ts[1]},
2179         new byte[][] {VALUES[1]},
2180         0, 0);
2181 
2182     Scan scan = new Scan(ROW);
2183     scan.addFamily(FAMILIES[0]);
2184     scan.setMaxVersions(Integer.MAX_VALUE);
2185     result = getSingleScanResult(ht, scan);
2186     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2187         new long [] {ts[1]},
2188         new byte[][] {VALUES[1]},
2189         0, 0);
2190 
2191     // Test delete latest version
2192     put = new Put(ROW);
2193     put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]);
2194     put.add(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]);
2195     put.add(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]);
2196     put.add(FAMILIES[0], null, ts[4], VALUES[4]);
2197     put.add(FAMILIES[0], null, ts[2], VALUES[2]);
2198     put.add(FAMILIES[0], null, ts[3], VALUES[3]);
2199     ht.put(put);
2200 
2201     delete = new Delete(ROW);
2202     delete.deleteColumn(FAMILIES[0], QUALIFIER); // ts[4]
2203     ht.delete(delete);
2204 
2205     get = new Get(ROW);
2206     get.addColumn(FAMILIES[0], QUALIFIER);
2207     get.setMaxVersions(Integer.MAX_VALUE);
2208     result = ht.get(get);
2209     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2210         new long [] {ts[1], ts[2], ts[3]},
2211         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2212         0, 2);
2213 
2214     scan = new Scan(ROW);
2215     scan.addColumn(FAMILIES[0], QUALIFIER);
2216     scan.setMaxVersions(Integer.MAX_VALUE);
2217     result = getSingleScanResult(ht, scan);
2218     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2219         new long [] {ts[1], ts[2], ts[3]},
2220         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2221         0, 2);
2222 
2223     // Test for HBASE-1847
2224     delete = new Delete(ROW);
2225     delete.deleteColumn(FAMILIES[0], null);
2226     ht.delete(delete);
2227 
2228     // Cleanup null qualifier
2229     delete = new Delete(ROW);
2230     delete.deleteColumns(FAMILIES[0], null);
2231     ht.delete(delete);
2232 
2233     // Expected client behavior might be that you can re-put deleted values
2234     // But alas, this is not to be.  We can't put them back in either case.
2235 
2236     put = new Put(ROW);
2237     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); // 1000
2238     put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); // 5000
2239     ht.put(put);
2240 
2241 
2242     // It used to be due to the internal implementation of Get, that
2243     // the Get() call would return ts[4] UNLIKE the Scan below. With
2244     // the switch to using Scan for Get this is no longer the case.
2245     get = new Get(ROW);
2246     get.addFamily(FAMILIES[0]);
2247     get.setMaxVersions(Integer.MAX_VALUE);
2248     result = ht.get(get);
2249     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2250         new long [] {ts[1], ts[2], ts[3]},
2251         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2252         0, 2);
2253 
2254     // The Scanner returns the previous values, the expected-naive-unexpected behavior
2255 
2256     scan = new Scan(ROW);
2257     scan.addFamily(FAMILIES[0]);
2258     scan.setMaxVersions(Integer.MAX_VALUE);
2259     result = getSingleScanResult(ht, scan);
2260     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
2261         new long [] {ts[1], ts[2], ts[3]},
2262         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
2263         0, 2);
2264 
2265     // Test deleting an entire family from one row but not the other various ways
2266 
2267     put = new Put(ROWS[0]);
2268     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
2269     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
2270     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
2271     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
2272     ht.put(put);
2273 
2274     put = new Put(ROWS[1]);
2275     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
2276     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
2277     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
2278     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
2279     ht.put(put);
2280 
2281     put = new Put(ROWS[2]);
2282     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
2283     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
2284     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
2285     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
2286     ht.put(put);
2287 
2288     // Assert that above went in.
2289     get = new Get(ROWS[2]);
2290     get.addFamily(FAMILIES[1]);
2291     get.addFamily(FAMILIES[2]);
2292     get.setMaxVersions(Integer.MAX_VALUE);
2293     result = ht.get(get);
2294     assertTrue("Expected 4 key but received " + result.size() + ": " + result,
2295         result.size() == 4);
2296 
2297     delete = new Delete(ROWS[0]);
2298     delete.deleteFamily(FAMILIES[2]);
2299     ht.delete(delete);
2300 
2301     delete = new Delete(ROWS[1]);
2302     delete.deleteColumns(FAMILIES[1], QUALIFIER);
2303     ht.delete(delete);
2304 
2305     delete = new Delete(ROWS[2]);
2306     delete.deleteColumn(FAMILIES[1], QUALIFIER);
2307     delete.deleteColumn(FAMILIES[1], QUALIFIER);
2308     delete.deleteColumn(FAMILIES[2], QUALIFIER);
2309     ht.delete(delete);
2310 
2311     get = new Get(ROWS[0]);
2312     get.addFamily(FAMILIES[1]);
2313     get.addFamily(FAMILIES[2]);
2314     get.setMaxVersions(Integer.MAX_VALUE);
2315     result = ht.get(get);
2316     assertTrue("Expected 2 keys but received " + result.size(),
2317         result.size() == 2);
2318     assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER,
2319         new long [] {ts[0], ts[1]},
2320         new byte[][] {VALUES[0], VALUES[1]},
2321         0, 1);
2322 
2323     scan = new Scan(ROWS[0]);
2324     scan.addFamily(FAMILIES[1]);
2325     scan.addFamily(FAMILIES[2]);
2326     scan.setMaxVersions(Integer.MAX_VALUE);
2327     result = getSingleScanResult(ht, scan);
2328     assertTrue("Expected 2 keys but received " + result.size(),
2329         result.size() == 2);
2330     assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER,
2331         new long [] {ts[0], ts[1]},
2332         new byte[][] {VALUES[0], VALUES[1]},
2333         0, 1);
2334 
2335     get = new Get(ROWS[1]);
2336     get.addFamily(FAMILIES[1]);
2337     get.addFamily(FAMILIES[2]);
2338     get.setMaxVersions(Integer.MAX_VALUE);
2339     result = ht.get(get);
2340     assertTrue("Expected 2 keys but received " + result.size(),
2341         result.size() == 2);
2342 
2343     scan = new Scan(ROWS[1]);
2344     scan.addFamily(FAMILIES[1]);
2345     scan.addFamily(FAMILIES[2]);
2346     scan.setMaxVersions(Integer.MAX_VALUE);
2347     result = getSingleScanResult(ht, scan);
2348     assertTrue("Expected 2 keys but received " + result.size(),
2349         result.size() == 2);
2350 
2351     get = new Get(ROWS[2]);
2352     get.addFamily(FAMILIES[1]);
2353     get.addFamily(FAMILIES[2]);
2354     get.setMaxVersions(Integer.MAX_VALUE);
2355     result = ht.get(get);
2356     assertEquals(1, result.size());
2357     assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
2358         new long [] {ts[2]},
2359         new byte[][] {VALUES[2]},
2360         0, 0);
2361 
2362     scan = new Scan(ROWS[2]);
2363     scan.addFamily(FAMILIES[1]);
2364     scan.addFamily(FAMILIES[2]);
2365     scan.setMaxVersions(Integer.MAX_VALUE);
2366     result = getSingleScanResult(ht, scan);
2367     assertEquals(1, result.size());
2368     assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
2369         new long [] {ts[2]},
2370         new byte[][] {VALUES[2]},
2371         0, 0);
2372 
2373     // Test if we delete the family first in one row (HBASE-1541)
2374 
2375     delete = new Delete(ROWS[3]);
2376     delete.deleteFamily(FAMILIES[1]);
2377     ht.delete(delete);
2378 
2379     put = new Put(ROWS[3]);
2380     put.add(FAMILIES[2], QUALIFIER, VALUES[0]);
2381     ht.put(put);
2382 
2383     put = new Put(ROWS[4]);
2384     put.add(FAMILIES[1], QUALIFIER, VALUES[1]);
2385     put.add(FAMILIES[2], QUALIFIER, VALUES[2]);
2386     ht.put(put);
2387 
2388     get = new Get(ROWS[3]);
2389     get.addFamily(FAMILIES[1]);
2390     get.addFamily(FAMILIES[2]);
2391     get.setMaxVersions(Integer.MAX_VALUE);
2392     result = ht.get(get);
2393     assertTrue("Expected 1 key but received " + result.size(),
2394         result.size() == 1);
2395 
2396     get = new Get(ROWS[4]);
2397     get.addFamily(FAMILIES[1]);
2398     get.addFamily(FAMILIES[2]);
2399     get.setMaxVersions(Integer.MAX_VALUE);
2400     result = ht.get(get);
2401     assertTrue("Expected 2 keys but received " + result.size(),
2402         result.size() == 2);
2403 
2404     scan = new Scan(ROWS[3]);
2405     scan.addFamily(FAMILIES[1]);
2406     scan.addFamily(FAMILIES[2]);
2407     scan.setMaxVersions(Integer.MAX_VALUE);
2408     ResultScanner scanner = ht.getScanner(scan);
2409     result = scanner.next();
2410     assertTrue("Expected 1 key but received " + result.size(),
2411         result.size() == 1);
2412     assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[3]));
2413     assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[0]));
2414     result = scanner.next();
2415     assertTrue("Expected 2 keys but received " + result.size(),
2416         result.size() == 2);
2417     assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[0]), ROWS[4]));
2418     assertTrue(Bytes.equals(CellUtil.cloneRow(result.rawCells()[1]), ROWS[4]));
2419     assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[0]), VALUES[1]));
2420     assertTrue(Bytes.equals(CellUtil.cloneValue(result.rawCells()[1]), VALUES[2]));
2421     scanner.close();
2422 
2423     // Add test of bulk deleting.
2424     for (int i = 0; i < 10; i++) {
2425       byte [] bytes = Bytes.toBytes(i);
2426       put = new Put(bytes);
2427       put.setDurability(Durability.SKIP_WAL);
2428       put.add(FAMILIES[0], QUALIFIER, bytes);
2429       ht.put(put);
2430     }
2431     for (int i = 0; i < 10; i++) {
2432       byte [] bytes = Bytes.toBytes(i);
2433       get = new Get(bytes);
2434       get.addFamily(FAMILIES[0]);
2435       result = ht.get(get);
2436       assertTrue(result.size() == 1);
2437     }
2438     ArrayList<Delete> deletes = new ArrayList<Delete>();
2439     for (int i = 0; i < 10; i++) {
2440       byte [] bytes = Bytes.toBytes(i);
2441       delete = new Delete(bytes);
2442       delete.deleteFamily(FAMILIES[0]);
2443       deletes.add(delete);
2444     }
2445     ht.delete(deletes);
2446     for (int i = 0; i < 10; i++) {
2447       byte [] bytes = Bytes.toBytes(i);
2448       get = new Get(bytes);
2449       get.addFamily(FAMILIES[0]);
2450       result = ht.get(get);
2451       assertTrue(result.size() == 0);
2452     }
2453   }
2454 
2455   /*
2456    * Baseline "scalability" test.
2457    *
2458    * Tests one hundred families, one million columns, one million versions
2459    */
2460   @Ignore @Test
2461   public void testMillions() throws Exception {
2462 
2463     // 100 families
2464 
2465     // millions of columns
2466 
2467     // millions of versions
2468 
2469   }
2470 
2471   @Ignore @Test
2472   public void testMultipleRegionsAndBatchPuts() throws Exception {
2473     // Two family table
2474 
2475     // Insert lots of rows
2476 
2477     // Insert to the same row with batched puts
2478 
2479     // Insert to multiple rows with batched puts
2480 
2481     // Split the table
2482 
2483     // Get row from first region
2484 
2485     // Get row from second region
2486 
2487     // Scan all rows
2488 
2489     // Insert to multiple regions with batched puts
2490 
2491     // Get row from first region
2492 
2493     // Get row from second region
2494 
2495     // Scan all rows
2496 
2497 
2498   }
2499 
2500   @Ignore @Test
2501   public void testMultipleRowMultipleFamily() throws Exception {
2502 
2503   }
2504 
2505   //
2506   // JIRA Testers
2507   //
2508 
2509   /**
2510    * HBASE-867
2511    *    If millions of columns in a column family, hbase scanner won't come up
2512    *
2513    *    Test will create numRows rows, each with numColsPerRow columns
2514    *    (1 version each), and attempt to scan them all.
2515    *
2516    *    To test at scale, up numColsPerRow to the millions
2517    *    (have not gotten that to work running as junit though)
2518    */
2519   @Test
2520   public void testJiraTest867() throws Exception {
2521     int numRows = 10;
2522     int numColsPerRow = 2000;
2523 
2524     byte [] TABLE = Bytes.toBytes("testJiraTest867");
2525 
2526     byte [][] ROWS = makeN(ROW, numRows);
2527     byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
2528 
2529     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
2530     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2531 
2532     // Insert rows
2533 
2534     for(int i=0;i<numRows;i++) {
2535       Put put = new Put(ROWS[i]);
2536       put.setDurability(Durability.SKIP_WAL);
2537       for(int j=0;j<numColsPerRow;j++) {
2538         put.add(FAMILY, QUALIFIERS[j], QUALIFIERS[j]);
2539       }
2540       assertTrue("Put expected to contain " + numColsPerRow + " columns but " +
2541           "only contains " + put.size(), put.size() == numColsPerRow);
2542       ht.put(put);
2543     }
2544 
2545     // Get a row
2546     Get get = new Get(ROWS[numRows-1]);
2547     Result result = ht.get(get);
2548     assertNumKeys(result, numColsPerRow);
2549     Cell [] keys = result.rawCells();
2550     for(int i=0;i<result.size();i++) {
2551       assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2552     }
2553 
2554     // Scan the rows
2555     Scan scan = new Scan();
2556     ResultScanner scanner = ht.getScanner(scan);
2557     int rowCount = 0;
2558     while((result = scanner.next()) != null) {
2559       assertNumKeys(result, numColsPerRow);
2560       Cell [] kvs = result.rawCells();
2561       for(int i=0;i<numColsPerRow;i++) {
2562         assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2563       }
2564       rowCount++;
2565     }
2566     scanner.close();
2567     assertTrue("Expected to scan " + numRows + " rows but actually scanned "
2568         + rowCount + " rows", rowCount == numRows);
2569 
2570     // flush and try again
2571 
2572     TEST_UTIL.flush();
2573 
2574     // Get a row
2575     get = new Get(ROWS[numRows-1]);
2576     result = ht.get(get);
2577     assertNumKeys(result, numColsPerRow);
2578     keys = result.rawCells();
2579     for(int i=0;i<result.size();i++) {
2580       assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2581     }
2582 
2583     // Scan the rows
2584     scan = new Scan();
2585     scanner = ht.getScanner(scan);
2586     rowCount = 0;
2587     while((result = scanner.next()) != null) {
2588       assertNumKeys(result, numColsPerRow);
2589       Cell [] kvs = result.rawCells();
2590       for(int i=0;i<numColsPerRow;i++) {
2591         assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
2592       }
2593       rowCount++;
2594     }
2595     scanner.close();
2596     assertTrue("Expected to scan " + numRows + " rows but actually scanned "
2597         + rowCount + " rows", rowCount == numRows);
2598 
2599   }
2600 
2601   /**
2602    * HBASE-861
2603    *    get with timestamp will return a value if there is a version with an
2604    *    earlier timestamp
2605    */
2606   @Test
2607   public void testJiraTest861() throws Exception {
2608 
2609     byte [] TABLE = Bytes.toBytes("testJiraTest861");
2610     byte [][] VALUES = makeNAscii(VALUE, 7);
2611     long [] STAMPS = makeStamps(7);
2612 
2613     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2614     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2615 
2616     // Insert three versions
2617 
2618     Put put = new Put(ROW);
2619     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2620     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2621     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2622     ht.put(put);
2623 
2624     // Get the middle value
2625     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2626 
2627     // Try to get one version before (expect fail)
2628     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2629 
2630     // Try to get one version after (expect fail)
2631     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2632 
2633     // Try same from storefile
2634     TEST_UTIL.flush();
2635     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2636     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2637     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2638 
2639     // Insert two more versions surrounding others, into memstore
2640     put = new Put(ROW);
2641     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2642     put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2643     ht.put(put);
2644 
2645     // Check we can get everything we should and can't get what we shouldn't
2646     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2647     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2648     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2649     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2650     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2651     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2652     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2653 
2654     // Try same from two storefiles
2655     TEST_UTIL.flush();
2656     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2657     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
2658     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2659     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2660     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2661     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
2662     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
2663 
2664   }
2665 
2666   /**
2667    * HBASE-33
2668    *    Add a HTable get/obtainScanner method that retrieves all versions of a
2669    *    particular column and row between two timestamps
2670    */
2671   @Test
2672   public void testJiraTest33() throws Exception {
2673 
2674     byte [] TABLE = Bytes.toBytes("testJiraTest33");
2675     byte [][] VALUES = makeNAscii(VALUE, 7);
2676     long [] STAMPS = makeStamps(7);
2677 
2678     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2679     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2680 
2681     // Insert lots versions
2682 
2683     Put put = new Put(ROW);
2684     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2685     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2686     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2687     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2688     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2689     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2690     ht.put(put);
2691 
2692     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2693     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2694     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2695     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2696 
2697     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2698     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2699     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2700     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2701 
2702     // Try same from storefile
2703     TEST_UTIL.flush();
2704 
2705     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2706     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2707     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2708     getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2709 
2710     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2711     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
2712     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2713     scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
2714 
2715   }
2716 
2717   /**
2718    * HBASE-1014
2719    *    commit(BatchUpdate) method should return timestamp
2720    */
2721   @Test
2722   public void testJiraTest1014() throws Exception {
2723 
2724     byte [] TABLE = Bytes.toBytes("testJiraTest1014");
2725 
2726     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2727     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2728 
2729     long manualStamp = 12345;
2730 
2731     // Insert lots versions
2732 
2733     Put put = new Put(ROW);
2734     put.add(FAMILY, QUALIFIER, manualStamp, VALUE);
2735     ht.put(put);
2736 
2737     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, manualStamp, VALUE);
2738     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp-1);
2739     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, manualStamp+1);
2740 
2741   }
2742 
2743   /**
2744    * HBASE-1182
2745    *    Scan for columns > some timestamp
2746    */
2747   @Test
2748   public void testJiraTest1182() throws Exception {
2749 
2750     byte [] TABLE = Bytes.toBytes("testJiraTest1182");
2751     byte [][] VALUES = makeNAscii(VALUE, 7);
2752     long [] STAMPS = makeStamps(7);
2753 
2754     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2755     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2756 
2757     // Insert lots versions
2758 
2759     Put put = new Put(ROW);
2760     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2761     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2762     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2763     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2764     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2765     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2766     ht.put(put);
2767 
2768     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2769     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2770     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2771 
2772     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2773     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2774     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2775 
2776     // Try same from storefile
2777     TEST_UTIL.flush();
2778 
2779     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2780     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2781     getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2782 
2783     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2784     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
2785     scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
2786   }
2787 
2788   /**
2789    * HBASE-52
2790    *    Add a means of scanning over all versions
2791    */
2792   @Test
2793   public void testJiraTest52() throws Exception {
2794     byte [] TABLE = Bytes.toBytes("testJiraTest52");
2795     byte [][] VALUES = makeNAscii(VALUE, 7);
2796     long [] STAMPS = makeStamps(7);
2797 
2798     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
2799     TEST_UTIL.waitTableAvailable(TABLE, 10000);
2800 
2801     // Insert lots versions
2802 
2803     Put put = new Put(ROW);
2804     put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
2805     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
2806     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
2807     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
2808     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
2809     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
2810     ht.put(put);
2811 
2812     getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2813 
2814     scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2815 
2816     // Try same from storefile
2817     TEST_UTIL.flush();
2818 
2819     getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2820 
2821     scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
2822   }
2823 
2824   //
2825   // Bulk Testers
2826   //
2827 
2828   private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
2829       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
2830       int start, int end)
2831   throws IOException {
2832     Get get = new Get(row);
2833     get.addColumn(family, qualifier);
2834     get.setMaxVersions(Integer.MAX_VALUE);
2835     get.setTimeRange(stamps[start+1], Long.MAX_VALUE);
2836     Result result = ht.get(get);
2837     assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
2838   }
2839 
2840   private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
2841       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2842   throws IOException {
2843     Get get = new Get(row);
2844     get.addColumn(family, qualifier);
2845     get.setMaxVersions(Integer.MAX_VALUE);
2846     get.setTimeRange(stamps[start], stamps[end]+1);
2847     Result result = ht.get(get);
2848     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2849   }
2850 
2851   private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
2852       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2853   throws IOException {
2854     Get get = new Get(row);
2855     get.addColumn(family, qualifier);
2856     get.setMaxVersions(Integer.MAX_VALUE);
2857     Result result = ht.get(get);
2858     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2859   }
2860 
2861   private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
2862       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
2863       int start, int end)
2864   throws IOException {
2865     Scan scan = new Scan(row);
2866     scan.addColumn(family, qualifier);
2867     scan.setMaxVersions(Integer.MAX_VALUE);
2868     scan.setTimeRange(stamps[start+1], Long.MAX_VALUE);
2869     Result result = getSingleScanResult(ht, scan);
2870     assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
2871   }
2872 
2873   private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
2874       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2875   throws IOException {
2876     Scan scan = new Scan(row);
2877     scan.addColumn(family, qualifier);
2878     scan.setMaxVersions(Integer.MAX_VALUE);
2879     scan.setTimeRange(stamps[start], stamps[end]+1);
2880     Result result = getSingleScanResult(ht, scan);
2881     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2882   }
2883 
2884   private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
2885       byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
2886   throws IOException {
2887     Scan scan = new Scan(row);
2888     scan.addColumn(family, qualifier);
2889     scan.setMaxVersions(Integer.MAX_VALUE);
2890     Result result = getSingleScanResult(ht, scan);
2891     assertNResult(result, row, family, qualifier, stamps, values, start, end);
2892   }
2893 
2894   private void getVersionAndVerify(Table ht, byte [] row, byte [] family,
2895       byte [] qualifier, long stamp, byte [] value)
2896   throws Exception {
2897     Get get = new Get(row);
2898     get.addColumn(family, qualifier);
2899     get.setTimeStamp(stamp);
2900     get.setMaxVersions(Integer.MAX_VALUE);
2901     Result result = ht.get(get);
2902     assertSingleResult(result, row, family, qualifier, stamp, value);
2903   }
2904 
2905   private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family,
2906       byte [] qualifier, long stamp)
2907   throws Exception {
2908     Get get = new Get(row);
2909     get.addColumn(family, qualifier);
2910     get.setTimeStamp(stamp);
2911     get.setMaxVersions(Integer.MAX_VALUE);
2912     Result result = ht.get(get);
2913     assertEmptyResult(result);
2914   }
2915 
2916   private void scanVersionAndVerify(Table ht, byte [] row, byte [] family,
2917       byte [] qualifier, long stamp, byte [] value)
2918   throws Exception {
2919     Scan scan = new Scan(row);
2920     scan.addColumn(family, qualifier);
2921     scan.setTimeStamp(stamp);
2922     scan.setMaxVersions(Integer.MAX_VALUE);
2923     Result result = getSingleScanResult(ht, scan);
2924     assertSingleResult(result, row, family, qualifier, stamp, value);
2925   }
2926 
2927   private void scanVersionAndVerifyMissing(Table ht, byte [] row,
2928       byte [] family, byte [] qualifier, long stamp)
2929   throws Exception {
2930     Scan scan = new Scan(row);
2931     scan.addColumn(family, qualifier);
2932     scan.setTimeStamp(stamp);
2933     scan.setMaxVersions(Integer.MAX_VALUE);
2934     Result result = getSingleScanResult(ht, scan);
2935     assertNullResult(result);
2936   }
2937 
2938   private void getTestNull(Table ht, byte [] row, byte [] family,
2939       byte [] value)
2940   throws Exception {
2941 
2942     Get get = new Get(row);
2943     get.addColumn(family, null);
2944     Result result = ht.get(get);
2945     assertSingleResult(result, row, family, null, value);
2946 
2947     get = new Get(row);
2948     get.addColumn(family, HConstants.EMPTY_BYTE_ARRAY);
2949     result = ht.get(get);
2950     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2951 
2952     get = new Get(row);
2953     get.addFamily(family);
2954     result = ht.get(get);
2955     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2956 
2957     get = new Get(row);
2958     result = ht.get(get);
2959     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2960 
2961   }
2962 
2963   private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value)
2964       throws Exception {
2965     scanTestNull(ht, row, family, value, false);
2966   }
2967 
2968   private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value,
2969       boolean isReversedScan) throws Exception {
2970 
2971     Scan scan = new Scan();
2972     scan.setReversed(isReversedScan);
2973     scan.addColumn(family, null);
2974     Result result = getSingleScanResult(ht, scan);
2975     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2976 
2977     scan = new Scan();
2978     scan.setReversed(isReversedScan);
2979     scan.addColumn(family, HConstants.EMPTY_BYTE_ARRAY);
2980     result = getSingleScanResult(ht, scan);
2981     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2982 
2983     scan = new Scan();
2984     scan.setReversed(isReversedScan);
2985     scan.addFamily(family);
2986     result = getSingleScanResult(ht, scan);
2987     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2988 
2989     scan = new Scan();
2990     scan.setReversed(isReversedScan);
2991     result = getSingleScanResult(ht, scan);
2992     assertSingleResult(result, row, family, HConstants.EMPTY_BYTE_ARRAY, value);
2993 
2994   }
2995 
2996   private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
2997       byte [][] QUALIFIERS, byte [][] VALUES)
2998   throws Exception {
2999 
3000     // Single column from memstore
3001     Get get = new Get(ROWS[0]);
3002     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
3003     Result result = ht.get(get);
3004     assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
3005 
3006     // Single column from storefile
3007     get = new Get(ROWS[0]);
3008     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
3009     result = ht.get(get);
3010     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
3011 
3012     // Single column from storefile, family match
3013     get = new Get(ROWS[0]);
3014     get.addFamily(FAMILIES[7]);
3015     result = ht.get(get);
3016     assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
3017 
3018     // Two columns, one from memstore one from storefile, same family,
3019     // wildcard match
3020     get = new Get(ROWS[0]);
3021     get.addFamily(FAMILIES[4]);
3022     result = ht.get(get);
3023     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
3024         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
3025 
3026     // Two columns, one from memstore one from storefile, same family,
3027     // explicit match
3028     get = new Get(ROWS[0]);
3029     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
3030     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
3031     result = ht.get(get);
3032     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
3033         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
3034 
3035     // Three column, one from memstore two from storefile, different families,
3036     // wildcard match
3037     get = new Get(ROWS[0]);
3038     get.addFamily(FAMILIES[4]);
3039     get.addFamily(FAMILIES[7]);
3040     result = ht.get(get);
3041     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3042         new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
3043 
3044     // Multiple columns from everywhere storefile, many family, wildcard
3045     get = new Get(ROWS[0]);
3046     get.addFamily(FAMILIES[2]);
3047     get.addFamily(FAMILIES[4]);
3048     get.addFamily(FAMILIES[6]);
3049     get.addFamily(FAMILIES[7]);
3050     result = ht.get(get);
3051     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3052         new int [][] {
3053           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
3054     });
3055 
3056     // Multiple columns from everywhere storefile, many family, wildcard
3057     get = new Get(ROWS[0]);
3058     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
3059     get.addColumn(FAMILIES[2], QUALIFIERS[4]);
3060     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
3061     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
3062     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
3063     get.addColumn(FAMILIES[6], QUALIFIERS[7]);
3064     get.addColumn(FAMILIES[7], QUALIFIERS[7]);
3065     get.addColumn(FAMILIES[7], QUALIFIERS[8]);
3066     result = ht.get(get);
3067     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3068         new int [][] {
3069           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
3070     });
3071 
3072     // Everything
3073     get = new Get(ROWS[0]);
3074     result = ht.get(get);
3075     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3076         new int [][] {
3077           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
3078     });
3079 
3080     // Get around inserted columns
3081 
3082     get = new Get(ROWS[1]);
3083     result = ht.get(get);
3084     assertEmptyResult(result);
3085 
3086     get = new Get(ROWS[0]);
3087     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
3088     get.addColumn(FAMILIES[2], QUALIFIERS[3]);
3089     result = ht.get(get);
3090     assertEmptyResult(result);
3091 
3092   }
3093 
3094   private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
3095       byte [][] QUALIFIERS, byte [][] VALUES)
3096   throws Exception {
3097 
3098     // Single column from memstore
3099     Scan scan = new Scan();
3100     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
3101     Result result = getSingleScanResult(ht, scan);
3102     assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]);
3103 
3104     // Single column from storefile
3105     scan = new Scan();
3106     scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
3107     result = getSingleScanResult(ht, scan);
3108     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
3109 
3110     // Single column from storefile, family match
3111     scan = new Scan();
3112     scan.addFamily(FAMILIES[7]);
3113     result = getSingleScanResult(ht, scan);
3114     assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
3115 
3116     // Two columns, one from memstore one from storefile, same family,
3117     // wildcard match
3118     scan = new Scan();
3119     scan.addFamily(FAMILIES[4]);
3120     result = getSingleScanResult(ht, scan);
3121     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
3122         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
3123 
3124     // Two columns, one from memstore one from storefile, same family,
3125     // explicit match
3126     scan = new Scan();
3127     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
3128     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
3129     result = getSingleScanResult(ht, scan);
3130     assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
3131         FAMILIES[4], QUALIFIERS[4], VALUES[4]);
3132 
3133     // Three column, one from memstore two from storefile, different families,
3134     // wildcard match
3135     scan = new Scan();
3136     scan.addFamily(FAMILIES[4]);
3137     scan.addFamily(FAMILIES[7]);
3138     result = getSingleScanResult(ht, scan);
3139     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3140         new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} });
3141 
3142     // Multiple columns from everywhere storefile, many family, wildcard
3143     scan = new Scan();
3144     scan.addFamily(FAMILIES[2]);
3145     scan.addFamily(FAMILIES[4]);
3146     scan.addFamily(FAMILIES[6]);
3147     scan.addFamily(FAMILIES[7]);
3148     result = getSingleScanResult(ht, scan);
3149     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3150         new int [][] {
3151           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
3152     });
3153 
3154     // Multiple columns from everywhere storefile, many family, wildcard
3155     scan = new Scan();
3156     scan.addColumn(FAMILIES[2], QUALIFIERS[2]);
3157     scan.addColumn(FAMILIES[2], QUALIFIERS[4]);
3158     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
3159     scan.addColumn(FAMILIES[4], QUALIFIERS[4]);
3160     scan.addColumn(FAMILIES[6], QUALIFIERS[6]);
3161     scan.addColumn(FAMILIES[6], QUALIFIERS[7]);
3162     scan.addColumn(FAMILIES[7], QUALIFIERS[7]);
3163     scan.addColumn(FAMILIES[7], QUALIFIERS[8]);
3164     result = getSingleScanResult(ht, scan);
3165     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3166         new int [][] {
3167           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
3168     });
3169 
3170     // Everything
3171     scan = new Scan();
3172     result = getSingleScanResult(ht, scan);
3173     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
3174         new int [][] {
3175           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
3176     });
3177 
3178     // Scan around inserted columns
3179 
3180     scan = new Scan(ROWS[1]);
3181     result = getSingleScanResult(ht, scan);
3182     assertNullResult(result);
3183 
3184     scan = new Scan();
3185     scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
3186     scan.addColumn(FAMILIES[2], QUALIFIERS[3]);
3187     result = getSingleScanResult(ht, scan);
3188     assertNullResult(result);
3189   }
3190 
3191   /**
3192    * Verify a single column using gets.
3193    * Expects family and qualifier arrays to be valid for at least
3194    * the range:  idx-2 < idx < idx+2
3195    */
3196   private void getVerifySingleColumn(Table ht,
3197       byte [][] ROWS, int ROWIDX,
3198       byte [][] FAMILIES, int FAMILYIDX,
3199       byte [][] QUALIFIERS, int QUALIFIERIDX,
3200       byte [][] VALUES, int VALUEIDX)
3201   throws Exception {
3202 
3203     Get get = new Get(ROWS[ROWIDX]);
3204     Result result = ht.get(get);
3205     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3206         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3207 
3208     get = new Get(ROWS[ROWIDX]);
3209     get.addFamily(FAMILIES[FAMILYIDX]);
3210     result = ht.get(get);
3211     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3212         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3213 
3214     get = new Get(ROWS[ROWIDX]);
3215     get.addFamily(FAMILIES[FAMILYIDX-2]);
3216     get.addFamily(FAMILIES[FAMILYIDX]);
3217     get.addFamily(FAMILIES[FAMILYIDX+2]);
3218     result = ht.get(get);
3219     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3220         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3221 
3222     get = new Get(ROWS[ROWIDX]);
3223     get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[0]);
3224     result = ht.get(get);
3225     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3226         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3227 
3228     get = new Get(ROWS[ROWIDX]);
3229     get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[1]);
3230     get.addFamily(FAMILIES[FAMILYIDX]);
3231     result = ht.get(get);
3232     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3233         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3234 
3235     get = new Get(ROWS[ROWIDX]);
3236     get.addFamily(FAMILIES[FAMILYIDX]);
3237     get.addColumn(FAMILIES[FAMILYIDX+1], QUALIFIERS[1]);
3238     get.addColumn(FAMILIES[FAMILYIDX-2], QUALIFIERS[1]);
3239     get.addFamily(FAMILIES[FAMILYIDX-1]);
3240     get.addFamily(FAMILIES[FAMILYIDX+2]);
3241     result = ht.get(get);
3242     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3243         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3244 
3245   }
3246 
3247 
3248   /**
3249    * Verify a single column using scanners.
3250    * Expects family and qualifier arrays to be valid for at least
3251    * the range:  idx-2 to idx+2
3252    * Expects row array to be valid for at least idx to idx+2
3253    */
3254   private void scanVerifySingleColumn(Table ht,
3255       byte [][] ROWS, int ROWIDX,
3256       byte [][] FAMILIES, int FAMILYIDX,
3257       byte [][] QUALIFIERS, int QUALIFIERIDX,
3258       byte [][] VALUES, int VALUEIDX)
3259   throws Exception {
3260 
3261     Scan scan = new Scan();
3262     Result result = getSingleScanResult(ht, scan);
3263     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3264         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3265 
3266     scan = new Scan(ROWS[ROWIDX]);
3267     result = getSingleScanResult(ht, scan);
3268     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3269         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3270 
3271     scan = new Scan(ROWS[ROWIDX], ROWS[ROWIDX+1]);
3272     result = getSingleScanResult(ht, scan);
3273     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3274         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3275 
3276     scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX+1]);
3277     result = getSingleScanResult(ht, scan);
3278     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3279         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3280 
3281     scan = new Scan();
3282     scan.addFamily(FAMILIES[FAMILYIDX]);
3283     result = getSingleScanResult(ht, scan);
3284     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3285         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3286 
3287     scan = new Scan();
3288     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
3289     result = getSingleScanResult(ht, scan);
3290     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3291         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3292 
3293     scan = new Scan();
3294     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
3295     scan.addFamily(FAMILIES[FAMILYIDX]);
3296     result = getSingleScanResult(ht, scan);
3297     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3298         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3299 
3300     scan = new Scan();
3301     scan.addColumn(FAMILIES[FAMILYIDX-1], QUALIFIERS[QUALIFIERIDX+1]);
3302     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]);
3303     scan.addFamily(FAMILIES[FAMILYIDX+1]);
3304     result = getSingleScanResult(ht, scan);
3305     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
3306         QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]);
3307 
3308   }
3309 
3310   /**
3311    * Verify we do not read any values by accident around a single column
3312    * Same requirements as getVerifySingleColumn
3313    */
3314   private void getVerifySingleEmpty(Table ht,
3315       byte [][] ROWS, int ROWIDX,
3316       byte [][] FAMILIES, int FAMILYIDX,
3317       byte [][] QUALIFIERS, int QUALIFIERIDX)
3318   throws Exception {
3319 
3320     Get get = new Get(ROWS[ROWIDX]);
3321     get.addFamily(FAMILIES[4]);
3322     get.addColumn(FAMILIES[4], QUALIFIERS[1]);
3323     Result result = ht.get(get);
3324     assertEmptyResult(result);
3325 
3326     get = new Get(ROWS[ROWIDX]);
3327     get.addFamily(FAMILIES[4]);
3328     get.addColumn(FAMILIES[4], QUALIFIERS[2]);
3329     result = ht.get(get);
3330     assertEmptyResult(result);
3331 
3332     get = new Get(ROWS[ROWIDX]);
3333     get.addFamily(FAMILIES[3]);
3334     get.addColumn(FAMILIES[4], QUALIFIERS[2]);
3335     get.addFamily(FAMILIES[5]);
3336     result = ht.get(get);
3337     assertEmptyResult(result);
3338 
3339     get = new Get(ROWS[ROWIDX+1]);
3340     result = ht.get(get);
3341     assertEmptyResult(result);
3342 
3343   }
3344 
3345   private void scanVerifySingleEmpty(Table ht,
3346       byte [][] ROWS, int ROWIDX,
3347       byte [][] FAMILIES, int FAMILYIDX,
3348       byte [][] QUALIFIERS, int QUALIFIERIDX)
3349   throws Exception {
3350 
3351     Scan scan = new Scan(ROWS[ROWIDX+1]);
3352     Result result = getSingleScanResult(ht, scan);
3353     assertNullResult(result);
3354 
3355     scan = new Scan(ROWS[ROWIDX+1],ROWS[ROWIDX+2]);
3356     result = getSingleScanResult(ht, scan);
3357     assertNullResult(result);
3358 
3359     scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX]);
3360     result = getSingleScanResult(ht, scan);
3361     assertNullResult(result);
3362 
3363     scan = new Scan();
3364     scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]);
3365     scan.addFamily(FAMILIES[FAMILYIDX-1]);
3366     result = getSingleScanResult(ht, scan);
3367     assertNullResult(result);
3368 
3369   }
3370 
3371   //
3372   // Verifiers
3373   //
3374 
3375   private void assertKey(Cell key, byte [] row, byte [] family,
3376       byte [] qualifier, byte [] value)
3377   throws Exception {
3378     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3379         "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
3380         equals(row, CellUtil.cloneRow(key)));
3381     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3382         "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3383         equals(family, CellUtil.cloneFamily(key)));
3384     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3385         "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]",
3386         equals(qualifier, CellUtil.cloneQualifier(key)));
3387     assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3388         "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]",
3389         equals(value, CellUtil.cloneValue(key)));
3390   }
3391 
3392   static void assertIncrementKey(Cell key, byte [] row, byte [] family,
3393       byte [] qualifier, long value)
3394   throws Exception {
3395     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3396         "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
3397         equals(row, CellUtil.cloneRow(key)));
3398     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3399         "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3400         equals(family, CellUtil.cloneFamily(key)));
3401     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3402         "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key)) + "]",
3403         equals(qualifier, CellUtil.cloneQualifier(key)));
3404     assertTrue("Expected value [" + value + "] " +
3405         "Got value [" + Bytes.toLong(CellUtil.cloneValue(key)) + "]",
3406         Bytes.toLong(CellUtil.cloneValue(key)) == value);
3407   }
3408 
3409   private void assertNumKeys(Result result, int n) throws Exception {
3410     assertTrue("Expected " + n + " keys but got " + result.size(),
3411         result.size() == n);
3412   }
3413 
3414   private void assertNResult(Result result, byte [] row,
3415       byte [][] families, byte [][] qualifiers, byte [][] values,
3416       int [][] idxs)
3417   throws Exception {
3418     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3419         "Got row [" + Bytes.toString(result.getRow()) +"]",
3420         equals(row, result.getRow()));
3421     assertTrue("Expected " + idxs.length + " keys but result contains "
3422         + result.size(), result.size() == idxs.length);
3423 
3424     Cell [] keys = result.rawCells();
3425 
3426     for(int i=0;i<keys.length;i++) {
3427       byte [] family = families[idxs[i][0]];
3428       byte [] qualifier = qualifiers[idxs[i][1]];
3429       byte [] value = values[idxs[i][2]];
3430       Cell key = keys[i];
3431 
3432       byte[] famb = CellUtil.cloneFamily(key);
3433       byte[] qualb = CellUtil.cloneQualifier(key);
3434       byte[] valb = CellUtil.cloneValue(key);
3435       assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
3436           + "] " + "Got family [" + Bytes.toString(famb) + "]",
3437           equals(family, famb));
3438       assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
3439           + "] " + "Got qualifier [" + Bytes.toString(qualb) + "]",
3440           equals(qualifier, qualb));
3441       assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
3442           + "Got value [" + Bytes.toString(valb) + "]",
3443           equals(value, valb));
3444     }
3445   }
3446 
3447   private void assertNResult(Result result, byte [] row,
3448       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
3449       int start, int end)
3450   throws IOException {
3451     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3452         "Got row [" + Bytes.toString(result.getRow()) +"]",
3453         equals(row, result.getRow()));
3454     int expectedResults = end - start + 1;
3455     assertEquals(expectedResults, result.size());
3456 
3457     Cell[] keys = result.rawCells();
3458 
3459     for (int i=0; i<keys.length; i++) {
3460       byte [] value = values[end-i];
3461       long ts = stamps[end-i];
3462       Cell key = keys[i];
3463 
3464       assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
3465           + "] " + "Got family [" + Bytes.toString(CellUtil.cloneFamily(key)) + "]",
3466           CellUtil.matchingFamily(key, family));
3467       assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
3468           + "] " + "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(key))+ "]",
3469           CellUtil.matchingQualifier(key, qualifier));
3470       assertTrue("Expected ts [" + ts + "] " +
3471           "Got ts [" + key.getTimestamp() + "]", ts == key.getTimestamp());
3472       assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
3473           + "Got value [" + Bytes.toString(CellUtil.cloneValue(key)) + "]",
3474           CellUtil.matchingValue(key,  value));
3475     }
3476   }
3477 
3478   /**
3479    * Validate that result contains two specified keys, exactly.
3480    * It is assumed key A sorts before key B.
3481    */
3482   private void assertDoubleResult(Result result, byte [] row,
3483       byte [] familyA, byte [] qualifierA, byte [] valueA,
3484       byte [] familyB, byte [] qualifierB, byte [] valueB)
3485   throws Exception {
3486     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3487         "Got row [" + Bytes.toString(result.getRow()) +"]",
3488         equals(row, result.getRow()));
3489     assertTrue("Expected two keys but result contains " + result.size(),
3490         result.size() == 2);
3491     Cell [] kv = result.rawCells();
3492     Cell kvA = kv[0];
3493     assertTrue("(A) Expected family [" + Bytes.toString(familyA) + "] " +
3494         "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvA)) + "]",
3495         equals(familyA, CellUtil.cloneFamily(kvA)));
3496     assertTrue("(A) Expected qualifier [" + Bytes.toString(qualifierA) + "] " +
3497         "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvA)) + "]",
3498         equals(qualifierA, CellUtil.cloneQualifier(kvA)));
3499     assertTrue("(A) Expected value [" + Bytes.toString(valueA) + "] " +
3500         "Got value [" + Bytes.toString(CellUtil.cloneValue(kvA)) + "]",
3501         equals(valueA, CellUtil.cloneValue(kvA)));
3502     Cell kvB = kv[1];
3503     assertTrue("(B) Expected family [" + Bytes.toString(familyB) + "] " +
3504         "Got family [" + Bytes.toString(CellUtil.cloneFamily(kvB)) + "]",
3505         equals(familyB, CellUtil.cloneFamily(kvB)));
3506     assertTrue("(B) Expected qualifier [" + Bytes.toString(qualifierB) + "] " +
3507         "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kvB)) + "]",
3508         equals(qualifierB, CellUtil.cloneQualifier(kvB)));
3509     assertTrue("(B) Expected value [" + Bytes.toString(valueB) + "] " +
3510         "Got value [" + Bytes.toString(CellUtil.cloneValue(kvB)) + "]",
3511         equals(valueB, CellUtil.cloneValue(kvB)));
3512   }
3513 
3514   private void assertSingleResult(Result result, byte [] row, byte [] family,
3515       byte [] qualifier, byte [] value)
3516   throws Exception {
3517     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3518         "Got row [" + Bytes.toString(result.getRow()) +"]",
3519         equals(row, result.getRow()));
3520     assertTrue("Expected a single key but result contains " + result.size(),
3521         result.size() == 1);
3522     Cell kv = result.rawCells()[0];
3523     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3524         "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]",
3525         equals(family, CellUtil.cloneFamily(kv)));
3526     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3527         "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]",
3528         equals(qualifier, CellUtil.cloneQualifier(kv)));
3529     assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3530         "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]",
3531         equals(value, CellUtil.cloneValue(kv)));
3532   }
3533 
3534   private void assertSingleResult(Result result, byte [] row, byte [] family,
3535       byte [] qualifier, long ts, byte [] value)
3536   throws Exception {
3537     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
3538         "Got row [" + Bytes.toString(result.getRow()) +"]",
3539         equals(row, result.getRow()));
3540     assertTrue("Expected a single key but result contains " + result.size(),
3541         result.size() == 1);
3542     Cell kv = result.rawCells()[0];
3543     assertTrue("Expected family [" + Bytes.toString(family) + "] " +
3544         "Got family [" + Bytes.toString(CellUtil.cloneFamily(kv)) + "]",
3545         equals(family, CellUtil.cloneFamily(kv)));
3546     assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
3547         "Got qualifier [" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "]",
3548         equals(qualifier, CellUtil.cloneQualifier(kv)));
3549     assertTrue("Expected ts [" + ts + "] " +
3550         "Got ts [" + kv.getTimestamp() + "]", ts == kv.getTimestamp());
3551     assertTrue("Expected value [" + Bytes.toString(value) + "] " +
3552         "Got value [" + Bytes.toString(CellUtil.cloneValue(kv)) + "]",
3553         equals(value, CellUtil.cloneValue(kv)));
3554   }
3555 
3556   private void assertEmptyResult(Result result) throws Exception {
3557     assertTrue("expected an empty result but result contains " +
3558         result.size() + " keys", result.isEmpty());
3559   }
3560 
3561   private void assertNullResult(Result result) throws Exception {
3562     assertTrue("expected null result but received a non-null result",
3563         result == null);
3564   }
3565 
3566   //
3567   // Helpers
3568   //
3569 
3570   private Result getSingleScanResult(Table ht, Scan scan) throws IOException {
3571     ResultScanner scanner = ht.getScanner(scan);
3572     Result result = scanner.next();
3573     scanner.close();
3574     return result;
3575   }
3576 
3577   private byte [][] makeNAscii(byte [] base, int n) {
3578     if(n > 256) {
3579       return makeNBig(base, n);
3580     }
3581     byte [][] ret = new byte[n][];
3582     for(int i=0;i<n;i++) {
3583       byte [] tail = Bytes.toBytes(Integer.toString(i));
3584       ret[i] = Bytes.add(base, tail);
3585     }
3586     return ret;
3587   }
3588 
3589   private byte [][] makeN(byte [] base, int n) {
3590     if (n > 256) {
3591       return makeNBig(base, n);
3592     }
3593     byte [][] ret = new byte[n][];
3594     for(int i=0;i<n;i++) {
3595       ret[i] = Bytes.add(base, new byte[]{(byte)i});
3596     }
3597     return ret;
3598   }
3599 
3600   private byte [][] makeNBig(byte [] base, int n) {
3601     byte [][] ret = new byte[n][];
3602     for(int i=0;i<n;i++) {
3603       int byteA = (i % 256);
3604       int byteB = (i >> 8);
3605       ret[i] = Bytes.add(base, new byte[]{(byte)byteB,(byte)byteA});
3606     }
3607     return ret;
3608   }
3609 
3610   private long [] makeStamps(int n) {
3611     long [] stamps = new long[n];
3612     for(int i=0;i<n;i++) stamps[i] = i+1;
3613     return stamps;
3614   }
3615 
3616   static boolean equals(byte [] left, byte [] right) {
3617     if (left == null && right == null) return true;
3618     if (left == null && right.length == 0) return true;
3619     if (right == null && left.length == 0) return true;
3620     return Bytes.equals(left, right);
3621   }
3622 
3623   @Test
3624   public void testDuplicateVersions() throws Exception {
3625     byte [] TABLE = Bytes.toBytes("testDuplicateVersions");
3626 
3627     long [] STAMPS = makeStamps(20);
3628     byte [][] VALUES = makeNAscii(VALUE, 20);
3629 
3630     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3631     TEST_UTIL.waitTableAvailable(TABLE, 10000);
3632 
3633     // Insert 4 versions of same column
3634     Put put = new Put(ROW);
3635     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3636     put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3637     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3638     put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3639     ht.put(put);
3640 
3641     // Verify we can get each one properly
3642     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3643     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3644     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3645     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3646     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3647     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3648     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3649     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3650 
3651     // Verify we don't accidentally get others
3652     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3653     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3654     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3655     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3656     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3657     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3658 
3659     // Ensure maxVersions in query is respected
3660     Get get = new Get(ROW);
3661     get.addColumn(FAMILY, QUALIFIER);
3662     get.setMaxVersions(2);
3663     Result result = ht.get(get);
3664     assertNResult(result, ROW, FAMILY, QUALIFIER,
3665         new long [] {STAMPS[4], STAMPS[5]},
3666         new byte[][] {VALUES[4], VALUES[5]},
3667         0, 1);
3668 
3669     Scan scan = new Scan(ROW);
3670     scan.addColumn(FAMILY, QUALIFIER);
3671     scan.setMaxVersions(2);
3672     result = getSingleScanResult(ht, scan);
3673     assertNResult(result, ROW, FAMILY, QUALIFIER,
3674         new long [] {STAMPS[4], STAMPS[5]},
3675         new byte[][] {VALUES[4], VALUES[5]},
3676         0, 1);
3677 
3678     // Flush and redo
3679 
3680     TEST_UTIL.flush();
3681 
3682     // Verify we can get each one properly
3683     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3684     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3685     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3686     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3687     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3688     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3689     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]);
3690     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]);
3691 
3692     // Verify we don't accidentally get others
3693     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3694     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3695     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3696     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3697     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]);
3698     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
3699 
3700     // Ensure maxVersions in query is respected
3701     get = new Get(ROW);
3702     get.addColumn(FAMILY, QUALIFIER);
3703     get.setMaxVersions(2);
3704     result = ht.get(get);
3705     assertNResult(result, ROW, FAMILY, QUALIFIER,
3706         new long [] {STAMPS[4], STAMPS[5]},
3707         new byte[][] {VALUES[4], VALUES[5]},
3708         0, 1);
3709 
3710     scan = new Scan(ROW);
3711     scan.addColumn(FAMILY, QUALIFIER);
3712     scan.setMaxVersions(2);
3713     result = getSingleScanResult(ht, scan);
3714     assertNResult(result, ROW, FAMILY, QUALIFIER,
3715         new long [] {STAMPS[4], STAMPS[5]},
3716         new byte[][] {VALUES[4], VALUES[5]},
3717         0, 1);
3718 
3719 
3720     // Add some memstore and retest
3721 
3722     // Insert 4 more versions of same column and a dupe
3723     put = new Put(ROW);
3724     put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]);
3725     put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3726     put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
3727     put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3728     put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]);
3729     ht.put(put);
3730 
3731     // Ensure maxVersions in query is respected
3732     get = new Get(ROW);
3733     get.addColumn(FAMILY, QUALIFIER);
3734     get.setMaxVersions(7);
3735     result = ht.get(get);
3736     assertNResult(result, ROW, FAMILY, QUALIFIER,
3737         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3738         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3739         0, 6);
3740 
3741     scan = new Scan(ROW);
3742     scan.addColumn(FAMILY, QUALIFIER);
3743     scan.setMaxVersions(7);
3744     result = getSingleScanResult(ht, scan);
3745     assertNResult(result, ROW, FAMILY, QUALIFIER,
3746         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3747         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3748         0, 6);
3749 
3750     get = new Get(ROW);
3751     get.setMaxVersions(7);
3752     result = ht.get(get);
3753     assertNResult(result, ROW, FAMILY, QUALIFIER,
3754         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3755         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3756         0, 6);
3757 
3758     scan = new Scan(ROW);
3759     scan.setMaxVersions(7);
3760     result = getSingleScanResult(ht, scan);
3761     assertNResult(result, ROW, FAMILY, QUALIFIER,
3762         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
3763         new byte[][] {VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
3764         0, 6);
3765 
3766     // Verify we can get each one properly
3767     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3768     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3769     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3770     getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3771     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
3772     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
3773     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[14]);
3774     scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]);
3775 
3776     // Verify we don't accidentally get others
3777     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3778     getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3779     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]);
3780     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]);
3781 
3782     // Ensure maxVersions of table is respected
3783 
3784     TEST_UTIL.flush();
3785 
3786     // Insert 4 more versions of same column and a dupe
3787     put = new Put(ROW);
3788     put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]);
3789     put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]);
3790     put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]);
3791     put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
3792     ht.put(put);
3793 
3794     get = new Get(ROW);
3795     get.addColumn(FAMILY, QUALIFIER);
3796     get.setMaxVersions(Integer.MAX_VALUE);
3797     result = ht.get(get);
3798     assertNResult(result, ROW, FAMILY, QUALIFIER,
3799         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3800         new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3801         0, 9);
3802 
3803     scan = new Scan(ROW);
3804     scan.addColumn(FAMILY, QUALIFIER);
3805     scan.setMaxVersions(Integer.MAX_VALUE);
3806     result = getSingleScanResult(ht, scan);
3807     assertNResult(result, ROW, FAMILY, QUALIFIER,
3808         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
3809         new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
3810         0, 9);
3811 
3812     // Delete a version in the memstore and a version in a storefile
3813     Delete delete = new Delete(ROW);
3814     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
3815     delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
3816     ht.delete(delete);
3817 
3818     // Test that it's gone
3819     get = new Get(ROW);
3820     get.addColumn(FAMILY, QUALIFIER);
3821     get.setMaxVersions(Integer.MAX_VALUE);
3822     result = ht.get(get);
3823     assertNResult(result, ROW, FAMILY, QUALIFIER,
3824         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3825         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3826         0, 9);
3827 
3828     scan = new Scan(ROW);
3829     scan.addColumn(FAMILY, QUALIFIER);
3830     scan.setMaxVersions(Integer.MAX_VALUE);
3831     result = getSingleScanResult(ht, scan);
3832     assertNResult(result, ROW, FAMILY, QUALIFIER,
3833         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
3834         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
3835         0, 9);
3836   }
3837 
3838   @Test
3839   public void testUpdates() throws Exception {
3840 
3841     byte [] TABLE = Bytes.toBytes("testUpdates");
3842     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3843     TEST_UTIL.waitTableAvailable(TABLE, 10000);
3844 
3845     // Write a column with values at timestamp 1, 2 and 3
3846     byte[] row = Bytes.toBytes("row1");
3847     byte[] qualifier = Bytes.toBytes("myCol");
3848     Put put = new Put(row);
3849     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3850     hTable.put(put);
3851 
3852     put = new Put(row);
3853     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3854     hTable.put(put);
3855 
3856     put = new Put(row);
3857     put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3858     hTable.put(put);
3859 
3860     Get get = new Get(row);
3861     get.addColumn(FAMILY, qualifier);
3862     get.setMaxVersions();
3863 
3864     // Check that the column indeed has the right values at timestamps 1 and
3865     // 2
3866     Result result = hTable.get(get);
3867     NavigableMap<Long, byte[]> navigableMap =
3868         result.getMap().get(FAMILY).get(qualifier);
3869     assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3870     assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3871 
3872     // Update the value at timestamp 1
3873     put = new Put(row);
3874     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3875     hTable.put(put);
3876 
3877     // Update the value at timestamp 2
3878     put = new Put(row);
3879     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3880     hTable.put(put);
3881 
3882     // Check that the values at timestamp 2 and 1 got updated
3883     result = hTable.get(get);
3884     navigableMap = result.getMap().get(FAMILY).get(qualifier);
3885     assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3886     assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3887   }
3888 
3889   @Test
3890   public void testUpdatesWithMajorCompaction() throws Exception {
3891 
3892     TableName TABLE = TableName.valueOf("testUpdatesWithMajorCompaction");
3893     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3894     TEST_UTIL.waitTableAvailable(TABLE, 10000);
3895 
3896     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3897 
3898     // Write a column with values at timestamp 1, 2 and 3
3899     byte[] row = Bytes.toBytes("row2");
3900     byte[] qualifier = Bytes.toBytes("myCol");
3901     Put put = new Put(row);
3902     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3903     hTable.put(put);
3904 
3905     put = new Put(row);
3906     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3907     hTable.put(put);
3908 
3909     put = new Put(row);
3910     put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3911     hTable.put(put);
3912 
3913     Get get = new Get(row);
3914     get.addColumn(FAMILY, qualifier);
3915     get.setMaxVersions();
3916 
3917     // Check that the column indeed has the right values at timestamps 1 and
3918     // 2
3919     Result result = hTable.get(get);
3920     NavigableMap<Long, byte[]> navigableMap =
3921         result.getMap().get(FAMILY).get(qualifier);
3922     assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3923     assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3924 
3925     // Trigger a major compaction
3926     admin.flush(TABLE);
3927     admin.majorCompact(TABLE);
3928     Thread.sleep(6000);
3929 
3930     // Update the value at timestamp 1
3931     put = new Put(row);
3932     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3933     hTable.put(put);
3934 
3935     // Update the value at timestamp 2
3936     put = new Put(row);
3937     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
3938     hTable.put(put);
3939 
3940     // Trigger a major compaction
3941     admin.flush(TABLE);
3942     admin.majorCompact(TABLE);
3943     Thread.sleep(6000);
3944 
3945     // Check that the values at timestamp 2 and 1 got updated
3946     result = hTable.get(get);
3947     navigableMap = result.getMap().get(FAMILY).get(qualifier);
3948     assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
3949     assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
3950   }
3951 
3952   @Test
3953   public void testMajorCompactionBetweenTwoUpdates() throws Exception {
3954 
3955     String tableName = "testMajorCompactionBetweenTwoUpdates";
3956     byte [] TABLE = Bytes.toBytes(tableName);
3957     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
3958     TEST_UTIL.waitTableAvailable(TABLE, 10000);
3959     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
3960 
3961     // Write a column with values at timestamp 1, 2 and 3
3962     byte[] row = Bytes.toBytes("row3");
3963     byte[] qualifier = Bytes.toBytes("myCol");
3964     Put put = new Put(row);
3965     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("AAA"));
3966     hTable.put(put);
3967 
3968     put = new Put(row);
3969     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("BBB"));
3970     hTable.put(put);
3971 
3972     put = new Put(row);
3973     put.add(FAMILY, qualifier, 3L, Bytes.toBytes("EEE"));
3974     hTable.put(put);
3975 
3976     Get get = new Get(row);
3977     get.addColumn(FAMILY, qualifier);
3978     get.setMaxVersions();
3979 
3980     // Check that the column indeed has the right values at timestamps 1 and
3981     // 2
3982     Result result = hTable.get(get);
3983     NavigableMap<Long, byte[]> navigableMap =
3984         result.getMap().get(FAMILY).get(qualifier);
3985     assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
3986     assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
3987 
3988     // Trigger a major compaction
3989     admin.flush(tableName);
3990     admin.majorCompact(tableName);
3991     Thread.sleep(6000);
3992 
3993     // Update the value at timestamp 1
3994     put = new Put(row);
3995     put.add(FAMILY, qualifier, 1L, Bytes.toBytes("CCC"));
3996     hTable.put(put);
3997 
3998     // Trigger a major compaction
3999     admin.flush(tableName);
4000     admin.majorCompact(tableName);
4001     Thread.sleep(6000);
4002 
4003     // Update the value at timestamp 2
4004     put = new Put(row);
4005     put.add(FAMILY, qualifier, 2L, Bytes.toBytes("DDD"));
4006     hTable.put(put);
4007 
4008     // Trigger a major compaction
4009     admin.flush(tableName);
4010     admin.majorCompact(tableName);
4011     Thread.sleep(6000);
4012 
4013     // Check that the values at timestamp 2 and 1 got updated
4014     result = hTable.get(get);
4015     navigableMap = result.getMap().get(FAMILY).get(qualifier);
4016 
4017     assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
4018     assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
4019   }
4020 
4021   @Test
4022   public void testGet_EmptyTable() throws IOException, InterruptedException {
4023     Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_EmptyTable"), FAMILY);
4024     TEST_UTIL.waitTableAvailable(TableName.valueOf("testGet_EmptyTable"), 10000);
4025     Get get = new Get(ROW);
4026     get.addFamily(FAMILY);
4027     Result r = table.get(get);
4028     assertTrue(r.isEmpty());
4029   }
4030 
4031   @Test
4032   public void testGet_NullQualifier() throws IOException, InterruptedException {
4033     Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_NullQualifier"), FAMILY);
4034     TEST_UTIL.waitTableAvailable(TableName.valueOf("testGet_NullQualifier"), 10000);
4035     Put put = new Put(ROW);
4036     put.add(FAMILY, QUALIFIER, VALUE);
4037     table.put(put);
4038 
4039     put = new Put(ROW);
4040     put.add(FAMILY, null, VALUE);
4041     table.put(put);
4042     LOG.info("Row put");
4043 
4044     Get get = new Get(ROW);
4045     get.addColumn(FAMILY, null);
4046     Result r = table.get(get);
4047     assertEquals(1, r.size());
4048 
4049     get = new Get(ROW);
4050     get.addFamily(FAMILY);
4051     r = table.get(get);
4052     assertEquals(2, r.size());
4053   }
4054 
4055   @Test
4056   public void testGet_NonExistentRow() throws IOException, InterruptedException {
4057     Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_NonExistentRow"), FAMILY);
4058     TEST_UTIL.waitTableAvailable(TableName.valueOf("testGet_NonExistentRow"), 10000);
4059     Put put = new Put(ROW);
4060     put.add(FAMILY, QUALIFIER, VALUE);
4061     table.put(put);
4062     LOG.info("Row put");
4063 
4064     Get get = new Get(ROW);
4065     get.addFamily(FAMILY);
4066     Result r = table.get(get);
4067     assertFalse(r.isEmpty());
4068     System.out.println("Row retrieved successfully");
4069 
4070     byte [] missingrow = Bytes.toBytes("missingrow");
4071     get = new Get(missingrow);
4072     get.addFamily(FAMILY);
4073     r = table.get(get);
4074     assertTrue(r.isEmpty());
4075     LOG.info("Row missing as it should be");
4076   }
4077 
4078   @Test
4079   public void testPut() throws IOException, InterruptedException {
4080     final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
4081     final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
4082     final byte [] row1 = Bytes.toBytes("row1");
4083     final byte [] row2 = Bytes.toBytes("row2");
4084     final byte [] value = Bytes.toBytes("abcd");
4085     Table table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
4086       new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
4087     TEST_UTIL.waitTableAvailable(Bytes.toBytes("testPut"), 10000);
4088     Put put = new Put(row1);
4089     put.add(CONTENTS_FAMILY, null, value);
4090     table.put(put);
4091 
4092     put = new Put(row2);
4093     put.add(CONTENTS_FAMILY, null, value);
4094 
4095     assertEquals(put.size(), 1);
4096     assertEquals(put.getFamilyCellMap().get(CONTENTS_FAMILY).size(), 1);
4097 
4098     // KeyValue v1 expectation.  Cast for now until we go all Cell all the time. TODO
4099     KeyValue kv = (KeyValue)put.getFamilyCellMap().get(CONTENTS_FAMILY).get(0);
4100 
4101     assertTrue(Bytes.equals(kv.getFamily(), CONTENTS_FAMILY));
4102     // will it return null or an empty byte array?
4103     assertTrue(Bytes.equals(kv.getQualifier(), new byte[0]));
4104 
4105     assertTrue(Bytes.equals(kv.getValue(), value));
4106 
4107     table.put(put);
4108 
4109     Scan scan = new Scan();
4110     scan.addColumn(CONTENTS_FAMILY, null);
4111     ResultScanner scanner = table.getScanner(scan);
4112     for (Result r : scanner) {
4113       for(Cell key : r.rawCells()) {
4114         System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString());
4115       }
4116     }
4117   }
4118 
4119   @Test
4120   public void testPutNoCF() throws IOException, InterruptedException {
4121     final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
4122     final byte[] VAL = Bytes.toBytes(100);
4123     Table table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), FAMILY);
4124     TEST_UTIL.waitTableAvailable(Bytes.toBytes("testPutNoCF"), 10000);
4125 
4126     boolean caughtNSCFE = false;
4127 
4128     try {
4129       Put p = new Put(ROW);
4130       p.add(BAD_FAM, QUALIFIER, VAL);
4131       table.put(p);
4132     } catch (RetriesExhaustedWithDetailsException e) {
4133       caughtNSCFE = e.getCause(0) instanceof NoSuchColumnFamilyException;
4134     }
4135     assertTrue("Should throw NoSuchColumnFamilyException", caughtNSCFE);
4136 
4137   }
4138 
4139   @Test
4140   public void testRowsPut() throws IOException, InterruptedException {
4141     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
4142     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
4143     final int NB_BATCH_ROWS = 10;
4144     final byte[] value = Bytes.toBytes("abcd");
4145     Table table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
4146       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
4147     TEST_UTIL.waitTableAvailable(Bytes.toBytes("testRowsPut"), 10000);
4148     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
4149     for (int i = 0; i < NB_BATCH_ROWS; i++) {
4150       byte[] row = Bytes.toBytes("row" + i);
4151       Put put = new Put(row);
4152       put.setDurability(Durability.SKIP_WAL);
4153       put.add(CONTENTS_FAMILY, null, value);
4154       rowsUpdate.add(put);
4155     }
4156     table.put(rowsUpdate);
4157     Scan scan = new Scan();
4158     scan.addFamily(CONTENTS_FAMILY);
4159     ResultScanner scanner = table.getScanner(scan);
4160     int nbRows = 0;
4161     for (@SuppressWarnings("unused")
4162     Result row : scanner)
4163       nbRows++;
4164     assertEquals(NB_BATCH_ROWS, nbRows);
4165   }
4166 
4167   @Test
4168   public void testRowsPutBufferedOneFlush() throws IOException, InterruptedException {
4169     final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
4170     final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
4171     final byte [] value = Bytes.toBytes("abcd");
4172     final int NB_BATCH_ROWS = 10;
4173     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
4174       new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
4175     TEST_UTIL.waitTableAvailable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
4176         10000);
4177     table.setAutoFlush(false);
4178     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
4179     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
4180       byte[] row = Bytes.toBytes("row" + i);
4181       Put put = new Put(row);
4182       put.setDurability(Durability.SKIP_WAL);
4183       put.add(CONTENTS_FAMILY, null, value);
4184       rowsUpdate.add(put);
4185     }
4186     table.put(rowsUpdate);
4187 
4188     Scan scan = new Scan();
4189     scan.addFamily(CONTENTS_FAMILY);
4190     ResultScanner scanner = table.getScanner(scan);
4191     int nbRows = 0;
4192     for (@SuppressWarnings("unused")
4193     Result row : scanner)
4194       nbRows++;
4195     assertEquals(0, nbRows);
4196     scanner.close();
4197 
4198     table.flushCommits();
4199 
4200     scan = new Scan();
4201     scan.addFamily(CONTENTS_FAMILY);
4202     scanner = table.getScanner(scan);
4203     nbRows = 0;
4204     for (@SuppressWarnings("unused")
4205     Result row : scanner)
4206       nbRows++;
4207     assertEquals(NB_BATCH_ROWS * 10, nbRows);
4208     table.close();
4209   }
4210 
4211   @Test
4212   public void testRowsPutBufferedManyManyFlushes() throws IOException, InterruptedException {
4213     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
4214     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
4215     final byte[] value = Bytes.toBytes("abcd");
4216     final int NB_BATCH_ROWS = 10;
4217     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
4218       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
4219     TEST_UTIL.waitTableAvailable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
4220         10000);
4221     table.setWriteBufferSize(10);
4222     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
4223     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
4224       byte[] row = Bytes.toBytes("row" + i);
4225       Put put = new Put(row);
4226       put.setDurability(Durability.SKIP_WAL);
4227       put.add(CONTENTS_FAMILY, null, value);
4228       rowsUpdate.add(put);
4229     }
4230     table.put(rowsUpdate);
4231 
4232     Scan scan = new Scan();
4233     scan.addFamily(CONTENTS_FAMILY);
4234     ResultScanner scanner = table.getScanner(scan);
4235     int nbRows = 0;
4236     for (@SuppressWarnings("unused")
4237     Result row : scanner)
4238       nbRows++;
4239     assertEquals(NB_BATCH_ROWS * 10, nbRows);
4240   }
4241 
4242   @Test
4243   public void testAddKeyValue() throws IOException {
4244     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
4245     final byte[] value = Bytes.toBytes("abcd");
4246     final byte[] row1 = Bytes.toBytes("row1");
4247     final byte[] row2 = Bytes.toBytes("row2");
4248     byte[] qualifier = Bytes.toBytes("qf1");
4249     Put put = new Put(row1);
4250 
4251     // Adding KeyValue with the same row
4252     KeyValue kv = new KeyValue(row1, CONTENTS_FAMILY, qualifier, value);
4253     boolean ok = true;
4254     try {
4255       put.add(kv);
4256     } catch (IOException e) {
4257       ok = false;
4258     }
4259     assertEquals(true, ok);
4260 
4261     // Adding KeyValue with the different row
4262     kv = new KeyValue(row2, CONTENTS_FAMILY, qualifier, value);
4263     ok = false;
4264     try {
4265       put.add(kv);
4266     } catch (IOException e) {
4267       ok = true;
4268     }
4269     assertEquals(true, ok);
4270   }
4271 
4272   private List<Result> doAppend(final boolean walUsed) throws IOException, InterruptedException {
4273     LOG.info("Starting testAppend, walUsed is " + walUsed);
4274     final TableName TABLENAME = TableName.valueOf(walUsed ? "testAppendWithWAL" : "testAppendWithoutWAL");
4275     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4276     TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
4277     final byte[] row1 = Bytes.toBytes("c");
4278     final byte[] row2 = Bytes.toBytes("b");
4279     final byte[] row3 = Bytes.toBytes("a");
4280     final byte[] qual = Bytes.toBytes("qual");
4281     Put put_0 = new Put(row2);
4282     put_0.addColumn(FAMILY, qual, Bytes.toBytes("put"));
4283     Put put_1 = new Put(row3);
4284     put_1.addColumn(FAMILY, qual, Bytes.toBytes("put"));
4285     Append append_0 = new Append(row1);
4286     append_0.add(FAMILY, qual, Bytes.toBytes("i"));
4287     Append append_1 = new Append(row1);
4288     append_1.add(FAMILY, qual, Bytes.toBytes("k"));
4289     Append append_2 = new Append(row1);
4290     append_2.add(FAMILY, qual, Bytes.toBytes("e"));
4291     if (!walUsed) {
4292       append_2.setDurability(Durability.SKIP_WAL);
4293     }
4294     Append append_3 = new Append(row1);
4295     append_3.add(FAMILY, qual, Bytes.toBytes("a"));
4296     Scan s = new Scan();
4297     s.setCaching(1);
4298     t.append(append_0);
4299     t.put(put_0);
4300     t.put(put_1);
4301     List<Result> results = new LinkedList<>();
4302     try (ResultScanner scanner = t.getScanner(s)) {
4303       t.append(append_1);
4304       t.append(append_2);
4305       t.append(append_3);
4306       for (Result r : scanner) {
4307         results.add(r);
4308       }
4309     }
4310     TEST_UTIL.deleteTable(TABLENAME);
4311     return results;
4312   }
4313 
4314   @Test
4315   public void testAppendWithoutWAL() throws Exception {
4316     List<Result> resultsWithWal = doAppend(true);
4317     List<Result> resultsWithoutWal = doAppend(false);
4318     assertEquals(resultsWithWal.size(), resultsWithoutWal.size());
4319     for (int i = 0; i != resultsWithWal.size(); ++i) {
4320       Result resultWithWal = resultsWithWal.get(i);
4321       Result resultWithoutWal = resultsWithoutWal.get(i);
4322       assertEquals(resultWithWal.rawCells().length, resultWithoutWal.rawCells().length);
4323       for (int j = 0; j != resultWithWal.rawCells().length; ++j) {
4324         Cell cellWithWal = resultWithWal.rawCells()[j];
4325         Cell cellWithoutWal = resultWithoutWal.rawCells()[j];
4326         assertTrue(Bytes.equals(CellUtil.cloneRow(cellWithWal), CellUtil.cloneRow(cellWithoutWal)));
4327         assertTrue(Bytes.equals(CellUtil.cloneFamily(cellWithWal), CellUtil.cloneFamily(cellWithoutWal)));
4328         assertTrue(Bytes.equals(CellUtil.cloneQualifier(cellWithWal), CellUtil.cloneQualifier(cellWithoutWal)));
4329         assertTrue(Bytes.equals(CellUtil.cloneValue(cellWithWal), CellUtil.cloneValue(cellWithoutWal)));
4330       }
4331     }
4332   }
4333 
4334   /**
4335    * test for HBASE-737
4336    * @throws IOException
4337    */
4338   @Test
4339   public void testHBase737() throws IOException, InterruptedException {
4340     final byte [] FAM1 = Bytes.toBytes("fam1");
4341     final byte [] FAM2 = Bytes.toBytes("fam2");
4342     // Open table
4343     Table table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
4344       new byte [][] {FAM1, FAM2});
4345     TEST_UTIL.waitTableAvailable(Bytes.toBytes("testHBase737"), 10000);
4346     // Insert some values
4347     Put put = new Put(ROW);
4348     put.add(FAM1, Bytes.toBytes("letters"), Bytes.toBytes("abcdefg"));
4349     table.put(put);
4350     try {
4351       Thread.sleep(1000);
4352     } catch (InterruptedException i) {
4353       //ignore
4354     }
4355 
4356     put = new Put(ROW);
4357     put.add(FAM1, Bytes.toBytes("numbers"), Bytes.toBytes("123456"));
4358     table.put(put);
4359 
4360     try {
4361       Thread.sleep(1000);
4362     } catch (InterruptedException i) {
4363       //ignore
4364     }
4365 
4366     put = new Put(ROW);
4367     put.add(FAM2, Bytes.toBytes("letters"), Bytes.toBytes("hijklmnop"));
4368     table.put(put);
4369 
4370     long times[] = new long[3];
4371 
4372     // First scan the memstore
4373 
4374     Scan scan = new Scan();
4375     scan.addFamily(FAM1);
4376     scan.addFamily(FAM2);
4377     ResultScanner s = table.getScanner(scan);
4378     try {
4379       int index = 0;
4380       Result r = null;
4381       while ((r = s.next()) != null) {
4382         for(Cell key : r.rawCells()) {
4383           times[index++] = key.getTimestamp();
4384         }
4385       }
4386     } finally {
4387       s.close();
4388     }
4389     for (int i = 0; i < times.length - 1; i++) {
4390       for (int j = i + 1; j < times.length; j++) {
4391         assertTrue(times[j] > times[i]);
4392       }
4393     }
4394 
4395     // Flush data to disk and try again
4396     TEST_UTIL.flush();
4397 
4398     // Reset times
4399     for(int i=0;i<times.length;i++) {
4400       times[i] = 0;
4401     }
4402 
4403     try {
4404       Thread.sleep(1000);
4405     } catch (InterruptedException i) {
4406       //ignore
4407     }
4408     scan = new Scan();
4409     scan.addFamily(FAM1);
4410     scan.addFamily(FAM2);
4411     s = table.getScanner(scan);
4412     try {
4413       int index = 0;
4414       Result r = null;
4415       while ((r = s.next()) != null) {
4416         for(Cell key : r.rawCells()) {
4417           times[index++] = key.getTimestamp();
4418         }
4419       }
4420     } finally {
4421       s.close();
4422     }
4423     for (int i = 0; i < times.length - 1; i++) {
4424       for (int j = i + 1; j < times.length; j++) {
4425         assertTrue(times[j] > times[i]);
4426       }
4427     }
4428   }
4429 
4430   @Test
4431   public void testListTables() throws IOException, InterruptedException {
4432     TableName t1 = TableName.valueOf("testListTables1");
4433     TableName t2 = TableName.valueOf("testListTables2");
4434     TableName t3 = TableName.valueOf("testListTables3");
4435     TableName [] tables = new TableName[] { t1, t2, t3 };
4436     for (int i = 0; i < tables.length; i++) {
4437       TEST_UTIL.createTable(tables[i], FAMILY);
4438       TEST_UTIL.waitTableAvailable(tables[i], 10000);
4439     }
4440     Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
4441     HTableDescriptor[] ts = admin.listTables();
4442     HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
4443     Collections.addAll(result, ts);
4444     int size = result.size();
4445     assertTrue(size >= tables.length);
4446     for (int i = 0; i < tables.length && i < size; i++) {
4447       boolean found = false;
4448       for (int j = 0; j < ts.length; j++) {
4449         if (ts[j].getTableName().equals(tables[i])) {
4450           found = true;
4451           break;
4452         }
4453       }
4454       assertTrue("Not found: " + tables[i], found);
4455     }
4456   }
4457 
4458   /**
4459    * creates an HTable for tableName using an unmanaged HConnection.
4460    *
4461    * @param tableName - table to create
4462    * @return the created HTable object
4463    * @throws IOException
4464    */
4465   HTable createUnmangedHConnectionHTable(final TableName tableName) throws IOException,
4466       InterruptedException {
4467     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
4468     TEST_UTIL.waitTableAvailable(tableName, 10000);
4469     HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
4470     return (HTable)conn.getTable(tableName);
4471   }
4472 
4473   /**
4474    * simple test that just executes parts of the client
4475    * API that accept a pre-created HConnection instance
4476    *
4477    * @throws IOException
4478    */
4479   @Test
4480   public void testUnmanagedHConnection() throws IOException, InterruptedException {
4481     final TableName tableName = TableName.valueOf("testUnmanagedHConnection");
4482     HTable t = createUnmangedHConnectionHTable(tableName);
4483     HBaseAdmin ha = new HBaseAdmin(t.getConnection());
4484     assertTrue(ha.tableExists(tableName));
4485     assertTrue(t.get(new Get(ROW)).isEmpty());
4486     ha.close();
4487   }
4488 
4489   /**
4490    * test of that unmanaged HConnections are able to reconnect
4491    * properly (see HBASE-5058)
4492    *
4493    * @throws Exception
4494    */
4495   @Test
4496   public void testUnmanagedHConnectionReconnect() throws Exception {
4497     Configuration conf = TEST_UTIL.getConfiguration();
4498     Class registryImpl = conf.getClass(
4499         HConstants.REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
4500     // This test does not make sense for MasterRegistry since it stops the only master in the
4501     // cluster and starts a new master without populating the underlying config for the connection.
4502     Assume.assumeFalse(registryImpl.equals(MasterRegistry.class));
4503     final TableName tableName = TableName.valueOf("testUnmanagedHConnectionReconnect");
4504     HTable t = createUnmangedHConnectionHTable(tableName);
4505     Connection conn = t.getConnection();
4506     try (HBaseAdmin ha = new HBaseAdmin(conn)) {
4507       assertTrue(ha.tableExists(tableName));
4508       assertTrue(t.get(new Get(ROW)).isEmpty());
4509     }
4510 
4511     // stop the master
4512     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
4513     cluster.stopMaster(0, false);
4514     cluster.waitOnMaster(0);
4515 
4516     // start up a new master
4517     cluster.startMaster();
4518     assertTrue(cluster.waitForActiveAndReadyMaster());
4519 
4520     // test that the same unmanaged connection works with a new
4521     // HBaseAdmin and can connect to the new master;
4522     try (HBaseAdmin newAdmin = new HBaseAdmin(conn)) {
4523       assertTrue(newAdmin.tableExists(tableName));
4524       assertTrue(newAdmin.getClusterStatus().getServersSize() == SLAVES);
4525     }
4526   }
4527 
4528   @Test
4529   public void testMiscHTableStuff() throws IOException, InterruptedException {
4530     final TableName tableAname = TableName.valueOf("testMiscHTableStuffA");
4531     final TableName tableBname = TableName.valueOf("testMiscHTableStuffB");
4532     final byte[] attrName = Bytes.toBytes("TESTATTR");
4533     final byte[] attrValue = Bytes.toBytes("somevalue");
4534     byte[] value = Bytes.toBytes("value");
4535 
4536     Table a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY);
4537     TEST_UTIL.waitTableAvailable(tableAname, 10000);
4538     Table b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY);
4539     TEST_UTIL.waitTableAvailable(tableBname, 10000);
4540     Put put = new Put(ROW);
4541     put.add(HConstants.CATALOG_FAMILY, null, value);
4542     a.put(put);
4543 
4544     // open a new connection to A and a connection to b
4545     Table newA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
4546 
4547     // copy data from A to B
4548     Scan scan = new Scan();
4549     scan.addFamily(HConstants.CATALOG_FAMILY);
4550     ResultScanner s = newA.getScanner(scan);
4551     try {
4552       for (Result r : s) {
4553         put = new Put(r.getRow());
4554         put.setDurability(Durability.SKIP_WAL);
4555         for (Cell kv : r.rawCells()) {
4556           put.add(kv);
4557         }
4558         b.put(put);
4559       }
4560     } finally {
4561       s.close();
4562     }
4563 
4564     // Opening a new connection to A will cause the tables to be reloaded
4565     Table anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
4566     Get get = new Get(ROW);
4567     get.addFamily(HConstants.CATALOG_FAMILY);
4568     anotherA.get(get);
4569 
4570     // We can still access A through newA because it has the table information
4571     // cached. And if it needs to recalibrate, that will cause the information
4572     // to be reloaded.
4573 
4574     // Test user metadata
4575     Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
4576     // make a modifiable descriptor
4577     HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor());
4578     // offline the table
4579     admin.disableTable(tableAname);
4580     // add a user attribute to HTD
4581     desc.setValue(attrName, attrValue);
4582     // add a user attribute to HCD
4583     for (HColumnDescriptor c : desc.getFamilies())
4584       c.setValue(attrName, attrValue);
4585     // update metadata for all regions of this table
4586     admin.modifyTable(tableAname, desc);
4587     // enable the table
4588     admin.enableTable(tableAname);
4589 
4590     // Test that attribute changes were applied
4591     desc = a.getTableDescriptor();
4592     assertEquals("wrong table descriptor returned", desc.getTableName(), tableAname);
4593     // check HTD attribute
4594     value = desc.getValue(attrName);
4595     assertFalse("missing HTD attribute value", value == null);
4596     assertFalse("HTD attribute value is incorrect",
4597       Bytes.compareTo(value, attrValue) != 0);
4598     // check HCD attribute
4599     for (HColumnDescriptor c : desc.getFamilies()) {
4600       value = c.getValue(attrName);
4601       assertFalse("missing HCD attribute value", value == null);
4602       assertFalse("HCD attribute value is incorrect",
4603         Bytes.compareTo(value, attrValue) != 0);
4604     }
4605   }
4606 
4607   @Test
4608   public void testGetClosestRowBefore() throws IOException, InterruptedException {
4609     final TableName tableAname = TableName.valueOf("testGetClosestRowBefore");
4610     final byte[] firstRow = Bytes.toBytes("row111");
4611     final byte[] secondRow = Bytes.toBytes("row222");
4612     final byte[] thirdRow = Bytes.toBytes("row333");
4613     final byte[] forthRow = Bytes.toBytes("row444");
4614     final byte[] beforeFirstRow = Bytes.toBytes("row");
4615     final byte[] beforeSecondRow = Bytes.toBytes("row22");
4616     final byte[] beforeThirdRow = Bytes.toBytes("row33");
4617     final byte[] beforeForthRow = Bytes.toBytes("row44");
4618 
4619     HTable table =
4620         TEST_UTIL.createTable(tableAname,
4621           new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1,
4622             1024);
4623     TEST_UTIL.waitTableAvailable(tableAname, 10000);
4624     // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
4625     // in Store.rowAtOrBeforeFromStoreFile
4626     String regionName = table.getRegionLocations().firstKey().getEncodedName();
4627     Region region =
4628         TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName);
4629     Put put1 = new Put(firstRow);
4630     Put put2 = new Put(secondRow);
4631     Put put3 = new Put(thirdRow);
4632     Put put4 = new Put(forthRow);
4633     byte[] one = new byte[] { 1 };
4634     byte[] two = new byte[] { 2 };
4635     byte[] three = new byte[] { 3 };
4636     byte[] four = new byte[] { 4 };
4637 
4638     put1.add(HConstants.CATALOG_FAMILY, null, one);
4639     put2.add(HConstants.CATALOG_FAMILY, null, two);
4640     put3.add(HConstants.CATALOG_FAMILY, null, three);
4641     put4.add(HConstants.CATALOG_FAMILY, null, four);
4642     table.put(put1);
4643     table.put(put2);
4644     table.put(put3);
4645     table.put(put4);
4646     region.flush(true);
4647     Result result = null;
4648 
4649     // Test before first that null is returned
4650     result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY);
4651     assertTrue(result == null);
4652 
4653     // Test at first that first is returned
4654     result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY);
4655     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4656     assertTrue(Bytes.equals(result.getRow(), firstRow));
4657     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
4658 
4659     // Test in between first and second that first is returned
4660     result = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY);
4661     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4662     assertTrue(Bytes.equals(result.getRow(), firstRow));
4663     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
4664 
4665     // Test at second make sure second is returned
4666     result = table.getRowOrBefore(secondRow, HConstants.CATALOG_FAMILY);
4667     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4668     assertTrue(Bytes.equals(result.getRow(), secondRow));
4669     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
4670 
4671     // Test in second and third, make sure second is returned
4672     result = table.getRowOrBefore(beforeThirdRow, HConstants.CATALOG_FAMILY);
4673     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4674     assertTrue(Bytes.equals(result.getRow(), secondRow));
4675     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
4676 
4677     // Test at third make sure third is returned
4678     result = table.getRowOrBefore(thirdRow, HConstants.CATALOG_FAMILY);
4679     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4680     assertTrue(Bytes.equals(result.getRow(), thirdRow));
4681     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
4682 
4683     // Test in third and forth, make sure third is returned
4684     result = table.getRowOrBefore(beforeForthRow, HConstants.CATALOG_FAMILY);
4685     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4686     assertTrue(Bytes.equals(result.getRow(), thirdRow));
4687     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
4688 
4689     // Test at forth make sure forth is returned
4690     result = table.getRowOrBefore(forthRow, HConstants.CATALOG_FAMILY);
4691     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4692     assertTrue(Bytes.equals(result.getRow(), forthRow));
4693     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
4694 
4695     // Test after forth make sure forth is returned
4696     result = table.getRowOrBefore(Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY);
4697     assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
4698     assertTrue(Bytes.equals(result.getRow(), forthRow));
4699     assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
4700 
4701     table.close();
4702   }
4703 
4704   /**
4705    * For HBASE-2156
4706    * @throws Exception
4707    */
4708   @Test
4709   public void testScanVariableReuse() throws Exception {
4710     Scan scan = new Scan();
4711     scan.addFamily(FAMILY);
4712     scan.addColumn(FAMILY, ROW);
4713 
4714     assertTrue(scan.getFamilyMap().get(FAMILY).size() == 1);
4715 
4716     scan = new Scan();
4717     scan.addFamily(FAMILY);
4718 
4719     assertTrue(scan.getFamilyMap().get(FAMILY) == null);
4720     assertTrue(scan.getFamilyMap().containsKey(FAMILY));
4721   }
4722 
4723   @Test
4724   public void testMultiRowMutation() throws Exception {
4725     LOG.info("Starting testMultiRowMutation");
4726     final TableName TABLENAME = TableName.valueOf("testMultiRowMutation");
4727     final byte [] ROW1 = Bytes.toBytes("testRow1");
4728 
4729     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4730     TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
4731     Put p = new Put(ROW);
4732     p.add(FAMILY, QUALIFIER, VALUE);
4733     MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p);
4734 
4735     p = new Put(ROW1);
4736     p.add(FAMILY, QUALIFIER, VALUE);
4737     MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, p);
4738 
4739     MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder();
4740     mrmBuilder.addMutationRequest(m1);
4741     mrmBuilder.addMutationRequest(m2);
4742     MutateRowsRequest mrm = mrmBuilder.build();
4743     CoprocessorRpcChannel channel = t.coprocessorService(ROW);
4744     MultiRowMutationService.BlockingInterface service =
4745        MultiRowMutationService.newBlockingStub(channel);
4746     service.mutateRows(null, mrm);
4747     Get g = new Get(ROW);
4748     Result r = t.get(g);
4749     assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)));
4750     g = new Get(ROW1);
4751     r = t.get(g);
4752     assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)));
4753   }
4754 
4755   @Test
4756   public void testRowMutation() throws Exception {
4757     LOG.info("Starting testRowMutation");
4758     final TableName TABLENAME = TableName.valueOf("testRowMutation");
4759     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4760     TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
4761     byte [][] QUALIFIERS = new byte [][] {
4762         Bytes.toBytes("a"), Bytes.toBytes("b")
4763     };
4764     RowMutations arm = new RowMutations(ROW);
4765     Put p = new Put(ROW);
4766     p.add(FAMILY, QUALIFIERS[0], VALUE);
4767     arm.add(p);
4768     t.mutateRow(arm);
4769 
4770     Get g = new Get(ROW);
4771     Result r = t.get(g);
4772     assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0])));
4773 
4774     arm = new RowMutations(ROW);
4775     p = new Put(ROW);
4776     p.add(FAMILY, QUALIFIERS[1], VALUE);
4777     arm.add(p);
4778     Delete d = new Delete(ROW);
4779     d.deleteColumns(FAMILY, QUALIFIERS[0]);
4780     arm.add(d);
4781     // TODO: Trying mutateRow again.  The batch was failing with a one try only.
4782     t.mutateRow(arm);
4783     r = t.get(g);
4784     assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1])));
4785     assertNull(r.getValue(FAMILY, QUALIFIERS[0]));
4786 
4787     //Test that we get a region level exception
4788     try {
4789       arm = new RowMutations(ROW);
4790       p = new Put(ROW);
4791       p.add(new byte[]{'b', 'o', 'g', 'u', 's'}, QUALIFIERS[0], VALUE);
4792       arm.add(p);
4793       t.mutateRow(arm);
4794       fail("Expected NoSuchColumnFamilyException");
4795     } catch(RetriesExhaustedWithDetailsException e) {
4796       for(Throwable rootCause: e.getCauses()){
4797         if(rootCause instanceof NoSuchColumnFamilyException){
4798           return;
4799         }
4800       }
4801       throw e;
4802     }
4803   }
4804 
4805   @Test
4806   public void testAppend() throws Exception {
4807     LOG.info("Starting testAppend");
4808     final TableName TABLENAME = TableName.valueOf("testAppend");
4809     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
4810     TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
4811     byte[] v1 = Bytes.toBytes("42");
4812     byte[] v2 = Bytes.toBytes("23");
4813     byte [][] QUALIFIERS = new byte [][] {
4814         Bytes.toBytes("b"), Bytes.toBytes("a"), Bytes.toBytes("c")
4815     };
4816     Append a = new Append(ROW);
4817     a.add(FAMILY, QUALIFIERS[0], v1);
4818     a.add(FAMILY, QUALIFIERS[1], v2);
4819     a.setReturnResults(false);
4820     assertNullResult(t.append(a));
4821 
4822     a = new Append(ROW);
4823     a.add(FAMILY, QUALIFIERS[0], v2);
4824     a.add(FAMILY, QUALIFIERS[1], v1);
4825     a.add(FAMILY, QUALIFIERS[2], v2);
4826     Result r = t.append(a);
4827     assertEquals(0, Bytes.compareTo(Bytes.add(v1,v2), r.getValue(FAMILY, QUALIFIERS[0])));
4828     assertEquals(0, Bytes.compareTo(Bytes.add(v2,v1), r.getValue(FAMILY, QUALIFIERS[1])));
4829     // QUALIFIERS[2] previously not exist, verify both value and timestamp are correct
4830     assertEquals(0, Bytes.compareTo(v2, r.getValue(FAMILY, QUALIFIERS[2])));
4831     assertEquals(r.getColumnLatest(FAMILY, QUALIFIERS[0]).getTimestamp(),
4832         r.getColumnLatest(FAMILY, QUALIFIERS[2]).getTimestamp());
4833   }
4834 
4835   @Test
4836   public void testClientPoolRoundRobin() throws IOException, InterruptedException {
4837     final TableName tableName = TableName.valueOf("testClientPoolRoundRobin");
4838 
4839     int poolSize = 3;
4840     int numVersions = poolSize * 2;
4841     Configuration conf = TEST_UTIL.getConfiguration();
4842     conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin");
4843     conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4844 
4845     Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, conf, Integer.MAX_VALUE);
4846     TEST_UTIL.waitTableAvailable(tableName, 10000);
4847 
4848     final long ts = EnvironmentEdgeManager.currentTime();
4849     Get get = new Get(ROW);
4850     get.addColumn(FAMILY, QUALIFIER);
4851     get.setMaxVersions();
4852 
4853     for (int versions = 1; versions <= numVersions; versions++) {
4854       Put put = new Put(ROW);
4855       put.add(FAMILY, QUALIFIER, ts + versions, VALUE);
4856       table.put(put);
4857 
4858       Result result = table.get(get);
4859       NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4860           .get(QUALIFIER);
4861 
4862       assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":"
4863           + Bytes.toString(QUALIFIER) + " did not match", versions, navigableMap.size());
4864       for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4865         assertTrue("The value at time " + entry.getKey()
4866             + " did not match what was put",
4867             Bytes.equals(VALUE, entry.getValue()));
4868       }
4869     }
4870   }
4871 
4872   @Ignore ("Flakey: HBASE-8989") @Test
4873   public void testClientPoolThreadLocal() throws IOException, InterruptedException {
4874     final TableName tableName = TableName.valueOf("testClientPoolThreadLocal");
4875 
4876     int poolSize = Integer.MAX_VALUE;
4877     int numVersions = 3;
4878     Configuration conf = TEST_UTIL.getConfiguration();
4879     conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local");
4880     conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
4881 
4882     final Table table = TEST_UTIL.createTable(tableName,
4883         new byte[][] { FAMILY }, conf, 3);
4884     TEST_UTIL.waitTableAvailable(tableName, 10000);
4885     final long ts = EnvironmentEdgeManager.currentTime();
4886     final Get get = new Get(ROW);
4887     get.addColumn(FAMILY, QUALIFIER);
4888     get.setMaxVersions();
4889 
4890     for (int versions = 1; versions <= numVersions; versions++) {
4891       Put put = new Put(ROW);
4892       put.add(FAMILY, QUALIFIER, ts + versions, VALUE);
4893       table.put(put);
4894 
4895       Result result = table.get(get);
4896       NavigableMap<Long, byte[]> navigableMap = result.getMap().get(FAMILY)
4897           .get(QUALIFIER);
4898 
4899       assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":"
4900           + Bytes.toString(QUALIFIER) + " did not match", versions, navigableMap.size());
4901       for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4902         assertTrue("The value at time " + entry.getKey()
4903             + " did not match what was put",
4904             Bytes.equals(VALUE, entry.getValue()));
4905       }
4906     }
4907 
4908     final Object waitLock = new Object();
4909     ExecutorService executorService = Executors.newFixedThreadPool(numVersions);
4910     final AtomicReference<AssertionError> error = new AtomicReference<AssertionError>(null);
4911     for (int versions = numVersions; versions < numVersions * 2; versions++) {
4912       final int versionsCopy = versions;
4913       executorService.submit(new Callable<Void>() {
4914         @Override
4915         public Void call() {
4916           try {
4917             Put put = new Put(ROW);
4918             put.add(FAMILY, QUALIFIER, ts + versionsCopy, VALUE);
4919             table.put(put);
4920 
4921             Result result = table.get(get);
4922             NavigableMap<Long, byte[]> navigableMap = result.getMap()
4923                 .get(FAMILY).get(QUALIFIER);
4924 
4925             assertEquals("The number of versions of '" + Bytes.toString(FAMILY) + ":"
4926                 + Bytes.toString(QUALIFIER) + " did not match " + versionsCopy, versionsCopy,
4927                 navigableMap.size());
4928             for (Map.Entry<Long, byte[]> entry : navigableMap.entrySet()) {
4929               assertTrue("The value at time " + entry.getKey()
4930                   + " did not match what was put",
4931                   Bytes.equals(VALUE, entry.getValue()));
4932             }
4933             synchronized (waitLock) {
4934               waitLock.wait();
4935             }
4936           } catch (Exception e) {
4937           } catch (AssertionError e) {
4938             // the error happens in a thread, it won't fail the test,
4939             // need to pass it to the caller for proper handling.
4940             error.set(e);
4941             LOG.error(e);
4942           }
4943 
4944           return null;
4945         }
4946       });
4947     }
4948     synchronized (waitLock) {
4949       waitLock.notifyAll();
4950     }
4951     executorService.shutdownNow();
4952     assertNull(error.get());
4953   }
4954 
4955   @Test
4956   public void testCheckAndPut() throws IOException, InterruptedException {
4957     final byte [] anotherrow = Bytes.toBytes("anotherrow");
4958     final byte [] value2 = Bytes.toBytes("abcd");
4959 
4960     Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPut"), FAMILY);
4961     TEST_UTIL.waitTableAvailable(TableName.valueOf("testCheckAndPut"), 10000);
4962     Put put1 = new Put(ROW);
4963     put1.add(FAMILY, QUALIFIER, VALUE);
4964 
4965     // row doesn't exist, so using non-null value should be considered "not match".
4966     boolean ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put1);
4967     assertEquals(ok, false);
4968 
4969     // row doesn't exist, so using "null" to check for existence should be considered "match".
4970     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4971     assertEquals(ok, true);
4972 
4973     // row now exists, so using "null" to check for existence should be considered "not match".
4974     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put1);
4975     assertEquals(ok, false);
4976 
4977     Put put2 = new Put(ROW);
4978     put2.add(FAMILY, QUALIFIER, value2);
4979 
4980     // row now exists, use the matching value to check
4981     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, VALUE, put2);
4982     assertEquals(ok, true);
4983 
4984     Put put3 = new Put(anotherrow);
4985     put3.add(FAMILY, QUALIFIER, VALUE);
4986 
4987     // try to do CheckAndPut on different rows
4988     try {
4989         ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, value2, put3);
4990         fail("trying to check and modify different rows should have failed.");
4991     } catch(Exception e) {}
4992   }
4993 
4994   @Test
4995   public void testCheckAndPutWithCompareOp() throws IOException, InterruptedException {
4996     final byte [] value1 = Bytes.toBytes("aaaa");
4997     final byte [] value2 = Bytes.toBytes("bbbb");
4998     final byte [] value3 = Bytes.toBytes("cccc");
4999     final byte [] value4 = Bytes.toBytes("dddd");
5000 
5001     Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPutWithCompareOp"), FAMILY);
5002     TEST_UTIL.waitTableAvailable(TableName.valueOf("testCheckAndPutWithCompareOp"), 10000);
5003 
5004     Put put2 = new Put(ROW);
5005     put2.add(FAMILY, QUALIFIER, value2);
5006 
5007     Put put3 = new Put(ROW);
5008     put3.add(FAMILY, QUALIFIER, value3);
5009 
5010     // row doesn't exist, so using "null" to check for existence should be considered "match".
5011     boolean ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put2);
5012     assertEquals(ok, true);
5013 
5014     // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL
5015     // turns out "match"
5016     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.GREATER, value1, put2);
5017     assertEquals(ok, false);
5018     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.EQUAL, value1, put2);
5019     assertEquals(ok, false);
5020     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.GREATER_OR_EQUAL, value1, put2);
5021     assertEquals(ok, false);
5022     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.LESS, value1, put2);
5023     assertEquals(ok, true);
5024     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value1, put2);
5025     assertEquals(ok, true);
5026     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.NOT_EQUAL, value1, put3);
5027     assertEquals(ok, true);
5028 
5029     // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL
5030     // turns out "match"
5031     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.LESS, value4, put3);
5032     assertEquals(ok, false);
5033     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value4, put3);
5034     assertEquals(ok, false);
5035     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.EQUAL, value4, put3);
5036     assertEquals(ok, false);
5037     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.GREATER, value4, put3);
5038     assertEquals(ok, true);
5039     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.GREATER_OR_EQUAL, value4, put3);
5040     assertEquals(ok, true);
5041     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.NOT_EQUAL, value4, put2);
5042     assertEquals(ok, true);
5043 
5044     // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL
5045     // turns out "match"
5046     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.GREATER, value2, put2);
5047     assertEquals(ok, false);
5048     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.NOT_EQUAL, value2, put2);
5049     assertEquals(ok, false);
5050     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.LESS, value2, put2);
5051     assertEquals(ok, false);
5052     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.GREATER_OR_EQUAL, value2, put2);
5053     assertEquals(ok, true);
5054     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value2, put2);
5055     assertEquals(ok, true);
5056     ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, CompareOp.EQUAL, value2, put3);
5057     assertEquals(ok, true);
5058   }
5059 
5060   @Test
5061   public void testCheckAndDeleteWithCompareOp() throws IOException, InterruptedException {
5062     final byte [] value1 = Bytes.toBytes("aaaa");
5063     final byte [] value2 = Bytes.toBytes("bbbb");
5064     final byte [] value3 = Bytes.toBytes("cccc");
5065     final byte [] value4 = Bytes.toBytes("dddd");
5066     Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndDeleteWithCompareOp"),
5067         FAMILY);
5068     TEST_UTIL.waitTableAvailable(TableName.valueOf("testCheckAndDeleteWithCompareOp"), 10000);
5069 
5070     Put bbbb = new Put(ROW);
5071     bbbb.add(FAMILY, QUALIFIER, value2);
5072 
5073     Put cccc = new Put(ROW);
5074     cccc.add(FAMILY, QUALIFIER, value3);
5075 
5076     Delete delete = new Delete(ROW);
5077     delete.deleteColumns(FAMILY, QUALIFIER);
5078 
5079     // cell = "bbbb", using "aaaa" to compare only LESS/LESS_OR_EQUAL/NOT_EQUAL
5080     // turns out "match"
5081     table.put(bbbb);
5082     assertTrue(verifyPut(table, bbbb, value2));
5083     boolean ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER, value1, delete);
5084     // aaaa is less than bbbb, > || >= should be false
5085     assertFalse(ok);
5086     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.EQUAL, value1, delete);
5087     assertFalse(ok);
5088     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER_OR_EQUAL, value1, delete);
5089     assertFalse(ok);
5090     // aaaa is less than bbbb, < || <= should be true
5091     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS, value1, delete);
5092     assertTrue(ok);
5093     table.put(bbbb);
5094     assertTrue(verifyPut(table, bbbb, value2));
5095     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value1, delete);
5096     assertTrue(ok);
5097 
5098     // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL
5099     // turns out "match"
5100     table.put(cccc);
5101     assertTrue(verifyPut(table, cccc, value3));
5102     // dddd is larger than cccc,  < || <= shoule be false
5103     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS, value4, delete);
5104     assertFalse(ok);
5105     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value4, delete);
5106     assertFalse(ok);
5107     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.EQUAL, value4, delete);
5108     assertFalse(ok);
5109     // dddd is larger than cccc, (> || >= || !=) shoule be true
5110     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER, value4, delete);
5111     assertTrue(ok);
5112     table.put(cccc);
5113     assertTrue(verifyPut(table, cccc, value3));
5114     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER_OR_EQUAL, value4, delete);
5115     assertTrue(ok);
5116     table.put(cccc);
5117     assertTrue(verifyPut(table, cccc, value3));
5118     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.NOT_EQUAL, value4, delete);
5119     assertTrue(ok);
5120 
5121     // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL
5122     // turns out "match"
5123     // bbbb equals to bbbb, != shoule be all false
5124     table.put(bbbb);
5125     assertTrue(verifyPut(table, bbbb, value2));
5126     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER, value2, delete);
5127     assertFalse(ok);
5128     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.NOT_EQUAL, value2, delete);
5129     assertFalse(ok);
5130     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS, value2, delete);
5131     assertFalse(ok);
5132     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER_OR_EQUAL, value2, delete);
5133     assertTrue(ok);
5134     table.put(bbbb);
5135     assertTrue(verifyPut(table, bbbb, value2));
5136     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value2, delete);
5137     assertTrue(ok);
5138     table.put(bbbb);
5139     assertTrue(verifyPut(table, bbbb, value2));
5140     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.EQUAL, value2, delete);
5141     assertTrue(ok);
5142   }
5143 
5144   /**
5145    * testCheckAndDeleteWithCompareOp will occasionally fail without this method.
5146    * Somehow checkAndDelete is executed faster than put, so it leads to assertion failure.
5147    * With this verification, we can make sure checkAndDelete is executed after put.
5148    */
5149   private boolean verifyPut(Table table, Put put, byte[] value) throws IOException {
5150     table.put(put);
5151     Get get = new Get(ROW);
5152     get.addColumn(FAMILY, QUALIFIER);
5153     Result result = table.get(get);
5154     return Bytes.equals(value, result.value());
5155   }
5156 
5157   /**
5158   * Test ScanMetrics
5159   * @throws Exception
5160   */
5161   @Test
5162   @SuppressWarnings ("unused")
5163   public void testScanMetrics() throws Exception {
5164     TableName TABLENAME = TableName.valueOf("testScanMetrics");
5165 
5166     Configuration conf = TEST_UTIL.getConfiguration();
5167 
5168     // Set up test table:
5169     // Create table:
5170     HTable ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILY);
5171     int numOfRegions = -1;
5172     try (RegionLocator r = ht.getRegionLocator()) {
5173       numOfRegions = r.getStartKeys().length;
5174     }
5175     // Create 3 rows in the table, with rowkeys starting with "zzz*" so that
5176     // scan are forced to hit all the regions.
5177     Put put1 = new Put(Bytes.toBytes("zzz1"));
5178     put1.add(FAMILY, QUALIFIER, VALUE);
5179     Put put2 = new Put(Bytes.toBytes("zzz2"));
5180     put2.add(FAMILY, QUALIFIER, VALUE);
5181     Put put3 = new Put(Bytes.toBytes("zzz3"));
5182     put3.add(FAMILY, QUALIFIER, VALUE);
5183     ht.put(Arrays.asList(put1, put2, put3));
5184 
5185     Scan scan1 = new Scan();
5186     int numRecords = 0;
5187     ResultScanner scanner = ht.getScanner(scan1);
5188     for(Result result : scanner) {
5189       numRecords++;
5190     }
5191     scanner.close();
5192     LOG.info("test data has " + numRecords + " records.");
5193 
5194     // by default, scan metrics collection is turned off
5195     assertEquals(null, scan1.getScanMetrics());
5196 
5197     // turn on scan metrics
5198     Scan scan2 = new Scan();
5199     scan2.setScanMetricsEnabled(true);
5200     scan2.setCaching(numRecords+1);
5201     scanner = ht.getScanner(scan2);
5202     for (Result result : scanner.next(numRecords - 1)) {
5203     }
5204     scanner.close();
5205     // closing the scanner will set the metrics.
5206     assertNotNull(scan2.getScanMetrics());
5207 
5208     // set caching to 1, because metrics are collected in each roundtrip only
5209     scan2 = new Scan();
5210     scan2.setScanMetricsEnabled(true);
5211     scan2.setCaching(1);
5212     scanner = ht.getScanner(scan2);
5213     // per HBASE-5717, this should still collect even if you don't run all the way to
5214     // the end of the scanner. So this is asking for 2 of the 3 rows we inserted.
5215     for (Result result : scanner.next(numRecords - 1)) {
5216     }
5217     scanner.close();
5218 
5219     ScanMetrics scanMetrics = scan2.getScanMetrics();
5220     assertEquals("Did not access all the regions in the table", numOfRegions,
5221         scanMetrics.countOfRegions.get());
5222 
5223     // check byte counters
5224     scan2 = new Scan();
5225     scan2.setScanMetricsEnabled(true);
5226     scan2.setCaching(1);
5227     scanner = ht.getScanner(scan2);
5228     int numBytes = 0;
5229     for (Result result : scanner.next(1)) {
5230       for (Cell cell: result.listCells()) {
5231         numBytes += CellUtil.estimatedSerializedSizeOf(cell);
5232       }
5233     }
5234     scanner.close();
5235     scanMetrics = scan2.getScanMetrics();
5236     assertEquals("Did not count the result bytes", numBytes,
5237       scanMetrics.countOfBytesInResults.get());
5238 
5239     // check byte counters on a small scan
5240     scan2 = new Scan();
5241     scan2.setScanMetricsEnabled(true);
5242     scan2.setCaching(1);
5243     scan2.setSmall(true);
5244     scanner = ht.getScanner(scan2);
5245     numBytes = 0;
5246     for (Result result : scanner.next(1)) {
5247       for (Cell cell: result.listCells()) {
5248         numBytes += CellUtil.estimatedSerializedSizeOf(cell);
5249       }
5250     }
5251     scanner.close();
5252     scanMetrics = scan2.getScanMetrics();
5253     assertEquals("Did not count the result bytes", numBytes,
5254       scanMetrics.countOfBytesInResults.get());
5255 
5256     // now, test that the metrics are still collected even if you don't call close, but do
5257     // run past the end of all the records
5258     /** There seems to be a timing issue here.  Comment out for now. Fix when time.
5259     Scan scanWithoutClose = new Scan();
5260     scanWithoutClose.setCaching(1);
5261     scanWithoutClose.setScanMetricsEnabled(true);
5262     ResultScanner scannerWithoutClose = ht.getScanner(scanWithoutClose);
5263     for (Result result : scannerWithoutClose.next(numRecords + 1)) {
5264     }
5265     ScanMetrics scanMetricsWithoutClose = getScanMetrics(scanWithoutClose);
5266     assertEquals("Did not access all the regions in the table", numOfRegions,
5267         scanMetricsWithoutClose.countOfRegions.get());
5268     */
5269 
5270     // finally, test that the metrics are collected correctly if you both run past all the records,
5271     // AND close the scanner
5272     Scan scanWithClose = new Scan();
5273     // make sure we can set caching up to the number of a scanned values
5274     scanWithClose.setCaching(numRecords);
5275     scanWithClose.setScanMetricsEnabled(true);
5276     ResultScanner scannerWithClose = ht.getScanner(scanWithClose);
5277     for (Result result : scannerWithClose.next(numRecords + 1)) {
5278     }
5279     scannerWithClose.close();
5280     ScanMetrics scanMetricsWithClose = getScanMetrics(scanWithClose);
5281     assertEquals("Did not access all the regions in the table", numOfRegions,
5282         scanMetricsWithClose.countOfRegions.get());
5283   }
5284 
5285   private ScanMetrics getScanMetrics(Scan scan) throws Exception {
5286     byte[] serializedMetrics = scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA);
5287     assertTrue("Serialized metrics were not found.", serializedMetrics != null);
5288 
5289     ScanMetrics scanMetrics = ProtobufUtil.toScanMetrics(serializedMetrics);
5290 
5291     return scanMetrics;
5292   }
5293 
5294   /**
5295    * Tests that cache on write works all the way up from the client-side.
5296    *
5297    * Performs inserts, flushes, and compactions, verifying changes in the block
5298    * cache along the way.
5299    *
5300    * @throws Exception
5301    */
5302   @Test
5303   public void testCacheOnWriteEvictOnClose() throws Exception {
5304     TableName tableName = TableName.valueOf("testCOWEOCfromClient");
5305     byte [] data = Bytes.toBytes("data");
5306     HTable table = TEST_UTIL.createTable(tableName, FAMILY);
5307     TEST_UTIL.waitTableAvailable(tableName, 10000);
5308     // get the block cache and region
5309     String regionName = table.getRegionLocations().firstKey().getEncodedName();
5310     Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName)
5311       .getFromOnlineRegions(regionName);
5312     Store store = region.getStores().iterator().next();
5313     CacheConfig cacheConf = store.getCacheConfig();
5314     cacheConf.setCacheDataOnWrite(true);
5315     cacheConf.setEvictOnClose(true);
5316     BlockCache cache = cacheConf.getBlockCache();
5317 
5318     // establish baseline stats
5319     long startBlockCount = cache.getBlockCount();
5320     long startBlockHits = cache.getStats().getHitCount();
5321     long startBlockMiss = cache.getStats().getMissCount();
5322 
5323     // wait till baseline is stable, (minimal 500 ms)
5324     for (int i = 0; i < 5; i++) {
5325       Thread.sleep(100);
5326       if (startBlockCount != cache.getBlockCount()
5327           || startBlockHits != cache.getStats().getHitCount()
5328           || startBlockMiss != cache.getStats().getMissCount()) {
5329         startBlockCount = cache.getBlockCount();
5330         startBlockHits = cache.getStats().getHitCount();
5331         startBlockMiss = cache.getStats().getMissCount();
5332         i = -1;
5333       }
5334     }
5335 
5336     // insert data
5337     Put put = new Put(ROW);
5338     put.add(FAMILY, QUALIFIER, data);
5339     table.put(put);
5340     assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
5341     // data was in memstore so don't expect any changes
5342     assertEquals(startBlockCount, cache.getBlockCount());
5343     assertEquals(startBlockHits, cache.getStats().getHitCount());
5344     assertEquals(startBlockMiss, cache.getStats().getMissCount());
5345     // flush the data
5346     LOG.debug("Flushing cache");
5347     region.flush(true);
5348     // expect two more blocks in cache - DATA and ROOT_INDEX
5349     // , no change in hits/misses
5350     long expectedBlockCount = startBlockCount + 2;
5351     long expectedBlockHits = startBlockHits;
5352     long expectedBlockMiss = startBlockMiss;
5353     assertEquals(expectedBlockCount, cache.getBlockCount());
5354     assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5355     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5356     // read the data and expect same blocks, one new hit, no misses
5357     assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
5358     assertEquals(expectedBlockCount, cache.getBlockCount());
5359     assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
5360     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5361     // insert a second column, read the row, no new blocks, one new hit
5362     byte [] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
5363     byte [] data2 = Bytes.add(data, data);
5364     put = new Put(ROW);
5365     put.add(FAMILY, QUALIFIER2, data2);
5366     table.put(put);
5367     Result r = table.get(new Get(ROW));
5368     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
5369     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
5370     assertEquals(expectedBlockCount, cache.getBlockCount());
5371     assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
5372     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5373     // flush, one new block
5374     System.out.println("Flushing cache");
5375     region.flush(true);
5376     // + 1 for Index Block, +1 for data block
5377     expectedBlockCount += 2;
5378     assertEquals(expectedBlockCount, cache.getBlockCount());
5379     assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5380     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5381     // compact, net minus two blocks, two hits, no misses
5382     System.out.println("Compacting");
5383     assertEquals(2, store.getStorefilesCount());
5384     store.triggerMajorCompaction();
5385     region.compact(true);
5386     waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max
5387     assertEquals(1, store.getStorefilesCount());
5388     // evicted two data blocks and two index blocks and compaction does not cache new blocks
5389     expectedBlockCount -= 4;
5390     assertEquals(expectedBlockCount, cache.getBlockCount());
5391     expectedBlockHits += 2;
5392     assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
5393     assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5394     // read the row, this should be a cache miss because we don't cache data
5395     // blocks on compaction
5396     r = table.get(new Get(ROW));
5397     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
5398     assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
5399     expectedBlockCount += 1; // cached one data block
5400     assertEquals(expectedBlockCount, cache.getBlockCount());
5401     assertEquals(expectedBlockHits, cache.getStats().getHitCount());
5402     assertEquals(++expectedBlockMiss, cache.getStats().getMissCount());
5403   }
5404 
5405   private void waitForStoreFileCount(Store store, int count, int timeout)
5406   throws InterruptedException {
5407     long start = System.currentTimeMillis();
5408     while (start + timeout > System.currentTimeMillis() &&
5409         store.getStorefilesCount() != count) {
5410       Thread.sleep(100);
5411     }
5412     System.out.println("start=" + start + ", now=" +
5413         System.currentTimeMillis() + ", cur=" + store.getStorefilesCount());
5414     assertEquals(count, store.getStorefilesCount());
5415   }
5416 
5417   @Test
5418   /**
5419    * Tests the non cached version of getRegionLocator by moving a region.
5420    */
5421   public void testNonCachedGetRegionLocation() throws Exception {
5422     // Test Initialization.
5423     TableName TABLE = TableName.valueOf("testNonCachedGetRegionLocation");
5424     byte [] family1 = Bytes.toBytes("f1");
5425     byte [] family2 = Bytes.toBytes("f2");
5426     try (HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
5427         Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration())) {
5428       TEST_UTIL.waitTableAvailable(TABLE, 10000);
5429       Map <HRegionInfo, ServerName> regionsMap = table.getRegionLocations();
5430       assertEquals(1, regionsMap.size());
5431       HRegionInfo regionInfo = regionsMap.keySet().iterator().next();
5432       ServerName addrBefore = regionsMap.get(regionInfo);
5433       // Verify region location before move.
5434       HRegionLocation addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
5435       HRegionLocation addrNoCache = table.getRegionLocation(regionInfo.getStartKey(),  true);
5436 
5437       assertEquals(addrBefore.getPort(), addrCache.getPort());
5438       assertEquals(addrBefore.getPort(), addrNoCache.getPort());
5439 
5440       ServerName addrAfter = null;
5441       // Now move the region to a different server.
5442       for (int i = 0; i < SLAVES; i++) {
5443         HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(i);
5444         ServerName addr = regionServer.getServerName();
5445         if (addr.getPort() != addrBefore.getPort()) {
5446           admin.move(regionInfo.getEncodedNameAsBytes(),
5447               Bytes.toBytes(addr.toString()));
5448           // Wait for the region to move.
5449           Thread.sleep(5000);
5450           addrAfter = addr;
5451           break;
5452         }
5453       }
5454 
5455       // Verify the region was moved.
5456       addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
5457       addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true);
5458       assertNotNull(addrAfter);
5459       assertTrue(addrAfter.getPort() != addrCache.getPort());
5460       assertEquals(addrAfter.getPort(), addrNoCache.getPort());
5461     }
5462   }
5463 
5464   @Test
5465   /**
5466    * Tests getRegionsInRange by creating some regions over which a range of
5467    * keys spans; then changing the key range.
5468    */
5469   public void testGetRegionsInRange() throws Exception {
5470     // Test Initialization.
5471     byte [] startKey = Bytes.toBytes("ddc");
5472     byte [] endKey = Bytes.toBytes("mmm");
5473     TableName TABLE = TableName.valueOf("testGetRegionsInRange");
5474     HTable table = TEST_UTIL.createMultiRegionTable(TABLE, new byte[][] { FAMILY }, 10);
5475     int numOfRegions = -1;
5476     try (RegionLocator r = table.getRegionLocator()) {
5477       numOfRegions = r.getStartKeys().length;
5478     }
5479     assertEquals(26, numOfRegions);
5480 
5481     // Get the regions in this range
5482     List<HRegionLocation> regionsList = table.getRegionsInRange(startKey,
5483       endKey);
5484     assertEquals(10, regionsList.size());
5485 
5486     // Change the start key
5487     startKey = Bytes.toBytes("fff");
5488     regionsList = table.getRegionsInRange(startKey, endKey);
5489     assertEquals(7, regionsList.size());
5490 
5491     // Change the end key
5492     endKey = Bytes.toBytes("nnn");
5493     regionsList = table.getRegionsInRange(startKey, endKey);
5494     assertEquals(8, regionsList.size());
5495 
5496     // Empty start key
5497     regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, endKey);
5498     assertEquals(13, regionsList.size());
5499 
5500     // Empty end key
5501     regionsList = table.getRegionsInRange(startKey, HConstants.EMPTY_END_ROW);
5502     assertEquals(21, regionsList.size());
5503 
5504     // Both start and end keys empty
5505     regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW,
5506       HConstants.EMPTY_END_ROW);
5507     assertEquals(26, regionsList.size());
5508 
5509     // Change the end key to somewhere in the last block
5510     endKey = Bytes.toBytes("zzz1");
5511     regionsList = table.getRegionsInRange(startKey, endKey);
5512     assertEquals(21, regionsList.size());
5513 
5514     // Change the start key to somewhere in the first block
5515     startKey = Bytes.toBytes("aac");
5516     regionsList = table.getRegionsInRange(startKey, endKey);
5517     assertEquals(26, regionsList.size());
5518 
5519     // Make start and end key the same
5520     startKey = endKey = Bytes.toBytes("ccc");
5521     regionsList = table.getRegionsInRange(startKey, endKey);
5522     assertEquals(1, regionsList.size());
5523   }
5524 
5525   @Test
5526   public void testJira6912() throws Exception {
5527     TableName TABLE = TableName.valueOf("testJira6912");
5528     Table foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
5529     TEST_UTIL.waitTableAvailable(TABLE, 10000);
5530     List<Put> puts = new ArrayList<Put>();
5531     for (int i=0;i !=100; i++){
5532       Put put = new Put(Bytes.toBytes(i));
5533       put.add(FAMILY, FAMILY, Bytes.toBytes(i));
5534       puts.add(put);
5535     }
5536     foo.put(puts);
5537     // If i comment this out it works
5538     TEST_UTIL.flush();
5539 
5540     Scan scan = new Scan();
5541     scan.setStartRow(Bytes.toBytes(1));
5542     scan.setStopRow(Bytes.toBytes(3));
5543     scan.addColumn(FAMILY, FAMILY);
5544     scan.setFilter(new RowFilter(CompareFilter.CompareOp.NOT_EQUAL, new BinaryComparator(Bytes.toBytes(1))));
5545 
5546     ResultScanner scanner = foo.getScanner(scan);
5547     Result[] bar = scanner.next(100);
5548     assertEquals(1, bar.length);
5549   }
5550 
5551   @Test
5552   public void testScan_NullQualifier() throws IOException, InterruptedException {
5553     Table table = TEST_UTIL.createTable(TableName.valueOf("testScan_NullQualifier"), FAMILY);
5554     TEST_UTIL.waitTableAvailable(TableName.valueOf("testScan_NullQualifier"), 10000);
5555 
5556     Put put = new Put(ROW);
5557     put.add(FAMILY, QUALIFIER, VALUE);
5558     table.put(put);
5559 
5560     put = new Put(ROW);
5561     put.add(FAMILY, null, VALUE);
5562     table.put(put);
5563     LOG.info("Row put");
5564 
5565     Scan scan = new Scan();
5566     scan.addColumn(FAMILY, null);
5567 
5568     ResultScanner scanner = table.getScanner(scan);
5569     Result[] bar = scanner.next(100);
5570     assertEquals(1, bar.length);
5571     assertEquals(1, bar[0].size());
5572 
5573     scan = new Scan();
5574     scan.addFamily(FAMILY);
5575 
5576     scanner = table.getScanner(scan);
5577     bar = scanner.next(100);
5578     assertEquals(1, bar.length);
5579     assertEquals(2, bar[0].size());
5580   }
5581 
5582   @Test
5583   public void testNegativeTimestamp() throws IOException, InterruptedException {
5584     Table table = TEST_UTIL.createTable(TableName.valueOf("testNegativeTimestamp"), FAMILY);
5585     TEST_UTIL.waitTableAvailable(TableName.valueOf("testNegativeTimestamp"), 10000);
5586     try {
5587       Put put = new Put(ROW, -1);
5588       put.add(FAMILY, QUALIFIER, VALUE);
5589       table.put(put);
5590       fail("Negative timestamps should not have been allowed");
5591     } catch (IllegalArgumentException ex) {
5592       assertTrue(ex.getMessage().contains("negative"));
5593     }
5594 
5595     try {
5596       Put put = new Put(ROW);
5597       put.add(FAMILY, QUALIFIER, -1, VALUE);
5598       table.put(put);
5599       fail("Negative timestamps should not have been allowed");
5600     } catch (IllegalArgumentException ex) {
5601       assertTrue(ex.getMessage().contains("negative"));
5602     }
5603 
5604     try {
5605       Delete delete = new Delete(ROW, -1);
5606       table.delete(delete);
5607       fail("Negative timestamps should not have been allowed");
5608     } catch (IllegalArgumentException ex) {
5609       assertTrue(ex.getMessage().contains("negative"));
5610     }
5611 
5612     try {
5613       Delete delete = new Delete(ROW);
5614       delete.deleteFamily(FAMILY, -1);
5615       table.delete(delete);
5616       fail("Negative timestamps should not have been allowed");
5617     } catch (IllegalArgumentException ex) {
5618       assertTrue(ex.getMessage().contains("negative"));
5619     }
5620 
5621     try {
5622       Scan scan = new Scan();
5623       scan.setTimeRange(-1, 1);
5624       table.getScanner(scan);
5625       fail("Negative timestamps should not have been allowed");
5626     } catch (IllegalArgumentException ex) {
5627       assertTrue(ex.getMessage().contains("negative"));
5628     }
5629 
5630     // KeyValue should allow negative timestamps for backwards compat. Otherwise, if the user
5631     // already has negative timestamps in cluster data, HBase won't be able to handle that
5632     try {
5633       new KeyValue(Bytes.toBytes(42), Bytes.toBytes(42), Bytes.toBytes(42), -1, Bytes.toBytes(42));
5634     } catch (IllegalArgumentException ex) {
5635       fail("KeyValue SHOULD allow negative timestamps");
5636     }
5637 
5638     table.close();
5639   }
5640 
5641   @Test
5642   public void testIllegalTableDescriptor() throws Exception {
5643     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testIllegalTableDescriptor"));
5644     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
5645 
5646     // create table with 0 families
5647     checkTableIsIllegal(htd);
5648     htd.addFamily(hcd);
5649     checkTableIsLegal(htd);
5650 
5651     htd.setMaxFileSize(1024); // 1K
5652     checkTableIsIllegal(htd);
5653     htd.setMaxFileSize(0);
5654     checkTableIsIllegal(htd);
5655     htd.setMaxFileSize(1024 * 1024 * 1024); // 1G
5656     checkTableIsLegal(htd);
5657 
5658     htd.setMemStoreFlushSize(1024);
5659     checkTableIsIllegal(htd);
5660     htd.setMemStoreFlushSize(0);
5661     checkTableIsIllegal(htd);
5662     htd.setMemStoreFlushSize(128 * 1024 * 1024); // 128M
5663     checkTableIsLegal(htd);
5664 
5665     htd.setRegionSplitPolicyClassName("nonexisting.foo.class");
5666     checkTableIsIllegal(htd);
5667     htd.setRegionSplitPolicyClassName(null);
5668     checkTableIsLegal(htd);
5669 
5670     hcd.setBlocksize(0);
5671     checkTableIsIllegal(htd);
5672     hcd.setBlocksize(1024 * 1024 * 128); // 128M
5673     checkTableIsIllegal(htd);
5674     hcd.setBlocksize(1024);
5675     checkTableIsLegal(htd);
5676 
5677     hcd.setTimeToLive(0);
5678     checkTableIsIllegal(htd);
5679     hcd.setTimeToLive(-1);
5680     checkTableIsIllegal(htd);
5681     hcd.setTimeToLive(1);
5682     checkTableIsLegal(htd);
5683 
5684     hcd.setMinVersions(-1);
5685     checkTableIsIllegal(htd);
5686     hcd.setMinVersions(3);
5687     try {
5688       hcd.setMaxVersions(2);
5689       fail();
5690     } catch (IllegalArgumentException ex) {
5691       // expected
5692       hcd.setMaxVersions(10);
5693     }
5694     checkTableIsLegal(htd);
5695 
5696     // HBASE-13776 Setting illegal versions for HColumnDescriptor
5697     //  does not throw IllegalArgumentException
5698     // finally, minVersions must be less than or equal to maxVersions
5699     hcd.setMaxVersions(4);
5700     hcd.setMinVersions(5);
5701     checkTableIsIllegal(htd);
5702     hcd.setMinVersions(3);
5703 
5704     try {
5705       hcd.setScope(-1);
5706       fail("Illegal value for setScope did not throw");
5707     } catch (IllegalArgumentException e) {
5708       // expected
5709       hcd.setScope(0);
5710     }
5711     checkTableIsLegal(htd);
5712 
5713     try {
5714       hcd.setDFSReplication((short) -1);
5715       fail("Illegal value for setDFSReplication did not throw");
5716     } catch (IllegalArgumentException e) {
5717       // pass
5718     }
5719     // set an illegal DFS replication value by hand
5720     hcd.setValue(HColumnDescriptor.DFS_REPLICATION, "-1");
5721     checkTableIsIllegal(htd);
5722     try {
5723       hcd.setDFSReplication((short) -1);
5724       fail("Should throw exception if an illegal value is explicitly being set");
5725     } catch (IllegalArgumentException e) {
5726       // pass
5727     }
5728 
5729     // check the conf settings to disable sanity checks
5730     htd.setMemStoreFlushSize(0);
5731 
5732     // Check that logs warn on invalid table but allow it.
5733     ListAppender listAppender = new ListAppender();
5734     Logger log = Logger.getLogger(HMaster.class);
5735     log.addAppender(listAppender);
5736     log.setLevel(Level.WARN);
5737 
5738     htd.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString());
5739     checkTableIsLegal(htd);
5740 
5741     assertFalse(listAppender.getMessages().isEmpty());
5742     assertTrue(listAppender.getMessages().get(0).startsWith("MEMSTORE_FLUSHSIZE for table "
5743         + "descriptor or \"hbase.hregion.memstore.flush.size\" (0) is too small, which might "
5744         + "cause very frequent flushing."));
5745 
5746     log.removeAppender(listAppender);
5747   }
5748 
5749   private static class ListAppender extends AppenderSkeleton {
5750     private final List<String> messages = new ArrayList<String>();
5751 
5752     @Override
5753     protected void append(LoggingEvent event) {
5754       messages.add(event.getMessage().toString());
5755     }
5756 
5757     @Override
5758     public void close() {
5759     }
5760 
5761     @Override
5762     public boolean requiresLayout() {
5763       return false;
5764     }
5765 
5766     public List<String> getMessages() {
5767       return messages;
5768     }
5769   }
5770 
5771   private void checkTableIsLegal(HTableDescriptor htd) throws IOException {
5772     Admin admin = TEST_UTIL.getHBaseAdmin();
5773     admin.createTable(htd);
5774     assertTrue(admin.tableExists(htd.getTableName()));
5775     admin.disableTable(htd.getTableName());
5776     admin.deleteTable(htd.getTableName());
5777   }
5778 
5779   private void checkTableIsIllegal(HTableDescriptor htd) throws IOException {
5780     Admin admin = TEST_UTIL.getHBaseAdmin();
5781     try {
5782       admin.createTable(htd);
5783       fail();
5784     } catch(Exception ex) {
5785       // should throw ex
5786     }
5787     assertFalse(admin.tableExists(htd.getTableName()));
5788   }
5789 
5790   @Test
5791   public void testRawScanRespectsVersions() throws Exception {
5792     TableName TABLE = TableName.valueOf("testRawScan");
5793     Table table = TEST_UTIL.createTable(TABLE, FAMILY);
5794     TEST_UTIL.waitTableAvailable(TABLE, 10000);
5795     byte[] row = Bytes.toBytes("row");
5796 
5797     // put the same row 4 times, with different values
5798     Put p = new Put(row);
5799     p.add(FAMILY, QUALIFIER, 10, VALUE);
5800     table.put(p);
5801     p = new Put(row);
5802     p.add(FAMILY, QUALIFIER, 11, ArrayUtils.add(VALUE, (byte) 2));
5803     table.put(p);
5804 
5805     p = new Put(row);
5806     p.add(FAMILY, QUALIFIER, 12, ArrayUtils.add(VALUE, (byte) 3));
5807     table.put(p);
5808 
5809     p = new Put(row);
5810     p.add(FAMILY, QUALIFIER, 13, ArrayUtils.add(VALUE, (byte) 4));
5811     table.put(p);
5812 
5813     int versions = 4;
5814     Scan s = new Scan(row);
5815     // get all the possible versions
5816     s.setMaxVersions();
5817     s.setRaw(true);
5818 
5819     ResultScanner scanner = table.getScanner(s);
5820     int count = 0;
5821     for (Result r : scanner) {
5822       assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5823       count++;
5824     }
5825     assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5826       count);
5827     scanner.close();
5828 
5829     // then if we decrease the number of versions, but keep the scan raw, we should see exactly that
5830     // number of versions
5831     versions = 2;
5832     s.setMaxVersions(versions);
5833     scanner = table.getScanner(s);
5834     count = 0;
5835     for (Result r : scanner) {
5836       assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5837       count++;
5838     }
5839     assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5840       count);
5841     scanner.close();
5842 
5843     // finally, if we turn off raw scanning, but max out the number of versions, we should go back
5844     // to seeing just three
5845     versions = 3;
5846     s.setMaxVersions(versions);
5847     scanner = table.getScanner(s);
5848     count = 0;
5849     for (Result r : scanner) {
5850       assertEquals("Found an unexpected number of results for the row!", versions, r.listCells().size());
5851       count++;
5852     }
5853     assertEquals("Found more than a single row when raw scanning the table with a single row!", 1,
5854       count);
5855     scanner.close();
5856 
5857     table.close();
5858     TEST_UTIL.deleteTable(TABLE);
5859   }
5860 
5861   @Test
5862   public void testEmptyFilterList() throws Exception {
5863     // Test Initialization.
5864     TableName TABLE = TableName.valueOf("testEmptyFilterList");
5865     Table table = TEST_UTIL.createTable(TABLE, FAMILY);
5866     TEST_UTIL.waitTableAvailable(TABLE, 10000);
5867 
5868     // Insert one row each region
5869     Put put = new Put(Bytes.toBytes("row"));
5870     put.addColumn(FAMILY, QUALIFIER, VALUE);
5871     table.put(put);
5872 
5873     List<Result> scanResults = new LinkedList<>();
5874     Scan scan = new Scan();
5875     scan.setFilter(new FilterList());
5876     try (ResultScanner scanner = table.getScanner(scan)) {
5877       for (Result r : scanner) {
5878         scanResults.add(r);
5879       }
5880     }
5881 
5882     Get g = new Get(Bytes.toBytes("row"));
5883     g.setFilter(new FilterList());
5884     Result getResult = table.get(g);
5885     if (scanResults.isEmpty()) {
5886       assertTrue(getResult.isEmpty());
5887     } else if (scanResults.size() == 1) {
5888       Result scanResult = scanResults.get(0);
5889       assertEquals(scanResult.rawCells().length, getResult.rawCells().length);
5890       for (int i = 0; i != scanResult.rawCells().length; ++i) {
5891         Cell scanCell = scanResult.rawCells()[i];
5892         Cell getCell = getResult.rawCells()[i];
5893         assertEquals(0, Bytes.compareTo(CellUtil.cloneRow(scanCell), CellUtil.cloneRow(getCell)));
5894         assertEquals(0,
5895           Bytes.compareTo(CellUtil.cloneFamily(scanCell), CellUtil.cloneFamily(getCell)));
5896         assertEquals(0,
5897           Bytes.compareTo(CellUtil.cloneQualifier(scanCell), CellUtil.cloneQualifier(getCell)));
5898         assertEquals(0,
5899           Bytes.compareTo(CellUtil.cloneValue(scanCell), CellUtil.cloneValue(getCell)));
5900       }
5901     } else {
5902       fail("The result retrieved from SCAN and Get should be same");
5903     }
5904   }
5905 
5906   @Test
5907   public void testSmallScan() throws Exception {
5908     // Test Initialization.
5909     TableName TABLE = TableName.valueOf("testSmallScan");
5910     Table table = TEST_UTIL.createTable(TABLE, FAMILY);
5911     TEST_UTIL.waitTableAvailable(TABLE, 10000);
5912 
5913     // Insert one row each region
5914     int insertNum = 10;
5915     for (int i = 0; i < 10; i++) {
5916       Put put = new Put(Bytes.toBytes("row" + String.format("%03d", i)));
5917       put.add(FAMILY, QUALIFIER, VALUE);
5918       table.put(put);
5919     }
5920 
5921     // nomal scan
5922     ResultScanner scanner = table.getScanner(new Scan());
5923     int count = 0;
5924     for (Result r : scanner) {
5925       assertTrue(!r.isEmpty());
5926       count++;
5927     }
5928     assertEquals(insertNum, count);
5929 
5930     // small scan
5931     Scan scan = new Scan(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
5932     scan.setSmall(true);
5933     scan.setCaching(2);
5934     scanner = table.getScanner(scan);
5935     count = 0;
5936     for (Result r : scanner) {
5937       assertTrue(!r.isEmpty());
5938       count++;
5939     }
5940     assertEquals(insertNum, count);
5941 
5942   }
5943 
5944   @Test
5945   public void testSuperSimpleWithReverseScan() throws Exception {
5946     TableName TABLE = TableName.valueOf("testSuperSimpleWithReverseScan");
5947     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
5948     TEST_UTIL.waitTableAvailable(TABLE, 10000);
5949 
5950     Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000"));
5951     put.add(FAMILY, QUALIFIER, VALUE);
5952     ht.put(put);
5953     put = new Put(Bytes.toBytes("0-b11111-0000000000000000002"));
5954     put.add(FAMILY, QUALIFIER, VALUE);
5955     ht.put(put);
5956     put = new Put(Bytes.toBytes("0-b11111-0000000000000000004"));
5957     put.add(FAMILY, QUALIFIER, VALUE);
5958     ht.put(put);
5959     put = new Put(Bytes.toBytes("0-b11111-0000000000000000006"));
5960     put.add(FAMILY, QUALIFIER, VALUE);
5961     ht.put(put);
5962     put = new Put(Bytes.toBytes("0-b11111-0000000000000000008"));
5963     put.add(FAMILY, QUALIFIER, VALUE);
5964     ht.put(put);
5965     put = new Put(Bytes.toBytes("0-b22222-0000000000000000001"));
5966     put.add(FAMILY, QUALIFIER, VALUE);
5967     ht.put(put);
5968     put = new Put(Bytes.toBytes("0-b22222-0000000000000000003"));
5969     put.add(FAMILY, QUALIFIER, VALUE);
5970     ht.put(put);
5971     put = new Put(Bytes.toBytes("0-b22222-0000000000000000005"));
5972     put.add(FAMILY, QUALIFIER, VALUE);
5973     ht.put(put);
5974     put = new Put(Bytes.toBytes("0-b22222-0000000000000000007"));
5975     put.add(FAMILY, QUALIFIER, VALUE);
5976     ht.put(put);
5977     put = new Put(Bytes.toBytes("0-b22222-0000000000000000009"));
5978     put.add(FAMILY, QUALIFIER, VALUE);
5979     ht.put(put);
5980     Scan scan = new Scan(Bytes.toBytes("0-b11111-9223372036854775807"),
5981         Bytes.toBytes("0-b11111-0000000000000000000"));
5982     scan.setReversed(true);
5983     ResultScanner scanner = ht.getScanner(scan);
5984     Result result = scanner.next();
5985     assertTrue(Bytes.equals(result.getRow(),
5986         Bytes.toBytes("0-b11111-0000000000000000008")));
5987     scanner.close();
5988     ht.close();
5989   }
5990 
5991   @Test
5992   public void testFiltersWithReverseScan() throws Exception {
5993     TableName TABLE = TableName.valueOf("testFiltersWithReverseScan");
5994     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
5995     TEST_UTIL.waitTableAvailable(TABLE, 10000);
5996 
5997     byte[][] ROWS = makeN(ROW, 10);
5998     byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
5999         Bytes.toBytes("col1-<d2v1>-<d3v2>"),
6000         Bytes.toBytes("col2-<d2v1>-<d3v2>"),
6001         Bytes.toBytes("col3-<d2v1>-<d3v2>"),
6002         Bytes.toBytes("col4-<d2v1>-<d3v2>"),
6003         Bytes.toBytes("col5-<d2v1>-<d3v2>"),
6004         Bytes.toBytes("col6-<d2v1>-<d3v2>"),
6005         Bytes.toBytes("col7-<d2v1>-<d3v2>"),
6006         Bytes.toBytes("col8-<d2v1>-<d3v2>"),
6007         Bytes.toBytes("col9-<d2v1>-<d3v2>") };
6008     for (int i = 0; i < 10; i++) {
6009       Put put = new Put(ROWS[i]);
6010       put.add(FAMILY, QUALIFIERS[i], VALUE);
6011       ht.put(put);
6012     }
6013     Scan scan = new Scan();
6014     scan.setReversed(true);
6015     scan.addFamily(FAMILY);
6016     Filter filter = new QualifierFilter(CompareOp.EQUAL,
6017         new RegexStringComparator("col[1-5]"));
6018     scan.setFilter(filter);
6019     ResultScanner scanner = ht.getScanner(scan);
6020     int expectedIndex = 5;
6021     for (Result result : scanner) {
6022       assertEquals(result.size(), 1);
6023       assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[expectedIndex]));
6024       assertTrue(Bytes.equals(result.raw()[0].getQualifier(),
6025           QUALIFIERS[expectedIndex]));
6026       expectedIndex--;
6027     }
6028     assertEquals(expectedIndex, 0);
6029     scanner.close();
6030     ht.close();
6031   }
6032 
6033   @Test
6034   public void testKeyOnlyFilterWithReverseScan() throws Exception {
6035     TableName TABLE = TableName.valueOf("testKeyOnlyFilterWithReverseScan");
6036     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
6037     TEST_UTIL.waitTableAvailable(TABLE, 10000);
6038 
6039     byte[][] ROWS = makeN(ROW, 10);
6040     byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
6041         Bytes.toBytes("col1-<d2v1>-<d3v2>"),
6042         Bytes.toBytes("col2-<d2v1>-<d3v2>"),
6043         Bytes.toBytes("col3-<d2v1>-<d3v2>"),
6044         Bytes.toBytes("col4-<d2v1>-<d3v2>"),
6045         Bytes.toBytes("col5-<d2v1>-<d3v2>"),
6046         Bytes.toBytes("col6-<d2v1>-<d3v2>"),
6047         Bytes.toBytes("col7-<d2v1>-<d3v2>"),
6048         Bytes.toBytes("col8-<d2v1>-<d3v2>"),
6049         Bytes.toBytes("col9-<d2v1>-<d3v2>") };
6050     for (int i = 0; i < 10; i++) {
6051       Put put = new Put(ROWS[i]);
6052       put.add(FAMILY, QUALIFIERS[i], VALUE);
6053       ht.put(put);
6054     }
6055     Scan scan = new Scan();
6056     scan.setReversed(true);
6057     scan.addFamily(FAMILY);
6058     Filter filter = new KeyOnlyFilter(true);
6059     scan.setFilter(filter);
6060     ResultScanner scanner = ht.getScanner(scan);
6061     int count = 0;
6062     for (Result result : ht.getScanner(scan)) {
6063       assertEquals(result.size(), 1);
6064       assertEquals(result.raw()[0].getValueLength(), Bytes.SIZEOF_INT);
6065       assertEquals(Bytes.toInt(result.raw()[0].getValue()), VALUE.length);
6066       count++;
6067     }
6068     assertEquals(count, 10);
6069     scanner.close();
6070     ht.close();
6071   }
6072 
6073   /**
6074    * Test simple table and non-existent row cases.
6075    */
6076   @Test
6077   public void testSimpleMissingWithReverseScan() throws Exception {
6078     TableName TABLE = TableName.valueOf("testSimpleMissingWithReverseScan");
6079     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
6080     TEST_UTIL.waitTableAvailable(TABLE, 10000);
6081 
6082     byte[][] ROWS = makeN(ROW, 4);
6083 
6084     // Try to get a row on an empty table
6085     Scan scan = new Scan();
6086     scan.setReversed(true);
6087     Result result = getSingleScanResult(ht, scan);
6088     assertNullResult(result);
6089 
6090     scan = new Scan(ROWS[0]);
6091     scan.setReversed(true);
6092     result = getSingleScanResult(ht, scan);
6093     assertNullResult(result);
6094 
6095     scan = new Scan(ROWS[0], ROWS[1]);
6096     scan.setReversed(true);
6097     result = getSingleScanResult(ht, scan);
6098     assertNullResult(result);
6099 
6100     scan = new Scan();
6101     scan.setReversed(true);
6102     scan.addFamily(FAMILY);
6103     result = getSingleScanResult(ht, scan);
6104     assertNullResult(result);
6105 
6106     scan = new Scan();
6107     scan.setReversed(true);
6108     scan.addColumn(FAMILY, QUALIFIER);
6109     result = getSingleScanResult(ht, scan);
6110     assertNullResult(result);
6111 
6112     // Insert a row
6113 
6114     Put put = new Put(ROWS[2]);
6115     put.add(FAMILY, QUALIFIER, VALUE);
6116     ht.put(put);
6117 
6118     // Make sure we can scan the row
6119     scan = new Scan();
6120     scan.setReversed(true);
6121     result = getSingleScanResult(ht, scan);
6122     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
6123 
6124     scan = new Scan(ROWS[3], ROWS[0]);
6125     scan.setReversed(true);
6126     result = getSingleScanResult(ht, scan);
6127     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
6128 
6129     scan = new Scan(ROWS[2], ROWS[1]);
6130     scan.setReversed(true);
6131     result = getSingleScanResult(ht, scan);
6132     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
6133 
6134     // Try to scan empty rows around it
6135     // Introduced MemStore#shouldSeekForReverseScan to fix the following
6136     scan = new Scan(ROWS[1]);
6137     scan.setReversed(true);
6138     result = getSingleScanResult(ht, scan);
6139     assertNullResult(result);
6140     ht.close();
6141   }
6142 
6143   @Test
6144   public void testNullWithReverseScan() throws Exception {
6145     TableName TABLE = TableName.valueOf("testNullWithReverseScan");
6146     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
6147     TEST_UTIL.waitTableAvailable(TABLE, 10000);
6148 
6149     // Null qualifier (should work)
6150     Put put = new Put(ROW);
6151     put.add(FAMILY, null, VALUE);
6152     ht.put(put);
6153     scanTestNull(ht, ROW, FAMILY, VALUE, true);
6154     Delete delete = new Delete(ROW);
6155     delete.deleteColumns(FAMILY, null);
6156     ht.delete(delete);
6157     // Use a new table
6158     byte[] TABLE2 = Bytes.toBytes("testNull2WithReverseScan");
6159     ht = TEST_UTIL.createTable(TableName.valueOf(TABLE2), FAMILY);
6160     TEST_UTIL.waitTableAvailable(TableName.valueOf(TABLE2), 10000);
6161 
6162     // Empty qualifier, byte[0] instead of null (should work)
6163     put = new Put(ROW);
6164     put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
6165     ht.put(put);
6166     scanTestNull(ht, ROW, FAMILY, VALUE, true);
6167     TEST_UTIL.flush();
6168     scanTestNull(ht, ROW, FAMILY, VALUE, true);
6169     delete = new Delete(ROW);
6170     delete.deleteColumns(FAMILY, HConstants.EMPTY_BYTE_ARRAY);
6171     ht.delete(delete);
6172     // Null value
6173     put = new Put(ROW);
6174     put.add(FAMILY, QUALIFIER, null);
6175     ht.put(put);
6176     Scan scan = new Scan();
6177     scan.setReversed(true);
6178     scan.addColumn(FAMILY, QUALIFIER);
6179     Result result = getSingleScanResult(ht, scan);
6180     assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
6181     ht.close();
6182   }
6183 
6184   @Test
6185   public void testDeletesWithReverseScan() throws Exception {
6186     TableName TABLE = TableName.valueOf("testDeletesWithReverseScan");
6187     byte[][] ROWS = makeNAscii(ROW, 6);
6188     byte[][] FAMILIES = makeNAscii(FAMILY, 3);
6189     byte[][] VALUES = makeN(VALUE, 5);
6190     long[] ts = { 1000, 2000, 3000, 4000, 5000 };
6191     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration(), 3);
6192     TEST_UTIL.waitTableAvailable(TABLE, 10000);
6193 
6194     Put put = new Put(ROW);
6195     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
6196     put.add(FAMILIES[0], QUALIFIER, ts[1], VALUES[1]);
6197     ht.put(put);
6198 
6199     Delete delete = new Delete(ROW);
6200     delete.deleteFamily(FAMILIES[0], ts[0]);
6201     ht.delete(delete);
6202 
6203     Scan scan = new Scan(ROW);
6204     scan.setReversed(true);
6205     scan.addFamily(FAMILIES[0]);
6206     scan.setMaxVersions(Integer.MAX_VALUE);
6207     Result result = getSingleScanResult(ht, scan);
6208     assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1] },
6209         new byte[][] { VALUES[1] }, 0, 0);
6210 
6211     // Test delete latest version
6212     put = new Put(ROW);
6213     put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]);
6214     put.add(FAMILIES[0], QUALIFIER, ts[2], VALUES[2]);
6215     put.add(FAMILIES[0], QUALIFIER, ts[3], VALUES[3]);
6216     put.add(FAMILIES[0], null, ts[4], VALUES[4]);
6217     put.add(FAMILIES[0], null, ts[2], VALUES[2]);
6218     put.add(FAMILIES[0], null, ts[3], VALUES[3]);
6219     ht.put(put);
6220 
6221     delete = new Delete(ROW);
6222     delete.deleteColumn(FAMILIES[0], QUALIFIER); // ts[4]
6223     ht.delete(delete);
6224 
6225     scan = new Scan(ROW);
6226     scan.setReversed(true);
6227     scan.addColumn(FAMILIES[0], QUALIFIER);
6228     scan.setMaxVersions(Integer.MAX_VALUE);
6229     result = getSingleScanResult(ht, scan);
6230     assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1],
6231         ts[2], ts[3] }, new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2);
6232 
6233     // Test for HBASE-1847
6234     delete = new Delete(ROW);
6235     delete.deleteColumn(FAMILIES[0], null);
6236     ht.delete(delete);
6237 
6238     // Cleanup null qualifier
6239     delete = new Delete(ROW);
6240     delete.deleteColumns(FAMILIES[0], null);
6241     ht.delete(delete);
6242 
6243     // Expected client behavior might be that you can re-put deleted values
6244     // But alas, this is not to be. We can't put them back in either case.
6245 
6246     put = new Put(ROW);
6247     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); // 1000
6248     put.add(FAMILIES[0], QUALIFIER, ts[4], VALUES[4]); // 5000
6249     ht.put(put);
6250 
6251     // The Scanner returns the previous values, the expected-naive-unexpected
6252     // behavior
6253 
6254     scan = new Scan(ROW);
6255     scan.setReversed(true);
6256     scan.addFamily(FAMILIES[0]);
6257     scan.setMaxVersions(Integer.MAX_VALUE);
6258     result = getSingleScanResult(ht, scan);
6259     assertNResult(result, ROW, FAMILIES[0], QUALIFIER, new long[] { ts[1],
6260         ts[2], ts[3] }, new byte[][] { VALUES[1], VALUES[2], VALUES[3] }, 0, 2);
6261 
6262     // Test deleting an entire family from one row but not the other various
6263     // ways
6264 
6265     put = new Put(ROWS[0]);
6266     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
6267     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
6268     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
6269     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
6270     ht.put(put);
6271 
6272     put = new Put(ROWS[1]);
6273     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
6274     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
6275     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
6276     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
6277     ht.put(put);
6278 
6279     put = new Put(ROWS[2]);
6280     put.add(FAMILIES[1], QUALIFIER, ts[0], VALUES[0]);
6281     put.add(FAMILIES[1], QUALIFIER, ts[1], VALUES[1]);
6282     put.add(FAMILIES[2], QUALIFIER, ts[2], VALUES[2]);
6283     put.add(FAMILIES[2], QUALIFIER, ts[3], VALUES[3]);
6284     ht.put(put);
6285 
6286     delete = new Delete(ROWS[0]);
6287     delete.deleteFamily(FAMILIES[2]);
6288     ht.delete(delete);
6289 
6290     delete = new Delete(ROWS[1]);
6291     delete.deleteColumns(FAMILIES[1], QUALIFIER);
6292     ht.delete(delete);
6293 
6294     delete = new Delete(ROWS[2]);
6295     delete.deleteColumn(FAMILIES[1], QUALIFIER);
6296     delete.deleteColumn(FAMILIES[1], QUALIFIER);
6297     delete.deleteColumn(FAMILIES[2], QUALIFIER);
6298     ht.delete(delete);
6299 
6300     scan = new Scan(ROWS[0]);
6301     scan.setReversed(true);
6302     scan.addFamily(FAMILIES[1]);
6303     scan.addFamily(FAMILIES[2]);
6304     scan.setMaxVersions(Integer.MAX_VALUE);
6305     result = getSingleScanResult(ht, scan);
6306     assertTrue("Expected 2 keys but received " + result.size(),
6307         result.size() == 2);
6308     assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER, new long[] { ts[0],
6309         ts[1] }, new byte[][] { VALUES[0], VALUES[1] }, 0, 1);
6310 
6311     scan = new Scan(ROWS[1]);
6312     scan.setReversed(true);
6313     scan.addFamily(FAMILIES[1]);
6314     scan.addFamily(FAMILIES[2]);
6315     scan.setMaxVersions(Integer.MAX_VALUE);
6316     result = getSingleScanResult(ht, scan);
6317     assertTrue("Expected 2 keys but received " + result.size(),
6318         result.size() == 2);
6319 
6320     scan = new Scan(ROWS[2]);
6321     scan.setReversed(true);
6322     scan.addFamily(FAMILIES[1]);
6323     scan.addFamily(FAMILIES[2]);
6324     scan.setMaxVersions(Integer.MAX_VALUE);
6325     result = getSingleScanResult(ht, scan);
6326     assertEquals(1, result.size());
6327     assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
6328         new long[] { ts[2] }, new byte[][] { VALUES[2] }, 0, 0);
6329 
6330     // Test if we delete the family first in one row (HBASE-1541)
6331 
6332     delete = new Delete(ROWS[3]);
6333     delete.deleteFamily(FAMILIES[1]);
6334     ht.delete(delete);
6335 
6336     put = new Put(ROWS[3]);
6337     put.add(FAMILIES[2], QUALIFIER, VALUES[0]);
6338     ht.put(put);
6339 
6340     put = new Put(ROWS[4]);
6341     put.add(FAMILIES[1], QUALIFIER, VALUES[1]);
6342     put.add(FAMILIES[2], QUALIFIER, VALUES[2]);
6343     ht.put(put);
6344 
6345     scan = new Scan(ROWS[4]);
6346     scan.setReversed(true);
6347     scan.addFamily(FAMILIES[1]);
6348     scan.addFamily(FAMILIES[2]);
6349     scan.setMaxVersions(Integer.MAX_VALUE);
6350     ResultScanner scanner = ht.getScanner(scan);
6351     result = scanner.next();
6352     assertTrue("Expected 2 keys but received " + result.size(),
6353         result.size() == 2);
6354     assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[4]));
6355     assertTrue(Bytes.equals(result.raw()[1].getRow(), ROWS[4]));
6356     assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[1]));
6357     assertTrue(Bytes.equals(result.raw()[1].getValue(), VALUES[2]));
6358     result = scanner.next();
6359     assertTrue("Expected 1 key but received " + result.size(),
6360         result.size() == 1);
6361     assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[3]));
6362     assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[0]));
6363     scanner.close();
6364     ht.close();
6365   }
6366 
6367   /**
6368    * Tests reversed scan under multi regions
6369    */
6370   @Test
6371   public void testReversedScanUnderMultiRegions() throws Exception {
6372     // Test Initialization.
6373     TableName TABLE = TableName.valueOf("testReversedScanUnderMultiRegions");
6374     byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY;
6375     byte[][] splitRows = new byte[][] { Bytes.toBytes("005"),
6376         Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)),
6377         Bytes.toBytes("006"),
6378         Bytes.add(Bytes.toBytes("006"), Bytes.multiple(maxByteArray, 8)),
6379         Bytes.toBytes("007"),
6380         Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)),
6381         Bytes.toBytes("008"), Bytes.multiple(maxByteArray, 2) };
6382     HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
6383     TEST_UTIL.waitTableAvailable(TABLE, 10000);
6384 
6385     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
6386 
6387     assertEquals(splitRows.length + 1, table.getRegionLocations().size());
6388     // Insert one row each region
6389     int insertNum = splitRows.length;
6390     for (int i = 0; i < insertNum; i++) {
6391       Put put = new Put(splitRows[i]);
6392       put.add(FAMILY, QUALIFIER, VALUE);
6393       table.put(put);
6394     }
6395 
6396     // scan forward
6397     ResultScanner scanner = table.getScanner(new Scan());
6398     int count = 0;
6399     for (Result r : scanner) {
6400       assertTrue(!r.isEmpty());
6401       count++;
6402     }
6403     assertEquals(insertNum, count);
6404 
6405     // scan backward
6406     Scan scan = new Scan();
6407     scan.setReversed(true);
6408     scanner = table.getScanner(scan);
6409     count = 0;
6410     byte[] lastRow = null;
6411     for (Result r : scanner) {
6412       assertTrue(!r.isEmpty());
6413       count++;
6414       byte[] thisRow = r.getRow();
6415       if (lastRow != null) {
6416         assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6417             + ",this row=" + Bytes.toString(thisRow),
6418             Bytes.compareTo(thisRow, lastRow) < 0);
6419       }
6420       lastRow = thisRow;
6421     }
6422     assertEquals(insertNum, count);
6423     table.close();
6424   }
6425 
6426   /**
6427    * Tests reversed scan under multi regions
6428    */
6429   @Test
6430   public void testSmallReversedScanUnderMultiRegions() throws Exception {
6431     // Test Initialization.
6432     TableName TABLE = TableName.valueOf("testSmallReversedScanUnderMultiRegions");
6433     byte[][] splitRows = new byte[][]{
6434         Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"),
6435         Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")};
6436     HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
6437     TEST_UTIL.waitTableAvailable(TABLE, 10000);
6438 
6439     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
6440 
6441     assertEquals(splitRows.length + 1, table.getRegionLocations().size());
6442     for (byte[] splitRow : splitRows) {
6443       Put put = new Put(splitRow);
6444       put.add(FAMILY, QUALIFIER, VALUE);
6445       table.put(put);
6446 
6447       byte[] nextRow = Bytes.copy(splitRow);
6448       nextRow[nextRow.length - 1]++;
6449 
6450       put = new Put(nextRow);
6451       put.add(FAMILY, QUALIFIER, VALUE);
6452       table.put(put);
6453     }
6454 
6455     // scan forward
6456     ResultScanner scanner = table.getScanner(new Scan());
6457     int count = 0;
6458     for (Result r : scanner) {
6459       assertTrue(!r.isEmpty());
6460       count++;
6461     }
6462     assertEquals(12, count);
6463 
6464     reverseScanTest(table, false);
6465     reverseScanTest(table, true);
6466 
6467     table.close();
6468   }
6469 
6470   private void reverseScanTest(Table table, boolean small) throws IOException {
6471     // scan backward
6472     Scan scan = new Scan();
6473     scan.setReversed(true);
6474     ResultScanner scanner = table.getScanner(scan);
6475     int count = 0;
6476     byte[] lastRow = null;
6477     for (Result r : scanner) {
6478       assertTrue(!r.isEmpty());
6479       count++;
6480       byte[] thisRow = r.getRow();
6481       if (lastRow != null) {
6482         assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6483             + ",this row=" + Bytes.toString(thisRow),
6484             Bytes.compareTo(thisRow, lastRow) < 0);
6485       }
6486       lastRow = thisRow;
6487     }
6488     assertEquals(12, count);
6489 
6490     scan = new Scan();
6491     scan.setSmall(small);
6492     scan.setReversed(true);
6493     scan.setStartRow(Bytes.toBytes("002"));
6494     scanner = table.getScanner(scan);
6495     count = 0;
6496     lastRow = null;
6497     for (Result r : scanner) {
6498       assertTrue(!r.isEmpty());
6499       count++;
6500       byte[] thisRow = r.getRow();
6501       if (lastRow != null) {
6502         assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6503             + ",this row=" + Bytes.toString(thisRow),
6504             Bytes.compareTo(thisRow, lastRow) < 0);
6505       }
6506       lastRow = thisRow;
6507     }
6508     assertEquals(3, count); // 000 001 002
6509 
6510     scan = new Scan();
6511     scan.setSmall(small);
6512     scan.setReversed(true);
6513     scan.setStartRow(Bytes.toBytes("002"));
6514     scan.setStopRow(Bytes.toBytes("000"));
6515     scanner = table.getScanner(scan);
6516     count = 0;
6517     lastRow = null;
6518     for (Result r : scanner) {
6519       assertTrue(!r.isEmpty());
6520       count++;
6521       byte[] thisRow = r.getRow();
6522       if (lastRow != null) {
6523         assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6524             + ",this row=" + Bytes.toString(thisRow),
6525             Bytes.compareTo(thisRow, lastRow) < 0);
6526       }
6527       lastRow = thisRow;
6528     }
6529     assertEquals(2, count); // 001 002
6530 
6531     scan = new Scan();
6532     scan.setSmall(small);
6533     scan.setReversed(true);
6534     scan.setStartRow(Bytes.toBytes("001"));
6535     scanner = table.getScanner(scan);
6536     count = 0;
6537     lastRow = null;
6538     for (Result r : scanner) {
6539       assertTrue(!r.isEmpty());
6540       count++;
6541       byte[] thisRow = r.getRow();
6542       if (lastRow != null) {
6543         assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6544             + ",this row=" + Bytes.toString(thisRow),
6545             Bytes.compareTo(thisRow, lastRow) < 0);
6546       }
6547       lastRow = thisRow;
6548     }
6549     assertEquals(2, count); // 000 001
6550 
6551     scan = new Scan();
6552     scan.setSmall(small);
6553     scan.setReversed(true);
6554     scan.setStartRow(Bytes.toBytes("000"));
6555     scanner = table.getScanner(scan);
6556     count = 0;
6557     lastRow = null;
6558     for (Result r : scanner) {
6559       assertTrue(!r.isEmpty());
6560       count++;
6561       byte[] thisRow = r.getRow();
6562       if (lastRow != null) {
6563         assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6564             + ",this row=" + Bytes.toString(thisRow),
6565             Bytes.compareTo(thisRow, lastRow) < 0);
6566       }
6567       lastRow = thisRow;
6568     }
6569     assertEquals(1, count); // 000
6570 
6571     scan = new Scan();
6572     scan.setSmall(small);
6573     scan.setReversed(true);
6574     scan.setStartRow(Bytes.toBytes("006"));
6575     scan.setStopRow(Bytes.toBytes("002"));
6576     scanner = table.getScanner(scan);
6577     count = 0;
6578     lastRow = null;
6579     for (Result r : scanner) {
6580       assertTrue(!r.isEmpty());
6581       count++;
6582       byte[] thisRow = r.getRow();
6583       if (lastRow != null) {
6584         assertTrue("Error scan order, last row= " + Bytes.toString(lastRow)
6585             + ",this row=" + Bytes.toString(thisRow),
6586             Bytes.compareTo(thisRow, lastRow) < 0);
6587       }
6588       lastRow = thisRow;
6589     }
6590     assertEquals(4, count); // 003 004 005 006
6591   }
6592 
6593   @Test
6594   public void testGetStartEndKeysWithRegionReplicas() throws IOException, InterruptedException {
6595     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testGetStartEndKeys"));
6596     HColumnDescriptor fam = new HColumnDescriptor(FAMILY);
6597     htd.addFamily(fam);
6598     byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
6599     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
6600     admin.createTable(htd, KEYS);
6601     TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
6602     List<HRegionInfo> regions = admin.getTableRegions(htd.getTableName());
6603 
6604     HRegionLocator locator =
6605         (HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName());
6606     for (int regionReplication = 1; regionReplication < 4; regionReplication++) {
6607       List<RegionLocations> regionLocations = new ArrayList<RegionLocations>();
6608 
6609       // mock region locations coming from meta with multiple replicas
6610       for (HRegionInfo region : regions) {
6611         HRegionLocation[] arr = new HRegionLocation[regionReplication];
6612         for (int i = 0; i < arr.length; i++) {
6613           arr[i] = new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(region, i), null);
6614         }
6615         regionLocations.add(new RegionLocations(arr));
6616       }
6617 
6618       Pair<byte[][], byte[][]> startEndKeys = locator.getStartEndKeys(regionLocations);
6619 
6620       assertEquals(KEYS.length + 1, startEndKeys.getFirst().length);
6621 
6622       for (int i = 0; i < KEYS.length + 1; i++) {
6623         byte[] startKey = i == 0 ? HConstants.EMPTY_START_ROW : KEYS[i - 1];
6624         byte[] endKey = i == KEYS.length ? HConstants.EMPTY_END_ROW : KEYS[i];
6625         assertArrayEquals(startKey, startEndKeys.getFirst()[i]);
6626         assertArrayEquals(endKey, startEndKeys.getSecond()[i]);
6627       }
6628     }
6629   }
6630 
6631   @Test
6632   public void testFilterAllRecords() throws IOException {
6633     Scan scan = new Scan();
6634     scan.setBatch(1);
6635     scan.setCaching(1);
6636     // Filter out any records
6637     scan.setFilter(new FilterList(new FirstKeyOnlyFilter(), new InclusiveStopFilter(new byte[0])));
6638     Table table = TEST_UTIL.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
6639     ResultScanner s = table.getScanner(scan);
6640     assertNull(s.next());
6641     table.close();
6642   }
6643 
6644   @Test
6645   public void testRegionCache() throws IOException, InterruptedException {
6646     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionCache"));
6647     HColumnDescriptor fam = new HColumnDescriptor(FAMILY);
6648     htd.addFamily(fam);
6649     byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
6650     Admin admin = TEST_UTIL.getHBaseAdmin();
6651     admin.createTable(htd, KEYS);
6652     TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
6653 
6654     HRegionLocator locator =
6655       (HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName());
6656     List<HRegionLocation> results = locator.getAllRegionLocations();
6657     int number = ((ConnectionManager.HConnectionImplementation)admin.getConnection())
6658       .getNumberOfCachedRegionLocations(htd.getTableName());
6659     assertEquals(results.size(), number);
6660   }
6661 
6662   @Test
6663   public void testCellSizeLimit() throws IOException, InterruptedException {
6664     final TableName tableName = TableName.valueOf("testCellSizeLimit");
6665     HTableDescriptor htd = new HTableDescriptor(tableName);
6666     htd.setConfiguration(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024)); // 10K
6667     HColumnDescriptor fam = new HColumnDescriptor(FAMILY);
6668     htd.addFamily(fam);
6669     Admin admin = TEST_UTIL.getHBaseAdmin();
6670     admin.createTable(htd);
6671     TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
6672 
6673     // Will succeed
6674     try (Table t = TEST_UTIL.getConnection().getTable(tableName)) {
6675       t.put(new Put(ROW).add(FAMILY, QUALIFIER, Bytes.toBytes(0L)));
6676       t.increment(new Increment(ROW).addColumn(FAMILY, QUALIFIER, 1L));
6677     }
6678     // Will succeed
6679     try (Table t = TEST_UTIL.getConnection().getTable(tableName)) {
6680       t.put(new Put(ROW).add(FAMILY, QUALIFIER, new byte[9*1024]));
6681     }
6682     // Will fail
6683     try (Table t = TEST_UTIL.getConnection().getTable(tableName)) {
6684       try {
6685         t.put(new Put(ROW).add(FAMILY, QUALIFIER, new byte[10 * 1024]));
6686         fail("Oversize cell failed to trigger exception");
6687       } catch (IOException e) {
6688         // expected
6689       }
6690       try {
6691         t.append(new Append(ROW).add(FAMILY, QUALIFIER, new byte[2 * 1024]));
6692         fail("Oversize cell failed to trigger exception");
6693       } catch (IOException e) {
6694         // expected
6695       }
6696     }
6697   }
6698 
6699   @Test(expected = DoNotRetryIOException.class)
6700   public void testCreateTableWithZeroRegionReplicas() throws Exception {
6701     TableName tableName = TableName.valueOf("testCreateTableWithZeroRegionReplicas");
6702     HTableDescriptor desc = new HTableDescriptor(tableName);
6703     desc.addFamily(new HColumnDescriptor("cf"));
6704     desc.setRegionReplication(0);
6705     TEST_UTIL.getHBaseAdmin().createTable(desc);
6706   }
6707 
6708   @Test(expected = DoNotRetryIOException.class)
6709   public void testModifyTableWithZeroRegionReplicas() throws Exception {
6710     TableName tableName = TableName.valueOf("testModifyTableWithZeroRegionReplicas");
6711     HTableDescriptor desc = new HTableDescriptor(tableName);
6712     desc.addFamily(new HColumnDescriptor("cf"));
6713 
6714     TEST_UTIL.getHBaseAdmin().createTable(desc);
6715     TEST_UTIL.waitTableAvailable(desc.getTableName(), 10000);
6716 
6717     HTableDescriptor newDesc = new HTableDescriptor(desc);
6718     newDesc.setRegionReplication(0);
6719 
6720     TEST_UTIL.getHBaseAdmin().modifyTable(tableName, newDesc);
6721   }
6722 }