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