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_heapSize() throws IOException {
1962     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
1963 
1964     initHRegion(tableName, getName(), fam1);
1965 
1966     long byAmount = 1L;
1967     long size;
1968 
1969     for( int i = 0; i < 1000 ; i++) {
1970       region.incrementColumnValue(row, fam1, qual1, byAmount, true);
1971 
1972       size = region.memstoreSize.get();
1973       assertTrue("memstore size: " + size, size >= 0);
1974     }
1975   }
1976 
1977   public void testIncrementColumnValue_UpdatingInPlace_Negative()
1978     throws IOException {
1979     initHRegion(tableName, getName(), fam1);
1980 
1981     long value = 3L;
1982     long amount = -1L;
1983 
1984     Put put = new Put(row);
1985     put.add(fam1, qual1, Bytes.toBytes(value));
1986     region.put(put);
1987 
1988     long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
1989     assertEquals(value+amount, result);
1990 
1991     assertICV(row, fam1, qual1, value+amount);
1992   }
1993 
1994   public void testIncrementColumnValue_AddingNew()
1995     throws IOException {
1996     initHRegion(tableName, getName(), fam1);
1997 
1998     long value = 1L;
1999     long amount = 3L;
2000 
2001     Put put = new Put(row);
2002     put.add(fam1, qual1, Bytes.toBytes(value));
2003     put.add(fam1, qual2, Bytes.toBytes(value));
2004     region.put(put);
2005 
2006     long result = region.incrementColumnValue(row, fam1, qual3, amount, true);
2007     assertEquals(amount, result);
2008 
2009     Get get = new Get(row);
2010     get.addColumn(fam1, qual3);
2011     Result rr = region.get(get, null);
2012     assertEquals(1, rr.size());
2013 
2014     // ensure none of the other cols were incremented.
2015     assertICV(row, fam1, qual1, value);
2016     assertICV(row, fam1, qual2, value);
2017     assertICV(row, fam1, qual3, amount);
2018   }
2019 
2020   public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
2021     initHRegion(tableName, getName(), fam1);
2022 
2023     long value = 1L;
2024     long amount = 3L;
2025 
2026     Put put = new Put(row);
2027     put.add(fam1, qual1, Bytes.toBytes(value));
2028     put.add(fam1, qual2, Bytes.toBytes(value));
2029     region.put(put);
2030 
2031     // flush to disk.
2032     region.flushcache();
2033 
2034     Store store = region.getStore(fam1);
2035     assertEquals(0, store.memstore.kvset.size());
2036 
2037     long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
2038     assertEquals(value+amount, r);
2039 
2040     assertICV(row, fam1, qual1, value+amount);
2041   }
2042 
2043   public void testIncrementColumnValue_AddingNewAfterSFCheck()
2044     throws IOException {
2045     initHRegion(tableName, getName(), fam1);
2046 
2047     long value = 1L;
2048     long amount = 3L;
2049 
2050     Put put = new Put(row);
2051     put.add(fam1, qual1, Bytes.toBytes(value));
2052     put.add(fam1, qual2, Bytes.toBytes(value));
2053     region.put(put);
2054     region.flushcache();
2055 
2056     Store store = region.getStore(fam1);
2057     assertEquals(0, store.memstore.kvset.size());
2058 
2059     long r = region.incrementColumnValue(row, fam1, qual3, amount, true);
2060     assertEquals(amount, r);
2061 
2062     assertICV(row, fam1, qual3, amount);
2063 
2064     region.flushcache();
2065 
2066     // ensure that this gets to disk.
2067     assertICV(row, fam1, qual3, amount);
2068   }
2069 
2070   private void assertICV(byte [] row,
2071                          byte [] familiy,
2072                          byte[] qualifier,
2073                          long amount) throws IOException {
2074     // run a get and see?
2075     Get get = new Get(row);
2076     get.addColumn(familiy, qualifier);
2077     Result result = region.get(get, null);
2078     assertEquals(1, result.size());
2079 
2080     KeyValue kv = result.raw()[0];
2081     long r = Bytes.toLong(kv.getValue());
2082     assertEquals(amount, r);
2083   }
2084 
2085 
2086 
2087   public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions()
2088   throws IOException {
2089     byte [] tableName = Bytes.toBytes("testtable");
2090     byte [] row1 = Bytes.toBytes("row1");
2091     byte [] fam1 = Bytes.toBytes("fam1");
2092     byte [] qf1 = Bytes.toBytes("qualifier1");
2093     byte [] qf2 = Bytes.toBytes("quateslifier2");
2094 
2095     long ts1 = 1;
2096     long ts2 = ts1 + 1;
2097     long ts3 = ts1 + 2;
2098     long ts4 = ts1 + 3;
2099 
2100     //Setting up region
2101     String method = this.getName();
2102     initHRegion(tableName, method, fam1);
2103 
2104     //Putting data in Region
2105     KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2106     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2107     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2108     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2109 
2110     KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2111     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2112     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2113     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2114 
2115     Put put = null;
2116     put = new Put(row1);
2117     put.add(kv14);
2118     put.add(kv24);
2119     region.put(put);
2120     region.flushcache();
2121 
2122     put = new Put(row1);
2123     put.add(kv23);
2124     put.add(kv13);
2125     region.put(put);
2126     region.flushcache();
2127 
2128     put = new Put(row1);
2129     put.add(kv22);
2130     put.add(kv12);
2131     region.put(put);
2132     region.flushcache();
2133 
2134     put = new Put(row1);
2135     put.add(kv21);
2136     put.add(kv11);
2137     region.put(put);
2138 
2139     //Expected
2140     List<KeyValue> expected = new ArrayList<KeyValue>();
2141     expected.add(kv14);
2142     expected.add(kv13);
2143     expected.add(kv12);
2144     expected.add(kv24);
2145     expected.add(kv23);
2146     expected.add(kv22);
2147 
2148     Scan scan = new Scan(row1);
2149     int versions = 3;
2150     scan.setMaxVersions(versions);
2151     List<KeyValue> actual = new ArrayList<KeyValue>();
2152     InternalScanner scanner = region.getScanner(scan);
2153 
2154     boolean hasNext = scanner.next(actual);
2155     assertEquals(false, hasNext);
2156 
2157     //Verify result
2158     for(int i=0; i<expected.size(); i++) {
2159       assertEquals(expected.get(i), actual.get(i));
2160     }
2161   }
2162 
2163   //////////////////////////////////////////////////////////////////////////////
2164   // Split test
2165   //////////////////////////////////////////////////////////////////////////////
2166   /**
2167    * Splits twice and verifies getting from each of the split regions.
2168    * @throws Exception
2169    */
2170   public void testBasicSplit() throws Exception {
2171     byte [] tableName = Bytes.toBytes("testtable");
2172     byte [][] families = {fam1, fam2, fam3};
2173 
2174     HBaseConfiguration hc = initSplit();
2175     //Setting up region
2176     String method = this.getName();
2177     initHRegion(tableName, method, hc, families);
2178 
2179     try {
2180       LOG.info("" + addContent(region, fam3));
2181       region.flushcache();
2182       byte [] splitRow = region.compactStores();
2183       assertNotNull(splitRow);
2184       LOG.info("SplitRow: " + Bytes.toString(splitRow));
2185       HRegion [] regions = splitRegion(region, splitRow);
2186       try {
2187         // Need to open the regions.
2188         // TODO: Add an 'open' to HRegion... don't do open by constructing
2189         // instance.
2190         for (int i = 0; i < regions.length; i++) {
2191           regions[i] = openClosedRegion(regions[i]);
2192         }
2193         // Assert can get rows out of new regions. Should be able to get first
2194         // row from first region and the midkey from second region.
2195         assertGet(regions[0], fam3, Bytes.toBytes(START_KEY));
2196         assertGet(regions[1], fam3, splitRow);
2197         // Test I can get scanner and that it starts at right place.
2198         assertScan(regions[0], fam3,
2199             Bytes.toBytes(START_KEY));
2200         assertScan(regions[1], fam3, splitRow);
2201         // Now prove can't split regions that have references.
2202         for (int i = 0; i < regions.length; i++) {
2203           // Add so much data to this region, we create a store file that is >
2204           // than one of our unsplitable references. it will.
2205           for (int j = 0; j < 2; j++) {
2206             addContent(regions[i], fam3);
2207           }
2208           addContent(regions[i], fam2);
2209           addContent(regions[i], fam1);
2210           regions[i].flushcache();
2211         }
2212 
2213         byte [][] midkeys = new byte [regions.length][];
2214         // To make regions splitable force compaction.
2215         for (int i = 0; i < regions.length; i++) {
2216           midkeys[i] = regions[i].compactStores();
2217         }
2218 
2219         TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
2220         // Split these two daughter regions so then I'll have 4 regions. Will
2221         // split because added data above.
2222         for (int i = 0; i < regions.length; i++) {
2223           HRegion[] rs = null;
2224           if (midkeys[i] != null) {
2225             rs = splitRegion(regions[i], midkeys[i]);
2226             for (int j = 0; j < rs.length; j++) {
2227               sortedMap.put(Bytes.toString(rs[j].getRegionName()),
2228                 openClosedRegion(rs[j]));
2229             }
2230           }
2231         }
2232         LOG.info("Made 4 regions");
2233         // The splits should have been even. Test I can get some arbitrary row
2234         // out of each.
2235         int interval = (LAST_CHAR - FIRST_CHAR) / 3;
2236         byte[] b = Bytes.toBytes(START_KEY);
2237         for (HRegion r : sortedMap.values()) {
2238           assertGet(r, fam3, b);
2239           b[0] += interval;
2240         }
2241       } finally {
2242         for (int i = 0; i < regions.length; i++) {
2243           try {
2244             regions[i].close();
2245           } catch (IOException e) {
2246             // Ignore.
2247           }
2248         }
2249       }
2250     } finally {
2251       if (region != null) {
2252         region.close();
2253         region.getLog().closeAndDelete();
2254       }
2255     }
2256   }
2257 
2258   public void testSplitRegion() throws IOException {
2259     byte [] tableName = Bytes.toBytes("testtable");
2260     byte [] qualifier = Bytes.toBytes("qualifier");
2261     HBaseConfiguration hc = initSplit();
2262     int numRows = 10;
2263     byte [][] families = {fam1, fam3};
2264 
2265     //Setting up region
2266     String method = this.getName();
2267     initHRegion(tableName, method, hc, families);
2268 
2269     //Put data in region
2270     int startRow = 100;
2271     putData(startRow, numRows, qualifier, families);
2272     int splitRow = startRow + numRows;
2273     putData(splitRow, numRows, qualifier, families);
2274     region.flushcache();
2275 
2276     HRegion [] regions = null;
2277     try {
2278       regions = splitRegion(region, Bytes.toBytes("" + splitRow));
2279       //Opening the regions returned.
2280       for (int i = 0; i < regions.length; i++) {
2281         regions[i] = openClosedRegion(regions[i]);
2282       }
2283       //Verifying that the region has been split
2284       assertEquals(2, regions.length);
2285 
2286       //Verifying that all data is still there and that data is in the right
2287       //place
2288       verifyData(regions[0], startRow, numRows, qualifier, families);
2289       verifyData(regions[1], splitRow, numRows, qualifier, families);
2290 
2291     } finally {
2292       if (region != null) {
2293         region.close();
2294         region.getLog().closeAndDelete();
2295       }
2296     }
2297   }
2298 
2299 
2300   /**
2301    * Flushes the cache in a thread while scanning. The tests verify that the
2302    * scan is coherent - e.g. the returned results are always of the same or
2303    * later update as the previous results.
2304    * @throws IOException scan / compact
2305    * @throws InterruptedException thread join
2306    */
2307   public void testFlushCacheWhileScanning() throws IOException, InterruptedException {
2308     byte[] tableName = Bytes.toBytes("testFlushCacheWhileScanning");
2309     byte[] family = Bytes.toBytes("family");
2310     int numRows = 1000;
2311     int flushAndScanInterval = 10;
2312     int compactInterval = 10 * flushAndScanInterval;
2313 
2314     String method = "testFlushCacheWhileScanning";
2315     initHRegion(tableName,method, family);
2316     FlushThread flushThread = new FlushThread();
2317     flushThread.start();
2318 
2319     Scan scan = new Scan();
2320     scan.addFamily(family);
2321     scan.setFilter(new SingleColumnValueFilter(family, qual1,
2322       CompareFilter.CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes(5L))));
2323 
2324     int expectedCount = 0;
2325     List<KeyValue> res = new ArrayList<KeyValue>();
2326 
2327     boolean toggle=true;
2328     for (long i = 0; i < numRows; i++) {
2329       Put put = new Put(Bytes.toBytes(i));
2330       put.add(family, qual1, Bytes.toBytes(i % 10));
2331       region.put(put);
2332 
2333       if (i != 0 && i % compactInterval == 0) {
2334         //System.out.println("iteration = " + i);
2335         region.compactStores(true);
2336       }
2337 
2338       if (i % 10 == 5L) {
2339         expectedCount++;
2340       }
2341 
2342       if (i != 0 && i % flushAndScanInterval == 0) {
2343         res.clear();
2344         InternalScanner scanner = region.getScanner(scan);
2345         if (toggle) {
2346           flushThread.flush();
2347         }
2348         while (scanner.next(res)) ;
2349         if (!toggle) {
2350           flushThread.flush();
2351         }
2352         assertEquals("i=" + i, expectedCount, res.size());
2353         toggle = !toggle;
2354       }
2355     }
2356 
2357     flushThread.done();
2358     flushThread.join();
2359     flushThread.checkNoError();
2360   }
2361 
2362   protected class FlushThread extends Thread {
2363     private volatile boolean done;
2364     private Throwable error = null;
2365 
2366     public void done() {
2367       done = true;
2368       synchronized (this) {
2369         interrupt();
2370       }
2371     }
2372 
2373     public void checkNoError() {
2374       if (error != null) {
2375         assertNull(error);
2376       }
2377     }
2378 
2379     @Override
2380     public void run() {
2381       done = false;
2382       while (!done) {
2383         synchronized (this) {
2384           try {
2385             wait();
2386           } catch (InterruptedException ignored) {
2387             if (done) {
2388               break;
2389             }
2390           }
2391         }
2392         try {
2393           region.flushcache();
2394         } catch (IOException e) {
2395           if (!done) {
2396             LOG.error("Error while flusing cache", e);
2397             error = e;
2398           }
2399           break;
2400         }
2401       }
2402 
2403     }
2404 
2405     public void flush() {
2406       synchronized (this) {
2407         notify();
2408       }
2409 
2410     }
2411   }
2412 
2413   /**
2414    * Writes very wide records and scans for the latest every time..
2415    * Flushes and compacts the region every now and then to keep things
2416    * realistic.
2417    *
2418    * @throws IOException          by flush / scan / compaction
2419    * @throws InterruptedException when joining threads
2420    */
2421   public void testWritesWhileScanning()
2422     throws IOException, InterruptedException {
2423     byte[] tableName = Bytes.toBytes("testWritesWhileScanning");
2424     int testCount = 100;
2425     int numRows = 1;
2426     int numFamilies = 10;
2427     int numQualifiers = 100;
2428     int flushInterval = 7;
2429     int compactInterval = 5 * flushInterval;
2430     byte[][] families = new byte[numFamilies][];
2431     for (int i = 0; i < numFamilies; i++) {
2432       families[i] = Bytes.toBytes("family" + i);
2433     }
2434     byte[][] qualifiers = new byte[numQualifiers][];
2435     for (int i = 0; i < numQualifiers; i++) {
2436       qualifiers[i] = Bytes.toBytes("qual" + i);
2437     }
2438 
2439     String method = "testWritesWhileScanning";
2440     initHRegion(tableName, method, families);
2441     PutThread putThread = new PutThread(numRows, families, qualifiers);
2442     putThread.start();
2443     putThread.waitForFirstPut();
2444 
2445     FlushThread flushThread = new FlushThread();
2446     flushThread.start();
2447 
2448     Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
2449 //    scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
2450 //      new BinaryComparator(Bytes.toBytes("row0"))));
2451 
2452     int expectedCount = numFamilies * numQualifiers;
2453     List<KeyValue> res = new ArrayList<KeyValue>();
2454 
2455     long prevTimestamp = 0L;
2456     for (int i = 0; i < testCount; i++) {
2457 
2458       if (i != 0 && i % compactInterval == 0) {
2459         region.compactStores(true);
2460       }
2461 
2462       if (i != 0 && i % flushInterval == 0) {
2463         //System.out.println("flush scan iteration = " + i);
2464         flushThread.flush();
2465       }
2466 
2467       boolean previousEmpty = res.isEmpty();
2468       res.clear();
2469       InternalScanner scanner = region.getScanner(scan);
2470       while (scanner.next(res)) ;
2471       if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
2472         assertEquals("i=" + i, expectedCount, res.size());
2473         long timestamp = res.get(0).getTimestamp();
2474         assertTrue("Timestamps were broke: " + timestamp + " prev: " + prevTimestamp,
2475             timestamp >= prevTimestamp);
2476         prevTimestamp = timestamp;
2477       }
2478     }
2479 
2480     putThread.done();
2481 
2482     region.flushcache();
2483 
2484     putThread.join();
2485     putThread.checkNoError();
2486 
2487     flushThread.done();
2488     flushThread.join();
2489     flushThread.checkNoError();
2490   }
2491 
2492   protected class PutThread extends Thread {
2493     private volatile boolean done;
2494     private volatile int numPutsFinished = 0;
2495 
2496     private Throwable error = null;
2497     private int numRows;
2498     private byte[][] families;
2499     private byte[][] qualifiers;
2500 
2501     private PutThread(int numRows, byte[][] families,
2502       byte[][] qualifiers) {
2503       this.numRows = numRows;
2504       this.families = families;
2505       this.qualifiers = qualifiers;
2506     }
2507 
2508     /**
2509      * Block until this thread has put at least one row.
2510      */
2511     public void waitForFirstPut() throws InterruptedException {
2512       // wait until put thread actually puts some data
2513       while (numPutsFinished == 0) {
2514         checkNoError();
2515         Thread.sleep(50);
2516       }
2517     }
2518 
2519     public void done() {
2520       done = true;
2521       synchronized (this) {
2522         interrupt();
2523       }
2524     }
2525 
2526     public void checkNoError() {
2527       if (error != null) {
2528         assertNull(error);
2529       }
2530     }
2531 
2532     @Override
2533     public void run() {
2534       done = false;
2535       while (!done) {
2536         try {
2537           for (int r = 0; r < numRows; r++) {
2538             byte[] row = Bytes.toBytes("row" + r);
2539             Put put = new Put(row);
2540             for (byte[] family : families) {
2541               for (byte[] qualifier : qualifiers) {
2542                 put.add(family, qualifier, (long) numPutsFinished,
2543                     Bytes.toBytes(numPutsFinished));
2544               }
2545             }
2546 //            System.out.println("Putting of kvsetsize=" + put.size());
2547             region.put(put);
2548             numPutsFinished++;
2549             if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
2550               System.out.println("put iteration = " + numPutsFinished);
2551               Delete delete = new Delete(row, (long)numPutsFinished-30, null);
2552               region.delete(delete, null, true);
2553             }
2554             numPutsFinished++;
2555           }
2556         } catch (IOException e) {
2557           LOG.error("error while putting records", e);
2558           error = e;
2559           break;
2560         }
2561       }
2562 
2563     }
2564 
2565   }
2566 
2567 
2568   /**
2569    * Writes very wide records and gets the latest row every time..
2570    * Flushes and compacts the region every now and then to keep things
2571    * realistic.
2572    *
2573    * @throws IOException          by flush / scan / compaction
2574    * @throws InterruptedException when joining threads
2575    */
2576   public void testWritesWhileGetting()
2577     throws IOException, InterruptedException {
2578     byte[] tableName = Bytes.toBytes("testWritesWhileScanning");
2579     int testCount = 100;
2580     int numRows = 1;
2581     int numFamilies = 10;
2582     int numQualifiers = 100;
2583     int flushInterval = 10;
2584     int compactInterval = 10 * flushInterval;
2585     byte[][] families = new byte[numFamilies][];
2586     for (int i = 0; i < numFamilies; i++) {
2587       families[i] = Bytes.toBytes("family" + i);
2588     }
2589     byte[][] qualifiers = new byte[numQualifiers][];
2590     for (int i = 0; i < numQualifiers; i++) {
2591       qualifiers[i] = Bytes.toBytes("qual" + i);
2592     }
2593 
2594     String method = "testWritesWhileScanning";
2595     initHRegion(tableName, method, families);
2596     PutThread putThread = new PutThread(numRows, families, qualifiers);
2597     putThread.start();
2598     putThread.waitForFirstPut();
2599 
2600     FlushThread flushThread = new FlushThread();
2601     flushThread.start();
2602 
2603     Get get = new Get(Bytes.toBytes("row0"));
2604     Result result = null;
2605 
2606     int expectedCount = numFamilies * numQualifiers;
2607 
2608     long prevTimestamp = 0L;
2609     for (int i = 0; i < testCount; i++) {
2610 
2611       if (i != 0 && i % compactInterval == 0) {
2612         region.compactStores(true);
2613       }
2614 
2615       if (i != 0 && i % flushInterval == 0) {
2616         //System.out.println("iteration = " + i);
2617         flushThread.flush();
2618       }
2619 
2620       boolean previousEmpty = result == null || result.isEmpty();
2621       result = region.get(get, null);
2622       if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
2623         assertEquals("i=" + i, expectedCount, result.size());
2624         // TODO this was removed, now what dangit?!
2625         // search looking for the qualifier in question?
2626         long timestamp = 0;
2627         for (KeyValue kv : result.sorted()) {
2628           if (Bytes.equals(kv.getFamily(), families[0])
2629             && Bytes.equals(kv.getQualifier(), qualifiers[0])) {
2630             timestamp = kv.getTimestamp();
2631           }
2632         }
2633         assertTrue(timestamp >= prevTimestamp);
2634         prevTimestamp = timestamp;
2635 
2636         byte [] gotValue = null;
2637         for (KeyValue kv : result.raw()) {
2638           byte [] thisValue = kv.getValue();
2639           if (gotValue != null) {
2640             assertEquals(gotValue, thisValue);
2641           }
2642           gotValue = thisValue;
2643         }
2644       }
2645     }
2646 
2647     putThread.done();
2648 
2649     region.flushcache();
2650 
2651     putThread.join();
2652     putThread.checkNoError();
2653 
2654     flushThread.done();
2655     flushThread.join();
2656     flushThread.checkNoError();
2657   }
2658 
2659 
2660   public void testIndexesScanWithOneDeletedRow() throws IOException {
2661     byte[] tableName = Bytes.toBytes("testIndexesScanWithOneDeletedRow");
2662     byte[] family = Bytes.toBytes("family");
2663 
2664     //Setting up region
2665     String method = "testIndexesScanWithOneDeletedRow";
2666     initHRegion(tableName, method, new HBaseConfiguration(), family);
2667 
2668     Put put = new Put(Bytes.toBytes(1L));
2669     put.add(family, qual1, 1L, Bytes.toBytes(1L));
2670     region.put(put);
2671 
2672     region.flushcache();
2673 
2674     Delete delete = new Delete(Bytes.toBytes(1L), 1L, null);
2675     //delete.deleteColumn(family, qual1);
2676     region.delete(delete, null, true);
2677 
2678     put = new Put(Bytes.toBytes(2L));
2679     put.add(family, qual1, 2L, Bytes.toBytes(2L));
2680     region.put(put);
2681 
2682     Scan idxScan = new Scan();
2683     idxScan.addFamily(family);
2684     idxScan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,
2685       Arrays.<Filter>asList(new SingleColumnValueFilter(family, qual1,
2686         CompareFilter.CompareOp.GREATER_OR_EQUAL,
2687         new BinaryComparator(Bytes.toBytes(0L))),
2688         new SingleColumnValueFilter(family, qual1,
2689           CompareFilter.CompareOp.LESS_OR_EQUAL,
2690           new BinaryComparator(Bytes.toBytes(3L)))
2691       )));
2692     InternalScanner scanner = region.getScanner(idxScan);
2693     List<KeyValue> res = new ArrayList<KeyValue>();
2694 
2695     //long start = System.nanoTime();
2696     while (scanner.next(res)) ;
2697     //long end = System.nanoTime();
2698     //System.out.println("memStoreEmpty=" + memStoreEmpty + ", time=" + (end - start)/1000000D);
2699     assertEquals(1L, res.size());
2700 
2701   }
2702 
2703   //////////////////////////////////////////////////////////////////////////////
2704   // Bloom filter test
2705   //////////////////////////////////////////////////////////////////////////////
2706 
2707   public void testAllColumnsWithBloomFilter() throws IOException {
2708     byte [] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
2709     byte [] FAMILY = Bytes.toBytes("family");
2710 
2711     //Create table
2712     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY, Integer.MAX_VALUE,
2713         HColumnDescriptor.DEFAULT_COMPRESSION,
2714         HColumnDescriptor.DEFAULT_IN_MEMORY,
2715         HColumnDescriptor.DEFAULT_BLOCKCACHE,
2716         Integer.MAX_VALUE, HColumnDescriptor.DEFAULT_TTL,
2717         "rowcol",
2718         HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
2719     HTableDescriptor htd = new HTableDescriptor(TABLE);
2720     htd.addFamily(hcd);
2721     HRegionInfo info = new HRegionInfo(htd, null, null, false);
2722     Path path = new Path(DIR + "testAllColumnsWithBloomFilter");
2723     region = HRegion.createHRegion(info, path, conf);
2724 
2725     // For row:0, col:0: insert versions 1 through 5.
2726     byte row[] = Bytes.toBytes("row:" + 0);
2727     byte column[] = Bytes.toBytes("column:" + 0);
2728     Put put = new Put(row);
2729     for (long idx = 1; idx <= 4; idx++) {
2730       put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
2731     }
2732     region.put(put);
2733 
2734     //Flush
2735     region.flushcache();
2736 
2737     //Get rows
2738     Get get = new Get(row);
2739     get.setMaxVersions();
2740     KeyValue[] kvs = region.get(get, null).raw();
2741 
2742     //Check if rows are correct
2743     assertEquals(4, kvs.length);
2744     checkOneCell(kvs[0], FAMILY, 0, 0, 4);
2745     checkOneCell(kvs[1], FAMILY, 0, 0, 3);
2746     checkOneCell(kvs[2], FAMILY, 0, 0, 2);
2747     checkOneCell(kvs[3], FAMILY, 0, 0, 1);
2748   }
2749 
2750   /**
2751     * Testcase to cover bug-fix for HBASE-2823
2752     * Ensures correct delete when issuing delete row
2753     * on columns with bloom filter set to row+col (BloomType.ROWCOL)
2754    */
2755   public void testDeleteRowWithBloomFilter() throws IOException {
2756     byte [] tableName = Bytes.toBytes("testDeleteRowWithBloomFilter");
2757     byte [] familyName = Bytes.toBytes("familyName");
2758 
2759     // Create Table
2760     HColumnDescriptor hcd = new HColumnDescriptor(familyName, Integer.MAX_VALUE,
2761         HColumnDescriptor.DEFAULT_COMPRESSION, false, true,
2762         HColumnDescriptor.DEFAULT_TTL, "rowcol");
2763 
2764     HTableDescriptor htd = new HTableDescriptor(tableName);
2765     htd.addFamily(hcd);
2766     HRegionInfo info = new HRegionInfo(htd, null, null, false);
2767     Path path = new Path(DIR + "TestDeleteRowWithBloomFilter");
2768     region = HRegion.createHRegion(info, path, conf);
2769 
2770     // Insert some data
2771     byte row[] = Bytes.toBytes("row1");
2772     byte col[] = Bytes.toBytes("col1");
2773 
2774     Put put = new Put(row);
2775     put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue"));
2776     region.put(put);
2777     region.flushcache();
2778 
2779     Delete del = new Delete(row);
2780     region.delete(del, null, true);
2781     region.flushcache();
2782 
2783     // Get remaining rows (should have none)
2784     Get get = new Get(row);
2785     get.addColumn(familyName, col);
2786 
2787     KeyValue[] keyValues = region.get(get, null).raw();
2788     assertTrue(keyValues.length == 0);
2789   }
2790 
2791   private void putData(int startRow, int numRows, byte [] qf,
2792       byte [] ...families)
2793   throws IOException {
2794     for(int i=startRow; i<startRow+numRows; i++) {
2795       Put put = new Put(Bytes.toBytes("" + i));
2796       for(byte [] family : families) {
2797         put.add(family, qf, null);
2798       }
2799       region.put(put);
2800     }
2801   }
2802 
2803   private void verifyData(HRegion newReg, int startRow, int numRows, byte [] qf,
2804       byte [] ... families)
2805   throws IOException {
2806     for(int i=startRow; i<startRow + numRows; i++) {
2807       byte [] row = Bytes.toBytes("" + i);
2808       Get get = new Get(row);
2809       for(byte [] family : families) {
2810         get.addColumn(family, qf);
2811       }
2812       Result result = newReg.get(get, null);
2813       KeyValue [] raw = result.sorted();
2814       assertEquals(families.length, result.size());
2815       for(int j=0; j<families.length; j++) {
2816         assertEquals(0, Bytes.compareTo(row, raw[j].getRow()));
2817         assertEquals(0, Bytes.compareTo(families[j], raw[j].getFamily()));
2818         assertEquals(0, Bytes.compareTo(qf, raw[j].getQualifier()));
2819       }
2820     }
2821   }
2822 
2823   private void assertGet(final HRegion r, final byte [] family, final byte [] k)
2824   throws IOException {
2825     // Now I have k, get values out and assert they are as expected.
2826     Get get = new Get(k).addFamily(family).setMaxVersions();
2827     KeyValue [] results = r.get(get, null).raw();
2828     for (int j = 0; j < results.length; j++) {
2829       byte [] tmp = results[j].getValue();
2830       // Row should be equal to value every time.
2831       assertTrue(Bytes.equals(k, tmp));
2832     }
2833   }
2834 
2835   /*
2836    * Assert first value in the passed region is <code>firstValue</code>.
2837    * @param r
2838    * @param fs
2839    * @param firstValue
2840    * @throws IOException
2841    */
2842   private void assertScan(final HRegion r, final byte [] fs,
2843       final byte [] firstValue)
2844   throws IOException {
2845     byte [][] families = {fs};
2846     Scan scan = new Scan();
2847     for (int i = 0; i < families.length; i++) scan.addFamily(families[i]);
2848     InternalScanner s = r.getScanner(scan);
2849     try {
2850       List<KeyValue> curVals = new ArrayList<KeyValue>();
2851       boolean first = true;
2852       OUTER_LOOP: while(s.next(curVals)) {
2853         for (KeyValue kv: curVals) {
2854           byte [] val = kv.getValue();
2855           byte [] curval = val;
2856           if (first) {
2857             first = false;
2858             assertTrue(Bytes.compareTo(curval, firstValue) == 0);
2859           } else {
2860             // Not asserting anything.  Might as well break.
2861             break OUTER_LOOP;
2862           }
2863         }
2864       }
2865     } finally {
2866       s.close();
2867     }
2868   }
2869 
2870   private HBaseConfiguration initSplit() {
2871     HBaseConfiguration conf = new HBaseConfiguration();
2872     // Always compact if there is more than one store file.
2873     conf.setInt("hbase.hstore.compactionThreshold", 2);
2874 
2875     // Make lease timeout longer, lease checks less frequent
2876     conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
2877 
2878     conf.setInt(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, 10 * 1000);
2879 
2880     // Increase the amount of time between client retries
2881     conf.setLong("hbase.client.pause", 15 * 1000);
2882 
2883     // This size should make it so we always split using the addContent
2884     // below.  After adding all data, the first region is 1.3M
2885     conf.setLong("hbase.hregion.max.filesize", 1024 * 128);
2886     return conf;
2887   }
2888 
2889   private void initHRegion (byte [] tableName, String callingMethod,
2890     byte[] ... families)
2891   throws IOException {
2892     initHRegion(tableName, callingMethod, new HBaseConfiguration(), families);
2893   }
2894 
2895   private void initHRegion (byte [] tableName, String callingMethod,
2896     HBaseConfiguration conf, byte [] ... families)
2897   throws IOException{
2898     HTableDescriptor htd = new HTableDescriptor(tableName);
2899     for(byte [] family : families) {
2900       htd.addFamily(new HColumnDescriptor(family));
2901     }
2902     HRegionInfo info = new HRegionInfo(htd, null, null, false);
2903     Path path = new Path(DIR + callingMethod);
2904     if (fs.exists(path)) {
2905       if (!fs.delete(path, true)) {
2906         throw new IOException("Failed delete of " + path);
2907       }
2908     }
2909     region = HRegion.createHRegion(info, path, conf);
2910   }
2911 
2912   /**
2913    * Assert that the passed in KeyValue has expected contents for the
2914    * specified row, column & timestamp.
2915    */
2916   private void checkOneCell(KeyValue kv, byte[] cf,
2917                              int rowIdx, int colIdx, long ts) {
2918     String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
2919     assertEquals("Row mismatch which checking: " + ctx,
2920                  "row:"+ rowIdx, Bytes.toString(kv.getRow()));
2921     assertEquals("ColumnFamily mismatch while checking: " + ctx,
2922                  Bytes.toString(cf), Bytes.toString(kv.getFamily()));
2923     assertEquals("Column qualifier mismatch while checking: " + ctx,
2924                  "column:" + colIdx, Bytes.toString(kv.getQualifier()));
2925     assertEquals("Timestamp mismatch while checking: " + ctx,
2926                  ts, kv.getTimestamp());
2927     assertEquals("Value mismatch while checking: " + ctx,
2928                  "value-version-" + ts, Bytes.toString(kv.getValue()));
2929   }
2930 
2931 }