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