1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver;
21  
22  import org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.fs.Path;
25  import org.apache.hadoop.hbase.HBaseConfiguration;
26  import org.apache.hadoop.hbase.HBaseTestCase;
27  import org.apache.hadoop.hbase.HBaseTestingUtility;
28  import org.apache.hadoop.hbase.HColumnDescriptor;
29  import org.apache.hadoop.hbase.HConstants;
30  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
31  import org.apache.hadoop.hbase.HRegionInfo;
32  import org.apache.hadoop.hbase.HTableDescriptor;
33  import org.apache.hadoop.hbase.KeyValue;
34  import org.apache.hadoop.hbase.MultithreadedTestUtil;
35  import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
36  import org.apache.hadoop.hbase.client.Delete;
37  import org.apache.hadoop.hbase.client.Get;
38  import org.apache.hadoop.hbase.client.Put;
39  import org.apache.hadoop.hbase.client.Result;
40  import org.apache.hadoop.hbase.client.Scan;
41  import org.apache.hadoop.hbase.filter.BinaryComparator;
42  import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
43  import org.apache.hadoop.hbase.filter.CompareFilter;
44  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
45  import org.apache.hadoop.hbase.filter.Filter;
46  import org.apache.hadoop.hbase.filter.FilterList;
47  import org.apache.hadoop.hbase.filter.PrefixFilter;
48  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
49  import org.apache.hadoop.hbase.regionserver.HRegion.RegionScanner;
50  import org.apache.hadoop.hbase.regionserver.wal.HLog;
51  import org.apache.hadoop.hbase.util.Bytes;
52  import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
53  import org.apache.hadoop.hbase.util.FSUtils;
54  import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
55  import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
56  import org.apache.hadoop.hbase.util.Pair;
57  import org.apache.hadoop.hbase.util.PairOfSameType;
58  import org.apache.hadoop.hbase.util.Threads;
59  
60  import com.google.common.base.Joiner;
61  import com.google.common.collect.Lists;
62  
63  import java.io.IOException;
64  import java.util.ArrayList;
65  import java.util.Arrays;
66  import java.util.HashMap;
67  import java.util.List;
68  import java.util.Map;
69  import java.util.TreeMap;
70  import java.util.concurrent.atomic.AtomicBoolean;
71  import java.util.concurrent.atomic.AtomicInteger;
72  import java.util.concurrent.atomic.AtomicReference;
73  
74  
75  /**
76   * Basic stand-alone testing of HRegion.
77   *
78   * A lot of the meta information for an HRegion now lives inside other
79   * HRegions or in the HBaseMaster, so only basic testing is possible.
80   */
81  public class TestHRegion extends HBaseTestCase {
82    static final Log LOG = LogFactory.getLog(TestHRegion.class);
83  
84    HRegion region = null;
85    private final String DIR = HBaseTestingUtility.getTestDir() +
86      "/TestHRegion/";
87  
88    private final int MAX_VERSIONS = 2;
89  
90    // Test names
91    protected final byte[] tableName = Bytes.toBytes("testtable");;
92    protected final byte[] qual1 = Bytes.toBytes("qual1");
93    protected final byte[] qual2 = Bytes.toBytes("qual2");
94    protected final byte[] qual3 = Bytes.toBytes("qual3");
95    protected final byte[] value1 = Bytes.toBytes("value1");
96    protected final byte[] value2 = Bytes.toBytes("value2");
97    protected final byte [] row = Bytes.toBytes("rowA");
98  
99    /**
100    * @see org.apache.hadoop.hbase.HBaseTestCase#setUp()
101    */
102   @Override
103   protected void setUp() throws Exception {
104     super.setUp();
105   }
106 
107   @Override
108   protected void tearDown() throws Exception {
109     super.tearDown();
110     EnvironmentEdgeManagerTestHelper.reset();
111   }
112 
113   //////////////////////////////////////////////////////////////////////////////
114   // New tests that doesn't spin up a mini cluster but rather just test the
115   // individual code pieces in the HRegion. Putting files locally in
116   // /tmp/testtable
117   //////////////////////////////////////////////////////////////////////////////
118 
119   public void testGetWhileRegionClose() throws IOException {
120     HBaseConfiguration hc = initSplit();
121     int numRows = 100;
122     byte [][] families = {fam1, fam2, fam3};
123 
124     //Setting up region
125     String method = this.getName();
126     initHRegion(tableName, method, hc, families);
127 
128     // Put data in region
129     final int startRow = 100;
130     putData(startRow, numRows, qual1, families);
131     putData(startRow, numRows, qual2, families);
132     putData(startRow, numRows, qual3, families);
133     // this.region.flushcache();
134     final AtomicBoolean done = new AtomicBoolean(false);
135     final AtomicInteger gets = new AtomicInteger(0);
136     GetTillDoneOrException [] threads = new GetTillDoneOrException[10];
137     try {
138       // Set ten threads running concurrently getting from the region.
139       for (int i = 0; i < threads.length / 2; i++) {
140         threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
141           done, gets);
142         threads[i].setDaemon(true);
143         threads[i].start();
144       }
145       // Artificially make the condition by setting closing flag explicitly.
146       // I can't make the issue happen with a call to region.close().
147       this.region.closing.set(true);
148       for (int i = threads.length / 2; i < threads.length; i++) {
149         threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
150           done, gets);
151         threads[i].setDaemon(true);
152         threads[i].start();
153       }
154     } finally {
155       if (this.region != null) {
156         this.region.close();
157         this.region.getLog().closeAndDelete();
158       }
159     }
160     done.set(true);
161     for (GetTillDoneOrException t: threads) {
162       try {
163         t.join();
164       } catch (InterruptedException e) {
165         e.printStackTrace();
166       }
167       if (t.e != null) {
168         LOG.info("Exception=" + t.e);
169         assertFalse("Found a NPE in " + t.getName(),
170           t.e instanceof NullPointerException);
171       }
172     }
173   }
174 
175   /*
176    * Thread that does get on single row until 'done' flag is flipped.  If an
177    * exception causes us to fail, it records it.
178    */
179   class GetTillDoneOrException extends Thread {
180     private final Get g;
181     private final AtomicBoolean done;
182     private final AtomicInteger count;
183     private Exception e;
184 
185     GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d,
186         final AtomicInteger c) {
187       super("getter." + i);
188       this.g = new Get(r);
189       this.done = d;
190       this.count = c;
191     }
192 
193     @Override
194     public void run() {
195       while (!this.done.get()) {
196         try {
197           assertTrue(region.get(g, null).size() > 0);
198           this.count.incrementAndGet();
199         } catch (Exception e) {
200           this.e = e;
201           break;
202         }
203       }
204     }
205   }
206 
207   /*
208    * An involved filter test.  Has multiple column families and deletes in mix.
209    */
210   public void testWeirdCacheBehaviour() throws Exception {
211     byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
212     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
213         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
214         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
215     initHRegion(TABLE, getName(), FAMILIES);
216     String value = "this is the value";
217     String value2 = "this is some other value";
218     String keyPrefix1 = "prefix1"; // UUID.randomUUID().toString();
219     String keyPrefix2 = "prefix2"; // UUID.randomUUID().toString();
220     String keyPrefix3 = "prefix3"; // UUID.randomUUID().toString();
221     putRows(this.region, 3, value, keyPrefix1);
222     putRows(this.region, 3, value, keyPrefix2);
223     putRows(this.region, 3, value, keyPrefix3);
224     // this.region.flushCommits();
225     putRows(this.region, 3, value2, keyPrefix1);
226     putRows(this.region, 3, value2, keyPrefix2);
227     putRows(this.region, 3, value2, keyPrefix3);
228     System.out.println("Checking values for key: " + keyPrefix1);
229     assertEquals("Got back incorrect number of rows from scan", 3,
230       getNumberOfRows(keyPrefix1, value2, this.region));
231     System.out.println("Checking values for key: " + keyPrefix2);
232     assertEquals("Got back incorrect number of rows from scan", 3,
233       getNumberOfRows(keyPrefix2, value2, this.region));
234     System.out.println("Checking values for key: " + keyPrefix3);
235     assertEquals("Got back incorrect number of rows from scan", 3,
236       getNumberOfRows(keyPrefix3, value2, this.region));
237     deleteColumns(this.region, value2, keyPrefix1);
238     deleteColumns(this.region, value2, keyPrefix2);
239     deleteColumns(this.region, value2, keyPrefix3);
240     System.out.println("Starting important checks.....");
241     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
242       0, getNumberOfRows(keyPrefix1, value2, this.region));
243     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
244       0, getNumberOfRows(keyPrefix2, value2, this.region));
245     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
246       0, getNumberOfRows(keyPrefix3, value2, this.region));
247   }
248 
249   private void deleteColumns(HRegion r, String value, String keyPrefix)
250   throws IOException {
251     InternalScanner scanner = buildScanner(keyPrefix, value, r);
252     int count = 0;
253     boolean more = false;
254     List<KeyValue> results = new ArrayList<KeyValue>();
255     do {
256       more = scanner.next(results);
257       if (results != null && !results.isEmpty())
258         count++;
259       else
260         break;
261       Delete delete = new Delete(results.get(0).getRow());
262       delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
263       r.delete(delete, null, false);
264       results.clear();
265     } while (more);
266     assertEquals("Did not perform correct number of deletes", 3, count);
267   }
268 
269   private int getNumberOfRows(String keyPrefix, String value, HRegion r) throws Exception {
270     InternalScanner resultScanner = buildScanner(keyPrefix, value, r);
271     int numberOfResults = 0;
272     List<KeyValue> results = new ArrayList<KeyValue>();
273     boolean more = false;
274     do {
275       more = resultScanner.next(results);
276       if (results != null && !results.isEmpty()) numberOfResults++;
277       else break;
278       for (KeyValue kv: results) {
279         System.out.println("kv=" + kv.toString() + ", " + Bytes.toString(kv.getValue()));
280       }
281       results.clear();
282     } while(more);
283     return numberOfResults;
284   }
285 
286   private InternalScanner buildScanner(String keyPrefix, String value, HRegion r)
287   throws IOException {
288     // Defaults FilterList.Operator.MUST_PASS_ALL.
289     FilterList allFilters = new FilterList();
290     allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
291     // Only return rows where this column value exists in the row.
292     SingleColumnValueFilter filter =
293       new SingleColumnValueFilter(Bytes.toBytes("trans-tags"),
294         Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value));
295     filter.setFilterIfMissing(true);
296     allFilters.addFilter(filter);
297     Scan scan = new Scan();
298     scan.addFamily(Bytes.toBytes("trans-blob"));
299     scan.addFamily(Bytes.toBytes("trans-type"));
300     scan.addFamily(Bytes.toBytes("trans-date"));
301     scan.addFamily(Bytes.toBytes("trans-tags"));
302     scan.addFamily(Bytes.toBytes("trans-group"));
303     scan.setFilter(allFilters);
304     return r.getScanner(scan);
305   }
306 
307   private void putRows(HRegion r, int numRows, String value, String key)
308   throws IOException {
309     for (int i = 0; i < numRows; i++) {
310       String row = key + "_" + i/* UUID.randomUUID().toString() */;
311       System.out.println(String.format("Saving row: %s, with value %s", row,
312         value));
313       Put put = new Put(Bytes.toBytes(row));
314       put.add(Bytes.toBytes("trans-blob"), null,
315         Bytes.toBytes("value for blob"));
316       put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
317       put.add(Bytes.toBytes("trans-date"), null,
318         Bytes.toBytes("20090921010101999"));
319       put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"),
320         Bytes.toBytes(value));
321       put.add(Bytes.toBytes("trans-group"), null,
322         Bytes.toBytes("adhocTransactionGroupId"));
323       r.put(put);
324     }
325   }
326 
327   public void testFamilyWithAndWithoutColon() throws Exception {
328     byte [] b = Bytes.toBytes(getName());
329     byte [] cf = Bytes.toBytes("cf");
330     initHRegion(b, getName(), cf);
331     Put p = new Put(b);
332     byte [] cfwithcolon = Bytes.toBytes("cf:");
333     p.add(cfwithcolon, cfwithcolon, cfwithcolon);
334     boolean exception = false;
335     try {
336       this.region.put(p);
337     } catch (NoSuchColumnFamilyException e) {
338       exception = true;
339     }
340     assertTrue(exception);
341   }
342 
343   @SuppressWarnings("unchecked")
344   public void testBatchPut() throws Exception {
345     byte[] b = Bytes.toBytes(getName());
346     byte[] cf = Bytes.toBytes("cf");
347     byte[] qual = Bytes.toBytes("qual");
348     byte[] val = Bytes.toBytes("val");
349     initHRegion(b, getName(), cf);
350 
351     HLog.getSyncOps(); // clear counter from prior tests
352     assertEquals(0, HLog.getSyncOps());
353 
354     LOG.info("First a batch put with all valid puts");
355     final Put[] puts = new Put[10];
356     for (int i = 0; i < 10; i++) {
357       puts[i] = new Put(Bytes.toBytes("row_" + i));
358       puts[i].add(cf, qual, val);
359     }
360 
361     OperationStatusCode[] codes = this.region.put(puts);
362     assertEquals(10, codes.length);
363     for (int i = 0; i < 10; i++) {
364       assertEquals(OperationStatusCode.SUCCESS, codes[i]);
365     }
366     assertEquals(1, HLog.getSyncOps());
367 
368     LOG.info("Next a batch put with one invalid family");
369     puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
370     codes = this.region.put(puts);
371     assertEquals(10, codes.length);
372     for (int i = 0; i < 10; i++) {
373       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
374         OperationStatusCode.SUCCESS, codes[i]);
375     }
376     assertEquals(1, HLog.getSyncOps());
377 
378     LOG.info("Next a batch put that has to break into two batches to avoid a lock");
379     Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
380 
381     MultithreadedTestUtil.TestContext ctx =
382       new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
383     final AtomicReference<OperationStatusCode[]> retFromThread =
384       new AtomicReference<OperationStatusCode[]>();
385     TestThread putter = new TestThread(ctx) {
386       @Override
387       public void doWork() throws IOException {
388         retFromThread.set(region.put(puts));
389       }
390     };
391     LOG.info("...starting put thread while holding lock");
392     ctx.addThread(putter);
393     ctx.startThreads();
394 
395     LOG.info("...waiting for put thread to sync first time");
396     long startWait = System.currentTimeMillis();
397     while (HLog.getSyncOps() == 0) {
398       Thread.sleep(100);
399       if (System.currentTimeMillis() - startWait > 10000) {
400         fail("Timed out waiting for thread to sync first minibatch");
401       }
402     }
403     LOG.info("...releasing row lock, which should let put thread continue");
404     region.releaseRowLock(lockedRow);
405     LOG.info("...joining on thread");
406     ctx.stop();
407     LOG.info("...checking that next batch was synced");
408     assertEquals(1, HLog.getSyncOps());
409     codes = retFromThread.get();
410     for (int i = 0; i < 10; i++) {
411       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
412         OperationStatusCode.SUCCESS, codes[i]);
413     }
414 
415     LOG.info("Nexta, a batch put which uses an already-held lock");
416     lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
417     LOG.info("...obtained row lock");
418     List<Pair<Put, Integer>> putsAndLocks = Lists.newArrayList();
419     for (int i = 0; i < 10; i++) {
420       Pair<Put, Integer> pair = new Pair<Put, Integer>(puts[i], null);
421       if (i == 2) pair.setSecond(lockedRow);
422       putsAndLocks.add(pair);
423     }
424 
425     codes = region.put(putsAndLocks.toArray(new Pair[0]));
426     LOG.info("...performed put");
427     for (int i = 0; i < 10; i++) {
428       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
429         OperationStatusCode.SUCCESS, codes[i]);
430     }
431     // Make sure we didn't do an extra batch
432     assertEquals(1, HLog.getSyncOps());
433 
434     // Make sure we still hold lock
435     assertTrue(region.isRowLocked(lockedRow));
436     LOG.info("...releasing lock");
437     region.releaseRowLock(lockedRow);
438   }
439 
440   //////////////////////////////////////////////////////////////////////////////
441   // checkAndMutate tests
442   //////////////////////////////////////////////////////////////////////////////
443   public void testCheckAndMutate_WithEmptyRowValue() throws IOException {
444     byte [] tableName = Bytes.toBytes("testtable");
445     byte [] row1 = Bytes.toBytes("row1");
446     byte [] fam1 = Bytes.toBytes("fam1");
447     byte [] qf1  = Bytes.toBytes("qualifier");
448     byte [] emptyVal  = new byte[] {};
449     byte [] val1  = Bytes.toBytes("value1");
450     byte [] val2  = Bytes.toBytes("value2");
451     Integer lockId = null;
452 
453     //Setting up region
454     String method = this.getName();
455     initHRegion(tableName, method, fam1);
456     //Putting data in key
457     Put put = new Put(row1);
458     put.add(fam1, qf1, val1);
459 
460     //checkAndPut with correct value
461     boolean res = region.checkAndMutate(row1, fam1, qf1, emptyVal, put, lockId,
462         true);
463     assertTrue(res);
464 
465     // not empty anymore
466     res = region.checkAndMutate(row1, fam1, qf1, emptyVal, put, lockId, true);
467     assertFalse(res);
468 
469     Delete delete = new Delete(row1);
470     delete.deleteColumn(fam1, qf1);
471     res = region.checkAndMutate(row1, fam1, qf1, emptyVal, delete, lockId,
472         true);
473     assertFalse(res);
474 
475     put = new Put(row1);
476     put.add(fam1, qf1, val2);
477     //checkAndPut with correct value
478     res = region.checkAndMutate(row1, fam1, qf1, val1, put, lockId, true);
479     assertTrue(res);
480 
481     //checkAndDelete with correct value
482     delete = new Delete(row1);
483     delete.deleteColumn(fam1, qf1);
484     delete.deleteColumn(fam1, qf1);
485     res = region.checkAndMutate(row1, fam1, qf1, val2, delete, lockId, true);
486     assertTrue(res);
487 
488     delete = new Delete(row1);
489     res = region.checkAndMutate(row1, fam1, qf1, emptyVal, delete, lockId,
490         true);
491     assertTrue(res);
492 
493     //checkAndPut looking for a null value
494     put = new Put(row1);
495     put.add(fam1, qf1, val1);
496 
497     res = region.checkAndMutate(row1, fam1, qf1, null, put, lockId, true);
498     assertTrue(res);
499     
500   }
501 
502   public void testCheckAndMutate_WithWrongValue() throws IOException{
503     byte [] tableName = Bytes.toBytes("testtable");
504     byte [] row1 = Bytes.toBytes("row1");
505     byte [] fam1 = Bytes.toBytes("fam1");
506     byte [] qf1  = Bytes.toBytes("qualifier");
507     byte [] val1  = Bytes.toBytes("value1");
508     byte [] val2  = Bytes.toBytes("value2");
509     Integer lockId = null;
510 
511     //Setting up region
512     String method = this.getName();
513     initHRegion(tableName, method, fam1);
514 
515     //Putting data in key
516     Put put = new Put(row1);
517     put.add(fam1, qf1, val1);
518     region.put(put);
519 
520     //checkAndPut with wrong value
521     boolean res = region.checkAndMutate(row1, fam1, qf1, val2, put, lockId, true);
522     assertEquals(false, res);
523 
524     //checkAndDelete with wrong value
525     Delete delete = new Delete(row1);
526     delete.deleteFamily(fam1);
527     res = region.checkAndMutate(row1, fam1, qf1, val2, delete, lockId, true);
528     assertEquals(false, res);
529   }
530 
531   public void testCheckAndMutate_WithCorrectValue() throws IOException{
532     byte [] tableName = Bytes.toBytes("testtable");
533     byte [] row1 = Bytes.toBytes("row1");
534     byte [] fam1 = Bytes.toBytes("fam1");
535     byte [] qf1  = Bytes.toBytes("qualifier");
536     byte [] val1  = Bytes.toBytes("value1");
537     Integer lockId = null;
538 
539     //Setting up region
540     String method = this.getName();
541     initHRegion(tableName, method, fam1);
542 
543     //Putting data in key
544     Put put = new Put(row1);
545     put.add(fam1, qf1, val1);
546     region.put(put);
547 
548     //checkAndPut with correct value
549     boolean res = region.checkAndMutate(row1, fam1, qf1, val1, put, lockId, true);
550     assertEquals(true, res);
551 
552     //checkAndDelete with correct value
553     Delete delete = new Delete(row1);
554     delete.deleteColumn(fam1, qf1);
555     res = region.checkAndMutate(row1, fam1, qf1, val1, put, lockId, true);
556     assertEquals(true, res);
557   }
558 
559   public void testCheckAndPut_ThatPutWasWritten() throws IOException{
560     byte [] tableName = Bytes.toBytes("testtable");
561     byte [] row1 = Bytes.toBytes("row1");
562     byte [] fam1 = Bytes.toBytes("fam1");
563     byte [] fam2 = Bytes.toBytes("fam2");
564     byte [] qf1  = Bytes.toBytes("qualifier");
565     byte [] val1  = Bytes.toBytes("value1");
566     byte [] val2  = Bytes.toBytes("value2");
567     Integer lockId = null;
568 
569     byte [][] families = {fam1, fam2};
570 
571     //Setting up region
572     String method = this.getName();
573     initHRegion(tableName, method, families);
574 
575     //Putting data in the key to check
576     Put put = new Put(row1);
577     put.add(fam1, qf1, val1);
578     region.put(put);
579 
580     //Creating put to add
581     long ts = System.currentTimeMillis();
582     KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
583     put = new Put(row1);
584     put.add(kv);
585 
586     //checkAndPut with wrong value
587     Store store = region.getStore(fam1);
588     store.memstore.kvset.size();
589 
590     boolean res = region.checkAndMutate(row1, fam1, qf1, val1, put, lockId, true);
591     assertEquals(true, res);
592     store.memstore.kvset.size();
593 
594     Get get = new Get(row1);
595     get.addColumn(fam2, qf1);
596     KeyValue [] actual = region.get(get, null).raw();
597 
598     KeyValue [] expected = {kv};
599 
600     assertEquals(expected.length, actual.length);
601     for(int i=0; i<actual.length; i++) {
602       assertEquals(expected[i], actual[i]);
603     }
604 
605   }
606 
607   public void testCheckAndDelete_ThatDeleteWasWritten() throws IOException{
608     byte [] tableName = Bytes.toBytes("testtable");
609     byte [] row1 = Bytes.toBytes("row1");
610     byte [] fam1 = Bytes.toBytes("fam1");
611     byte [] fam2 = Bytes.toBytes("fam2");
612     byte [] qf1  = Bytes.toBytes("qualifier1");
613     byte [] qf2  = Bytes.toBytes("qualifier2");
614     byte [] qf3  = Bytes.toBytes("qualifier3");
615     byte [] val1  = Bytes.toBytes("value1");
616     byte [] val2  = Bytes.toBytes("value2");
617     byte [] val3  = Bytes.toBytes("value3");
618     byte[] emptyVal = new byte[] { };
619     Integer lockId = null;
620 
621     byte [][] families = {fam1, fam2};
622 
623     //Setting up region
624     String method = this.getName();
625     initHRegion(tableName, method, families);
626 
627     //Put content
628     Put put = new Put(row1);
629     put.add(fam1, qf1, val1);
630     region.put(put);
631     Threads.sleep(2);
632 
633     put = new Put(row1);
634     put.add(fam1, qf1, val2);
635     put.add(fam2, qf1, val3);
636     put.add(fam2, qf2, val2);
637     put.add(fam2, qf3, val1);
638     put.add(fam1, qf3, val1);
639     region.put(put);
640 
641     //Multi-column delete
642     Delete delete = new Delete(row1);
643     delete.deleteColumn(fam1, qf1);
644     delete.deleteColumn(fam2, qf1);
645     delete.deleteColumn(fam1, qf3);
646     boolean res = region.checkAndMutate(row1, fam1, qf1, val2, delete, lockId,
647         true);
648     assertEquals(true, res);
649 
650     Get get = new Get(row1);
651     get.addColumn(fam1, qf1);
652     get.addColumn(fam1, qf3);
653     get.addColumn(fam2, qf2);
654     Result r = region.get(get, null);
655     assertEquals(2, r.size());
656     assertEquals(val1, r.getValue(fam1, qf1));
657     assertEquals(val2, r.getValue(fam2, qf2));
658 
659     //Family delete
660     delete = new Delete(row1);
661     delete.deleteFamily(fam2);
662     res = region.checkAndMutate(row1, fam2, qf1, emptyVal, delete, lockId,
663         true);
664     assertEquals(true, res);
665 
666     get = new Get(row1);
667     r = region.get(get, null);
668     assertEquals(1, r.size());
669     assertEquals(val1, r.getValue(fam1, qf1));
670 
671     //Row delete
672     delete = new Delete(row1);
673     res = region.checkAndMutate(row1, fam1, qf1, val1, delete, lockId,
674         true);
675     assertEquals(true, res);
676     get = new Get(row1);
677     r = region.get(get, null);
678     assertEquals(0, r.size());
679   }
680 
681   //////////////////////////////////////////////////////////////////////////////
682   // Delete tests
683   //////////////////////////////////////////////////////////////////////////////
684   public void testDelete_multiDeleteColumn() throws IOException {
685     byte [] tableName = Bytes.toBytes("testtable");
686     byte [] row1 = Bytes.toBytes("row1");
687     byte [] fam1 = Bytes.toBytes("fam1");
688     byte [] qual = Bytes.toBytes("qualifier");
689     byte [] value = Bytes.toBytes("value");
690 
691     Put put = new Put(row1);
692     put.add(fam1, qual, 1, value);
693     put.add(fam1, qual, 2, value);
694 
695     String method = this.getName();
696     initHRegion(tableName, method, fam1);
697 
698     region.put(put);
699 
700     // We do support deleting more than 1 'latest' version
701     Delete delete = new Delete(row1);
702     delete.deleteColumn(fam1, qual);
703     delete.deleteColumn(fam1, qual);
704     region.delete(delete, null, false);
705 
706     Get get = new Get(row1);
707     get.addFamily(fam1);
708     Result r = region.get(get, null);
709     assertEquals(0, r.size());
710   }
711 
712   public void testDelete_CheckFamily() throws IOException {
713     byte [] tableName = Bytes.toBytes("testtable");
714     byte [] row1 = Bytes.toBytes("row1");
715     byte [] fam1 = Bytes.toBytes("fam1");
716     byte [] fam2 = Bytes.toBytes("fam2");
717     byte [] fam3 = Bytes.toBytes("fam3");
718     byte [] fam4 = Bytes.toBytes("fam4");
719 
720     //Setting up region
721     String method = this.getName();
722     initHRegion(tableName, method, fam1, fam2, fam3);
723 
724     List<KeyValue> kvs  = new ArrayList<KeyValue>();
725     kvs.add(new KeyValue(row1, fam4, null, null));
726 
727 
728     //testing existing family
729     byte [] family = fam2;
730     try {
731       Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
732       deleteMap.put(family, kvs);
733       region.delete(deleteMap, true);
734     } catch (Exception e) {
735       assertTrue("Family " +new String(family)+ " does not exist", false);
736     }
737 
738     //testing non existing family
739     boolean ok = false;
740     family = fam4;
741     try {
742       Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
743       deleteMap.put(family, kvs);
744       region.delete(deleteMap, true);
745     } catch (Exception e) {
746       ok = true;
747     }
748     assertEquals("Family " +new String(family)+ " does exist", true, ok);
749   }
750 
751   public void testDelete_mixed() throws IOException, InterruptedException {
752     byte [] tableName = Bytes.toBytes("testtable");
753     byte [] fam = Bytes.toBytes("info");
754     byte [][] families = {fam};
755     String method = this.getName();
756     initHRegion(tableName, method, families);
757     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
758 
759     byte [] row = Bytes.toBytes("table_name");
760     // column names
761     byte [] serverinfo = Bytes.toBytes("serverinfo");
762     byte [] splitA = Bytes.toBytes("splitA");
763     byte [] splitB = Bytes.toBytes("splitB");
764 
765     // add some data:
766     Put put = new Put(row);
767     put.add(fam, splitA, Bytes.toBytes("reference_A"));
768     region.put(put);
769 
770     put = new Put(row);
771     put.add(fam, splitB, Bytes.toBytes("reference_B"));
772     region.put(put);
773 
774     put = new Put(row);
775     put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
776     region.put(put);
777 
778     // ok now delete a split:
779     Delete delete = new Delete(row);
780     delete.deleteColumns(fam, splitA);
781     region.delete(delete, null, true);
782 
783     // assert some things:
784     Get get = new Get(row).addColumn(fam, serverinfo);
785     Result result = region.get(get, null);
786     assertEquals(1, result.size());
787 
788     get = new Get(row).addColumn(fam, splitA);
789     result = region.get(get, null);
790     assertEquals(0, result.size());
791 
792     get = new Get(row).addColumn(fam, splitB);
793     result = region.get(get, null);
794     assertEquals(1, result.size());
795 
796     // Assert that after a delete, I can put.
797     put = new Put(row);
798     put.add(fam, splitA, Bytes.toBytes("reference_A"));
799     region.put(put);
800     get = new Get(row);
801     result = region.get(get, null);
802     assertEquals(3, result.size());
803 
804     // Now delete all... then test I can add stuff back
805     delete = new Delete(row);
806     region.delete(delete, null, false);
807     assertEquals(0, region.get(get, null).size());
808 
809     region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
810     result = region.get(get, null);
811     assertEquals(1, result.size());
812   }
813 
814   public void testDeleteRowWithFutureTs() throws IOException {
815     byte [] tableName = Bytes.toBytes("testtable");
816     byte [] fam = Bytes.toBytes("info");
817     byte [][] families = {fam};
818     String method = this.getName();
819     initHRegion(tableName, method, families);
820 
821     byte [] row = Bytes.toBytes("table_name");
822     // column names
823     byte [] serverinfo = Bytes.toBytes("serverinfo");
824 
825     // add data in the far future
826     Put put = new Put(row);
827     put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP-5,Bytes.toBytes("value"));
828     region.put(put);
829 
830     // now delete something in the present
831     Delete delete = new Delete(row);
832     region.delete(delete, null, true);
833 
834     // make sure we still see our data
835     Get get = new Get(row).addColumn(fam, serverinfo);
836     Result result = region.get(get, null);
837     assertEquals(1, result.size());
838 
839     // delete the future row
840     delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
841     region.delete(delete, null, true);
842 
843     // make sure it is gone
844     get = new Get(row).addColumn(fam, serverinfo);
845     result = region.get(get, null);
846     assertEquals(0, result.size());
847   }
848 
849   /**
850    * Tests that the special LATEST_TIMESTAMP option for puts gets
851    * replaced by the actual timestamp
852    */
853   public void testPutWithLatestTS() throws IOException {
854     byte [] tableName = Bytes.toBytes("testtable");
855     byte [] fam = Bytes.toBytes("info");
856     byte [][] families = {fam};
857     String method = this.getName();
858     initHRegion(tableName, method, families);
859 
860     byte [] row = Bytes.toBytes("row1");
861     // column names
862     byte [] qual = Bytes.toBytes("qual");
863 
864     // add data with LATEST_TIMESTAMP, put without WAL
865     Put put = new Put(row);
866     put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
867     region.put(put, false);
868 
869     // Make sure it shows up with an actual timestamp
870     Get get = new Get(row).addColumn(fam, qual);
871     Result result = region.get(get, null);
872     assertEquals(1, result.size());
873     KeyValue kv = result.raw()[0];
874     LOG.info("Got: " + kv);
875     assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
876         kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
877 
878     // Check same with WAL enabled (historically these took different
879     // code paths, so check both)
880     row = Bytes.toBytes("row2");
881     put = new Put(row);
882     put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
883     region.put(put, true);
884 
885     // Make sure it shows up with an actual timestamp
886     get = new Get(row).addColumn(fam, qual);
887     result = region.get(get, null);
888     assertEquals(1, result.size());
889     kv = result.raw()[0];
890     LOG.info("Got: " + kv);
891     assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
892         kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
893 
894   }
895 
896   public void testScanner_DeleteOneFamilyNotAnother() throws IOException {
897     byte [] tableName = Bytes.toBytes("test_table");
898     byte [] fam1 = Bytes.toBytes("columnA");
899     byte [] fam2 = Bytes.toBytes("columnB");
900     initHRegion(tableName, getName(), fam1, fam2);
901 
902     byte [] rowA = Bytes.toBytes("rowA");
903     byte [] rowB = Bytes.toBytes("rowB");
904 
905     byte [] value = Bytes.toBytes("value");
906 
907     Delete delete = new Delete(rowA);
908     delete.deleteFamily(fam1);
909 
910     region.delete(delete, null, true);
911 
912     // now create data.
913     Put put = new Put(rowA);
914     put.add(fam2, null, value);
915     region.put(put);
916 
917     put = new Put(rowB);
918     put.add(fam1, null, value);
919     put.add(fam2, null, value);
920     region.put(put);
921 
922     Scan scan = new Scan();
923     scan.addFamily(fam1).addFamily(fam2);
924     InternalScanner s = region.getScanner(scan);
925     List<KeyValue> results = new ArrayList<KeyValue>();
926     s.next(results);
927     assertTrue(Bytes.equals(rowA, results.get(0).getRow()));
928 
929     results.clear();
930     s.next(results);
931     assertTrue(Bytes.equals(rowB, results.get(0).getRow()));
932 
933   }
934 
935   public void testDeleteColumns_PostInsert() throws IOException,
936       InterruptedException {
937     Delete delete = new Delete(row);
938     delete.deleteColumns(fam1, qual1);
939     doTestDelete_AndPostInsert(delete);
940   }
941 
942   public void testDeleteFamily_PostInsert() throws IOException, InterruptedException {
943     Delete delete = new Delete(row);
944     delete.deleteFamily(fam1);
945     doTestDelete_AndPostInsert(delete);
946   }
947 
948   public void doTestDelete_AndPostInsert(Delete delete)
949       throws IOException, InterruptedException {
950     initHRegion(tableName, getName(), fam1);
951     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
952     Put put = new Put(row);
953     put.add(fam1, qual1, value1);
954     region.put(put);
955 
956     // now delete the value:
957     region.delete(delete, null, true);
958 
959 
960     // ok put data:
961     put = new Put(row);
962     put.add(fam1, qual1, value2);
963     region.put(put);
964 
965     // ok get:
966     Get get = new Get(row);
967     get.addColumn(fam1, qual1);
968 
969     Result r = region.get(get, null);
970     assertEquals(1, r.size());
971     assertByteEquals(value2, r.getValue(fam1, qual1));
972 
973     // next:
974     Scan scan = new Scan(row);
975     scan.addColumn(fam1, qual1);
976     InternalScanner s = region.getScanner(scan);
977 
978     List<KeyValue> results = new ArrayList<KeyValue>();
979     assertEquals(false, s.next(results));
980     assertEquals(1, results.size());
981     KeyValue kv = results.get(0);
982 
983     assertByteEquals(value2, kv.getValue());
984     assertByteEquals(fam1, kv.getFamily());
985     assertByteEquals(qual1, kv.getQualifier());
986     assertByteEquals(row, kv.getRow());
987   }
988 
989 
990 
991   public void testDelete_CheckTimestampUpdated()
992   throws IOException {
993     byte [] row1 = Bytes.toBytes("row1");
994     byte [] col1 = Bytes.toBytes("col1");
995     byte [] col2 = Bytes.toBytes("col2");
996     byte [] col3 = Bytes.toBytes("col3");
997 
998     //Setting up region
999     String method = this.getName();
1000     initHRegion(tableName, method, fam1);
1001 
1002     //Building checkerList
1003     List<KeyValue> kvs  = new ArrayList<KeyValue>();
1004     kvs.add(new KeyValue(row1, fam1, col1, null));
1005     kvs.add(new KeyValue(row1, fam1, col2, null));
1006     kvs.add(new KeyValue(row1, fam1, col3, null));
1007 
1008     Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
1009     deleteMap.put(fam1, kvs);
1010     region.delete(deleteMap, true);
1011 
1012     // extract the key values out the memstore:
1013     // This is kinda hacky, but better than nothing...
1014     long now = System.currentTimeMillis();
1015     KeyValue firstKv = region.getStore(fam1).memstore.kvset.first();
1016     assertTrue(firstKv.getTimestamp() <= now);
1017     now = firstKv.getTimestamp();
1018     for (KeyValue kv: region.getStore(fam1).memstore.kvset) {
1019       assertTrue(kv.getTimestamp() <= now);
1020       now = kv.getTimestamp();
1021     }
1022   }
1023 
1024   //////////////////////////////////////////////////////////////////////////////
1025   // Get tests
1026   //////////////////////////////////////////////////////////////////////////////
1027   public void testGet_FamilyChecker() throws IOException {
1028     byte [] tableName = Bytes.toBytes("testtable");
1029     byte [] row1 = Bytes.toBytes("row1");
1030     byte [] fam1 = Bytes.toBytes("fam1");
1031     byte [] fam2 = Bytes.toBytes("False");
1032     byte [] col1 = Bytes.toBytes("col1");
1033 
1034     //Setting up region
1035     String method = this.getName();
1036     initHRegion(tableName, method, fam1);
1037 
1038     Get get = new Get(row1);
1039     get.addColumn(fam2, col1);
1040 
1041     //Test
1042     try {
1043       region.get(get, null);
1044     } catch (NoSuchColumnFamilyException e){
1045       assertFalse(false);
1046       return;
1047     }
1048     assertFalse(true);
1049   }
1050 
1051   public void testGet_Basic() throws IOException {
1052     byte [] tableName = Bytes.toBytes("testtable");
1053     byte [] row1 = Bytes.toBytes("row1");
1054     byte [] fam1 = Bytes.toBytes("fam1");
1055     byte [] col1 = Bytes.toBytes("col1");
1056     byte [] col2 = Bytes.toBytes("col2");
1057     byte [] col3 = Bytes.toBytes("col3");
1058     byte [] col4 = Bytes.toBytes("col4");
1059     byte [] col5 = Bytes.toBytes("col5");
1060 
1061     //Setting up region
1062     String method = this.getName();
1063     initHRegion(tableName, method, fam1);
1064 
1065     //Add to memstore
1066     Put put = new Put(row1);
1067     put.add(fam1, col1, null);
1068     put.add(fam1, col2, null);
1069     put.add(fam1, col3, null);
1070     put.add(fam1, col4, null);
1071     put.add(fam1, col5, null);
1072     region.put(put);
1073 
1074     Get get = new Get(row1);
1075     get.addColumn(fam1, col2);
1076     get.addColumn(fam1, col4);
1077     //Expected result
1078     KeyValue kv1 = new KeyValue(row1, fam1, col2);
1079     KeyValue kv2 = new KeyValue(row1, fam1, col4);
1080     KeyValue [] expected = {kv1, kv2};
1081 
1082     //Test
1083     Result res = region.get(get, null);
1084     assertEquals(expected.length, res.size());
1085     for(int i=0; i<res.size(); i++){
1086       assertEquals(0,
1087           Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1088       assertEquals(0,
1089           Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1090       assertEquals(0,
1091           Bytes.compareTo(
1092               expected[i].getQualifier(), res.raw()[i].getQualifier()));
1093     }
1094 
1095     // Test using a filter on a Get
1096     Get g = new Get(row1);
1097     final int count = 2;
1098     g.setFilter(new ColumnCountGetFilter(count));
1099     res = region.get(g, null);
1100     assertEquals(count, res.size());
1101   }
1102 
1103   public void testGet_Empty() throws IOException {
1104     byte [] tableName = Bytes.toBytes("emptytable");
1105     byte [] row = Bytes.toBytes("row");
1106     byte [] fam = Bytes.toBytes("fam");
1107 
1108     String method = this.getName();
1109     initHRegion(tableName, method, fam);
1110 
1111     Get get = new Get(row);
1112     get.addFamily(fam);
1113     Result r = region.get(get, null);
1114 
1115     assertTrue(r.isEmpty());
1116   }
1117 
1118   //Test that checked if there was anything special when reading from the ROOT
1119   //table. To be able to use this test you need to comment the part in
1120   //HTableDescriptor that checks for '-' and '.'. You also need to remove the
1121   //s in the beginning of the name.
1122   public void stestGet_Root() throws IOException {
1123     //Setting up region
1124     String method = this.getName();
1125     initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
1126 
1127     //Add to memstore
1128     Put put = new Put(HConstants.EMPTY_START_ROW);
1129     put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, null);
1130     region.put(put);
1131 
1132     Get get = new Get(HConstants.EMPTY_START_ROW);
1133     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1134 
1135     //Expected result
1136     KeyValue kv1 = new KeyValue(HConstants.EMPTY_START_ROW,
1137         HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1138     KeyValue [] expected = {kv1};
1139 
1140     //Test from memstore
1141     Result res = region.get(get, null);
1142 
1143     assertEquals(expected.length, res.size());
1144     for(int i=0; i<res.size(); i++){
1145       assertEquals(0,
1146           Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1147       assertEquals(0,
1148           Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1149       assertEquals(0,
1150           Bytes.compareTo(
1151               expected[i].getQualifier(), res.raw()[i].getQualifier()));
1152     }
1153 
1154     //flush
1155     region.flushcache();
1156 
1157     //test2
1158     res = region.get(get, null);
1159 
1160     assertEquals(expected.length, res.size());
1161     for(int i=0; i<res.size(); i++){
1162       assertEquals(0,
1163           Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1164       assertEquals(0,
1165           Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1166       assertEquals(0,
1167           Bytes.compareTo(
1168               expected[i].getQualifier(), res.raw()[i].getQualifier()));
1169     }
1170 
1171     //Scan
1172     Scan scan = new Scan();
1173     scan.addFamily(HConstants.CATALOG_FAMILY);
1174     InternalScanner s = region.getScanner(scan);
1175     List<KeyValue> result = new ArrayList<KeyValue>();
1176     s.next(result);
1177 
1178     assertEquals(expected.length, result.size());
1179     for(int i=0; i<res.size(); i++){
1180       assertEquals(0,
1181           Bytes.compareTo(expected[i].getRow(), result.get(i).getRow()));
1182       assertEquals(0,
1183           Bytes.compareTo(expected[i].getFamily(), result.get(i).getFamily()));
1184       assertEquals(0,
1185           Bytes.compareTo(
1186               expected[i].getQualifier(), result.get(i).getQualifier()));
1187     }
1188   }
1189 
1190   //////////////////////////////////////////////////////////////////////////////
1191   // Lock test
1192   //////////////////////////////////////////////////////////////////////////////
1193   public void testLocks() throws IOException{
1194     byte [] tableName = Bytes.toBytes("testtable");
1195     byte [][] families = {fam1, fam2, fam3};
1196 
1197     HBaseConfiguration hc = initSplit();
1198     //Setting up region
1199     String method = this.getName();
1200     initHRegion(tableName, method, hc, families);
1201 
1202     final int threadCount = 10;
1203     final int lockCount = 10;
1204 
1205     List<Thread>threads = new ArrayList<Thread>(threadCount);
1206     for (int i = 0; i < threadCount; i++) {
1207       threads.add(new Thread(Integer.toString(i)) {
1208         @Override
1209         public void run() {
1210           Integer [] lockids = new Integer[lockCount];
1211           // Get locks.
1212           for (int i = 0; i < lockCount; i++) {
1213             try {
1214               byte [] rowid = Bytes.toBytes(Integer.toString(i));
1215               lockids[i] = region.obtainRowLock(rowid);
1216               assertEquals(rowid, region.getRowFromLock(lockids[i]));
1217               LOG.debug(getName() + " locked " + Bytes.toString(rowid));
1218             } catch (IOException e) {
1219               e.printStackTrace();
1220             }
1221           }
1222           LOG.debug(getName() + " set " +
1223               Integer.toString(lockCount) + " locks");
1224 
1225           // Abort outstanding locks.
1226           for (int i = lockCount - 1; i >= 0; i--) {
1227             region.releaseRowLock(lockids[i]);
1228             LOG.debug(getName() + " unlocked " + i);
1229           }
1230           LOG.debug(getName() + " released " +
1231               Integer.toString(lockCount) + " locks");
1232         }
1233       });
1234     }
1235 
1236     // Startup all our threads.
1237     for (Thread t : threads) {
1238       t.start();
1239     }
1240 
1241     // Now wait around till all are done.
1242     for (Thread t: threads) {
1243       while (t.isAlive()) {
1244         try {
1245           Thread.sleep(1);
1246         } catch (InterruptedException e) {
1247           // Go around again.
1248         }
1249       }
1250     }
1251     LOG.info("locks completed.");
1252   }
1253 
1254   //////////////////////////////////////////////////////////////////////////////
1255   // Merge test
1256   //////////////////////////////////////////////////////////////////////////////
1257   public void testMerge() throws IOException {
1258     byte [] tableName = Bytes.toBytes("testtable");
1259     byte [][] families = {fam1, fam2, fam3};
1260     HBaseConfiguration hc = initSplit();
1261     //Setting up region
1262     String method = this.getName();
1263     initHRegion(tableName, method, hc, families);
1264     try {
1265       LOG.info("" + addContent(region, fam3));
1266       region.flushcache();
1267       byte [] splitRow = region.compactStores();
1268       assertNotNull(splitRow);
1269       LOG.info("SplitRow: " + Bytes.toString(splitRow));
1270       HRegion [] subregions = splitRegion(region, splitRow);
1271       try {
1272         // Need to open the regions.
1273         for (int i = 0; i < subregions.length; i++) {
1274           openClosedRegion(subregions[i]);
1275           subregions[i].compactStores();
1276         }
1277         Path oldRegionPath = region.getRegionDir();
1278         Path oldRegion1 = subregions[0].getRegionDir();
1279         Path oldRegion2 = subregions[1].getRegionDir();
1280         long startTime = System.currentTimeMillis();
1281         region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
1282         LOG.info("Merge regions elapsed time: " +
1283             ((System.currentTimeMillis() - startTime) / 1000.0));
1284         fs.delete(oldRegion1, true);
1285         fs.delete(oldRegion2, true);
1286         fs.delete(oldRegionPath, true);
1287         LOG.info("splitAndMerge completed.");
1288       } finally {
1289         for (int i = 0; i < subregions.length; i++) {
1290           try {
1291             subregions[i].close();
1292           } catch (IOException e) {
1293             // Ignore.
1294           }
1295         }
1296       }
1297     } finally {
1298       if (region != null) {
1299         region.close();
1300         region.getLog().closeAndDelete();
1301       }
1302     }
1303   }
1304 
1305   /**
1306    * @param parent Region to split.
1307    * @param midkey Key to split around.
1308    * @return The Regions we created.
1309    * @throws IOException
1310    */
1311   HRegion [] splitRegion(final HRegion parent, final byte [] midkey)
1312   throws IOException {
1313     PairOfSameType<HRegion> result = null;
1314     SplitTransaction st = new SplitTransaction(parent, midkey);
1315     // If prepare does not return true, for some reason -- logged inside in
1316     // the prepare call -- we are not ready to split just now.  Just return.
1317     if (!st.prepare()) return null;
1318     try {
1319       result = st.execute(null);
1320     } catch (IOException ioe) {
1321       try {
1322         LOG.info("Running rollback of failed split of " +
1323           parent.getRegionNameAsString() + "; " + ioe.getMessage());
1324         st.rollback(null);
1325         LOG.info("Successful rollback of failed split of " +
1326           parent.getRegionNameAsString());
1327         return null;
1328       } catch (RuntimeException e) {
1329         // If failed rollback, kill this server to avoid having a hole in table.
1330         LOG.info("Failed rollback of failed split of " +
1331           parent.getRegionNameAsString() + " -- aborting server", e);
1332       }
1333     }
1334     return new HRegion [] {result.getFirst(), result.getSecond()};
1335   }
1336 
1337   //////////////////////////////////////////////////////////////////////////////
1338   // Scanner tests
1339   //////////////////////////////////////////////////////////////////////////////
1340   public void testGetScanner_WithOkFamilies() throws IOException {
1341     byte [] tableName = Bytes.toBytes("testtable");
1342     byte [] fam1 = Bytes.toBytes("fam1");
1343     byte [] fam2 = Bytes.toBytes("fam2");
1344 
1345     byte [][] families = {fam1, fam2};
1346 
1347     //Setting up region
1348     String method = this.getName();
1349     initHRegion(tableName, method, families);
1350 
1351     Scan scan = new Scan();
1352     scan.addFamily(fam1);
1353     scan.addFamily(fam2);
1354     try {
1355       region.getScanner(scan);
1356     } catch (Exception e) {
1357       assertTrue("Families could not be found in Region", false);
1358     }
1359   }
1360 
1361   public void testGetScanner_WithNotOkFamilies() throws IOException {
1362     byte [] tableName = Bytes.toBytes("testtable");
1363     byte [] fam1 = Bytes.toBytes("fam1");
1364     byte [] fam2 = Bytes.toBytes("fam2");
1365 
1366     byte [][] families = {fam1};
1367 
1368     //Setting up region
1369     String method = this.getName();
1370     initHRegion(tableName, method, families);
1371 
1372     Scan scan = new Scan();
1373     scan.addFamily(fam2);
1374     boolean ok = false;
1375     try {
1376       region.getScanner(scan);
1377     } catch (Exception e) {
1378       ok = true;
1379     }
1380     assertTrue("Families could not be found in Region", ok);
1381   }
1382 
1383   public void testGetScanner_WithNoFamilies() throws IOException {
1384     byte [] tableName = Bytes.toBytes("testtable");
1385     byte [] row1 = Bytes.toBytes("row1");
1386     byte [] fam1 = Bytes.toBytes("fam1");
1387     byte [] fam2 = Bytes.toBytes("fam2");
1388     byte [] fam3 = Bytes.toBytes("fam3");
1389     byte [] fam4 = Bytes.toBytes("fam4");
1390 
1391     byte [][] families = {fam1, fam2, fam3, fam4};
1392 
1393     //Setting up region
1394     String method = this.getName();
1395     initHRegion(tableName, method, families);
1396 
1397 
1398     //Putting data in Region
1399     Put put = new Put(row1);
1400     put.add(fam1, null, null);
1401     put.add(fam2, null, null);
1402     put.add(fam3, null, null);
1403     put.add(fam4, null, null);
1404     region.put(put);
1405 
1406     Scan scan = null;
1407     HRegion.RegionScanner is = null;
1408 
1409     //Testing to see how many scanners that is produced by getScanner, starting
1410     //with known number, 2 - current = 1
1411     scan = new Scan();
1412     scan.addFamily(fam2);
1413     scan.addFamily(fam4);
1414     is = (RegionScanner) region.getScanner(scan);
1415     ReadWriteConsistencyControl.resetThreadReadPoint(region.getRWCC());
1416     assertEquals(1, ((RegionScanner)is).storeHeap.getHeap().size());
1417 
1418     scan = new Scan();
1419     is = (RegionScanner) region.getScanner(scan);
1420     ReadWriteConsistencyControl.resetThreadReadPoint(region.getRWCC());
1421     assertEquals(families.length -1,
1422         ((RegionScanner)is).storeHeap.getHeap().size());
1423   }
1424 
1425   /**
1426    * This method tests https://issues.apache.org/jira/browse/HBASE-2516.
1427    */
1428   public void testGetScanner_WithRegionClosed() {
1429     byte[] tableName = Bytes.toBytes("testtable");
1430     byte[] fam1 = Bytes.toBytes("fam1");
1431     byte[] fam2 = Bytes.toBytes("fam2");
1432 
1433     byte[][] families = {fam1, fam2};
1434 
1435     //Setting up region
1436     String method = this.getName();
1437     try {
1438       initHRegion(tableName, method, families);
1439     } catch (IOException e) {
1440       e.printStackTrace();
1441       fail("Got IOException during initHRegion, " + e.getMessage());
1442     }
1443     region.closed.set(true);
1444     try {
1445       region.getScanner(null);
1446       fail("Expected to get an exception during getScanner on a region that is closed");
1447     } catch (org.apache.hadoop.hbase.NotServingRegionException e) {
1448       //this is the correct exception that is expected
1449     } catch (IOException e) {
1450       fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
1451               + e.getMessage());
1452     }
1453   }
1454 
1455   public void testRegionScanner_Next() throws IOException {
1456     byte [] tableName = Bytes.toBytes("testtable");
1457     byte [] row1 = Bytes.toBytes("row1");
1458     byte [] row2 = Bytes.toBytes("row2");
1459     byte [] fam1 = Bytes.toBytes("fam1");
1460     byte [] fam2 = Bytes.toBytes("fam2");
1461     byte [] fam3 = Bytes.toBytes("fam3");
1462     byte [] fam4 = Bytes.toBytes("fam4");
1463 
1464     byte [][] families = {fam1, fam2, fam3, fam4};
1465     long ts = System.currentTimeMillis();
1466 
1467     //Setting up region
1468     String method = this.getName();
1469     initHRegion(tableName, method, families);
1470 
1471     //Putting data in Region
1472     Put put = null;
1473     put = new Put(row1);
1474     put.add(fam1, null, ts, null);
1475     put.add(fam2, null, ts, null);
1476     put.add(fam3, null, ts, null);
1477     put.add(fam4, null, ts, null);
1478     region.put(put);
1479 
1480     put = new Put(row2);
1481     put.add(fam1, null, ts, null);
1482     put.add(fam2, null, ts, null);
1483     put.add(fam3, null, ts, null);
1484     put.add(fam4, null, ts, null);
1485     region.put(put);
1486 
1487     Scan scan = new Scan();
1488     scan.addFamily(fam2);
1489     scan.addFamily(fam4);
1490     InternalScanner is = region.getScanner(scan);
1491 
1492     List<KeyValue> res = null;
1493 
1494     //Result 1
1495     List<KeyValue> expected1 = new ArrayList<KeyValue>();
1496     expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
1497     expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
1498 
1499     res = new ArrayList<KeyValue>();
1500     is.next(res);
1501     for(int i=0; i<res.size(); i++) {
1502       assertEquals(expected1.get(i), res.get(i));
1503     }
1504 
1505     //Result 2
1506     List<KeyValue> expected2 = new ArrayList<KeyValue>();
1507     expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
1508     expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
1509 
1510     res = new ArrayList<KeyValue>();
1511     is.next(res);
1512     for(int i=0; i<res.size(); i++) {
1513       assertEquals(expected2.get(i), res.get(i));
1514     }
1515 
1516   }
1517 
1518   public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions()
1519   throws IOException {
1520     byte [] tableName = Bytes.toBytes("testtable");
1521     byte [] row1 = Bytes.toBytes("row1");
1522     byte [] qf1 = Bytes.toBytes("qualifier1");
1523     byte [] qf2 = Bytes.toBytes("qualifier2");
1524     byte [] fam1 = Bytes.toBytes("fam1");
1525     byte [][] families = {fam1};
1526 
1527     long ts1 = System.currentTimeMillis();
1528     long ts2 = ts1 + 1;
1529     long ts3 = ts1 + 2;
1530 
1531     //Setting up region
1532     String method = this.getName();
1533     initHRegion(tableName, method, families);
1534 
1535     //Putting data in Region
1536     Put put = null;
1537     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1538     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1539     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1540 
1541     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1542     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1543     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1544 
1545     put = new Put(row1);
1546     put.add(kv13);
1547     put.add(kv12);
1548     put.add(kv11);
1549     put.add(kv23);
1550     put.add(kv22);
1551     put.add(kv21);
1552     region.put(put);
1553 
1554     //Expected
1555     List<KeyValue> expected = new ArrayList<KeyValue>();
1556     expected.add(kv13);
1557     expected.add(kv12);
1558 
1559     Scan scan = new Scan(row1);
1560     scan.addColumn(fam1, qf1);
1561     scan.setMaxVersions(MAX_VERSIONS);
1562     List<KeyValue> actual = new ArrayList<KeyValue>();
1563     InternalScanner scanner = region.getScanner(scan);
1564 
1565     boolean hasNext = scanner.next(actual);
1566     assertEquals(false, hasNext);
1567 
1568     //Verify result
1569     for(int i=0; i<expected.size(); i++) {
1570       assertEquals(expected.get(i), actual.get(i));
1571     }
1572   }
1573 
1574   public void testScanner_ExplicitColumns_FromFilesOnly_EnforceVersions()
1575   throws IOException{
1576     byte [] tableName = Bytes.toBytes("testtable");
1577     byte [] row1 = Bytes.toBytes("row1");
1578     byte [] qf1 = Bytes.toBytes("qualifier1");
1579     byte [] qf2 = Bytes.toBytes("qualifier2");
1580     byte [] fam1 = Bytes.toBytes("fam1");
1581     byte [][] families = {fam1};
1582 
1583     long ts1 = 1; //System.currentTimeMillis();
1584     long ts2 = ts1 + 1;
1585     long ts3 = ts1 + 2;
1586 
1587     //Setting up region
1588     String method = this.getName();
1589     initHRegion(tableName, method, families);
1590 
1591     //Putting data in Region
1592     Put put = null;
1593     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1594     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1595     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1596 
1597     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1598     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1599     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1600 
1601     put = new Put(row1);
1602     put.add(kv13);
1603     put.add(kv12);
1604     put.add(kv11);
1605     put.add(kv23);
1606     put.add(kv22);
1607     put.add(kv21);
1608     region.put(put);
1609     region.flushcache();
1610 
1611     //Expected
1612     List<KeyValue> expected = new ArrayList<KeyValue>();
1613     expected.add(kv13);
1614     expected.add(kv12);
1615     expected.add(kv23);
1616     expected.add(kv22);
1617 
1618     Scan scan = new Scan(row1);
1619     scan.addColumn(fam1, qf1);
1620     scan.addColumn(fam1, qf2);
1621     scan.setMaxVersions(MAX_VERSIONS);
1622     List<KeyValue> actual = new ArrayList<KeyValue>();
1623     InternalScanner scanner = region.getScanner(scan);
1624 
1625     boolean hasNext = scanner.next(actual);
1626     assertEquals(false, hasNext);
1627 
1628     //Verify result
1629     for(int i=0; i<expected.size(); i++) {
1630       assertEquals(expected.get(i), actual.get(i));
1631     }
1632   }
1633 
1634   public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions()
1635   throws IOException {
1636     byte [] tableName = Bytes.toBytes("testtable");
1637     byte [] row1 = Bytes.toBytes("row1");
1638     byte [] fam1 = Bytes.toBytes("fam1");
1639     byte [][] families = {fam1};
1640     byte [] qf1 = Bytes.toBytes("qualifier1");
1641     byte [] qf2 = Bytes.toBytes("qualifier2");
1642 
1643     long ts1 = 1;
1644     long ts2 = ts1 + 1;
1645     long ts3 = ts1 + 2;
1646     long ts4 = ts1 + 3;
1647 
1648     //Setting up region
1649     String method = this.getName();
1650     initHRegion(tableName, method, families);
1651 
1652     //Putting data in Region
1653     KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
1654     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1655     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1656     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1657 
1658     KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
1659     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1660     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1661     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1662 
1663     Put put = null;
1664     put = new Put(row1);
1665     put.add(kv14);
1666     put.add(kv24);
1667     region.put(put);
1668     region.flushcache();
1669 
1670     put = new Put(row1);
1671     put.add(kv23);
1672     put.add(kv13);
1673     region.put(put);
1674     region.flushcache();
1675 
1676     put = new Put(row1);
1677     put.add(kv22);
1678     put.add(kv12);
1679     region.put(put);
1680     region.flushcache();
1681 
1682     put = new Put(row1);
1683     put.add(kv21);
1684     put.add(kv11);
1685     region.put(put);
1686 
1687     //Expected
1688     List<KeyValue> expected = new ArrayList<KeyValue>();
1689     expected.add(kv14);
1690     expected.add(kv13);
1691     expected.add(kv12);
1692     expected.add(kv24);
1693     expected.add(kv23);
1694     expected.add(kv22);
1695 
1696     Scan scan = new Scan(row1);
1697     scan.addColumn(fam1, qf1);
1698     scan.addColumn(fam1, qf2);
1699     int versions = 3;
1700     scan.setMaxVersions(versions);
1701     List<KeyValue> actual = new ArrayList<KeyValue>();
1702     InternalScanner scanner = region.getScanner(scan);
1703 
1704     boolean hasNext = scanner.next(actual);
1705     assertEquals(false, hasNext);
1706 
1707     //Verify result
1708     for(int i=0; i<expected.size(); i++) {
1709       assertEquals(expected.get(i), actual.get(i));
1710     }
1711   }
1712 
1713   public void testScanner_Wildcard_FromMemStore_EnforceVersions()
1714   throws IOException {
1715     byte [] tableName = Bytes.toBytes("testtable");
1716     byte [] row1 = Bytes.toBytes("row1");
1717     byte [] qf1 = Bytes.toBytes("qualifier1");
1718     byte [] qf2 = Bytes.toBytes("qualifier2");
1719     byte [] fam1 = Bytes.toBytes("fam1");
1720     byte [][] families = {fam1};
1721 
1722     long ts1 = System.currentTimeMillis();
1723     long ts2 = ts1 + 1;
1724     long ts3 = ts1 + 2;
1725 
1726     //Setting up region
1727     String method = this.getName();
1728     initHRegion(tableName, method, families);
1729 
1730     //Putting data in Region
1731     Put put = null;
1732     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1733     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1734     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1735 
1736     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1737     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1738     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1739 
1740     put = new Put(row1);
1741     put.add(kv13);
1742     put.add(kv12);
1743     put.add(kv11);
1744     put.add(kv23);
1745     put.add(kv22);
1746     put.add(kv21);
1747     region.put(put);
1748 
1749     //Expected
1750     List<KeyValue> expected = new ArrayList<KeyValue>();
1751     expected.add(kv13);
1752     expected.add(kv12);
1753     expected.add(kv23);
1754     expected.add(kv22);
1755 
1756     Scan scan = new Scan(row1);
1757     scan.addFamily(fam1);
1758     scan.setMaxVersions(MAX_VERSIONS);
1759     List<KeyValue> actual = new ArrayList<KeyValue>();
1760     InternalScanner scanner = region.getScanner(scan);
1761 
1762     boolean hasNext = scanner.next(actual);
1763     assertEquals(false, hasNext);
1764 
1765     //Verify result
1766     for(int i=0; i<expected.size(); i++) {
1767       assertEquals(expected.get(i), actual.get(i));
1768     }
1769   }
1770 
1771   public void testScanner_Wildcard_FromFilesOnly_EnforceVersions()
1772   throws IOException{
1773     byte [] tableName = Bytes.toBytes("testtable");
1774     byte [] row1 = Bytes.toBytes("row1");
1775     byte [] qf1 = Bytes.toBytes("qualifier1");
1776     byte [] qf2 = Bytes.toBytes("qualifier2");
1777     byte [] fam1 = Bytes.toBytes("fam1");
1778 
1779     long ts1 = 1; //System.currentTimeMillis();
1780     long ts2 = ts1 + 1;
1781     long ts3 = ts1 + 2;
1782 
1783     //Setting up region
1784     String method = this.getName();
1785     initHRegion(tableName, method, fam1);
1786 
1787     //Putting data in Region
1788     Put put = null;
1789     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1790     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1791     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1792 
1793     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1794     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1795     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1796 
1797     put = new Put(row1);
1798     put.add(kv13);
1799     put.add(kv12);
1800     put.add(kv11);
1801     put.add(kv23);
1802     put.add(kv22);
1803     put.add(kv21);
1804     region.put(put);
1805     region.flushcache();
1806 
1807     //Expected
1808     List<KeyValue> expected = new ArrayList<KeyValue>();
1809     expected.add(kv13);
1810     expected.add(kv12);
1811     expected.add(kv23);
1812     expected.add(kv22);
1813 
1814     Scan scan = new Scan(row1);
1815     scan.addFamily(fam1);
1816     scan.setMaxVersions(MAX_VERSIONS);
1817     List<KeyValue> actual = new ArrayList<KeyValue>();
1818     InternalScanner scanner = region.getScanner(scan);
1819 
1820     boolean hasNext = scanner.next(actual);
1821     assertEquals(false, hasNext);
1822 
1823     //Verify result
1824     for(int i=0; i<expected.size(); i++) {
1825       assertEquals(expected.get(i), actual.get(i));
1826     }
1827   }
1828 
1829   public void testScanner_StopRow1542() throws IOException {
1830     byte [] tableName = Bytes.toBytes("test_table");
1831     byte [] family = Bytes.toBytes("testFamily");
1832     initHRegion(tableName, getName(), family);
1833 
1834     byte [] row1 = Bytes.toBytes("row111");
1835     byte [] row2 = Bytes.toBytes("row222");
1836     byte [] row3 = Bytes.toBytes("row333");
1837     byte [] row4 = Bytes.toBytes("row444");
1838     byte [] row5 = Bytes.toBytes("row555");
1839 
1840     byte [] col1 = Bytes.toBytes("Pub111");
1841     byte [] col2 = Bytes.toBytes("Pub222");
1842 
1843 
1844     Put put = new Put(row1);
1845     put.add(family, col1, Bytes.toBytes(10L));
1846     region.put(put);
1847 
1848     put = new Put(row2);
1849     put.add(family, col1, Bytes.toBytes(15L));
1850     region.put(put);
1851 
1852     put = new Put(row3);
1853     put.add(family, col2, Bytes.toBytes(20L));
1854     region.put(put);
1855 
1856     put = new Put(row4);
1857     put.add(family, col2, Bytes.toBytes(30L));
1858     region.put(put);
1859 
1860     put = new Put(row5);
1861     put.add(family, col1, Bytes.toBytes(40L));
1862     region.put(put);
1863 
1864     Scan scan = new Scan(row3, row4);
1865     scan.setMaxVersions();
1866     scan.addColumn(family, col1);
1867     InternalScanner s = region.getScanner(scan);
1868 
1869     List<KeyValue> results = new ArrayList<KeyValue>();
1870     assertEquals(false, s.next(results));
1871     assertEquals(0, results.size());
1872   }
1873 
1874   public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
1875     initHRegion(tableName, getName(), fam1);
1876 
1877     long value = 1L;
1878     long amount = 3L;
1879 
1880     Put put = new Put(row);
1881     put.add(fam1, qual1, Bytes.toBytes(value));
1882     region.put(put);
1883 
1884     long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
1885 
1886     assertEquals(value+amount, result);
1887 
1888     Store store = region.getStore(fam1);
1889     // ICV removes any extra values floating around in there.
1890     assertEquals(1, store.memstore.kvset.size());
1891     assertTrue(store.memstore.snapshot.isEmpty());
1892 
1893     assertICV(row, fam1, qual1, value+amount);
1894   }
1895 
1896   public void testIncrementColumnValue_BumpSnapshot() throws IOException {
1897     ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
1898     EnvironmentEdgeManagerTestHelper.injectEdge(mee);
1899     initHRegion(tableName, getName(), fam1);
1900 
1901     long value = 42L;
1902     long incr = 44L;
1903 
1904     // first put something in kvset, then snapshot it.
1905     Put put = new Put(row);
1906     put.add(fam1, qual1, Bytes.toBytes(value));
1907     region.put(put);
1908 
1909     // get the store in question:
1910     Store s = region.getStore(fam1);
1911     s.snapshot(); //bam
1912 
1913     // now increment:
1914     long newVal = region.incrementColumnValue(row, fam1, qual1,
1915         incr, false);
1916 
1917     assertEquals(value+incr, newVal);
1918 
1919     // get both versions:
1920     Get get = new Get(row);
1921     get.setMaxVersions();
1922     get.addColumn(fam1,qual1);
1923 
1924     Result r = region.get(get, null);
1925     assertEquals(2, r.size());
1926     KeyValue first = r.raw()[0];
1927     KeyValue second = r.raw()[1];
1928 
1929     assertTrue("ICV failed to upgrade timestamp",
1930         first.getTimestamp() != second.getTimestamp());
1931   }
1932 
1933   public void testIncrementColumnValue_ConcurrentFlush() throws IOException {
1934     initHRegion(tableName, getName(), fam1);
1935 
1936     long value = 1L;
1937     long amount = 3L;
1938 
1939     Put put = new Put(row);
1940     put.add(fam1, qual1, Bytes.toBytes(value));
1941     region.put(put);
1942 
1943     // now increment during a flush
1944     Thread t = new Thread() {
1945       public void run() {
1946         try {
1947           region.flushcache();
1948         } catch (IOException e) {
1949           LOG.info("test ICV, got IOE during flushcache()");
1950         }
1951       }
1952     };
1953     t.start();
1954     long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
1955     assertEquals(value+amount, r);
1956 
1957     // this also asserts there is only 1 KeyValue in the set.
1958     assertICV(row, fam1, qual1, value+amount);
1959   }
1960 
1961   public void testIncrementColumnValue_UpdatingInPlace_Negative()
1962     throws IOException {
1963     initHRegion(tableName, getName(), fam1);
1964 
1965     long value = 3L;
1966     long amount = -1L;
1967 
1968     Put put = new Put(row);
1969     put.add(fam1, qual1, Bytes.toBytes(value));
1970     region.put(put);
1971 
1972     long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
1973     assertEquals(value+amount, result);
1974 
1975     assertICV(row, fam1, qual1, value+amount);
1976   }
1977 
1978   public void testIncrementColumnValue_AddingNew()
1979     throws IOException {
1980     initHRegion(tableName, getName(), fam1);
1981 
1982     long value = 1L;
1983     long amount = 3L;
1984 
1985     Put put = new Put(row);
1986     put.add(fam1, qual1, Bytes.toBytes(value));
1987     put.add(fam1, qual2, Bytes.toBytes(value));
1988     region.put(put);
1989 
1990     long result = region.incrementColumnValue(row, fam1, qual3, amount, true);
1991     assertEquals(amount, result);
1992 
1993     Get get = new Get(row);
1994     get.addColumn(fam1, qual3);
1995     Result rr = region.get(get, null);
1996     assertEquals(1, rr.size());
1997 
1998     // ensure none of the other cols were incremented.
1999     assertICV(row, fam1, qual1, value);
2000     assertICV(row, fam1, qual2, value);
2001     assertICV(row, fam1, qual3, amount);
2002   }
2003 
2004   public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
2005     initHRegion(tableName, getName(), fam1);
2006 
2007     long value = 1L;
2008     long amount = 3L;
2009 
2010     Put put = new Put(row);
2011     put.add(fam1, qual1, Bytes.toBytes(value));
2012     put.add(fam1, qual2, Bytes.toBytes(value));
2013     region.put(put);
2014 
2015     // flush to disk.
2016     region.flushcache();
2017 
2018     Store store = region.getStore(fam1);
2019     assertEquals(0, store.memstore.kvset.size());
2020 
2021     long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
2022     assertEquals(value+amount, r);
2023 
2024     assertICV(row, fam1, qual1, value+amount);
2025   }
2026 
2027   public void testIncrementColumnValue_AddingNewAfterSFCheck()
2028     throws IOException {
2029     initHRegion(tableName, getName(), fam1);
2030 
2031     long value = 1L;
2032     long amount = 3L;
2033 
2034     Put put = new Put(row);
2035     put.add(fam1, qual1, Bytes.toBytes(value));
2036     put.add(fam1, qual2, Bytes.toBytes(value));
2037     region.put(put);
2038     region.flushcache();
2039 
2040     Store store = region.getStore(fam1);
2041     assertEquals(0, store.memstore.kvset.size());
2042 
2043     long r = region.incrementColumnValue(row, fam1, qual3, amount, true);
2044     assertEquals(amount, r);
2045 
2046     assertICV(row, fam1, qual3, amount);
2047 
2048     region.flushcache();
2049 
2050     // ensure that this gets to disk.
2051     assertICV(row, fam1, qual3, amount);
2052   }
2053 
2054   private void assertICV(byte [] row,
2055                          byte [] familiy,
2056                          byte[] qualifier,
2057                          long amount) throws IOException {
2058     // run a get and see?
2059     Get get = new Get(row);
2060     get.addColumn(familiy, qualifier);
2061     Result result = region.get(get, null);
2062     assertEquals(1, result.size());
2063 
2064     KeyValue kv = result.raw()[0];
2065     long r = Bytes.toLong(kv.getValue());
2066     assertEquals(amount, r);
2067   }
2068 
2069 
2070 
2071   public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions()
2072   throws IOException {
2073     byte [] tableName = Bytes.toBytes("testtable");
2074     byte [] row1 = Bytes.toBytes("row1");
2075     byte [] fam1 = Bytes.toBytes("fam1");
2076     byte [] qf1 = Bytes.toBytes("qualifier1");
2077     byte [] qf2 = Bytes.toBytes("quateslifier2");
2078 
2079     long ts1 = 1;
2080     long ts2 = ts1 + 1;
2081     long ts3 = ts1 + 2;
2082     long ts4 = ts1 + 3;
2083 
2084     //Setting up region
2085     String method = this.getName();
2086     initHRegion(tableName, method, fam1);
2087 
2088     //Putting data in Region
2089     KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2090     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2091     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2092     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2093 
2094     KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2095     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2096     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2097     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2098 
2099     Put put = null;
2100     put = new Put(row1);
2101     put.add(kv14);
2102     put.add(kv24);
2103     region.put(put);
2104     region.flushcache();
2105 
2106     put = new Put(row1);
2107     put.add(kv23);
2108     put.add(kv13);
2109     region.put(put);
2110     region.flushcache();
2111 
2112     put = new Put(row1);
2113     put.add(kv22);
2114     put.add(kv12);
2115     region.put(put);
2116     region.flushcache();
2117 
2118     put = new Put(row1);
2119     put.add(kv21);
2120     put.add(kv11);
2121     region.put(put);
2122 
2123     //Expected
2124     List<KeyValue> expected = new ArrayList<KeyValue>();
2125     expected.add(kv14);
2126     expected.add(kv13);
2127     expected.add(kv12);
2128     expected.add(kv24);
2129     expected.add(kv23);
2130     expected.add(kv22);
2131 
2132     Scan scan = new Scan(row1);
2133     int versions = 3;
2134     scan.setMaxVersions(versions);
2135     List<KeyValue> actual = new ArrayList<KeyValue>();
2136     InternalScanner scanner = region.getScanner(scan);
2137 
2138     boolean hasNext = scanner.next(actual);
2139     assertEquals(false, hasNext);
2140 
2141     //Verify result
2142     for(int i=0; i<expected.size(); i++) {
2143       assertEquals(expected.get(i), actual.get(i));
2144     }
2145   }
2146 
2147   //////////////////////////////////////////////////////////////////////////////
2148   // Split test
2149   //////////////////////////////////////////////////////////////////////////////
2150   /**
2151    * Splits twice and verifies getting from each of the split regions.
2152    * @throws Exception
2153    */
2154   public void testBasicSplit() throws Exception {
2155     byte [] tableName = Bytes.toBytes("testtable");
2156     byte [][] families = {fam1, fam2, fam3};
2157 
2158     HBaseConfiguration hc = initSplit();
2159     //Setting up region
2160     String method = this.getName();
2161     initHRegion(tableName, method, hc, families);
2162 
2163     try {
2164       LOG.info("" + addContent(region, fam3));
2165       region.flushcache();
2166       byte [] splitRow = region.compactStores();
2167       assertNotNull(splitRow);
2168       LOG.info("SplitRow: " + Bytes.toString(splitRow));
2169       HRegion [] regions = splitRegion(region, splitRow);
2170       try {
2171         // Need to open the regions.
2172         // TODO: Add an 'open' to HRegion... don't do open by constructing
2173         // instance.
2174         for (int i = 0; i < regions.length; i++) {
2175           regions[i] = openClosedRegion(regions[i]);
2176         }
2177         // Assert can get rows out of new regions. Should be able to get first
2178         // row from first region and the midkey from second region.
2179         assertGet(regions[0], fam3, Bytes.toBytes(START_KEY));
2180         assertGet(regions[1], fam3, splitRow);
2181         // Test I can get scanner and that it starts at right place.
2182         assertScan(regions[0], fam3,
2183             Bytes.toBytes(START_KEY));
2184         assertScan(regions[1], fam3, splitRow);
2185         // Now prove can't split regions that have references.
2186         for (int i = 0; i < regions.length; i++) {
2187           // Add so much data to this region, we create a store file that is >
2188           // than one of our unsplitable references. it will.
2189           for (int j = 0; j < 2; j++) {
2190             addContent(regions[i], fam3);
2191           }
2192           addContent(regions[i], fam2);
2193           addContent(regions[i], fam1);
2194           regions[i].flushcache();
2195         }
2196 
2197         byte [][] midkeys = new byte [regions.length][];
2198         // To make regions splitable force compaction.
2199         for (int i = 0; i < regions.length; i++) {
2200           midkeys[i] = regions[i].compactStores();
2201         }
2202 
2203         TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
2204         // Split these two daughter regions so then I'll have 4 regions. Will
2205         // split because added data above.
2206         for (int i = 0; i < regions.length; i++) {
2207           HRegion[] rs = null;
2208           if (midkeys[i] != null) {
2209             rs = splitRegion(regions[i], midkeys[i]);
2210             for (int j = 0; j < rs.length; j++) {
2211               sortedMap.put(Bytes.toString(rs[j].getRegionName()),
2212                 openClosedRegion(rs[j]));
2213             }
2214           }
2215         }
2216         LOG.info("Made 4 regions");
2217         // The splits should have been even. Test I can get some arbitrary row
2218         // out of each.
2219         int interval = (LAST_CHAR - FIRST_CHAR) / 3;
2220         byte[] b = Bytes.toBytes(START_KEY);
2221         for (HRegion r : sortedMap.values()) {
2222           assertGet(r, fam3, b);
2223           b[0] += interval;
2224         }
2225       } finally {
2226         for (int i = 0; i < regions.length; i++) {
2227           try {
2228             regions[i].close();
2229           } catch (IOException e) {
2230             // Ignore.
2231           }
2232         }
2233       }
2234     } finally {
2235       if (region != null) {
2236         region.close();
2237         region.getLog().closeAndDelete();
2238       }
2239     }
2240   }
2241 
2242   public void testSplitRegion() throws IOException {
2243     byte [] tableName = Bytes.toBytes("testtable");
2244     byte [] qualifier = Bytes.toBytes("qualifier");
2245     HBaseConfiguration hc = initSplit();
2246     int numRows = 10;
2247     byte [][] families = {fam1, fam3};
2248 
2249     //Setting up region
2250     String method = this.getName();
2251     initHRegion(tableName, method, hc, families);
2252 
2253     //Put data in region
2254     int startRow = 100;
2255     putData(startRow, numRows, qualifier, families);
2256     int splitRow = startRow + numRows;
2257     putData(splitRow, numRows, qualifier, families);
2258     region.flushcache();
2259 
2260     HRegion [] regions = null;
2261     try {
2262       regions = splitRegion(region, Bytes.toBytes("" + splitRow));
2263       //Opening the regions returned.
2264       for (int i = 0; i < regions.length; i++) {
2265         regions[i] = openClosedRegion(regions[i]);
2266       }
2267       //Verifying that the region has been split
2268       assertEquals(2, regions.length);
2269 
2270       //Verifying that all data is still there and that data is in the right
2271       //place
2272       verifyData(regions[0], startRow, numRows, qualifier, families);
2273       verifyData(regions[1], splitRow, numRows, qualifier, families);
2274 
2275     } finally {
2276       if (region != null) {
2277         region.close();
2278         region.getLog().closeAndDelete();
2279       }
2280     }
2281   }
2282 
2283 
2284   /**
2285    * Flushes the cache in a thread while scanning. The tests verify that the
2286    * scan is coherent - e.g. the returned results are always of the same or
2287    * later update as the previous results.
2288    * @throws IOException scan / compact
2289    * @throws InterruptedException thread join
2290    */
2291   public void testFlushCacheWhileScanning() throws IOException, InterruptedException {
2292     byte[] tableName = Bytes.toBytes("testFlushCacheWhileScanning");
2293     byte[] family = Bytes.toBytes("family");
2294     int numRows = 1000;
2295     int flushAndScanInterval = 10;
2296     int compactInterval = 10 * flushAndScanInterval;
2297 
2298     String method = "testFlushCacheWhileScanning";
2299     initHRegion(tableName,method, family);
2300     FlushThread flushThread = new FlushThread();
2301     flushThread.start();
2302 
2303     Scan scan = new Scan();
2304     scan.addFamily(family);
2305     scan.setFilter(new SingleColumnValueFilter(family, qual1,
2306       CompareFilter.CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes(5L))));
2307 
2308     int expectedCount = 0;
2309     List<KeyValue> res = new ArrayList<KeyValue>();
2310 
2311     boolean toggle=true;
2312     for (long i = 0; i < numRows; i++) {
2313       Put put = new Put(Bytes.toBytes(i));
2314       put.add(family, qual1, Bytes.toBytes(i % 10));
2315       region.put(put);
2316 
2317       if (i != 0 && i % compactInterval == 0) {
2318         //System.out.println("iteration = " + i);
2319         region.compactStores(true);
2320       }
2321 
2322       if (i % 10 == 5L) {
2323         expectedCount++;
2324       }
2325 
2326       if (i != 0 && i % flushAndScanInterval == 0) {
2327         res.clear();
2328         InternalScanner scanner = region.getScanner(scan);
2329         if (toggle) {
2330           flushThread.flush();
2331         }
2332         while (scanner.next(res)) ;
2333         if (!toggle) {
2334           flushThread.flush();
2335         }
2336         assertEquals("i=" + i, expectedCount, res.size());
2337         toggle = !toggle;
2338       }
2339     }
2340 
2341     flushThread.done();
2342     flushThread.join();
2343     flushThread.checkNoError();
2344   }
2345 
2346   protected class FlushThread extends Thread {
2347     private volatile boolean done;
2348     private Throwable error = null;
2349 
2350     public void done() {
2351       done = true;
2352       synchronized (this) {
2353         interrupt();
2354       }
2355     }
2356 
2357     public void checkNoError() {
2358       if (error != null) {
2359         assertNull(error);
2360       }
2361     }
2362 
2363     @Override
2364     public void run() {
2365       done = false;
2366       while (!done) {
2367         synchronized (this) {
2368           try {
2369             wait();
2370           } catch (InterruptedException ignored) {
2371             if (done) {
2372               break;
2373             }
2374           }
2375         }
2376         try {
2377           region.flushcache();
2378         } catch (IOException e) {
2379           if (!done) {
2380             LOG.error("Error while flusing cache", e);
2381             error = e;
2382           }
2383           break;
2384         }
2385       }
2386 
2387     }
2388 
2389     public void flush() {
2390       synchronized (this) {
2391         notify();
2392       }
2393 
2394     }
2395   }
2396 
2397   /**
2398    * Writes very wide records and scans for the latest every time..
2399    * Flushes and compacts the region every now and then to keep things
2400    * realistic.
2401    *
2402    * @throws IOException          by flush / scan / compaction
2403    * @throws InterruptedException when joining threads
2404    */
2405   public void testWritesWhileScanning()
2406     throws IOException, InterruptedException {
2407     byte[] tableName = Bytes.toBytes("testWritesWhileScanning");
2408     int testCount = 100;
2409     int numRows = 1;
2410     int numFamilies = 10;
2411     int numQualifiers = 100;
2412     int flushInterval = 7;
2413     int compactInterval = 5 * flushInterval;
2414     byte[][] families = new byte[numFamilies][];
2415     for (int i = 0; i < numFamilies; i++) {
2416       families[i] = Bytes.toBytes("family" + i);
2417     }
2418     byte[][] qualifiers = new byte[numQualifiers][];
2419     for (int i = 0; i < numQualifiers; i++) {
2420       qualifiers[i] = Bytes.toBytes("qual" + i);
2421     }
2422 
2423     String method = "testWritesWhileScanning";
2424     initHRegion(tableName, method, families);
2425     PutThread putThread = new PutThread(numRows, families, qualifiers);
2426     putThread.start();
2427     putThread.waitForFirstPut();
2428 
2429     FlushThread flushThread = new FlushThread();
2430     flushThread.start();
2431 
2432     Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
2433 //    scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
2434 //      new BinaryComparator(Bytes.toBytes("row0"))));
2435 
2436     int expectedCount = numFamilies * numQualifiers;
2437     List<KeyValue> res = new ArrayList<KeyValue>();
2438 
2439     long prevTimestamp = 0L;
2440     for (int i = 0; i < testCount; i++) {
2441 
2442       if (i != 0 && i % compactInterval == 0) {
2443         region.compactStores(true);
2444       }
2445 
2446       if (i != 0 && i % flushInterval == 0) {
2447         //System.out.println("flush scan iteration = " + i);
2448         flushThread.flush();
2449       }
2450 
2451       boolean previousEmpty = res.isEmpty();
2452       res.clear();
2453       InternalScanner scanner = region.getScanner(scan);
2454       while (scanner.next(res)) ;
2455       if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
2456         assertEquals("i=" + i, expectedCount, res.size());
2457         long timestamp = res.get(0).getTimestamp();
2458         assertTrue("Timestamps were broke: " + timestamp + " prev: " + prevTimestamp,
2459             timestamp >= prevTimestamp);
2460         prevTimestamp = timestamp;
2461       }
2462     }
2463 
2464     putThread.done();
2465 
2466     region.flushcache();
2467 
2468     putThread.join();
2469     putThread.checkNoError();
2470 
2471     flushThread.done();
2472     flushThread.join();
2473     flushThread.checkNoError();
2474   }
2475 
2476   protected class PutThread extends Thread {
2477     private volatile boolean done;
2478     private volatile int numPutsFinished = 0;
2479 
2480     private Throwable error = null;
2481     private int numRows;
2482     private byte[][] families;
2483     private byte[][] qualifiers;
2484 
2485     private PutThread(int numRows, byte[][] families,
2486       byte[][] qualifiers) {
2487       this.numRows = numRows;
2488       this.families = families;
2489       this.qualifiers = qualifiers;
2490     }
2491 
2492     /**
2493      * Block until this thread has put at least one row.
2494      */
2495     public void waitForFirstPut() throws InterruptedException {
2496       // wait until put thread actually puts some data
2497       while (numPutsFinished == 0) {
2498         checkNoError();
2499         Thread.sleep(50);
2500       }
2501     }
2502 
2503     public void done() {
2504       done = true;
2505       synchronized (this) {
2506         interrupt();
2507       }
2508     }
2509 
2510     public void checkNoError() {
2511       if (error != null) {
2512         assertNull(error);
2513       }
2514     }
2515 
2516     @Override
2517     public void run() {
2518       done = false;
2519       while (!done) {
2520         try {
2521           for (int r = 0; r < numRows; r++) {
2522             byte[] row = Bytes.toBytes("row" + r);
2523             Put put = new Put(row);
2524             for (byte[] family : families) {
2525               for (byte[] qualifier : qualifiers) {
2526                 put.add(family, qualifier, (long) numPutsFinished,
2527                     Bytes.toBytes(numPutsFinished));
2528               }
2529             }
2530 //            System.out.println("Putting of kvsetsize=" + put.size());
2531             region.put(put);
2532             numPutsFinished++;
2533             if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
2534               System.out.println("put iteration = " + numPutsFinished);
2535               Delete delete = new Delete(row, (long)numPutsFinished-30, null);
2536               region.delete(delete, null, true);
2537             }
2538             numPutsFinished++;
2539           }
2540         } catch (IOException e) {
2541           LOG.error("error while putting records", e);
2542           error = e;
2543           break;
2544         }
2545       }
2546 
2547     }
2548 
2549   }
2550 
2551 
2552   /**
2553    * Writes very wide records and gets the latest row every time..
2554    * Flushes and compacts the region every now and then to keep things
2555    * realistic.
2556    *
2557    * @throws IOException          by flush / scan / compaction
2558    * @throws InterruptedException when joining threads
2559    */
2560   public void testWritesWhileGetting()
2561     throws IOException, InterruptedException {
2562     byte[] tableName = Bytes.toBytes("testWritesWhileScanning");
2563     int testCount = 100;
2564     int numRows = 1;
2565     int numFamilies = 10;
2566     int numQualifiers = 100;
2567     int flushInterval = 10;
2568     int compactInterval = 10 * flushInterval;
2569     byte[][] families = new byte[numFamilies][];
2570     for (int i = 0; i < numFamilies; i++) {
2571       families[i] = Bytes.toBytes("family" + i);
2572     }
2573     byte[][] qualifiers = new byte[numQualifiers][];
2574     for (int i = 0; i < numQualifiers; i++) {
2575       qualifiers[i] = Bytes.toBytes("qual" + i);
2576     }
2577 
2578     String method = "testWritesWhileScanning";
2579     initHRegion(tableName, method, families);
2580     PutThread putThread = new PutThread(numRows, families, qualifiers);
2581     putThread.start();
2582     putThread.waitForFirstPut();
2583 
2584     FlushThread flushThread = new FlushThread();
2585     flushThread.start();
2586 
2587     Get get = new Get(Bytes.toBytes("row0"));
2588     Result result = null;
2589 
2590     int expectedCount = numFamilies * numQualifiers;
2591 
2592     long prevTimestamp = 0L;
2593     for (int i = 0; i < testCount; i++) {
2594 
2595       if (i != 0 && i % compactInterval == 0) {
2596         region.compactStores(true);
2597       }
2598 
2599       if (i != 0 && i % flushInterval == 0) {
2600         //System.out.println("iteration = " + i);
2601         flushThread.flush();
2602       }
2603 
2604       boolean previousEmpty = result == null || result.isEmpty();
2605       result = region.get(get, null);
2606       if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
2607         assertEquals("i=" + i, expectedCount, result.size());
2608         // TODO this was removed, now what dangit?!
2609         // search looking for the qualifier in question?
2610         long timestamp = 0;
2611         for (KeyValue kv : result.sorted()) {
2612           if (Bytes.equals(kv.getFamily(), families[0])
2613             && Bytes.equals(kv.getQualifier(), qualifiers[0])) {
2614             timestamp = kv.getTimestamp();
2615           }
2616         }
2617         assertTrue(timestamp >= prevTimestamp);
2618         prevTimestamp = timestamp;
2619 
2620         byte [] gotValue = null;
2621         for (KeyValue kv : result.raw()) {
2622           byte [] thisValue = kv.getValue();
2623           if (gotValue != null) {
2624             assertEquals(gotValue, thisValue);
2625           }
2626           gotValue = thisValue;
2627         }
2628       }
2629     }
2630 
2631     putThread.done();
2632 
2633     region.flushcache();
2634 
2635     putThread.join();
2636     putThread.checkNoError();
2637 
2638     flushThread.done();
2639     flushThread.join();
2640     flushThread.checkNoError();
2641   }
2642 
2643 
2644   public void testIndexesScanWithOneDeletedRow() throws IOException {
2645     byte[] tableName = Bytes.toBytes("testIndexesScanWithOneDeletedRow");
2646     byte[] family = Bytes.toBytes("family");
2647 
2648     //Setting up region
2649     String method = "testIndexesScanWithOneDeletedRow";
2650     initHRegion(tableName, method, new HBaseConfiguration(), family);
2651 
2652     Put put = new Put(Bytes.toBytes(1L));
2653     put.add(family, qual1, 1L, Bytes.toBytes(1L));
2654     region.put(put);
2655 
2656     region.flushcache();
2657 
2658     Delete delete = new Delete(Bytes.toBytes(1L), 1L, null);
2659     //delete.deleteColumn(family, qual1);
2660     region.delete(delete, null, true);
2661 
2662     put = new Put(Bytes.toBytes(2L));
2663     put.add(family, qual1, 2L, Bytes.toBytes(2L));
2664     region.put(put);
2665 
2666     Scan idxScan = new Scan();
2667     idxScan.addFamily(family);
2668     idxScan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,
2669       Arrays.<Filter>asList(new SingleColumnValueFilter(family, qual1,
2670         CompareFilter.CompareOp.GREATER_OR_EQUAL,
2671         new BinaryComparator(Bytes.toBytes(0L))),
2672         new SingleColumnValueFilter(family, qual1,
2673           CompareFilter.CompareOp.LESS_OR_EQUAL,
2674           new BinaryComparator(Bytes.toBytes(3L)))
2675       )));
2676     InternalScanner scanner = region.getScanner(idxScan);
2677     List<KeyValue> res = new ArrayList<KeyValue>();
2678 
2679     //long start = System.nanoTime();
2680     while (scanner.next(res)) ;
2681     //long end = System.nanoTime();
2682     //System.out.println("memStoreEmpty=" + memStoreEmpty + ", time=" + (end - start)/1000000D);
2683     assertEquals(1L, res.size());
2684 
2685   }
2686 
2687   //////////////////////////////////////////////////////////////////////////////
2688   // Bloom filter test
2689   //////////////////////////////////////////////////////////////////////////////
2690 
2691   public void testAllColumnsWithBloomFilter() throws IOException {
2692     byte [] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
2693     byte [] FAMILY = Bytes.toBytes("family");
2694 
2695     //Create table
2696     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY, Integer.MAX_VALUE,
2697         HColumnDescriptor.DEFAULT_COMPRESSION,
2698         HColumnDescriptor.DEFAULT_IN_MEMORY,
2699         HColumnDescriptor.DEFAULT_BLOCKCACHE,
2700         Integer.MAX_VALUE, HColumnDescriptor.DEFAULT_TTL,
2701         "rowcol",
2702         HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
2703     HTableDescriptor htd = new HTableDescriptor(TABLE);
2704     htd.addFamily(hcd);
2705     HRegionInfo info = new HRegionInfo(htd, null, null, false);
2706     Path path = new Path(DIR + "testAllColumnsWithBloomFilter");
2707     region = HRegion.createHRegion(info, path, conf);
2708 
2709     // For row:0, col:0: insert versions 1 through 5.
2710     byte row[] = Bytes.toBytes("row:" + 0);
2711     byte column[] = Bytes.toBytes("column:" + 0);
2712     Put put = new Put(row);
2713     for (long idx = 1; idx <= 4; idx++) {
2714       put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
2715     }
2716     region.put(put);
2717 
2718     //Flush
2719     region.flushcache();
2720 
2721     //Get rows
2722     Get get = new Get(row);
2723     get.setMaxVersions();
2724     KeyValue[] kvs = region.get(get, null).raw();
2725 
2726     //Check if rows are correct
2727     assertEquals(4, kvs.length);
2728     checkOneCell(kvs[0], FAMILY, 0, 0, 4);
2729     checkOneCell(kvs[1], FAMILY, 0, 0, 3);
2730     checkOneCell(kvs[2], FAMILY, 0, 0, 2);
2731     checkOneCell(kvs[3], FAMILY, 0, 0, 1);
2732   }
2733 
2734   /**
2735     * Testcase to cover bug-fix for HBASE-2823
2736     * Ensures correct delete when issuing delete row
2737     * on columns with bloom filter set to row+col (BloomType.ROWCOL)
2738    */
2739   public void testDeleteRowWithBloomFilter() throws IOException {
2740     byte [] tableName = Bytes.toBytes("testDeleteRowWithBloomFilter");
2741     byte [] familyName = Bytes.toBytes("familyName");
2742 
2743     // Create Table
2744     HColumnDescriptor hcd = new HColumnDescriptor(familyName, Integer.MAX_VALUE,
2745         HColumnDescriptor.DEFAULT_COMPRESSION, false, true,
2746         HColumnDescriptor.DEFAULT_TTL, "rowcol");
2747 
2748     HTableDescriptor htd = new HTableDescriptor(tableName);
2749     htd.addFamily(hcd);
2750     HRegionInfo info = new HRegionInfo(htd, null, null, false);
2751     Path path = new Path(DIR + "TestDeleteRowWithBloomFilter");
2752     region = HRegion.createHRegion(info, path, conf);
2753 
2754     // Insert some data
2755     byte row[] = Bytes.toBytes("row1");
2756     byte col[] = Bytes.toBytes("col1");
2757 
2758     Put put = new Put(row);
2759     put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue"));
2760     region.put(put);
2761     region.flushcache();
2762 
2763     Delete del = new Delete(row);
2764     region.delete(del, null, true);
2765     region.flushcache();
2766 
2767     // Get remaining rows (should have none)
2768     Get get = new Get(row);
2769     get.addColumn(familyName, col);
2770 
2771     KeyValue[] keyValues = region.get(get, null).raw();
2772     assertTrue(keyValues.length == 0);
2773   }
2774 
2775   private void putData(int startRow, int numRows, byte [] qf,
2776       byte [] ...families)
2777   throws IOException {
2778     for(int i=startRow; i<startRow+numRows; i++) {
2779       Put put = new Put(Bytes.toBytes("" + i));
2780       for(byte [] family : families) {
2781         put.add(family, qf, null);
2782       }
2783       region.put(put);
2784     }
2785   }
2786 
2787   private void verifyData(HRegion newReg, int startRow, int numRows, byte [] qf,
2788       byte [] ... families)
2789   throws IOException {
2790     for(int i=startRow; i<startRow + numRows; i++) {
2791       byte [] row = Bytes.toBytes("" + i);
2792       Get get = new Get(row);
2793       for(byte [] family : families) {
2794         get.addColumn(family, qf);
2795       }
2796       Result result = newReg.get(get, null);
2797       KeyValue [] raw = result.sorted();
2798       assertEquals(families.length, result.size());
2799       for(int j=0; j<families.length; j++) {
2800         assertEquals(0, Bytes.compareTo(row, raw[j].getRow()));
2801         assertEquals(0, Bytes.compareTo(families[j], raw[j].getFamily()));
2802         assertEquals(0, Bytes.compareTo(qf, raw[j].getQualifier()));
2803       }
2804     }
2805   }
2806 
2807   private void assertGet(final HRegion r, final byte [] family, final byte [] k)
2808   throws IOException {
2809     // Now I have k, get values out and assert they are as expected.
2810     Get get = new Get(k).addFamily(family).setMaxVersions();
2811     KeyValue [] results = r.get(get, null).raw();
2812     for (int j = 0; j < results.length; j++) {
2813       byte [] tmp = results[j].getValue();
2814       // Row should be equal to value every time.
2815       assertTrue(Bytes.equals(k, tmp));
2816     }
2817   }
2818 
2819   /*
2820    * Assert first value in the passed region is <code>firstValue</code>.
2821    * @param r
2822    * @param fs
2823    * @param firstValue
2824    * @throws IOException
2825    */
2826   private void assertScan(final HRegion r, final byte [] fs,
2827       final byte [] firstValue)
2828   throws IOException {
2829     byte [][] families = {fs};
2830     Scan scan = new Scan();
2831     for (int i = 0; i < families.length; i++) scan.addFamily(families[i]);
2832     InternalScanner s = r.getScanner(scan);
2833     try {
2834       List<KeyValue> curVals = new ArrayList<KeyValue>();
2835       boolean first = true;
2836       OUTER_LOOP: while(s.next(curVals)) {
2837         for (KeyValue kv: curVals) {
2838           byte [] val = kv.getValue();
2839           byte [] curval = val;
2840           if (first) {
2841             first = false;
2842             assertTrue(Bytes.compareTo(curval, firstValue) == 0);
2843           } else {
2844             // Not asserting anything.  Might as well break.
2845             break OUTER_LOOP;
2846           }
2847         }
2848       }
2849     } finally {
2850       s.close();
2851     }
2852   }
2853 
2854   private HBaseConfiguration initSplit() {
2855     HBaseConfiguration conf = new HBaseConfiguration();
2856     // Always compact if there is more than one store file.
2857     conf.setInt("hbase.hstore.compactionThreshold", 2);
2858 
2859     // Make lease timeout longer, lease checks less frequent
2860     conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
2861 
2862     conf.setInt(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, 10 * 1000);
2863 
2864     // Increase the amount of time between client retries
2865     conf.setLong("hbase.client.pause", 15 * 1000);
2866 
2867     // This size should make it so we always split using the addContent
2868     // below.  After adding all data, the first region is 1.3M
2869     conf.setLong("hbase.hregion.max.filesize", 1024 * 128);
2870     return conf;
2871   }
2872 
2873   private void initHRegion (byte [] tableName, String callingMethod,
2874     byte[] ... families)
2875   throws IOException {
2876     initHRegion(tableName, callingMethod, new HBaseConfiguration(), families);
2877   }
2878 
2879   private void initHRegion (byte [] tableName, String callingMethod,
2880     HBaseConfiguration conf, byte [] ... families)
2881   throws IOException{
2882     HTableDescriptor htd = new HTableDescriptor(tableName);
2883     for(byte [] family : families) {
2884       htd.addFamily(new HColumnDescriptor(family));
2885     }
2886     HRegionInfo info = new HRegionInfo(htd, null, null, false);
2887     Path path = new Path(DIR + callingMethod);
2888     if (fs.exists(path)) {
2889       if (!fs.delete(path, true)) {
2890         throw new IOException("Failed delete of " + path);
2891       }
2892     }
2893     region = HRegion.createHRegion(info, path, conf);
2894   }
2895 
2896   /**
2897    * Assert that the passed in KeyValue has expected contents for the
2898    * specified row, column & timestamp.
2899    */
2900   private void checkOneCell(KeyValue kv, byte[] cf,
2901                              int rowIdx, int colIdx, long ts) {
2902     String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
2903     assertEquals("Row mismatch which checking: " + ctx,
2904                  "row:"+ rowIdx, Bytes.toString(kv.getRow()));
2905     assertEquals("ColumnFamily mismatch while checking: " + ctx,
2906                  Bytes.toString(cf), Bytes.toString(kv.getFamily()));
2907     assertEquals("Column qualifier mismatch while checking: " + ctx,
2908                  "column:" + colIdx, Bytes.toString(kv.getQualifier()));
2909     assertEquals("Timestamp mismatch while checking: " + ctx,
2910                  ts, kv.getTimestamp());
2911     assertEquals("Value mismatch while checking: " + ctx,
2912                  "value-version-" + ts, Bytes.toString(kv.getValue()));
2913   }
2914 
2915 }