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