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  
23  import java.io.DataInput;
24  import java.io.DataOutput;
25  import java.io.IOException;
26  import java.io.InterruptedIOException;
27  import java.util.ArrayList;
28  import java.util.Arrays;
29  import java.util.HashMap;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.TreeMap;
33  import java.util.concurrent.atomic.AtomicBoolean;
34  import java.util.concurrent.atomic.AtomicInteger;
35  import java.util.concurrent.atomic.AtomicReference;
36  
37  import org.apache.commons.logging.Log;
38  import org.apache.commons.logging.LogFactory;
39  import org.apache.hadoop.conf.Configuration;
40  import org.apache.hadoop.fs.FSDataOutputStream;
41  import org.apache.hadoop.fs.FileSystem;
42  import org.apache.hadoop.fs.Path;
43  import org.apache.hadoop.hbase.DoNotRetryIOException;
44  import org.apache.hadoop.hbase.HBaseConfiguration;
45  import org.apache.hadoop.hbase.HBaseTestCase;
46  import org.apache.hadoop.hbase.HBaseTestingUtility;
47  import org.apache.hadoop.hbase.HColumnDescriptor;
48  import org.apache.hadoop.hbase.HConstants;
49  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
50  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.HTableDescriptor;
53  import org.apache.hadoop.hbase.KeyValue;
54  import org.apache.hadoop.hbase.MediumTests;
55  import org.apache.hadoop.hbase.MiniHBaseCluster;
56  import org.apache.hadoop.hbase.MultithreadedTestUtil;
57  import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
58  import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
59  import org.apache.hadoop.hbase.client.Append;
60  import org.apache.hadoop.hbase.client.Delete;
61  import org.apache.hadoop.hbase.client.Get;
62  import org.apache.hadoop.hbase.client.HTable;
63  import org.apache.hadoop.hbase.client.Increment;
64  import org.apache.hadoop.hbase.client.Put;
65  import org.apache.hadoop.hbase.client.Result;
66  import org.apache.hadoop.hbase.client.RowMutations;
67  import org.apache.hadoop.hbase.client.Scan;
68  import org.apache.hadoop.hbase.filter.BinaryComparator;
69  import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
70  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
71  import org.apache.hadoop.hbase.filter.Filter;
72  import org.apache.hadoop.hbase.filter.FilterBase;
73  import org.apache.hadoop.hbase.filter.FilterList;
74  import org.apache.hadoop.hbase.filter.NullComparator;
75  import org.apache.hadoop.hbase.filter.PrefixFilter;
76  import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
77  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
78  import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
79  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
80  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
81  import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
82  import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
83  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
84  import org.apache.hadoop.hbase.regionserver.wal.HLog;
85  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
86  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
87  import org.apache.hadoop.hbase.util.Bytes;
88  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
89  import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
90  import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
91  import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
92  import org.apache.hadoop.hbase.util.Pair;
93  import org.apache.hadoop.hbase.util.PairOfSameType;
94  import org.apache.hadoop.hbase.util.Threads;
95  import org.junit.Assert;
96  import org.junit.Test;
97  import org.junit.experimental.categories.Category;
98  import org.mockito.Mockito;
99  
100 import com.google.common.collect.Lists;
101 
102 
103 /**
104  * Basic stand-alone testing of HRegion.
105  *
106  * A lot of the meta information for an HRegion now lives inside other
107  * HRegions or in the HBaseMaster, so only basic testing is possible.
108  */
109 @Category(MediumTests.class)
110 @SuppressWarnings("deprecation")
111 public class TestHRegion extends HBaseTestCase {
112   // Do not spin up clusters in here.  If you need to spin up a cluster, do it
113   // over in TestHRegionOnCluster.
114   static final Log LOG = LogFactory.getLog(TestHRegion.class);
115 
116   private static final String COLUMN_FAMILY = "MyCF";
117 
118   HRegion region = null;
119   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
120   private static final String DIR = TEST_UTIL.getDataTestDir("TestHRegion").toString();
121 
122   private final int MAX_VERSIONS = 2;
123 
124   // Test names
125   protected final byte[] tableName = Bytes.toBytes("testtable");;
126   protected final byte[] qual1 = Bytes.toBytes("qual1");
127   protected final byte[] qual2 = Bytes.toBytes("qual2");
128   protected final byte[] qual3 = Bytes.toBytes("qual3");
129   protected final byte[] value1 = Bytes.toBytes("value1");
130   protected final byte[] value2 = Bytes.toBytes("value2");
131   protected final byte [] row = Bytes.toBytes("rowA");
132   protected final byte [] row2 = Bytes.toBytes("rowB");
133 
134 
135   private Map<String, Long> startingMetrics;
136 
137   /**
138    * @see org.apache.hadoop.hbase.HBaseTestCase#setUp()
139    */
140   @Override
141   protected void setUp() throws Exception {
142     startingMetrics = SchemaMetrics.getMetricsSnapshot();
143     super.setUp();
144   }
145 
146   @Override
147   protected void tearDown() throws Exception {
148     super.tearDown();
149     EnvironmentEdgeManagerTestHelper.reset();
150     SchemaMetrics.validateMetricChanges(startingMetrics);
151   }
152 
153   //////////////////////////////////////////////////////////////////////////////
154   // New tests that doesn't spin up a mini cluster but rather just test the
155   // individual code pieces in the HRegion. Putting files locally in
156   // /tmp/testtable
157   //////////////////////////////////////////////////////////////////////////////
158 
159   public void testCompactionAffectedByScanners() throws Exception {
160     String method = "testCompactionAffectedByScanners";
161     byte[] tableName = Bytes.toBytes(method);
162     byte[] family = Bytes.toBytes("family");
163     this.region = initHRegion(tableName, method, conf, family);
164 
165     Put put = new Put(Bytes.toBytes("r1"));
166     put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
167     region.put(put);
168     region.flushcache();
169 
170 
171     Scan scan = new Scan();
172     scan.setMaxVersions(3);
173     // open the first scanner
174     RegionScanner scanner1 = region.getScanner(scan);
175 
176     Delete delete = new Delete(Bytes.toBytes("r1"));
177     region.delete(delete, null, false);
178     region.flushcache();
179 
180     // open the second scanner
181     RegionScanner scanner2 = region.getScanner(scan);
182 
183     List<KeyValue> results = new ArrayList<KeyValue>();
184 
185     System.out.println("Smallest read point:" + region.getSmallestReadPoint());
186 
187     // make a major compaction
188     region.compactStores(true);
189 
190     // open the third scanner
191     RegionScanner scanner3 = region.getScanner(scan);
192 
193     // get data from scanner 1, 2, 3 after major compaction
194     scanner1.next(results);
195     System.out.println(results);
196     assertEquals(1, results.size());
197 
198     results.clear();
199     scanner2.next(results);
200     System.out.println(results);
201     assertEquals(0, results.size());
202 
203     results.clear();
204     scanner3.next(results);
205     System.out.println(results);
206     assertEquals(0, results.size());
207   }
208 
209   @Test
210   public void testToShowNPEOnRegionScannerReseek() throws Exception{
211     String method = "testToShowNPEOnRegionScannerReseek";
212     byte[] tableName = Bytes.toBytes(method);
213     byte[] family = Bytes.toBytes("family");
214     this.region = initHRegion(tableName, method, conf, family);
215 
216     Put put = new Put(Bytes.toBytes("r1"));
217     put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
218     region.put(put);
219     put = new Put(Bytes.toBytes("r2"));
220     put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
221     region.put(put);
222     region.flushcache();
223 
224 
225     Scan scan = new Scan();
226     scan.setMaxVersions(3);
227     // open the first scanner
228     RegionScanner scanner1 = region.getScanner(scan);
229 
230     System.out.println("Smallest read point:" + region.getSmallestReadPoint());
231     
232     region.compactStores(true);
233 
234     scanner1.reseek(Bytes.toBytes("r2"));
235     List<KeyValue> results = new ArrayList<KeyValue>();
236     scanner1.next(results);
237     KeyValue keyValue = results.get(0);
238     Assert.assertTrue(Bytes.compareTo(keyValue.getRow(), Bytes.toBytes("r2")) == 0);
239     scanner1.close();
240   }
241 
242   public void testSkipRecoveredEditsReplay() throws Exception {
243     String method = "testSkipRecoveredEditsReplay";
244     byte[] tableName = Bytes.toBytes(method);
245     byte[] family = Bytes.toBytes("family");
246     this.region = initHRegion(tableName, method, conf, family);
247     try {
248       Path regiondir = region.getRegionDir();
249       FileSystem fs = region.getFilesystem();
250       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
251 
252       Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
253 
254       long maxSeqId = 1050;
255       long minSeqId = 1000;
256 
257       for (long i = minSeqId; i <= maxSeqId; i += 10) {
258         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
259         fs.create(recoveredEdits);
260         HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
261 
262         long time = System.nanoTime();
263         WALEdit edit = new WALEdit();
264         edit.add(new KeyValue(row, family, Bytes.toBytes(i),
265             time, KeyValue.Type.Put, Bytes.toBytes(i)));
266         writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
267             i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
268 
269         writer.close();
270       }
271       MonitoredTask status = TaskMonitor.get().createStatus(method);
272       Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
273           Bytes.BYTES_COMPARATOR);
274       for (Store store : region.getStores().values()) {
275         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
276             minSeqId - 1);
277       }
278       long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
279       assertEquals(maxSeqId, seqId);
280       Get get = new Get(row);
281       Result result = region.get(get, null);
282       for (long i = minSeqId; i <= maxSeqId; i += 10) {
283         List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
284         assertEquals(1, kvs.size());
285         assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
286       }
287     } finally {
288       HRegion.closeHRegion(this.region);
289       this.region = null;
290     }
291   }
292 
293   public void testSkipRecoveredEditsReplaySomeIgnored() throws Exception {
294     String method = "testSkipRecoveredEditsReplaySomeIgnored";
295     byte[] tableName = Bytes.toBytes(method);
296     byte[] family = Bytes.toBytes("family");
297     this.region = initHRegion(tableName, method, conf, family);
298     try {
299       Path regiondir = region.getRegionDir();
300       FileSystem fs = region.getFilesystem();
301       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
302 
303       Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
304 
305       long maxSeqId = 1050;
306       long minSeqId = 1000;
307 
308       for (long i = minSeqId; i <= maxSeqId; i += 10) {
309         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
310         fs.create(recoveredEdits);
311         HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
312 
313         long time = System.nanoTime();
314         WALEdit edit = new WALEdit();
315         edit.add(new KeyValue(row, family, Bytes.toBytes(i),
316             time, KeyValue.Type.Put, Bytes.toBytes(i)));
317         writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
318             i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
319 
320         writer.close();
321       }
322       long recoverSeqId = 1030;
323       MonitoredTask status = TaskMonitor.get().createStatus(method);
324       Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
325           Bytes.BYTES_COMPARATOR);
326       for (Store store : region.getStores().values()) {
327         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
328             recoverSeqId - 1);
329       }
330       long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
331       assertEquals(maxSeqId, seqId);
332       Get get = new Get(row);
333       Result result = region.get(get, null);
334       for (long i = minSeqId; i <= maxSeqId; i += 10) {
335         List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
336         if (i < recoverSeqId) {
337           assertEquals(0, kvs.size());
338         } else {
339           assertEquals(1, kvs.size());
340           assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
341         }
342       }
343     } finally {
344       HRegion.closeHRegion(this.region);
345       this.region = null;
346     }
347   }
348 
349   public void testSkipRecoveredEditsReplayAllIgnored() throws Exception {
350     String method = "testSkipRecoveredEditsReplayAllIgnored";
351     byte[] tableName = Bytes.toBytes(method);
352     byte[] family = Bytes.toBytes("family");
353     this.region = initHRegion(tableName, method, conf, family);
354     try {
355       Path regiondir = region.getRegionDir();
356       FileSystem fs = region.getFilesystem();
357 
358       Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
359       for (int i = 1000; i < 1050; i += 10) {
360         Path recoveredEdits = new Path(
361             recoveredEditsDir, String.format("%019d", i));
362         FSDataOutputStream dos=  fs.create(recoveredEdits);
363         dos.writeInt(i);
364         dos.close();
365       }
366       long minSeqId = 2000;
367       Path recoveredEdits = new Path(
368           recoveredEditsDir, String.format("%019d", minSeqId-1));
369       FSDataOutputStream dos=  fs.create(recoveredEdits);
370       dos.close();
371       
372       Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
373         Bytes.BYTES_COMPARATOR);
374       for (Store store : region.getStores().values()) {
375         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId);
376       }
377       long seqId = region.replayRecoveredEditsIfAny(regiondir,
378           maxSeqIdInStores, null, null);
379       assertEquals(minSeqId, seqId);
380     } finally {
381       HRegion.closeHRegion(this.region);
382       this.region = null;
383     }
384   }
385 
386   public void testGetWhileRegionClose() throws IOException {
387     Configuration hc = initSplit();
388     int numRows = 100;
389     byte [][] families = {fam1, fam2, fam3};
390 
391     //Setting up region
392     String method = this.getName();
393     this.region = initHRegion(tableName, method, hc, families);
394     try {
395       // Put data in region
396       final int startRow = 100;
397       putData(startRow, numRows, qual1, families);
398       putData(startRow, numRows, qual2, families);
399       putData(startRow, numRows, qual3, families);
400       // this.region.flushcache();
401       final AtomicBoolean done = new AtomicBoolean(false);
402       final AtomicInteger gets = new AtomicInteger(0);
403       GetTillDoneOrException [] threads = new GetTillDoneOrException[10];
404       try {
405         // Set ten threads running concurrently getting from the region.
406         for (int i = 0; i < threads.length / 2; i++) {
407           threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
408               done, gets);
409           threads[i].setDaemon(true);
410           threads[i].start();
411         }
412         // Artificially make the condition by setting closing flag explicitly.
413         // I can't make the issue happen with a call to region.close().
414         this.region.closing.set(true);
415         for (int i = threads.length / 2; i < threads.length; i++) {
416           threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
417               done, gets);
418           threads[i].setDaemon(true);
419           threads[i].start();
420         }
421       } finally {
422         if (this.region != null) {
423           this.region.close();
424           this.region.getLog().closeAndDelete();
425         }
426       }
427       done.set(true);
428       for (GetTillDoneOrException t: threads) {
429         try {
430           t.join();
431         } catch (InterruptedException e) {
432           e.printStackTrace();
433         }
434         if (t.e != null) {
435           LOG.info("Exception=" + t.e);
436           assertFalse("Found a NPE in " + t.getName(),
437               t.e instanceof NullPointerException);
438         }
439       }
440     } finally {
441       HRegion.closeHRegion(this.region);
442       this.region = null;
443     }
444   }
445 
446   /*
447    * Thread that does get on single row until 'done' flag is flipped.  If an
448    * exception causes us to fail, it records it.
449    */
450   class GetTillDoneOrException extends Thread {
451     private final Get g;
452     private final AtomicBoolean done;
453     private final AtomicInteger count;
454     private Exception e;
455 
456     GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d,
457         final AtomicInteger c) {
458       super("getter." + i);
459       this.g = new Get(r);
460       this.done = d;
461       this.count = c;
462     }
463 
464     @Override
465     public void run() {
466       while (!this.done.get()) {
467         try {
468           assertTrue(region.get(g, null).size() > 0);
469           this.count.incrementAndGet();
470         } catch (Exception e) {
471           this.e = e;
472           break;
473         }
474       }
475     }
476   }
477 
478   /*
479    * An involved filter test.  Has multiple column families and deletes in mix.
480    */
481   public void testWeirdCacheBehaviour() throws Exception {
482     byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
483     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
484         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
485         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
486     this.region = initHRegion(TABLE, getName(), conf, FAMILIES);
487     try {
488       String value = "this is the value";
489       String value2 = "this is some other value";
490       String keyPrefix1 = "prefix1"; // UUID.randomUUID().toString();
491       String keyPrefix2 = "prefix2"; // UUID.randomUUID().toString();
492       String keyPrefix3 = "prefix3"; // UUID.randomUUID().toString();
493       putRows(this.region, 3, value, keyPrefix1);
494       putRows(this.region, 3, value, keyPrefix2);
495       putRows(this.region, 3, value, keyPrefix3);
496       // this.region.flushCommits();
497       putRows(this.region, 3, value2, keyPrefix1);
498       putRows(this.region, 3, value2, keyPrefix2);
499       putRows(this.region, 3, value2, keyPrefix3);
500       System.out.println("Checking values for key: " + keyPrefix1);
501       assertEquals("Got back incorrect number of rows from scan", 3,
502           getNumberOfRows(keyPrefix1, value2, this.region));
503       System.out.println("Checking values for key: " + keyPrefix2);
504       assertEquals("Got back incorrect number of rows from scan", 3,
505           getNumberOfRows(keyPrefix2, value2, this.region));
506       System.out.println("Checking values for key: " + keyPrefix3);
507       assertEquals("Got back incorrect number of rows from scan", 3,
508           getNumberOfRows(keyPrefix3, value2, this.region));
509       deleteColumns(this.region, value2, keyPrefix1);
510       deleteColumns(this.region, value2, keyPrefix2);
511       deleteColumns(this.region, value2, keyPrefix3);
512       System.out.println("Starting important checks.....");
513       assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
514           0, getNumberOfRows(keyPrefix1, value2, this.region));
515       assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
516           0, getNumberOfRows(keyPrefix2, value2, this.region));
517       assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
518           0, getNumberOfRows(keyPrefix3, value2, this.region));
519     } finally {
520       HRegion.closeHRegion(this.region);
521       this.region = null;
522     }
523   }
524 
525   public void testAppendWithReadOnlyTable() throws Exception {
526     byte[] TABLE = Bytes.toBytes("readOnlyTable");
527     this.region = initHRegion(TABLE, getName(), conf, true, Bytes.toBytes("somefamily"));
528     boolean exceptionCaught = false;
529     Append append = new Append(Bytes.toBytes("somerow"));
530     append.add(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 
531         Bytes.toBytes("somevalue"));
532     try {
533       region.append(append, false);
534     } catch (IOException e) {
535       exceptionCaught = true;
536     } finally {
537       HRegion.closeHRegion(this.region);
538       this.region = null;
539     }
540     assertTrue(exceptionCaught == true);
541   }
542 
543   public void testIncrWithReadOnlyTable() throws Exception {
544     byte[] TABLE = Bytes.toBytes("readOnlyTable");
545     this.region = initHRegion(TABLE, getName(), conf, true, Bytes.toBytes("somefamily"));
546     boolean exceptionCaught = false;    
547     Increment inc = new Increment(Bytes.toBytes("somerow"));
548     inc.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
549     try {
550       region.increment(inc, false);
551     } catch (IOException e) {
552       exceptionCaught = true;
553     } finally {
554       HRegion.closeHRegion(this.region);
555       this.region = null;
556     }
557     assertTrue(exceptionCaught == true);
558   }
559 
560   private void deleteColumns(HRegion r, String value, String keyPrefix)
561   throws IOException {
562     InternalScanner scanner = buildScanner(keyPrefix, value, r);
563     int count = 0;
564     boolean more = false;
565     List<KeyValue> results = new ArrayList<KeyValue>();
566     do {
567       more = scanner.next(results);
568       if (results != null && !results.isEmpty())
569         count++;
570       else
571         break;
572       Delete delete = new Delete(results.get(0).getRow());
573       delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
574       r.delete(delete, null, false);
575       results.clear();
576     } while (more);
577     assertEquals("Did not perform correct number of deletes", 3, count);
578   }
579 
580   private int getNumberOfRows(String keyPrefix, String value, HRegion r) throws Exception {
581     InternalScanner resultScanner = buildScanner(keyPrefix, value, r);
582     int numberOfResults = 0;
583     List<KeyValue> results = new ArrayList<KeyValue>();
584     boolean more = false;
585     do {
586       more = resultScanner.next(results);
587       if (results != null && !results.isEmpty()) numberOfResults++;
588       else break;
589       for (KeyValue kv: results) {
590         System.out.println("kv=" + kv.toString() + ", " + Bytes.toString(kv.getValue()));
591       }
592       results.clear();
593     } while(more);
594     return numberOfResults;
595   }
596 
597   private InternalScanner buildScanner(String keyPrefix, String value, HRegion r)
598   throws IOException {
599     // Defaults FilterList.Operator.MUST_PASS_ALL.
600     FilterList allFilters = new FilterList();
601     allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
602     // Only return rows where this column value exists in the row.
603     SingleColumnValueFilter filter =
604       new SingleColumnValueFilter(Bytes.toBytes("trans-tags"),
605         Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value));
606     filter.setFilterIfMissing(true);
607     allFilters.addFilter(filter);
608     Scan scan = new Scan();
609     scan.addFamily(Bytes.toBytes("trans-blob"));
610     scan.addFamily(Bytes.toBytes("trans-type"));
611     scan.addFamily(Bytes.toBytes("trans-date"));
612     scan.addFamily(Bytes.toBytes("trans-tags"));
613     scan.addFamily(Bytes.toBytes("trans-group"));
614     scan.setFilter(allFilters);
615     return r.getScanner(scan);
616   }
617 
618   private void putRows(HRegion r, int numRows, String value, String key)
619   throws IOException {
620     for (int i = 0; i < numRows; i++) {
621       String row = key + "_" + i/* UUID.randomUUID().toString() */;
622       System.out.println(String.format("Saving row: %s, with value %s", row,
623         value));
624       Put put = new Put(Bytes.toBytes(row));
625       put.setWriteToWAL(false);
626       put.add(Bytes.toBytes("trans-blob"), null,
627         Bytes.toBytes("value for blob"));
628       put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
629       put.add(Bytes.toBytes("trans-date"), null,
630         Bytes.toBytes("20090921010101999"));
631       put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"),
632         Bytes.toBytes(value));
633       put.add(Bytes.toBytes("trans-group"), null,
634         Bytes.toBytes("adhocTransactionGroupId"));
635       r.put(put);
636     }
637   }
638 
639   public void testFamilyWithAndWithoutColon() throws Exception {
640     byte [] b = Bytes.toBytes(getName());
641     byte [] cf = Bytes.toBytes(COLUMN_FAMILY);
642     this.region = initHRegion(b, getName(), conf, cf);
643     try {
644       Put p = new Put(b);
645       byte [] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
646       p.add(cfwithcolon, cfwithcolon, cfwithcolon);
647       boolean exception = false;
648       try {
649         this.region.put(p);
650       } catch (NoSuchColumnFamilyException e) {
651         exception = true;
652       }
653       assertTrue(exception);
654     } finally {
655        HRegion.closeHRegion(this.region);
656       this.region = null;
657     }
658   }
659 
660   @SuppressWarnings("unchecked")
661   public void testBatchPut() throws Exception {
662     byte[] b = Bytes.toBytes(getName());
663     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
664     byte[] qual = Bytes.toBytes("qual");
665     byte[] val = Bytes.toBytes("val");
666     this.region = initHRegion(b, getName(), conf, cf);
667     try {
668       HLog.getSyncTime(); // clear counter from prior tests
669       assertEquals(0, HLog.getSyncTime().count);
670 
671       LOG.info("First a batch put with all valid puts");
672       final Put[] puts = new Put[10];
673       for (int i = 0; i < 10; i++) {
674         puts[i] = new Put(Bytes.toBytes("row_" + i));
675         puts[i].add(cf, qual, val);
676       }
677 
678       OperationStatus[] codes = this.region.put(puts);
679       assertEquals(10, codes.length);
680       for (int i = 0; i < 10; i++) {
681         assertEquals(OperationStatusCode.SUCCESS, codes[i]
682             .getOperationStatusCode());
683       }
684       assertEquals(1, HLog.getSyncTime().count);
685 
686       LOG.info("Next a batch put with one invalid family");
687       puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
688       codes = this.region.put(puts);
689       assertEquals(10, codes.length);
690       for (int i = 0; i < 10; i++) {
691         assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
692           OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
693       }
694       assertEquals(1, HLog.getSyncTime().count);
695 
696       LOG.info("Next a batch put that has to break into two batches to avoid a lock");
697       Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
698 
699       MultithreadedTestUtil.TestContext ctx =
700         new MultithreadedTestUtil.TestContext(conf);
701       final AtomicReference<OperationStatus[]> retFromThread =
702         new AtomicReference<OperationStatus[]>();
703       TestThread putter = new TestThread(ctx) {
704         @Override
705         public void doWork() throws IOException {
706           retFromThread.set(region.put(puts));
707         }
708       };
709       LOG.info("...starting put thread while holding lock");
710       ctx.addThread(putter);
711       ctx.startThreads();
712   
713       LOG.info("...waiting for put thread to sync first time");
714       long startWait = System.currentTimeMillis();
715       while (HLog.getSyncTime().count == 0) {
716         Thread.sleep(100);
717         if (System.currentTimeMillis() - startWait > 10000) {
718           fail("Timed out waiting for thread to sync first minibatch");
719         }
720       }
721       LOG.info("...releasing row lock, which should let put thread continue");
722       region.releaseRowLock(lockedRow);
723       LOG.info("...joining on thread");
724       ctx.stop();
725       LOG.info("...checking that next batch was synced");
726       assertEquals(1, HLog.getSyncTime().count);
727       codes = retFromThread.get();
728       for (int i = 0; i < 10; i++) {
729         assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
730           OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
731       }
732   
733       LOG.info("Nexta, a batch put which uses an already-held lock");
734       lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
735       LOG.info("...obtained row lock");
736       List<Pair<Put, Integer>> putsAndLocks = Lists.newArrayList();
737       for (int i = 0; i < 10; i++) {
738         Pair<Put, Integer> pair = new Pair<Put, Integer>(puts[i], null);
739         if (i == 2) pair.setSecond(lockedRow);
740         putsAndLocks.add(pair);
741       }
742   
743       codes = region.put(putsAndLocks.toArray(new Pair[0]));
744       LOG.info("...performed put");
745       for (int i = 0; i < 10; i++) {
746         assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
747           OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
748       }
749       // Make sure we didn't do an extra batch
750       assertEquals(1, HLog.getSyncTime().count);
751   
752       // Make sure we still hold lock
753       assertTrue(region.isRowLocked(lockedRow));
754       LOG.info("...releasing lock");
755       region.releaseRowLock(lockedRow);
756     } finally {
757       HRegion.closeHRegion(this.region);
758        this.region = null;
759     }
760   }
761 
762   public void testBatchPutWithTsSlop() throws Exception {
763     byte[] b = Bytes.toBytes(getName());
764     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
765     byte[] qual = Bytes.toBytes("qual");
766     byte[] val = Bytes.toBytes("val");
767     Configuration conf = HBaseConfiguration.create(this.conf);
768 
769     // add data with a timestamp that is too recent for range. Ensure assert
770     conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
771     this.region = initHRegion(b, getName(), conf, cf);
772 
773     try{
774       HLog.getSyncTime(); // clear counter from prior tests
775       assertEquals(0, HLog.getSyncTime().count);
776 
777       final Put[] puts = new Put[10];
778       for (int i = 0; i < 10; i++) {
779         puts[i] = new Put(Bytes.toBytes("row_" + i), Long.MAX_VALUE - 100);
780         puts[i].add(cf, qual, val);
781       }
782 
783       OperationStatus[] codes = this.region.put(puts);
784       assertEquals(10, codes.length);
785       for (int i = 0; i < 10; i++) {
786         assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i]
787             .getOperationStatusCode());
788       }
789       assertEquals(0, HLog.getSyncTime().count);
790 
791 
792     } finally {
793       HRegion.closeHRegion(this.region);
794       this.region = null;
795     }
796 
797   }
798 
799   //////////////////////////////////////////////////////////////////////////////
800   // checkAndMutate tests
801   //////////////////////////////////////////////////////////////////////////////
802   public void testCheckAndMutate_WithEmptyRowValue() throws IOException {
803     byte [] tableName = Bytes.toBytes("testtable");
804     byte [] row1 = Bytes.toBytes("row1");
805     byte [] fam1 = Bytes.toBytes("fam1");
806     byte [] qf1  = Bytes.toBytes("qualifier");
807     byte [] emptyVal  = new byte[] {};
808     byte [] val1  = Bytes.toBytes("value1");
809     byte [] val2  = Bytes.toBytes("value2");
810     Integer lockId = null;
811 
812     //Setting up region
813     String method = this.getName();
814     this.region = initHRegion(tableName, method, conf, fam1);
815     try {
816       //Putting empty data in key
817       Put put = new Put(row1);
818       put.add(fam1, qf1, emptyVal);
819 
820       //checkAndPut with empty value
821       boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
822           new BinaryComparator(emptyVal), put, lockId, true);
823       assertTrue(res);
824 
825       //Putting data in key
826       put = new Put(row1);
827       put.add(fam1, qf1, val1);
828 
829       //checkAndPut with correct value
830       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
831           new BinaryComparator(emptyVal), put, lockId, true);
832       assertTrue(res);
833 
834       // not empty anymore
835       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
836           new BinaryComparator(emptyVal), put, lockId, true);
837       assertFalse(res);
838 
839       Delete delete = new Delete(row1);
840       delete.deleteColumn(fam1, qf1);
841       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
842           new BinaryComparator(emptyVal), delete, lockId, true);
843       assertFalse(res);
844 
845       put = new Put(row1);
846       put.add(fam1, qf1, val2);
847       //checkAndPut with correct value
848       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
849           new BinaryComparator(val1), put, lockId, true);
850       assertTrue(res);
851 
852       //checkAndDelete with correct value
853       delete = new Delete(row1);
854       delete.deleteColumn(fam1, qf1);
855       delete.deleteColumn(fam1, qf1);
856       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
857           new BinaryComparator(val2), delete, lockId, true);
858       assertTrue(res);
859 
860       delete = new Delete(row1);
861       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
862           new BinaryComparator(emptyVal), delete, lockId, true);
863       assertTrue(res);
864 
865       //checkAndPut looking for a null value
866       put = new Put(row1);
867       put.add(fam1, qf1, val1);
868 
869       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
870           new NullComparator(), put, lockId, true);
871       assertTrue(res);
872     } finally {
873       HRegion.closeHRegion(this.region);
874       this.region = null;
875     }
876   }
877 
878   public void testCheckAndMutate_WithWrongValue() throws IOException{
879     byte [] tableName = Bytes.toBytes("testtable");
880     byte [] row1 = Bytes.toBytes("row1");
881     byte [] fam1 = Bytes.toBytes("fam1");
882     byte [] qf1  = Bytes.toBytes("qualifier");
883     byte [] val1  = Bytes.toBytes("value1");
884     byte [] val2  = Bytes.toBytes("value2");
885     Integer lockId = null;
886 
887     //Setting up region
888     String method = this.getName();
889     this.region = initHRegion(tableName, method, conf, fam1);
890     try {
891       //Putting data in key
892       Put put = new Put(row1);
893       put.add(fam1, qf1, val1);
894       region.put(put);
895 
896       //checkAndPut with wrong value
897       boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
898           new BinaryComparator(val2), put, lockId, true);
899       assertEquals(false, res);
900 
901       //checkAndDelete with wrong value
902       Delete delete = new Delete(row1);
903       delete.deleteFamily(fam1);
904       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
905           new BinaryComparator(val2), delete, lockId, true);
906       assertEquals(false, res);
907     } finally {
908       HRegion.closeHRegion(this.region);
909       this.region = null;
910     }
911   }
912 
913   public void testCheckAndMutate_WithCorrectValue() throws IOException{
914     byte [] tableName = Bytes.toBytes("testtable");
915     byte [] row1 = Bytes.toBytes("row1");
916     byte [] fam1 = Bytes.toBytes("fam1");
917     byte [] qf1  = Bytes.toBytes("qualifier");
918     byte [] val1  = Bytes.toBytes("value1");
919     Integer lockId = null;
920 
921     //Setting up region
922     String method = this.getName();
923     this.region = initHRegion(tableName, method, conf, fam1);
924     try {
925       //Putting data in key
926       Put put = new Put(row1);
927       put.add(fam1, qf1, val1);
928       region.put(put);
929 
930       //checkAndPut with correct value
931       boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
932           new BinaryComparator(val1), put, lockId, true);
933       assertEquals(true, res);
934 
935       //checkAndDelete with correct value
936       Delete delete = new Delete(row1);
937       delete.deleteColumn(fam1, qf1);
938       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
939           new BinaryComparator(val1), put, lockId, true);
940       assertEquals(true, res);
941     } finally {
942       HRegion.closeHRegion(this.region);
943       this.region = null;
944     }
945   }
946 
947   public void testCheckAndPut_ThatPutWasWritten() throws IOException{
948     byte [] tableName = Bytes.toBytes("testtable");
949     byte [] row1 = Bytes.toBytes("row1");
950     byte [] fam1 = Bytes.toBytes("fam1");
951     byte [] fam2 = Bytes.toBytes("fam2");
952     byte [] qf1  = Bytes.toBytes("qualifier");
953     byte [] val1  = Bytes.toBytes("value1");
954     byte [] val2  = Bytes.toBytes("value2");
955     Integer lockId = null;
956 
957     byte [][] families = {fam1, fam2};
958 
959     //Setting up region
960     String method = this.getName();
961     this.region = initHRegion(tableName, method, conf, families);
962     try {
963       //Putting data in the key to check
964       Put put = new Put(row1);
965       put.add(fam1, qf1, val1);
966       region.put(put);
967 
968       //Creating put to add
969       long ts = System.currentTimeMillis();
970       KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
971       put = new Put(row1);
972       put.add(kv);
973 
974       //checkAndPut with wrong value
975       Store store = region.getStore(fam1);
976       store.memstore.kvset.size();
977 
978       boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
979           new BinaryComparator(val1), put, lockId, true);
980       assertEquals(true, res);
981       store.memstore.kvset.size();
982 
983       Get get = new Get(row1);
984       get.addColumn(fam2, qf1);
985       KeyValue [] actual = region.get(get, null).raw();
986 
987       KeyValue [] expected = {kv};
988 
989       assertEquals(expected.length, actual.length);
990       for(int i=0; i<actual.length; i++) {
991         assertEquals(expected[i], actual[i]);
992       }
993     } finally {
994       HRegion.closeHRegion(this.region);
995       this.region = null;
996     }
997   }
998 
999   public void testCheckAndPut_wrongRowInPut() throws IOException {
1000     this.region = initHRegion(tableName, this.getName(), conf, COLUMNS);
1001     try {
1002       Put put = new Put(row2);
1003       put.add(fam1, qual1, value1);
1004       try {
1005         boolean res = region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
1006             new BinaryComparator(value2), put, null, false);
1007         fail();
1008       } catch (DoNotRetryIOException expected) {
1009         // expected exception.
1010       }
1011     } finally {
1012       HRegion.closeHRegion(this.region);
1013       this.region = null;
1014     }
1015   }
1016 
1017   public void testmutateRowsWithLocks_wrongCF() throws IOException {
1018     this.region = initHRegion(tableName, this.getName(), conf, fam1, fam2);
1019     try {
1020       Put put = new Put(row2);
1021       put.add(fam3, qual1, value1);
1022       RowMutations rm = new RowMutations(row2);
1023       rm.add(put);
1024       try {
1025         region.mutateRow(rm);
1026         fail();
1027       } catch (DoNotRetryIOException expected) {
1028         // expected exception.
1029         LOG.debug("Caught expected exception: " + expected.getMessage());
1030       }
1031     } finally {
1032       HRegion.closeHRegion(this.region);
1033       this.region = null;
1034     }
1035   }
1036   
1037   public void testCheckAndDelete_ThatDeleteWasWritten() throws IOException{
1038     byte [] tableName = Bytes.toBytes("testtable");
1039     byte [] row1 = Bytes.toBytes("row1");
1040     byte [] fam1 = Bytes.toBytes("fam1");
1041     byte [] fam2 = Bytes.toBytes("fam2");
1042     byte [] qf1  = Bytes.toBytes("qualifier1");
1043     byte [] qf2  = Bytes.toBytes("qualifier2");
1044     byte [] qf3  = Bytes.toBytes("qualifier3");
1045     byte [] val1  = Bytes.toBytes("value1");
1046     byte [] val2  = Bytes.toBytes("value2");
1047     byte [] val3  = Bytes.toBytes("value3");
1048     byte[] emptyVal = new byte[] { };
1049     Integer lockId = null;
1050 
1051     byte [][] families = {fam1, fam2};
1052 
1053     //Setting up region
1054     String method = this.getName();
1055     this.region = initHRegion(tableName, method, conf, families);
1056     try {
1057       //Put content
1058       Put put = new Put(row1);
1059       put.add(fam1, qf1, val1);
1060       region.put(put);
1061       Threads.sleep(2);
1062 
1063       put = new Put(row1);
1064       put.add(fam1, qf1, val2);
1065       put.add(fam2, qf1, val3);
1066       put.add(fam2, qf2, val2);
1067       put.add(fam2, qf3, val1);
1068       put.add(fam1, qf3, val1);
1069       region.put(put);
1070 
1071       //Multi-column delete
1072       Delete delete = new Delete(row1);
1073       delete.deleteColumn(fam1, qf1);
1074       delete.deleteColumn(fam2, qf1);
1075       delete.deleteColumn(fam1, qf3);
1076       boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
1077           new BinaryComparator(val2), delete, lockId, true);
1078       assertEquals(true, res);
1079 
1080       Get get = new Get(row1);
1081       get.addColumn(fam1, qf1);
1082       get.addColumn(fam1, qf3);
1083       get.addColumn(fam2, qf2);
1084       Result r = region.get(get, null);
1085       assertEquals(2, r.size());
1086       assertEquals(val1, r.getValue(fam1, qf1));
1087       assertEquals(val2, r.getValue(fam2, qf2));
1088 
1089       //Family delete
1090       delete = new Delete(row1);
1091       delete.deleteFamily(fam2);
1092       res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL,
1093           new BinaryComparator(emptyVal), delete, lockId, true);
1094       assertEquals(true, res);
1095 
1096       get = new Get(row1);
1097       r = region.get(get, null);
1098       assertEquals(1, r.size());
1099       assertEquals(val1, r.getValue(fam1, qf1));
1100 
1101       //Row delete
1102       delete = new Delete(row1);
1103       res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
1104           new BinaryComparator(val1), delete, lockId, true);
1105       assertEquals(true, res);
1106       get = new Get(row1);
1107       r = region.get(get, null);
1108       assertEquals(0, r.size());
1109     } finally {
1110       HRegion.closeHRegion(this.region);
1111       this.region = null;
1112     }
1113   }
1114 
1115   //////////////////////////////////////////////////////////////////////////////
1116   // Delete tests
1117   //////////////////////////////////////////////////////////////////////////////
1118   public void testDelete_multiDeleteColumn() throws IOException {
1119     byte [] tableName = Bytes.toBytes("testtable");
1120     byte [] row1 = Bytes.toBytes("row1");
1121     byte [] fam1 = Bytes.toBytes("fam1");
1122     byte [] qual = Bytes.toBytes("qualifier");
1123     byte [] value = Bytes.toBytes("value");
1124 
1125     Put put = new Put(row1);
1126     put.add(fam1, qual, 1, value);
1127     put.add(fam1, qual, 2, value);
1128 
1129     String method = this.getName();
1130     this.region = initHRegion(tableName, method, conf, fam1);
1131     try {
1132       region.put(put);
1133 
1134       // We do support deleting more than 1 'latest' version
1135       Delete delete = new Delete(row1);
1136       delete.deleteColumn(fam1, qual);
1137       delete.deleteColumn(fam1, qual);
1138       region.delete(delete, null, false);
1139 
1140       Get get = new Get(row1);
1141       get.addFamily(fam1);
1142       Result r = region.get(get, null);
1143       assertEquals(0, r.size());
1144     } finally {
1145       HRegion.closeHRegion(this.region);
1146       this.region = null;
1147     }
1148   }
1149 
1150   public void testDelete_CheckFamily() throws IOException {
1151     byte [] tableName = Bytes.toBytes("testtable");
1152     byte [] row1 = Bytes.toBytes("row1");
1153     byte [] fam1 = Bytes.toBytes("fam1");
1154     byte [] fam2 = Bytes.toBytes("fam2");
1155     byte [] fam3 = Bytes.toBytes("fam3");
1156     byte [] fam4 = Bytes.toBytes("fam4");
1157 
1158     //Setting up region
1159     String method = this.getName();
1160     this.region = initHRegion(tableName, method, conf, fam1, fam2, fam3);
1161     try {
1162       List<KeyValue> kvs  = new ArrayList<KeyValue>();
1163       kvs.add(new KeyValue(row1, fam4, null, null));
1164 
1165 
1166       //testing existing family
1167       byte [] family = fam2;
1168       try {
1169         Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
1170         deleteMap.put(family, kvs);
1171         region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
1172       } catch (Exception e) {
1173         assertTrue("Family " +new String(family)+ " does not exist", false);
1174       }
1175 
1176       //testing non existing family
1177       boolean ok = false;
1178       family = fam4;
1179       try {
1180         Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
1181         deleteMap.put(family, kvs);
1182         region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
1183       } catch (Exception e) {
1184         ok = true;
1185       }
1186       assertEquals("Family " +new String(family)+ " does exist", true, ok);
1187     } finally {
1188       HRegion.closeHRegion(this.region);
1189       this.region = null;
1190     }
1191   }
1192 
1193   public void testDelete_mixed() throws IOException, InterruptedException {
1194     byte [] tableName = Bytes.toBytes("testtable");
1195     byte [] fam = Bytes.toBytes("info");
1196     byte [][] families = {fam};
1197     String method = this.getName();
1198     this.region = initHRegion(tableName, method, conf, families);
1199     try {
1200       EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
1201 
1202       byte [] row = Bytes.toBytes("table_name");
1203       // column names
1204       byte [] serverinfo = Bytes.toBytes("serverinfo");
1205       byte [] splitA = Bytes.toBytes("splitA");
1206       byte [] splitB = Bytes.toBytes("splitB");
1207 
1208       // add some data:
1209       Put put = new Put(row);
1210       put.add(fam, splitA, Bytes.toBytes("reference_A"));
1211       region.put(put);
1212 
1213       put = new Put(row);
1214       put.add(fam, splitB, Bytes.toBytes("reference_B"));
1215       region.put(put);
1216 
1217       put = new Put(row);
1218       put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
1219       region.put(put);
1220 
1221       // ok now delete a split:
1222       Delete delete = new Delete(row);
1223       delete.deleteColumns(fam, splitA);
1224       region.delete(delete, null, true);
1225 
1226       // assert some things:
1227       Get get = new Get(row).addColumn(fam, serverinfo);
1228       Result result = region.get(get, null);
1229       assertEquals(1, result.size());
1230 
1231       get = new Get(row).addColumn(fam, splitA);
1232       result = region.get(get, null);
1233       assertEquals(0, result.size());
1234 
1235       get = new Get(row).addColumn(fam, splitB);
1236       result = region.get(get, null);
1237       assertEquals(1, result.size());
1238 
1239       // Assert that after a delete, I can put.
1240       put = new Put(row);
1241       put.add(fam, splitA, Bytes.toBytes("reference_A"));
1242       region.put(put);
1243       get = new Get(row);
1244       result = region.get(get, null);
1245       assertEquals(3, result.size());
1246 
1247       // Now delete all... then test I can add stuff back
1248       delete = new Delete(row);
1249       region.delete(delete, null, false);
1250       assertEquals(0, region.get(get, null).size());
1251 
1252       region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
1253       result = region.get(get, null);
1254       assertEquals(1, result.size());
1255     } finally {
1256       HRegion.closeHRegion(this.region);
1257       this.region = null;
1258     }
1259   }
1260 
1261   public void testDeleteRowWithFutureTs() throws IOException {
1262     byte [] tableName = Bytes.toBytes("testtable");
1263     byte [] fam = Bytes.toBytes("info");
1264     byte [][] families = {fam};
1265     String method = this.getName();
1266     this.region = initHRegion(tableName, method, conf, families);
1267     try {
1268       byte [] row = Bytes.toBytes("table_name");
1269       // column names
1270       byte [] serverinfo = Bytes.toBytes("serverinfo");
1271 
1272       // add data in the far future
1273       Put put = new Put(row);
1274       put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP-5,Bytes.toBytes("value"));
1275       region.put(put);
1276 
1277       // now delete something in the present
1278       Delete delete = new Delete(row);
1279       region.delete(delete, null, true);
1280 
1281       // make sure we still see our data
1282       Get get = new Get(row).addColumn(fam, serverinfo);
1283       Result result = region.get(get, null);
1284       assertEquals(1, result.size());
1285 
1286       // delete the future row
1287       delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
1288       region.delete(delete, null, true);
1289 
1290       // make sure it is gone
1291       get = new Get(row).addColumn(fam, serverinfo);
1292       result = region.get(get, null);
1293       assertEquals(0, result.size());
1294     } finally {
1295       HRegion.closeHRegion(this.region);
1296       this.region = null;
1297     }
1298   }
1299 
1300   /**
1301    * Tests that the special LATEST_TIMESTAMP option for puts gets
1302    * replaced by the actual timestamp
1303    */
1304   public void testPutWithLatestTS() throws IOException {
1305     byte [] tableName = Bytes.toBytes("testtable");
1306     byte [] fam = Bytes.toBytes("info");
1307     byte [][] families = {fam};
1308     String method = this.getName();
1309     this.region = initHRegion(tableName, method, conf, families);
1310     try {
1311       byte [] row = Bytes.toBytes("row1");
1312       // column names
1313       byte [] qual = Bytes.toBytes("qual");
1314 
1315       // add data with LATEST_TIMESTAMP, put without WAL
1316       Put put = new Put(row);
1317       put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
1318       region.put(put, false);
1319 
1320       // Make sure it shows up with an actual timestamp
1321       Get get = new Get(row).addColumn(fam, qual);
1322       Result result = region.get(get, null);
1323       assertEquals(1, result.size());
1324       KeyValue kv = result.raw()[0];
1325       LOG.info("Got: " + kv);
1326       assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
1327           kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
1328 
1329       // Check same with WAL enabled (historically these took different
1330       // code paths, so check both)
1331       row = Bytes.toBytes("row2");
1332       put = new Put(row);
1333       put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
1334       region.put(put, true);
1335 
1336       // Make sure it shows up with an actual timestamp
1337       get = new Get(row).addColumn(fam, qual);
1338       result = region.get(get, null);
1339       assertEquals(1, result.size());
1340       kv = result.raw()[0];
1341       LOG.info("Got: " + kv);
1342       assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
1343           kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
1344     } finally {
1345       HRegion.closeHRegion(this.region);
1346       this.region = null;
1347     }
1348 
1349   }
1350 
1351 
1352   /**
1353    * Tests that there is server-side filtering for invalid timestamp upper
1354    * bound. Note that the timestamp lower bound is automatically handled for us
1355    * by the TTL field.
1356    */
1357   public void testPutWithTsSlop() throws IOException {
1358     byte[] tableName = Bytes.toBytes("testtable");
1359     byte[] fam = Bytes.toBytes("info");
1360     byte[][] families = { fam };
1361     String method = this.getName();
1362     Configuration conf = HBaseConfiguration.create(this.conf);
1363 
1364     // add data with a timestamp that is too recent for range. Ensure assert
1365     conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
1366     this.region = initHRegion(tableName, method, conf, families);
1367     boolean caughtExcep = false;
1368     try {
1369       try {
1370         // no TS specified == use latest. should not error
1371         region.put(new Put(row).add(fam, Bytes.toBytes("qual"), Bytes
1372             .toBytes("value")), false);
1373         // TS out of range. should error
1374         region.put(new Put(row).add(fam, Bytes.toBytes("qual"),
1375             System.currentTimeMillis() + 2000,
1376             Bytes.toBytes("value")), false);
1377         fail("Expected IOE for TS out of configured timerange");
1378       } catch (DoNotRetryIOException ioe) {
1379         LOG.debug("Received expected exception", ioe);
1380         caughtExcep = true;
1381       }
1382       assertTrue("Should catch FailedSanityCheckException", caughtExcep);
1383     } finally {
1384       HRegion.closeHRegion(this.region);
1385       this.region = null;
1386     }
1387   }
1388 
1389   public void testScanner_DeleteOneFamilyNotAnother() throws IOException {
1390     byte [] tableName = Bytes.toBytes("test_table");
1391     byte [] fam1 = Bytes.toBytes("columnA");
1392     byte [] fam2 = Bytes.toBytes("columnB");
1393     this.region = initHRegion(tableName, getName(), conf, fam1, fam2);
1394     try {
1395       byte [] rowA = Bytes.toBytes("rowA");
1396       byte [] rowB = Bytes.toBytes("rowB");
1397 
1398       byte [] value = Bytes.toBytes("value");
1399 
1400       Delete delete = new Delete(rowA);
1401       delete.deleteFamily(fam1);
1402 
1403       region.delete(delete, null, true);
1404 
1405       // now create data.
1406       Put put = new Put(rowA);
1407       put.add(fam2, null, value);
1408       region.put(put);
1409 
1410       put = new Put(rowB);
1411       put.add(fam1, null, value);
1412       put.add(fam2, null, value);
1413       region.put(put);
1414 
1415       Scan scan = new Scan();
1416       scan.addFamily(fam1).addFamily(fam2);
1417       InternalScanner s = region.getScanner(scan);
1418       List<KeyValue> results = new ArrayList<KeyValue>();
1419       s.next(results);
1420       assertTrue(Bytes.equals(rowA, results.get(0).getRow()));
1421 
1422       results.clear();
1423       s.next(results);
1424       assertTrue(Bytes.equals(rowB, results.get(0).getRow()));
1425     } finally {
1426       HRegion.closeHRegion(this.region);
1427       this.region = null;
1428     }
1429   }
1430 
1431   public void testDeleteColumns_PostInsert() throws IOException,
1432       InterruptedException {
1433     Delete delete = new Delete(row);
1434     delete.deleteColumns(fam1, qual1);
1435     doTestDelete_AndPostInsert(delete);
1436   }
1437 
1438   public void testDeleteFamily_PostInsert() throws IOException, InterruptedException {
1439     Delete delete = new Delete(row);
1440     delete.deleteFamily(fam1);
1441     doTestDelete_AndPostInsert(delete);
1442   }
1443 
1444   public void doTestDelete_AndPostInsert(Delete delete)
1445       throws IOException, InterruptedException {
1446     this.region = initHRegion(tableName, getName(), conf, fam1);
1447     try {
1448       EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
1449       Put put = new Put(row);
1450       put.add(fam1, qual1, value1);
1451       region.put(put);
1452 
1453       // now delete the value:
1454       region.delete(delete, null, true);
1455 
1456 
1457       // ok put data:
1458       put = new Put(row);
1459       put.add(fam1, qual1, value2);
1460       region.put(put);
1461 
1462       // ok get:
1463       Get get = new Get(row);
1464       get.addColumn(fam1, qual1);
1465 
1466       Result r = region.get(get, null);
1467       assertEquals(1, r.size());
1468       assertByteEquals(value2, r.getValue(fam1, qual1));
1469 
1470       // next:
1471       Scan scan = new Scan(row);
1472       scan.addColumn(fam1, qual1);
1473       InternalScanner s = region.getScanner(scan);
1474 
1475       List<KeyValue> results = new ArrayList<KeyValue>();
1476       assertEquals(false, s.next(results));
1477       assertEquals(1, results.size());
1478       KeyValue kv = results.get(0);
1479 
1480       assertByteEquals(value2, kv.getValue());
1481       assertByteEquals(fam1, kv.getFamily());
1482       assertByteEquals(qual1, kv.getQualifier());
1483       assertByteEquals(row, kv.getRow());
1484     } finally {
1485       HRegion.closeHRegion(this.region);
1486       this.region = null;
1487     }
1488   }
1489 
1490   public void testDelete_CheckTimestampUpdated()
1491   throws IOException {
1492     byte [] row1 = Bytes.toBytes("row1");
1493     byte [] col1 = Bytes.toBytes("col1");
1494     byte [] col2 = Bytes.toBytes("col2");
1495     byte [] col3 = Bytes.toBytes("col3");
1496 
1497     //Setting up region
1498     String method = this.getName();
1499     this.region = initHRegion(tableName, method, conf, fam1);
1500     try {
1501       //Building checkerList
1502       List<KeyValue> kvs  = new ArrayList<KeyValue>();
1503       kvs.add(new KeyValue(row1, fam1, col1, null));
1504       kvs.add(new KeyValue(row1, fam1, col2, null));
1505       kvs.add(new KeyValue(row1, fam1, col3, null));
1506 
1507       Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
1508       deleteMap.put(fam1, kvs);
1509       region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
1510 
1511       // extract the key values out the memstore:
1512       // This is kinda hacky, but better than nothing...
1513       long now = System.currentTimeMillis();
1514       KeyValue firstKv = region.getStore(fam1).memstore.kvset.first();
1515       assertTrue(firstKv.getTimestamp() <= now);
1516       now = firstKv.getTimestamp();
1517       for (KeyValue kv: region.getStore(fam1).memstore.kvset) {
1518         assertTrue(kv.getTimestamp() <= now);
1519         now = kv.getTimestamp();
1520       }
1521     } finally {
1522       HRegion.closeHRegion(this.region);
1523       this.region = null;
1524     }
1525   }
1526 
1527   //////////////////////////////////////////////////////////////////////////////
1528   // Get tests
1529   //////////////////////////////////////////////////////////////////////////////
1530   public void testGet_FamilyChecker() throws IOException {
1531     byte [] tableName = Bytes.toBytes("testtable");
1532     byte [] row1 = Bytes.toBytes("row1");
1533     byte [] fam1 = Bytes.toBytes("fam1");
1534     byte [] fam2 = Bytes.toBytes("False");
1535     byte [] col1 = Bytes.toBytes("col1");
1536 
1537     //Setting up region
1538     String method = this.getName();
1539     this.region = initHRegion(tableName, method, conf, fam1);
1540     try {
1541       Get get = new Get(row1);
1542       get.addColumn(fam2, col1);
1543 
1544       //Test
1545       try {
1546         region.get(get, null);
1547       } catch (DoNotRetryIOException e) {
1548         assertFalse(false);
1549         return;
1550       }
1551       assertFalse(true);
1552     } finally {
1553       HRegion.closeHRegion(this.region);
1554       this.region = null;
1555     }
1556   }
1557 
1558   public void testGet_Basic() throws IOException {
1559     byte [] tableName = Bytes.toBytes("testtable");
1560     byte [] row1 = Bytes.toBytes("row1");
1561     byte [] fam1 = Bytes.toBytes("fam1");
1562     byte [] col1 = Bytes.toBytes("col1");
1563     byte [] col2 = Bytes.toBytes("col2");
1564     byte [] col3 = Bytes.toBytes("col3");
1565     byte [] col4 = Bytes.toBytes("col4");
1566     byte [] col5 = Bytes.toBytes("col5");
1567 
1568     //Setting up region
1569     String method = this.getName();
1570     this.region = initHRegion(tableName, method, conf, fam1);
1571     try {
1572       //Add to memstore
1573       Put put = new Put(row1);
1574       put.add(fam1, col1, null);
1575       put.add(fam1, col2, null);
1576       put.add(fam1, col3, null);
1577       put.add(fam1, col4, null);
1578       put.add(fam1, col5, null);
1579       region.put(put);
1580 
1581       Get get = new Get(row1);
1582       get.addColumn(fam1, col2);
1583       get.addColumn(fam1, col4);
1584       //Expected result
1585       KeyValue kv1 = new KeyValue(row1, fam1, col2);
1586       KeyValue kv2 = new KeyValue(row1, fam1, col4);
1587       KeyValue [] expected = {kv1, kv2};
1588 
1589       //Test
1590       Result res = region.get(get, null);
1591       assertEquals(expected.length, res.size());
1592       for(int i=0; i<res.size(); i++){
1593         assertEquals(0,
1594             Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1595         assertEquals(0,
1596             Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1597         assertEquals(0,
1598             Bytes.compareTo(
1599                 expected[i].getQualifier(), res.raw()[i].getQualifier()));
1600       }
1601 
1602       // Test using a filter on a Get
1603       Get g = new Get(row1);
1604       final int count = 2;
1605       g.setFilter(new ColumnCountGetFilter(count));
1606       res = region.get(g, null);
1607       assertEquals(count, res.size());
1608     } finally {
1609       HRegion.closeHRegion(this.region);
1610       this.region = null;
1611     }
1612   }
1613 
1614   public void testGet_Empty() throws IOException {
1615     byte [] tableName = Bytes.toBytes("emptytable");
1616     byte [] row = Bytes.toBytes("row");
1617     byte [] fam = Bytes.toBytes("fam");
1618 
1619     String method = this.getName();
1620     this.region = initHRegion(tableName, method, conf, fam);
1621     try {
1622       Get get = new Get(row);
1623       get.addFamily(fam);
1624       Result r = region.get(get, null);
1625 
1626       assertTrue(r.isEmpty());
1627     } finally {
1628       HRegion.closeHRegion(this.region);
1629       this.region = null;
1630     }
1631   }
1632 
1633   //Test that checked if there was anything special when reading from the ROOT
1634   //table. To be able to use this test you need to comment the part in
1635   //HTableDescriptor that checks for '-' and '.'. You also need to remove the
1636   //s in the beginning of the name.
1637   public void stestGet_Root() throws IOException {
1638     //Setting up region
1639     String method = this.getName();
1640     this.region = initHRegion(HConstants.ROOT_TABLE_NAME,
1641       method, conf, HConstants.CATALOG_FAMILY);
1642     try {
1643       //Add to memstore
1644       Put put = new Put(HConstants.EMPTY_START_ROW);
1645       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, null);
1646       region.put(put);
1647 
1648       Get get = new Get(HConstants.EMPTY_START_ROW);
1649       get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1650 
1651       //Expected result
1652       KeyValue kv1 = new KeyValue(HConstants.EMPTY_START_ROW,
1653           HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1654       KeyValue [] expected = {kv1};
1655 
1656       //Test from memstore
1657       Result res = region.get(get, null);
1658 
1659       assertEquals(expected.length, res.size());
1660       for(int i=0; i<res.size(); i++){
1661         assertEquals(0,
1662             Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1663         assertEquals(0,
1664             Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1665         assertEquals(0,
1666             Bytes.compareTo(
1667                 expected[i].getQualifier(), res.raw()[i].getQualifier()));
1668       }
1669 
1670       //flush
1671       region.flushcache();
1672 
1673       //test2
1674       res = region.get(get, null);
1675 
1676       assertEquals(expected.length, res.size());
1677       for(int i=0; i<res.size(); i++){
1678         assertEquals(0,
1679             Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1680         assertEquals(0,
1681             Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1682         assertEquals(0,
1683             Bytes.compareTo(
1684                 expected[i].getQualifier(), res.raw()[i].getQualifier()));
1685       }
1686 
1687       //Scan
1688       Scan scan = new Scan();
1689       scan.addFamily(HConstants.CATALOG_FAMILY);
1690       InternalScanner s = region.getScanner(scan);
1691       List<KeyValue> result = new ArrayList<KeyValue>();
1692       s.next(result);
1693 
1694       assertEquals(expected.length, result.size());
1695       for(int i=0; i<res.size(); i++){
1696         assertEquals(0,
1697             Bytes.compareTo(expected[i].getRow(), result.get(i).getRow()));
1698         assertEquals(0,
1699             Bytes.compareTo(expected[i].getFamily(), result.get(i).getFamily()));
1700         assertEquals(0,
1701             Bytes.compareTo(
1702                 expected[i].getQualifier(), result.get(i).getQualifier()));
1703       }
1704     } finally {
1705       HRegion.closeHRegion(this.region);
1706       this.region = null;
1707     }
1708   }
1709 
1710   //////////////////////////////////////////////////////////////////////////////
1711   // Lock test
1712   //////////////////////////////////////////////////////////////////////////////
1713   public void testLocks() throws IOException{
1714     byte [] tableName = Bytes.toBytes("testtable");
1715     byte [][] families = {fam1, fam2, fam3};
1716 
1717     Configuration hc = initSplit();
1718     //Setting up region
1719     String method = this.getName();
1720     this.region = initHRegion(tableName, method, hc, families);
1721     try {
1722       final int threadCount = 10;
1723       final int lockCount = 10;
1724 
1725       List<Thread>threads = new ArrayList<Thread>(threadCount);
1726       for (int i = 0; i < threadCount; i++) {
1727         threads.add(new Thread(Integer.toString(i)) {
1728           @Override
1729           public void run() {
1730             Integer [] lockids = new Integer[lockCount];
1731             // Get locks.
1732             for (int i = 0; i < lockCount; i++) {
1733               try {
1734                 byte [] rowid = Bytes.toBytes(Integer.toString(i));
1735                 lockids[i] = region.obtainRowLock(rowid);
1736                 assertEquals(rowid, region.getRowFromLock(lockids[i]));
1737                 LOG.debug(getName() + " locked " + Bytes.toString(rowid));
1738               } catch (IOException e) {
1739                 e.printStackTrace();
1740               }
1741             }
1742             LOG.debug(getName() + " set " +
1743                 Integer.toString(lockCount) + " locks");
1744 
1745             // Abort outstanding locks.
1746             for (int i = lockCount - 1; i >= 0; i--) {
1747               region.releaseRowLock(lockids[i]);
1748               LOG.debug(getName() + " unlocked " + i);
1749             }
1750             LOG.debug(getName() + " released " +
1751                 Integer.toString(lockCount) + " locks");
1752           }
1753         });
1754       }
1755 
1756       // Startup all our threads.
1757       for (Thread t : threads) {
1758         t.start();
1759       }
1760 
1761       // Now wait around till all are done.
1762       for (Thread t: threads) {
1763         while (t.isAlive()) {
1764           try {
1765             Thread.sleep(1);
1766           } catch (InterruptedException e) {
1767             // Go around again.
1768           }
1769         }
1770       }
1771       LOG.info("locks completed.");
1772     } finally {
1773       HRegion.closeHRegion(this.region);
1774       this.region = null;
1775     }
1776   }
1777 
1778   //////////////////////////////////////////////////////////////////////////////
1779   // Merge test
1780   //////////////////////////////////////////////////////////////////////////////
1781   public void testMerge() throws IOException {
1782     byte [] tableName = Bytes.toBytes("testtable");
1783     byte [][] families = {fam1, fam2, fam3};
1784     Configuration hc = initSplit();
1785     //Setting up region
1786     String method = this.getName();
1787     this.region = initHRegion(tableName, method, hc, families);
1788     try {
1789       LOG.info("" + addContent(region, fam3));
1790       region.flushcache();
1791       region.compactStores();
1792       byte [] splitRow = region.checkSplit();
1793       assertNotNull(splitRow);
1794       LOG.info("SplitRow: " + Bytes.toString(splitRow));
1795       HRegion [] subregions = splitRegion(region, splitRow);
1796       try {
1797         // Need to open the regions.
1798         for (int i = 0; i < subregions.length; i++) {
1799           openClosedRegion(subregions[i]);
1800           subregions[i].compactStores();
1801         }
1802         Path oldRegionPath = region.getRegionDir();
1803         Path oldRegion1 = subregions[0].getRegionDir();
1804         Path oldRegion2 = subregions[1].getRegionDir();
1805         long startTime = System.currentTimeMillis();
1806         region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
1807         LOG.info("Merge regions elapsed time: " +
1808             ((System.currentTimeMillis() - startTime) / 1000.0));
1809         fs.delete(oldRegion1, true);
1810         fs.delete(oldRegion2, true);
1811         fs.delete(oldRegionPath, true);
1812         LOG.info("splitAndMerge completed.");
1813       } finally {
1814         for (int i = 0; i < subregions.length; i++) {
1815           try {
1816             subregions[i].close();
1817             subregions[i].getLog().closeAndDelete();
1818           } catch (IOException e) {
1819             // Ignore.
1820           }
1821         }
1822       }
1823     } finally {
1824       HRegion.closeHRegion(this.region);
1825       this.region = null;
1826     }
1827   }
1828 
1829   /**
1830    * @param parent Region to split.
1831    * @param midkey Key to split around.
1832    * @return The Regions we created.
1833    * @throws IOException
1834    */
1835   HRegion [] splitRegion(final HRegion parent, final byte [] midkey)
1836   throws IOException {
1837     PairOfSameType<HRegion> result = null;
1838     SplitTransaction st = new SplitTransaction(parent, midkey);
1839     // If prepare does not return true, for some reason -- logged inside in
1840     // the prepare call -- we are not ready to split just now.  Just return.
1841     if (!st.prepare()) return null;
1842     try {
1843       result = st.execute(null, null);
1844     } catch (IOException ioe) {
1845       try {
1846         LOG.info("Running rollback of failed split of " +
1847           parent.getRegionNameAsString() + "; " + ioe.getMessage());
1848         st.rollback(null, null);
1849         LOG.info("Successful rollback of failed split of " +
1850           parent.getRegionNameAsString());
1851         return null;
1852       } catch (RuntimeException e) {
1853         // If failed rollback, kill this server to avoid having a hole in table.
1854         LOG.info("Failed rollback of failed split of " +
1855           parent.getRegionNameAsString() + " -- aborting server", e);
1856       }
1857     }
1858     return new HRegion [] {result.getFirst(), result.getSecond()};
1859   }
1860 
1861   //////////////////////////////////////////////////////////////////////////////
1862   // Scanner tests
1863   //////////////////////////////////////////////////////////////////////////////
1864   public void testGetScanner_WithOkFamilies() throws IOException {
1865     byte [] tableName = Bytes.toBytes("testtable");
1866     byte [] fam1 = Bytes.toBytes("fam1");
1867     byte [] fam2 = Bytes.toBytes("fam2");
1868 
1869     byte [][] families = {fam1, fam2};
1870 
1871     //Setting up region
1872     String method = this.getName();
1873     this.region = initHRegion(tableName, method, conf, families);
1874     try {
1875       Scan scan = new Scan();
1876       scan.addFamily(fam1);
1877       scan.addFamily(fam2);
1878       try {
1879         region.getScanner(scan);
1880       } catch (Exception e) {
1881         assertTrue("Families could not be found in Region", false);
1882       }
1883     } finally {
1884       HRegion.closeHRegion(this.region);
1885       this.region = null;
1886     }
1887   }
1888 
1889   public void testGetScanner_WithNotOkFamilies() throws IOException {
1890     byte [] tableName = Bytes.toBytes("testtable");
1891     byte [] fam1 = Bytes.toBytes("fam1");
1892     byte [] fam2 = Bytes.toBytes("fam2");
1893 
1894     byte [][] families = {fam1};
1895 
1896     //Setting up region
1897     String method = this.getName();
1898     this.region = initHRegion(tableName, method, conf, families);
1899     try {
1900       Scan scan = new Scan();
1901       scan.addFamily(fam2);
1902       boolean ok = false;
1903       try {
1904         region.getScanner(scan);
1905       } catch (Exception e) {
1906         ok = true;
1907       }
1908       assertTrue("Families could not be found in Region", ok);
1909     } finally {
1910       HRegion.closeHRegion(this.region);
1911       this.region = null;
1912     }
1913   }
1914 
1915   public void testGetScanner_WithNoFamilies() throws IOException {
1916     byte [] tableName = Bytes.toBytes("testtable");
1917     byte [] row1 = Bytes.toBytes("row1");
1918     byte [] fam1 = Bytes.toBytes("fam1");
1919     byte [] fam2 = Bytes.toBytes("fam2");
1920     byte [] fam3 = Bytes.toBytes("fam3");
1921     byte [] fam4 = Bytes.toBytes("fam4");
1922 
1923     byte [][] families = {fam1, fam2, fam3, fam4};
1924 
1925     //Setting up region
1926     String method = this.getName();
1927     this.region = initHRegion(tableName, method, conf, families);
1928     try {
1929 
1930       //Putting data in Region
1931       Put put = new Put(row1);
1932       put.add(fam1, null, null);
1933       put.add(fam2, null, null);
1934       put.add(fam3, null, null);
1935       put.add(fam4, null, null);
1936       region.put(put);
1937 
1938       Scan scan = null;
1939       HRegion.RegionScannerImpl is = null;
1940 
1941       //Testing to see how many scanners that is produced by getScanner, starting
1942       //with known number, 2 - current = 1
1943       scan = new Scan();
1944       scan.addFamily(fam2);
1945       scan.addFamily(fam4);
1946       is = (RegionScannerImpl) region.getScanner(scan);
1947       MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
1948       assertEquals(1, ((RegionScannerImpl)is).storeHeap.getHeap().size());
1949 
1950       scan = new Scan();
1951       is = (RegionScannerImpl) region.getScanner(scan);
1952       MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
1953       assertEquals(families.length -1,
1954           ((RegionScannerImpl)is).storeHeap.getHeap().size());
1955     } finally {
1956       HRegion.closeHRegion(this.region);
1957       this.region = null;
1958     }
1959   }
1960 
1961   /**
1962    * This method tests https://issues.apache.org/jira/browse/HBASE-2516.
1963    * @throws IOException 
1964    */
1965   public void testGetScanner_WithRegionClosed() throws IOException {
1966     byte[] tableName = Bytes.toBytes("testtable");
1967     byte[] fam1 = Bytes.toBytes("fam1");
1968     byte[] fam2 = Bytes.toBytes("fam2");
1969 
1970     byte[][] families = {fam1, fam2};
1971 
1972     //Setting up region
1973     String method = this.getName();
1974     try {
1975       this.region = initHRegion(tableName, method, conf, families);
1976     } catch (IOException e) {
1977       e.printStackTrace();
1978       fail("Got IOException during initHRegion, " + e.getMessage());
1979     }
1980     try {
1981       region.closed.set(true);
1982       try {
1983         region.getScanner(null);
1984         fail("Expected to get an exception during getScanner on a region that is closed");
1985       } catch (org.apache.hadoop.hbase.NotServingRegionException e) {
1986         //this is the correct exception that is expected
1987       } catch (IOException e) {
1988         fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
1989             + e.getMessage());
1990       }
1991     } finally {
1992       HRegion.closeHRegion(this.region);
1993       this.region = null;
1994     }
1995   }
1996 
1997   public void testRegionScanner_Next() throws IOException {
1998     byte [] tableName = Bytes.toBytes("testtable");
1999     byte [] row1 = Bytes.toBytes("row1");
2000     byte [] row2 = Bytes.toBytes("row2");
2001     byte [] fam1 = Bytes.toBytes("fam1");
2002     byte [] fam2 = Bytes.toBytes("fam2");
2003     byte [] fam3 = Bytes.toBytes("fam3");
2004     byte [] fam4 = Bytes.toBytes("fam4");
2005 
2006     byte [][] families = {fam1, fam2, fam3, fam4};
2007     long ts = System.currentTimeMillis();
2008 
2009     //Setting up region
2010     String method = this.getName();
2011     this.region = initHRegion(tableName, method, conf, families);
2012     try {
2013       //Putting data in Region
2014       Put put = null;
2015       put = new Put(row1);
2016       put.add(fam1, null, ts, null);
2017       put.add(fam2, null, ts, null);
2018       put.add(fam3, null, ts, null);
2019       put.add(fam4, null, ts, null);
2020       region.put(put);
2021 
2022       put = new Put(row2);
2023       put.add(fam1, null, ts, null);
2024       put.add(fam2, null, ts, null);
2025       put.add(fam3, null, ts, null);
2026       put.add(fam4, null, ts, null);
2027       region.put(put);
2028 
2029       Scan scan = new Scan();
2030       scan.addFamily(fam2);
2031       scan.addFamily(fam4);
2032       InternalScanner is = region.getScanner(scan);
2033 
2034       List<KeyValue> res = null;
2035 
2036       //Result 1
2037       List<KeyValue> expected1 = new ArrayList<KeyValue>();
2038       expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
2039       expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
2040 
2041       res = new ArrayList<KeyValue>();
2042       is.next(res);
2043       for(int i=0; i<res.size(); i++) {
2044         assertEquals(expected1.get(i), res.get(i));
2045       }
2046 
2047       //Result 2
2048       List<KeyValue> expected2 = new ArrayList<KeyValue>();
2049       expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
2050       expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
2051 
2052       res = new ArrayList<KeyValue>();
2053       is.next(res);
2054       for(int i=0; i<res.size(); i++) {
2055         assertEquals(expected2.get(i), res.get(i));
2056       }
2057     } finally {
2058       HRegion.closeHRegion(this.region);
2059       this.region = null;
2060     }
2061   }
2062 
2063   public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions()
2064   throws IOException {
2065     byte [] tableName = Bytes.toBytes("testtable");
2066     byte [] row1 = Bytes.toBytes("row1");
2067     byte [] qf1 = Bytes.toBytes("qualifier1");
2068     byte [] qf2 = Bytes.toBytes("qualifier2");
2069     byte [] fam1 = Bytes.toBytes("fam1");
2070     byte [][] families = {fam1};
2071 
2072     long ts1 = System.currentTimeMillis();
2073     long ts2 = ts1 + 1;
2074     long ts3 = ts1 + 2;
2075 
2076     //Setting up region
2077     String method = this.getName();
2078     this.region = initHRegion(tableName, method, conf, families);
2079     try {
2080       //Putting data in Region
2081       Put put = null;
2082       KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2083       KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2084       KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2085 
2086       KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2087       KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2088       KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2089 
2090       put = new Put(row1);
2091       put.add(kv13);
2092       put.add(kv12);
2093       put.add(kv11);
2094       put.add(kv23);
2095       put.add(kv22);
2096       put.add(kv21);
2097       region.put(put);
2098 
2099       //Expected
2100       List<KeyValue> expected = new ArrayList<KeyValue>();
2101       expected.add(kv13);
2102       expected.add(kv12);
2103 
2104       Scan scan = new Scan(row1);
2105       scan.addColumn(fam1, qf1);
2106       scan.setMaxVersions(MAX_VERSIONS);
2107       List<KeyValue> actual = new ArrayList<KeyValue>();
2108       InternalScanner scanner = region.getScanner(scan);
2109 
2110       boolean hasNext = scanner.next(actual);
2111       assertEquals(false, hasNext);
2112 
2113       //Verify result
2114       for(int i=0; i<expected.size(); i++) {
2115         assertEquals(expected.get(i), actual.get(i));
2116       }
2117     } finally {
2118       HRegion.closeHRegion(this.region);
2119       this.region = null;
2120     }
2121   }
2122 
2123   public void testScanner_ExplicitColumns_FromFilesOnly_EnforceVersions()
2124   throws IOException{
2125     byte [] tableName = Bytes.toBytes("testtable");
2126     byte [] row1 = Bytes.toBytes("row1");
2127     byte [] qf1 = Bytes.toBytes("qualifier1");
2128     byte [] qf2 = Bytes.toBytes("qualifier2");
2129     byte [] fam1 = Bytes.toBytes("fam1");
2130     byte [][] families = {fam1};
2131 
2132     long ts1 = 1; //System.currentTimeMillis();
2133     long ts2 = ts1 + 1;
2134     long ts3 = ts1 + 2;
2135 
2136     //Setting up region
2137     String method = this.getName();
2138     this.region = initHRegion(tableName, method, conf, families);
2139     try {
2140       //Putting data in Region
2141       Put put = null;
2142       KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2143       KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2144       KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2145 
2146       KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2147       KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2148       KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2149 
2150       put = new Put(row1);
2151       put.add(kv13);
2152       put.add(kv12);
2153       put.add(kv11);
2154       put.add(kv23);
2155       put.add(kv22);
2156       put.add(kv21);
2157       region.put(put);
2158       region.flushcache();
2159 
2160       //Expected
2161       List<KeyValue> expected = new ArrayList<KeyValue>();
2162       expected.add(kv13);
2163       expected.add(kv12);
2164       expected.add(kv23);
2165       expected.add(kv22);
2166 
2167       Scan scan = new Scan(row1);
2168       scan.addColumn(fam1, qf1);
2169       scan.addColumn(fam1, qf2);
2170       scan.setMaxVersions(MAX_VERSIONS);
2171       List<KeyValue> actual = new ArrayList<KeyValue>();
2172       InternalScanner scanner = region.getScanner(scan);
2173 
2174       boolean hasNext = scanner.next(actual);
2175       assertEquals(false, hasNext);
2176 
2177       //Verify result
2178       for(int i=0; i<expected.size(); i++) {
2179         assertEquals(expected.get(i), actual.get(i));
2180       }
2181     } finally {
2182       HRegion.closeHRegion(this.region);
2183       this.region = null;
2184     }
2185   }
2186 
2187   public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions()
2188   throws IOException {
2189     byte [] tableName = Bytes.toBytes("testtable");
2190     byte [] row1 = Bytes.toBytes("row1");
2191     byte [] fam1 = Bytes.toBytes("fam1");
2192     byte [][] families = {fam1};
2193     byte [] qf1 = Bytes.toBytes("qualifier1");
2194     byte [] qf2 = Bytes.toBytes("qualifier2");
2195 
2196     long ts1 = 1;
2197     long ts2 = ts1 + 1;
2198     long ts3 = ts1 + 2;
2199     long ts4 = ts1 + 3;
2200 
2201     //Setting up region
2202     String method = this.getName();
2203     this.region = initHRegion(tableName, method, conf, families);
2204     try {
2205       //Putting data in Region
2206       KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2207       KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2208       KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2209       KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2210 
2211       KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2212       KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2213       KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2214       KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2215 
2216       Put put = null;
2217       put = new Put(row1);
2218       put.add(kv14);
2219       put.add(kv24);
2220       region.put(put);
2221       region.flushcache();
2222 
2223       put = new Put(row1);
2224       put.add(kv23);
2225       put.add(kv13);
2226       region.put(put);
2227       region.flushcache();
2228 
2229       put = new Put(row1);
2230       put.add(kv22);
2231       put.add(kv12);
2232       region.put(put);
2233       region.flushcache();
2234 
2235       put = new Put(row1);
2236       put.add(kv21);
2237       put.add(kv11);
2238       region.put(put);
2239 
2240       //Expected
2241       List<KeyValue> expected = new ArrayList<KeyValue>();
2242       expected.add(kv14);
2243       expected.add(kv13);
2244       expected.add(kv12);
2245       expected.add(kv24);
2246       expected.add(kv23);
2247       expected.add(kv22);
2248 
2249       Scan scan = new Scan(row1);
2250       scan.addColumn(fam1, qf1);
2251       scan.addColumn(fam1, qf2);
2252       int versions = 3;
2253       scan.setMaxVersions(versions);
2254       List<KeyValue> actual = new ArrayList<KeyValue>();
2255       InternalScanner scanner = region.getScanner(scan);
2256 
2257       boolean hasNext = scanner.next(actual);
2258       assertEquals(false, hasNext);
2259 
2260       //Verify result
2261       for(int i=0; i<expected.size(); i++) {
2262         assertEquals(expected.get(i), actual.get(i));
2263       }
2264     } finally {
2265       HRegion.closeHRegion(this.region);
2266       this.region = null;
2267     }
2268   }
2269 
2270   public void testScanner_Wildcard_FromMemStore_EnforceVersions()
2271   throws IOException {
2272     byte [] tableName = Bytes.toBytes("testtable");
2273     byte [] row1 = Bytes.toBytes("row1");
2274     byte [] qf1 = Bytes.toBytes("qualifier1");
2275     byte [] qf2 = Bytes.toBytes("qualifier2");
2276     byte [] fam1 = Bytes.toBytes("fam1");
2277     byte [][] families = {fam1};
2278 
2279     long ts1 = System.currentTimeMillis();
2280     long ts2 = ts1 + 1;
2281     long ts3 = ts1 + 2;
2282 
2283     //Setting up region
2284     String method = this.getName();
2285     this.region = initHRegion(tableName, method, conf, families);
2286     try {
2287       //Putting data in Region
2288       Put put = null;
2289       KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2290       KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2291       KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2292 
2293       KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2294       KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2295       KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2296 
2297       put = new Put(row1);
2298       put.add(kv13);
2299       put.add(kv12);
2300       put.add(kv11);
2301       put.add(kv23);
2302       put.add(kv22);
2303       put.add(kv21);
2304       region.put(put);
2305 
2306       //Expected
2307       List<KeyValue> expected = new ArrayList<KeyValue>();
2308       expected.add(kv13);
2309       expected.add(kv12);
2310       expected.add(kv23);
2311       expected.add(kv22);
2312 
2313       Scan scan = new Scan(row1);
2314       scan.addFamily(fam1);
2315       scan.setMaxVersions(MAX_VERSIONS);
2316       List<KeyValue> actual = new ArrayList<KeyValue>();
2317       InternalScanner scanner = region.getScanner(scan);
2318 
2319       boolean hasNext = scanner.next(actual);
2320       assertEquals(false, hasNext);
2321 
2322       //Verify result
2323       for(int i=0; i<expected.size(); i++) {
2324         assertEquals(expected.get(i), actual.get(i));
2325       }
2326     } finally {
2327       HRegion.closeHRegion(this.region);
2328       this.region = null;
2329     }
2330   }
2331 
2332   public void testScanner_Wildcard_FromFilesOnly_EnforceVersions()
2333   throws IOException{
2334     byte [] tableName = Bytes.toBytes("testtable");
2335     byte [] row1 = Bytes.toBytes("row1");
2336     byte [] qf1 = Bytes.toBytes("qualifier1");
2337     byte [] qf2 = Bytes.toBytes("qualifier2");
2338     byte [] fam1 = Bytes.toBytes("fam1");
2339 
2340     long ts1 = 1; //System.currentTimeMillis();
2341     long ts2 = ts1 + 1;
2342     long ts3 = ts1 + 2;
2343 
2344     //Setting up region
2345     String method = this.getName();
2346     this.region = initHRegion(tableName, method, conf, fam1);
2347     try {
2348       //Putting data in Region
2349       Put put = null;
2350       KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2351       KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2352       KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2353 
2354       KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2355       KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2356       KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2357 
2358       put = new Put(row1);
2359       put.add(kv13);
2360       put.add(kv12);
2361       put.add(kv11);
2362       put.add(kv23);
2363       put.add(kv22);
2364       put.add(kv21);
2365       region.put(put);
2366       region.flushcache();
2367 
2368       //Expected
2369       List<KeyValue> expected = new ArrayList<KeyValue>();
2370       expected.add(kv13);
2371       expected.add(kv12);
2372       expected.add(kv23);
2373       expected.add(kv22);
2374 
2375       Scan scan = new Scan(row1);
2376       scan.addFamily(fam1);
2377       scan.setMaxVersions(MAX_VERSIONS);
2378       List<KeyValue> actual = new ArrayList<KeyValue>();
2379       InternalScanner scanner = region.getScanner(scan);
2380 
2381       boolean hasNext = scanner.next(actual);
2382       assertEquals(false, hasNext);
2383 
2384       //Verify result
2385       for(int i=0; i<expected.size(); i++) {
2386         assertEquals(expected.get(i), actual.get(i));
2387       }
2388     } finally {
2389       HRegion.closeHRegion(this.region);
2390       this.region = null;
2391     }
2392   }
2393 
2394   public void testScanner_StopRow1542() throws IOException {
2395     byte [] tableName = Bytes.toBytes("test_table");
2396     byte [] family = Bytes.toBytes("testFamily");
2397     this.region = initHRegion(tableName, getName(), conf, family);
2398     try {
2399       byte [] row1 = Bytes.toBytes("row111");
2400       byte [] row2 = Bytes.toBytes("row222");
2401       byte [] row3 = Bytes.toBytes("row333");
2402       byte [] row4 = Bytes.toBytes("row444");
2403       byte [] row5 = Bytes.toBytes("row555");
2404 
2405       byte [] col1 = Bytes.toBytes("Pub111");
2406       byte [] col2 = Bytes.toBytes("Pub222");
2407 
2408 
2409       Put put = new Put(row1);
2410       put.add(family, col1, Bytes.toBytes(10L));
2411       region.put(put);
2412 
2413       put = new Put(row2);
2414       put.add(family, col1, Bytes.toBytes(15L));
2415       region.put(put);
2416 
2417       put = new Put(row3);
2418       put.add(family, col2, Bytes.toBytes(20L));
2419       region.put(put);
2420 
2421       put = new Put(row4);
2422       put.add(family, col2, Bytes.toBytes(30L));
2423       region.put(put);
2424 
2425       put = new Put(row5);
2426       put.add(family, col1, Bytes.toBytes(40L));
2427       region.put(put);
2428 
2429       Scan scan = new Scan(row3, row4);
2430       scan.setMaxVersions();
2431       scan.addColumn(family, col1);
2432       InternalScanner s = region.getScanner(scan);
2433 
2434       List<KeyValue> results = new ArrayList<KeyValue>();
2435       assertEquals(false, s.next(results));
2436       assertEquals(0, results.size());
2437     } finally {
2438       HRegion.closeHRegion(this.region);
2439       this.region = null;
2440     }
2441   }
2442 
2443   public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
2444     this.region = initHRegion(tableName, getName(), conf, fam1);
2445     try {
2446       long value = 1L;
2447       long amount = 3L;
2448 
2449       Put put = new Put(row);
2450       put.add(fam1, qual1, Bytes.toBytes(value));
2451       region.put(put);
2452 
2453       long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
2454 
2455       assertEquals(value+amount, result);
2456 
2457       Store store = region.getStore(fam1);
2458       // ICV removes any extra values floating around in there.
2459       assertEquals(1, store.memstore.kvset.size());
2460       assertTrue(store.memstore.snapshot.isEmpty());
2461 
2462       assertICV(row, fam1, qual1, value+amount);
2463     } finally {
2464       HRegion.closeHRegion(this.region);
2465       this.region = null;
2466     }
2467   }
2468 
2469   public void testIncrementColumnValue_BumpSnapshot() throws IOException {
2470     ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
2471     EnvironmentEdgeManagerTestHelper.injectEdge(mee);
2472     this.region = initHRegion(tableName, getName(), conf, fam1);
2473     try {
2474       long value = 42L;
2475       long incr = 44L;
2476 
2477       // first put something in kvset, then snapshot it.
2478       Put put = new Put(row);
2479       put.add(fam1, qual1, Bytes.toBytes(value));
2480       region.put(put);
2481 
2482       // get the store in question:
2483       Store s = region.getStore(fam1);
2484       s.snapshot(); //bam
2485 
2486       // now increment:
2487       long newVal = region.incrementColumnValue(row, fam1, qual1,
2488           incr, false);
2489 
2490       assertEquals(value+incr, newVal);
2491 
2492       // get both versions:
2493       Get get = new Get(row);
2494       get.setMaxVersions();
2495       get.addColumn(fam1,qual1);
2496 
2497       Result r = region.get(get, null);
2498       assertEquals(2, r.size());
2499       KeyValue first = r.raw()[0];
2500       KeyValue second = r.raw()[1];
2501 
2502       assertTrue("ICV failed to upgrade timestamp",
2503           first.getTimestamp() != second.getTimestamp());
2504     } finally {
2505       HRegion.closeHRegion(this.region);
2506       this.region = null;
2507     }
2508   }
2509 
2510   public void testIncrementColumnValue_ConcurrentFlush() throws IOException {
2511     this.region = initHRegion(tableName, getName(), conf, fam1);
2512     try {
2513       long value = 1L;
2514       long amount = 3L;
2515 
2516       Put put = new Put(row);
2517       put.add(fam1, qual1, Bytes.toBytes(value));
2518       region.put(put);
2519 
2520       // now increment during a flush
2521       Thread t = new Thread() {
2522         public void run() {
2523           try {
2524             region.flushcache();
2525           } catch (IOException e) {
2526             LOG.info("test ICV, got IOE during flushcache()");
2527           }
2528         }
2529       };
2530       t.start();
2531       long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
2532       assertEquals(value+amount, r);
2533 
2534       // this also asserts there is only 1 KeyValue in the set.
2535       assertICV(row, fam1, qual1, value+amount);
2536     } finally {
2537       HRegion.closeHRegion(this.region);
2538       this.region = null;
2539     }
2540   }
2541 
2542   public void testIncrementColumnValue_heapSize() throws IOException {
2543     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
2544 
2545     this.region = initHRegion(tableName, getName(), conf, fam1);
2546     try {
2547       long byAmount = 1L;
2548       long size;
2549 
2550       for( int i = 0; i < 1000 ; i++) {
2551         region.incrementColumnValue(row, fam1, qual1, byAmount, true);
2552 
2553         size = region.memstoreSize.get();
2554         assertTrue("memstore size: " + size, size >= 0);
2555       }
2556     } finally {
2557       HRegion.closeHRegion(this.region);
2558       this.region = null;
2559     }
2560   }
2561 
2562   public void testIncrementColumnValue_UpdatingInPlace_Negative()
2563     throws IOException {
2564     this.region = initHRegion(tableName, getName(), conf, fam1);
2565     try {
2566       long value = 3L;
2567       long amount = -1L;
2568 
2569       Put put = new Put(row);
2570       put.add(fam1, qual1, Bytes.toBytes(value));
2571       region.put(put);
2572 
2573       long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
2574       assertEquals(value+amount, result);
2575 
2576       assertICV(row, fam1, qual1, value+amount);
2577     } finally {
2578       HRegion.closeHRegion(this.region);
2579       this.region = null;
2580     }
2581   }
2582 
2583   public void testIncrementColumnValue_AddingNew()
2584     throws IOException {
2585     this.region = initHRegion(tableName, getName(), conf, fam1);
2586     try {
2587       long value = 1L;
2588       long amount = 3L;
2589 
2590       Put put = new Put(row);
2591       put.add(fam1, qual1, Bytes.toBytes(value));
2592       put.add(fam1, qual2, Bytes.toBytes(value));
2593       region.put(put);
2594 
2595       long result = region.incrementColumnValue(row, fam1, qual3, amount, true);
2596       assertEquals(amount, result);
2597 
2598       Get get = new Get(row);
2599       get.addColumn(fam1, qual3);
2600       Result rr = region.get(get, null);
2601       assertEquals(1, rr.size());
2602 
2603       // ensure none of the other cols were incremented.
2604       assertICV(row, fam1, qual1, value);
2605       assertICV(row, fam1, qual2, value);
2606       assertICV(row, fam1, qual3, amount);
2607     } finally {
2608       HRegion.closeHRegion(this.region);
2609       this.region = null;
2610     }
2611   }
2612 
2613   public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
2614     this.region = initHRegion(tableName, getName(), conf, fam1);
2615     try {
2616       long value = 1L;
2617       long amount = 3L;
2618 
2619       Put put = new Put(row);
2620       put.add(fam1, qual1, Bytes.toBytes(value));
2621       put.add(fam1, qual2, Bytes.toBytes(value));
2622       region.put(put);
2623 
2624       // flush to disk.
2625       region.flushcache();
2626 
2627       Store store = region.getStore(fam1);
2628       assertEquals(0, store.memstore.kvset.size());
2629 
2630       long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
2631       assertEquals(value+amount, r);
2632 
2633       assertICV(row, fam1, qual1, value+amount);
2634     } finally {
2635       HRegion.closeHRegion(this.region);
2636       this.region = null;
2637     }
2638   }
2639 
2640   public void testIncrementColumnValue_AddingNewAfterSFCheck()
2641     throws IOException {
2642     this.region = initHRegion(tableName, getName(), conf, fam1);
2643     try {
2644       long value = 1L;
2645       long amount = 3L;
2646 
2647       Put put = new Put(row);
2648       put.add(fam1, qual1, Bytes.toBytes(value));
2649       put.add(fam1, qual2, Bytes.toBytes(value));
2650       region.put(put);
2651       region.flushcache();
2652 
2653       Store store = region.getStore(fam1);
2654       assertEquals(0, store.memstore.kvset.size());
2655 
2656       long r = region.incrementColumnValue(row, fam1, qual3, amount, true);
2657       assertEquals(amount, r);
2658 
2659       assertICV(row, fam1, qual3, amount);
2660 
2661       region.flushcache();
2662 
2663       // ensure that this gets to disk.
2664       assertICV(row, fam1, qual3, amount);
2665     } finally {
2666       HRegion.closeHRegion(this.region);
2667       this.region = null;
2668     }
2669   }
2670 
2671   /**
2672    * Added for HBASE-3235.
2673    *
2674    * When the initial put and an ICV update were arriving with the same timestamp,
2675    * the initial Put KV was being skipped during {@link MemStore#upsert(KeyValue)}
2676    * causing the iteration for matching KVs, causing the update-in-place to not
2677    * happen and the ICV put to effectively disappear.
2678    * @throws IOException
2679    */
2680   public void testIncrementColumnValue_UpdatingInPlace_TimestampClobber() throws IOException {
2681     this.region = initHRegion(tableName, getName(), conf, fam1);
2682     try {
2683       long value = 1L;
2684       long amount = 3L;
2685       long now = EnvironmentEdgeManager.currentTimeMillis();
2686       ManualEnvironmentEdge mock = new ManualEnvironmentEdge();
2687       mock.setValue(now);
2688       EnvironmentEdgeManagerTestHelper.injectEdge(mock);
2689 
2690       // verify we catch an ICV on a put with the same timestamp
2691       Put put = new Put(row);
2692       put.add(fam1, qual1, now, Bytes.toBytes(value));
2693       region.put(put);
2694 
2695       long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
2696 
2697       assertEquals(value+amount, result);
2698 
2699       Store store = region.getStore(fam1);
2700       // ICV should update the existing Put with the same timestamp
2701       assertEquals(1, store.memstore.kvset.size());
2702       assertTrue(store.memstore.snapshot.isEmpty());
2703 
2704       assertICV(row, fam1, qual1, value+amount);
2705 
2706       // verify we catch an ICV even when the put ts > now
2707       put = new Put(row);
2708       put.add(fam1, qual2, now+1, Bytes.toBytes(value));
2709       region.put(put);
2710 
2711       result = region.incrementColumnValue(row, fam1, qual2, amount, true);
2712 
2713       assertEquals(value+amount, result);
2714 
2715       store = region.getStore(fam1);
2716       // ICV should update the existing Put with the same timestamp
2717       assertEquals(2, store.memstore.kvset.size());
2718       assertTrue(store.memstore.snapshot.isEmpty());
2719 
2720       assertICV(row, fam1, qual2, value+amount);
2721       EnvironmentEdgeManagerTestHelper.reset();
2722     } finally {
2723       HRegion.closeHRegion(this.region);
2724       this.region = null;
2725     }
2726   }
2727 
2728   public void testIncrementColumnValue_WrongInitialSize() throws IOException {
2729     this.region = initHRegion(tableName, getName(), conf, fam1);
2730     try {
2731       byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
2732       int row1Field1 = 0;
2733       int row1Field2 = 1;
2734       Put put1 = new Put(row1);
2735       put1.add(fam1, qual1, Bytes.toBytes(row1Field1));
2736       put1.add(fam1, qual2, Bytes.toBytes(row1Field2));
2737       region.put(put1);
2738 
2739       long result;
2740       try {
2741         result = region.incrementColumnValue(row1, fam1, qual1, 1, true);
2742         fail("Expected to fail here");
2743       } catch (Exception exception) {
2744         // Expected.
2745       }
2746 
2747 
2748       assertICV(row1, fam1, qual1, row1Field1);
2749       assertICV(row1, fam1, qual2, row1Field2);
2750     } finally {
2751       HRegion.closeHRegion(this.region);
2752       this.region = null;
2753     }
2754   }
2755 
2756   public void testIncrement_WrongInitialSize() throws IOException {
2757     this.region = initHRegion(tableName, getName(), conf, fam1);
2758     try {
2759       byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
2760       long row1Field1 = 0;
2761       int row1Field2 = 1;
2762       Put put1 = new Put(row1);
2763       put1.add(fam1, qual1, Bytes.toBytes(row1Field1));
2764       put1.add(fam1, qual2, Bytes.toBytes(row1Field2));
2765       region.put(put1);
2766       Increment increment = new Increment(row1);
2767       increment.addColumn(fam1, qual1, 1);
2768 
2769       //here we should be successful as normal
2770       region.increment(increment, null, true);
2771       assertICV(row1, fam1, qual1, row1Field1 + 1);
2772 
2773       //failed to increment
2774       increment = new Increment(row1);
2775       increment.addColumn(fam1, qual2, 1);
2776       try {
2777         region.increment(increment, null, true);
2778         fail("Expected to fail here");
2779       } catch (Exception exception) {
2780         // Expected.
2781       }
2782       assertICV(row1, fam1, qual2, row1Field2);
2783     } finally {
2784       HRegion.closeHRegion(this.region);
2785       this.region = null;
2786     }
2787   }
2788   private void assertICV(byte [] row,
2789                          byte [] familiy,
2790                          byte[] qualifier,
2791                          long amount) throws IOException {
2792     // run a get and see?
2793     Get get = new Get(row);
2794     get.addColumn(familiy, qualifier);
2795     Result result = region.get(get, null);
2796     assertEquals(1, result.size());
2797 
2798     KeyValue kv = result.raw()[0];
2799     long r = Bytes.toLong(kv.getValue());
2800     assertEquals(amount, r);
2801   }
2802 
2803   private void assertICV(byte [] row,
2804                          byte [] familiy,
2805                          byte[] qualifier,
2806                          int amount) throws IOException {
2807     // run a get and see?
2808     Get get = new Get(row);
2809     get.addColumn(familiy, qualifier);
2810     Result result = region.get(get, null);
2811     assertEquals(1, result.size());
2812 
2813     KeyValue kv = result.raw()[0];
2814     int r = Bytes.toInt(kv.getValue());
2815     assertEquals(amount, r);
2816   }
2817 
2818   public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions()
2819   throws IOException {
2820     byte [] tableName = Bytes.toBytes("testtable");
2821     byte [] row1 = Bytes.toBytes("row1");
2822     byte [] fam1 = Bytes.toBytes("fam1");
2823     byte [] qf1 = Bytes.toBytes("qualifier1");
2824     byte [] qf2 = Bytes.toBytes("quateslifier2");
2825 
2826     long ts1 = 1;
2827     long ts2 = ts1 + 1;
2828     long ts3 = ts1 + 2;
2829     long ts4 = ts1 + 3;
2830 
2831     //Setting up region
2832     String method = this.getName();
2833     this.region = initHRegion(tableName, method, conf, fam1);
2834     try {
2835       //Putting data in Region
2836       KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2837       KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2838       KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2839       KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2840 
2841       KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2842       KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2843       KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2844       KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2845 
2846       Put put = null;
2847       put = new Put(row1);
2848       put.add(kv14);
2849       put.add(kv24);
2850       region.put(put);
2851       region.flushcache();
2852 
2853       put = new Put(row1);
2854       put.add(kv23);
2855       put.add(kv13);
2856       region.put(put);
2857       region.flushcache();
2858 
2859       put = new Put(row1);
2860       put.add(kv22);
2861       put.add(kv12);
2862       region.put(put);
2863       region.flushcache();
2864 
2865       put = new Put(row1);
2866       put.add(kv21);
2867       put.add(kv11);
2868       region.put(put);
2869 
2870       //Expected
2871       List<KeyValue> expected = new ArrayList<KeyValue>();
2872       expected.add(kv14);
2873       expected.add(kv13);
2874       expected.add(kv12);
2875       expected.add(kv24);
2876       expected.add(kv23);
2877       expected.add(kv22);
2878 
2879       Scan scan = new Scan(row1);
2880       int versions = 3;
2881       scan.setMaxVersions(versions);
2882       List<KeyValue> actual = new ArrayList<KeyValue>();
2883       InternalScanner scanner = region.getScanner(scan);
2884 
2885       boolean hasNext = scanner.next(actual);
2886       assertEquals(false, hasNext);
2887 
2888       //Verify result
2889       for(int i=0; i<expected.size(); i++) {
2890         assertEquals(expected.get(i), actual.get(i));
2891       }
2892     } finally {
2893       HRegion.closeHRegion(this.region);
2894       this.region = null;
2895     }
2896   }
2897 
2898   /**
2899    * Added for HBASE-5416
2900    *
2901    * Here we test scan optimization when only subset of CFs are used in filter
2902    * conditions.
2903    */
2904   public void testScanner_JoinedScanners() throws IOException {
2905     byte [] tableName = Bytes.toBytes("testTable");
2906     byte [] cf_essential = Bytes.toBytes("essential");
2907     byte [] cf_joined = Bytes.toBytes("joined");
2908     byte [] cf_alpha = Bytes.toBytes("alpha");
2909     this.region = initHRegion(tableName, getName(), conf, cf_essential, cf_joined, cf_alpha);
2910     try {
2911       byte [] row1 = Bytes.toBytes("row1");
2912       byte [] row2 = Bytes.toBytes("row2");
2913       byte [] row3 = Bytes.toBytes("row3");
2914 
2915       byte [] col_normal = Bytes.toBytes("d");
2916       byte [] col_alpha = Bytes.toBytes("a");
2917 
2918       byte [] filtered_val = Bytes.toBytes(3);
2919 
2920       Put put = new Put(row1);
2921       put.add(cf_essential, col_normal, Bytes.toBytes(1));
2922       put.add(cf_joined, col_alpha, Bytes.toBytes(1));
2923       region.put(put);
2924 
2925       put = new Put(row2);
2926       put.add(cf_essential, col_alpha, Bytes.toBytes(2));
2927       put.add(cf_joined, col_normal, Bytes.toBytes(2));
2928       put.add(cf_alpha, col_alpha, Bytes.toBytes(2));
2929       region.put(put);
2930 
2931       put = new Put(row3);
2932       put.add(cf_essential, col_normal, filtered_val);
2933       put.add(cf_joined, col_normal, filtered_val);
2934       region.put(put);
2935 
2936       // Check two things:
2937       // 1. result list contains expected values
2938       // 2. result list is sorted properly
2939 
2940       Scan scan = new Scan();
2941       Filter filter = new SingleColumnValueExcludeFilter(cf_essential, col_normal,
2942                                                          CompareOp.NOT_EQUAL, filtered_val);
2943       scan.setFilter(filter);
2944       scan.setLoadColumnFamiliesOnDemand(true);
2945       InternalScanner s = region.getScanner(scan);
2946 
2947       List<KeyValue> results = new ArrayList<KeyValue>();
2948       assertTrue(s.next(results));
2949       assertEquals(results.size(), 1);
2950       results.clear();
2951 
2952       assertTrue(s.next(results));
2953       assertEquals(results.size(), 3);
2954       assertTrue("orderCheck", results.get(0).matchingFamily(cf_alpha));
2955       assertTrue("orderCheck", results.get(1).matchingFamily(cf_essential));
2956       assertTrue("orderCheck", results.get(2).matchingFamily(cf_joined));
2957       results.clear();
2958 
2959       assertFalse(s.next(results));
2960       assertEquals(results.size(), 0);
2961     } finally {
2962       HRegion.closeHRegion(this.region);
2963       this.region = null;
2964     }
2965   }
2966 
2967   /**
2968    * HBASE-5416
2969    *
2970    * Test case when scan limits amount of KVs returned on each next() call.
2971    */
2972   public void testScanner_JoinedScannersWithLimits() throws IOException {
2973     final byte [] tableName = Bytes.toBytes("testTable");
2974     final byte [] cf_first = Bytes.toBytes("first");
2975     final byte [] cf_second = Bytes.toBytes("second");
2976 
2977     this.region = initHRegion(tableName, getName(), conf, cf_first, cf_second);
2978     try {
2979       final byte [] col_a = Bytes.toBytes("a");
2980       final byte [] col_b = Bytes.toBytes("b");
2981 
2982       Put put;
2983 
2984       for (int i = 0; i < 10; i++) {
2985         put = new Put(Bytes.toBytes("r" + Integer.toString(i)));
2986         put.add(cf_first, col_a, Bytes.toBytes(i));
2987         if (i < 5) {
2988           put.add(cf_first, col_b, Bytes.toBytes(i));
2989           put.add(cf_second, col_a, Bytes.toBytes(i));
2990           put.add(cf_second, col_b, Bytes.toBytes(i));
2991         }
2992         region.put(put);
2993       }
2994 
2995       Scan scan = new Scan();
2996       scan.setLoadColumnFamiliesOnDemand(true);
2997       Filter bogusFilter = new FilterBase() {
2998         @Override
2999         public boolean isFamilyEssential(byte[] name) {
3000           return Bytes.equals(name, cf_first);
3001         }
3002         @Override
3003         public void readFields(DataInput arg0) throws IOException {
3004         }
3005 
3006         @Override
3007         public void write(DataOutput arg0) throws IOException {
3008         }
3009       };
3010 
3011       scan.setFilter(bogusFilter);
3012       InternalScanner s = region.getScanner(scan);
3013 
3014       // Our data looks like this:
3015       // r0: first:a, first:b, second:a, second:b
3016       // r1: first:a, first:b, second:a, second:b
3017       // r2: first:a, first:b, second:a, second:b
3018       // r3: first:a, first:b, second:a, second:b
3019       // r4: first:a, first:b, second:a, second:b
3020       // r5: first:a
3021       // r6: first:a
3022       // r7: first:a
3023       // r8: first:a
3024       // r9: first:a
3025 
3026       // But due to next's limit set to 3, we should get this:
3027       // r0: first:a, first:b, second:a
3028       // r0: second:b
3029       // r1: first:a, first:b, second:a
3030       // r1: second:b
3031       // r2: first:a, first:b, second:a
3032       // r2: second:b
3033       // r3: first:a, first:b, second:a
3034       // r3: second:b
3035       // r4: first:a, first:b, second:a
3036       // r4: second:b
3037       // r5: first:a
3038       // r6: first:a
3039       // r7: first:a
3040       // r8: first:a
3041       // r9: first:a
3042 
3043       List<KeyValue> results = new ArrayList<KeyValue>();
3044       int index = 0;
3045       while (true) {
3046         boolean more = s.next(results, 3);
3047         if ((index >> 1) < 5) {
3048           if (index % 2 == 0)
3049             assertEquals(results.size(), 3);
3050           else
3051             assertEquals(results.size(), 1);
3052         }
3053         else
3054           assertEquals(results.size(), 1);
3055         results.clear();
3056         index++;
3057         if (!more) break;
3058       }
3059     } finally {
3060       HRegion.closeHRegion(this.region);
3061       this.region = null;
3062     }
3063   }
3064 
3065   //////////////////////////////////////////////////////////////////////////////
3066   // Split test
3067   //////////////////////////////////////////////////////////////////////////////
3068   /**
3069    * Splits twice and verifies getting from each of the split regions.
3070    * @throws Exception
3071    */
3072   public void testBasicSplit() throws Exception {
3073     byte [] tableName = Bytes.toBytes("testtable");
3074     byte [][] families = {fam1, fam2, fam3};
3075 
3076     Configuration hc = initSplit();
3077     //Setting up region
3078     String method = this.getName();
3079     this.region = initHRegion(tableName, method, hc, families);
3080 
3081     try {
3082       LOG.info("" + addContent(region, fam3));
3083       region.flushcache();
3084       region.compactStores();
3085       byte [] splitRow = region.checkSplit();
3086       assertNotNull(splitRow);
3087       LOG.info("SplitRow: " + Bytes.toString(splitRow));
3088       HRegion [] regions = splitRegion(region, splitRow);
3089       try {
3090         // Need to open the regions.
3091         // TODO: Add an 'open' to HRegion... don't do open by constructing
3092         // instance.
3093         for (int i = 0; i < regions.length; i++) {
3094           regions[i] = openClosedRegion(regions[i]);
3095         }
3096         // Assert can get rows out of new regions. Should be able to get first
3097         // row from first region and the midkey from second region.
3098         assertGet(regions[0], fam3, Bytes.toBytes(START_KEY));
3099         assertGet(regions[1], fam3, splitRow);
3100         // Test I can get scanner and that it starts at right place.
3101         assertScan(regions[0], fam3,
3102             Bytes.toBytes(START_KEY));
3103         assertScan(regions[1], fam3, splitRow);
3104         // Now prove can't split regions that have references.
3105         for (int i = 0; i < regions.length; i++) {
3106           // Add so much data to this region, we create a store file that is >
3107           // than one of our unsplitable references. it will.
3108           for (int j = 0; j < 2; j++) {
3109             addContent(regions[i], fam3);
3110           }
3111           addContent(regions[i], fam2);
3112           addContent(regions[i], fam1);
3113           regions[i].flushcache();
3114         }
3115 
3116         byte [][] midkeys = new byte [regions.length][];
3117         // To make regions splitable force compaction.
3118         for (int i = 0; i < regions.length; i++) {
3119           regions[i].compactStores();
3120           midkeys[i] = regions[i].checkSplit();
3121         }
3122 
3123         TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
3124         // Split these two daughter regions so then I'll have 4 regions. Will
3125         // split because added data above.
3126         for (int i = 0; i < regions.length; i++) {
3127           HRegion[] rs = null;
3128           if (midkeys[i] != null) {
3129             rs = splitRegion(regions[i], midkeys[i]);
3130             for (int j = 0; j < rs.length; j++) {
3131               sortedMap.put(Bytes.toString(rs[j].getRegionName()),
3132                 openClosedRegion(rs[j]));
3133             }
3134           }
3135         }
3136         LOG.info("Made 4 regions");
3137         // The splits should have been even. Test I can get some arbitrary row
3138         // out of each.
3139         int interval = (LAST_CHAR - FIRST_CHAR) / 3;
3140         byte[] b = Bytes.toBytes(START_KEY);
3141         for (HRegion r : sortedMap.values()) {
3142           assertGet(r, fam3, b);
3143           b[0] += interval;
3144         }
3145       } finally {
3146         for (int i = 0; i < regions.length; i++) {
3147           try {
3148             regions[i].close();
3149           } catch (IOException e) {
3150             // Ignore.
3151           }
3152         }
3153       }
3154     } finally {
3155       HRegion.closeHRegion(this.region);
3156       this.region = null;
3157     }
3158   }
3159 
3160   public void testSplitRegion() throws IOException {
3161     byte [] tableName = Bytes.toBytes("testtable");
3162     byte [] qualifier = Bytes.toBytes("qualifier");
3163     Configuration hc = initSplit();
3164     int numRows = 10;
3165     byte [][] families = {fam1, fam3};
3166 
3167     //Setting up region
3168     String method = this.getName();
3169     this.region = initHRegion(tableName, method, hc, families);
3170 
3171     //Put data in region
3172     int startRow = 100;
3173     putData(startRow, numRows, qualifier, families);
3174     int splitRow = startRow + numRows;
3175     putData(splitRow, numRows, qualifier, families);
3176     region.flushcache();
3177 
3178     HRegion [] regions = null;
3179     try {
3180       regions = splitRegion(region, Bytes.toBytes("" + splitRow));
3181       //Opening the regions returned.
3182       for (int i = 0; i < regions.length; i++) {
3183         regions[i] = openClosedRegion(regions[i]);
3184       }
3185       //Verifying that the region has been split
3186       assertEquals(2, regions.length);
3187 
3188       //Verifying that all data is still there and that data is in the right
3189       //place
3190       verifyData(regions[0], startRow, numRows, qualifier, families);
3191       verifyData(regions[1], splitRow, numRows, qualifier, families);
3192 
3193     } finally {
3194       HRegion.closeHRegion(this.region);
3195       this.region = null;
3196     }
3197   }
3198 
3199 
3200   /**
3201    * Flushes the cache in a thread while scanning. The tests verify that the
3202    * scan is coherent - e.g. the returned results are always of the same or
3203    * later update as the previous results.
3204    * @throws IOException scan / compact
3205    * @throws InterruptedException thread join
3206    */
3207   public void testFlushCacheWhileScanning() throws IOException, InterruptedException {
3208     byte[] tableName = Bytes.toBytes("testFlushCacheWhileScanning");
3209     byte[] family = Bytes.toBytes("family");
3210     int numRows = 1000;
3211     int flushAndScanInterval = 10;
3212     int compactInterval = 10 * flushAndScanInterval;
3213 
3214     String method = "testFlushCacheWhileScanning";
3215     this.region = initHRegion(tableName,method, conf, family);
3216     try {
3217       FlushThread flushThread = new FlushThread();
3218       flushThread.start();
3219 
3220       Scan scan = new Scan();
3221       scan.addFamily(family);
3222       scan.setFilter(new SingleColumnValueFilter(family, qual1,
3223           CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes(5L))));
3224 
3225       int expectedCount = 0;
3226       List<KeyValue> res = new ArrayList<KeyValue>();
3227 
3228       boolean toggle=true;
3229       for (long i = 0; i < numRows; i++) {
3230         Put put = new Put(Bytes.toBytes(i));
3231         put.setWriteToWAL(false);
3232         put.add(family, qual1, Bytes.toBytes(i % 10));
3233         region.put(put);
3234 
3235         if (i != 0 && i % compactInterval == 0) {
3236           //System.out.println("iteration = " + i);
3237           region.compactStores(true);
3238         }
3239 
3240         if (i % 10 == 5L) {
3241           expectedCount++;
3242         }
3243 
3244         if (i != 0 && i % flushAndScanInterval == 0) {
3245           res.clear();
3246           InternalScanner scanner = region.getScanner(scan);
3247           if (toggle) {
3248             flushThread.flush();
3249           }
3250           while (scanner.next(res)) ;
3251           if (!toggle) {
3252             flushThread.flush();
3253           }
3254           assertEquals("i=" + i, expectedCount, res.size());
3255           toggle = !toggle;
3256         }
3257       }
3258 
3259       flushThread.done();
3260       flushThread.join();
3261       flushThread.checkNoError();
3262     } finally {
3263       HRegion.closeHRegion(this.region);
3264       this.region = null;
3265     }
3266   }
3267 
3268   protected class FlushThread extends Thread {
3269     private volatile boolean done;
3270     private Throwable error = null;
3271 
3272     public void done() {
3273       done = true;
3274       synchronized (this) {
3275         interrupt();
3276       }
3277     }
3278 
3279     public void checkNoError() {
3280       if (error != null) {
3281         assertNull(error);
3282       }
3283     }
3284 
3285     @Override
3286     public void run() {
3287       done = false;
3288       while (!done) {
3289         synchronized (this) {
3290           try {
3291             wait();
3292           } catch (InterruptedException ignored) {
3293             if (done) {
3294               break;
3295             }
3296           }
3297         }
3298         try {
3299           region.flushcache();
3300         } catch (IOException e) {
3301           if (!done) {
3302             LOG.error("Error while flusing cache", e);
3303             error = e;
3304           }
3305           break;
3306         }
3307       }
3308 
3309     }
3310 
3311     public void flush() {
3312       synchronized (this) {
3313         notify();
3314       }
3315 
3316     }
3317   }
3318 
3319   /**
3320    * Writes very wide records and scans for the latest every time..
3321    * Flushes and compacts the region every now and then to keep things
3322    * realistic.
3323    *
3324    * @throws IOException          by flush / scan / compaction
3325    * @throws InterruptedException when joining threads
3326    */
3327   public void testWritesWhileScanning()
3328     throws IOException, InterruptedException {
3329     byte[] tableName = Bytes.toBytes("testWritesWhileScanning");
3330     int testCount = 100;
3331     int numRows = 1;
3332     int numFamilies = 10;
3333     int numQualifiers = 100;
3334     int flushInterval = 7;
3335     int compactInterval = 5 * flushInterval;
3336     byte[][] families = new byte[numFamilies][];
3337     for (int i = 0; i < numFamilies; i++) {
3338       families[i] = Bytes.toBytes("family" + i);
3339     }
3340     byte[][] qualifiers = new byte[numQualifiers][];
3341     for (int i = 0; i < numQualifiers; i++) {
3342       qualifiers[i] = Bytes.toBytes("qual" + i);
3343     }
3344 
3345     String method = "testWritesWhileScanning";
3346     this.region = initHRegion(tableName, method, conf, families);
3347     try {
3348       PutThread putThread = new PutThread(numRows, families, qualifiers);
3349       putThread.start();
3350       putThread.waitForFirstPut();
3351 
3352       FlushThread flushThread = new FlushThread();
3353       flushThread.start();
3354 
3355       Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
3356       //    scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
3357       //      new BinaryComparator(Bytes.toBytes("row0"))));
3358 
3359       int expectedCount = numFamilies * numQualifiers;
3360       List<KeyValue> res = new ArrayList<KeyValue>();
3361 
3362       long prevTimestamp = 0L;
3363       for (int i = 0; i < testCount; i++) {
3364 
3365         if (i != 0 && i % compactInterval == 0) {
3366           region.compactStores(true);
3367         }
3368 
3369         if (i != 0 && i % flushInterval == 0) {
3370           //System.out.println("flush scan iteration = " + i);
3371           flushThread.flush();
3372         }
3373 
3374         boolean previousEmpty = res.isEmpty();
3375         res.clear();
3376         InternalScanner scanner = region.getScanner(scan);
3377         while (scanner.next(res)) ;
3378         if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
3379           assertEquals("i=" + i, expectedCount, res.size());
3380           long timestamp = res.get(0).getTimestamp();
3381           assertTrue("Timestamps were broke: " + timestamp + " prev: " + prevTimestamp,
3382               timestamp >= prevTimestamp);
3383           prevTimestamp = timestamp;
3384         }
3385       }
3386 
3387       putThread.done();
3388 
3389       region.flushcache();
3390 
3391       putThread.join();
3392       putThread.checkNoError();
3393 
3394       flushThread.done();
3395       flushThread.join();
3396       flushThread.checkNoError();
3397     } finally {
3398       HRegion.closeHRegion(this.region);
3399       this.region = null;
3400     }
3401   }
3402 
3403   protected class PutThread extends Thread {
3404     private volatile boolean done;
3405     private volatile int numPutsFinished = 0;
3406 
3407     private Throwable error = null;
3408     private int numRows;
3409     private byte[][] families;
3410     private byte[][] qualifiers;
3411 
3412     private PutThread(int numRows, byte[][] families,
3413       byte[][] qualifiers) {
3414       this.numRows = numRows;
3415       this.families = families;
3416       this.qualifiers = qualifiers;
3417     }
3418 
3419     /**
3420      * Block until this thread has put at least one row.
3421      */
3422     public void waitForFirstPut() throws InterruptedException {
3423       // wait until put thread actually puts some data
3424       while (numPutsFinished == 0) {
3425         checkNoError();
3426         Thread.sleep(50);
3427       }
3428     }
3429 
3430     public void done() {
3431       done = true;
3432       synchronized (this) {
3433         interrupt();
3434       }
3435     }
3436 
3437     public void checkNoError() {
3438       if (error != null) {
3439         assertNull(error);
3440       }
3441     }
3442 
3443     @Override
3444     public void run() {
3445       done = false;
3446       while (!done) {
3447         try {
3448           for (int r = 0; r < numRows; r++) {
3449             byte[] row = Bytes.toBytes("row" + r);
3450             Put put = new Put(row);
3451             put.setWriteToWAL(false);
3452             byte[] value = Bytes.toBytes(String.valueOf(numPutsFinished));
3453             for (byte[] family : families) {
3454               for (byte[] qualifier : qualifiers) {
3455                 put.add(family, qualifier, (long) numPutsFinished, value);
3456               }
3457             }
3458 //            System.out.println("Putting of kvsetsize=" + put.size());
3459             region.put(put);
3460             numPutsFinished++;
3461             if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
3462               System.out.println("put iteration = " + numPutsFinished);
3463               Delete delete = new Delete(row, (long)numPutsFinished-30, null);
3464               region.delete(delete, null, true);
3465             }
3466             numPutsFinished++;
3467           }
3468         } catch (InterruptedIOException e) {
3469           // This is fine. It means we are done, or didn't get the lock on time
3470         } catch (IOException e) {
3471           LOG.error("error while putting records", e);
3472           error = e;
3473           break;
3474         }
3475       }
3476 
3477     }
3478 
3479   }
3480 
3481 
3482   /**
3483    * Writes very wide records and gets the latest row every time..
3484    * Flushes and compacts the region aggressivly to catch issues.
3485    *
3486    * @throws IOException          by flush / scan / compaction
3487    * @throws InterruptedException when joining threads
3488    */
3489   public void testWritesWhileGetting()
3490     throws Exception {
3491     byte[] tableName = Bytes.toBytes("testWritesWhileGetting");
3492     int testCount = 100;
3493     int numRows = 1;
3494     int numFamilies = 10;
3495     int numQualifiers = 100;
3496     int compactInterval = 100;
3497     byte[][] families = new byte[numFamilies][];
3498     for (int i = 0; i < numFamilies; i++) {
3499       families[i] = Bytes.toBytes("family" + i);
3500     }
3501     byte[][] qualifiers = new byte[numQualifiers][];
3502     for (int i = 0; i < numQualifiers; i++) {
3503       qualifiers[i] = Bytes.toBytes("qual" + i);
3504     }
3505 
3506     Configuration conf = HBaseConfiguration.create(this.conf);
3507 
3508     String method = "testWritesWhileGetting";
3509     // This test flushes constantly and can cause many files to be created, possibly
3510     // extending over the ulimit.  Make sure compactions are aggressive in reducing
3511     // the number of HFiles created.
3512     conf.setInt("hbase.hstore.compaction.min", 1);
3513     conf.setInt("hbase.hstore.compaction.max", 1000);
3514     this.region = initHRegion(tableName, method, conf, families);
3515     PutThread putThread = null;
3516     MultithreadedTestUtil.TestContext ctx =
3517       new MultithreadedTestUtil.TestContext(conf);
3518     try {
3519       putThread = new PutThread(numRows, families, qualifiers);
3520       putThread.start();
3521       putThread.waitForFirstPut();
3522 
3523       // Add a thread that flushes as fast as possible
3524       ctx.addThread(new RepeatingTestThread(ctx) {
3525     	private int flushesSinceCompact = 0;
3526     	private final int maxFlushesSinceCompact = 20;
3527         public void doAnAction() throws Exception {
3528           if (region.flushcache()) {
3529             ++flushesSinceCompact;
3530           }
3531           // Compact regularly to avoid creating too many files and exceeding the ulimit.
3532           if (flushesSinceCompact == maxFlushesSinceCompact) {
3533             region.compactStores(false);
3534             flushesSinceCompact = 0;
3535           }
3536         }
3537       });
3538       ctx.startThreads();
3539 
3540       Get get = new Get(Bytes.toBytes("row0"));
3541       Result result = null;
3542 
3543       int expectedCount = numFamilies * numQualifiers;
3544 
3545       long prevTimestamp = 0L;
3546       for (int i = 0; i < testCount; i++) {
3547 
3548         boolean previousEmpty = result == null || result.isEmpty();
3549         result = region.get(get, null);
3550         if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
3551           assertEquals("i=" + i, expectedCount, result.size());
3552           // TODO this was removed, now what dangit?!
3553           // search looking for the qualifier in question?
3554           long timestamp = 0;
3555           for (KeyValue kv : result.raw()) {
3556             if (Bytes.equals(kv.getFamily(), families[0])
3557                 && Bytes.equals(kv.getQualifier(), qualifiers[0])) {
3558               timestamp = kv.getTimestamp();
3559             }
3560           }
3561           assertTrue(timestamp >= prevTimestamp);
3562           prevTimestamp = timestamp;
3563           KeyValue previousKV = null;
3564 
3565           for (KeyValue kv : result.raw()) {
3566             byte[] thisValue = kv.getValue();
3567             if (previousKV != null) {
3568               if (Bytes.compareTo(previousKV.getValue(), thisValue) != 0) {
3569                 LOG.warn("These two KV should have the same value." +
3570                     " Previous KV:" +
3571                     previousKV + "(memStoreTS:" + previousKV.getMemstoreTS() + ")" +
3572                     ", New KV: " +
3573                     kv + "(memStoreTS:" + kv.getMemstoreTS() + ")"
3574                     );
3575                 assertEquals(0, Bytes.compareTo(previousKV.getValue(), thisValue));
3576               }
3577             }
3578             previousKV = kv;
3579           }
3580         }
3581       }
3582     } finally {
3583       if (putThread != null) putThread.done();
3584 
3585       region.flushcache();
3586 
3587       if (putThread != null) {
3588         putThread.join();
3589         putThread.checkNoError();
3590       }
3591 
3592       ctx.stop();
3593       HRegion.closeHRegion(this.region);
3594       this.region = null;
3595     }
3596   }
3597 
3598   public void testHolesInMeta() throws Exception {
3599     String method = "testHolesInMeta";
3600     byte[] tableName = Bytes.toBytes(method);
3601     byte[] family = Bytes.toBytes("family");
3602     this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method,
3603         conf, false, family);
3604     try {
3605       byte[] rowNotServed = Bytes.toBytes("a");
3606       Get g = new Get(rowNotServed);
3607       try {
3608         region.get(g, null);
3609         fail();
3610       } catch (WrongRegionException x) {
3611         // OK
3612       }
3613       byte[] row = Bytes.toBytes("y");
3614       g = new Get(row);
3615       region.get(g, null);
3616     } finally {
3617       HRegion.closeHRegion(this.region);
3618       this.region = null;
3619     }
3620   }
3621   
3622   /**
3623    * Testcase to check state of region initialization task set to ABORTED or not if any exceptions
3624    * during initialization
3625    * 
3626    * @throws Exception
3627    */
3628   @Test
3629   public void testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization() throws Exception {
3630     HRegionInfo info = null;
3631     try {
3632       FileSystem fs = Mockito.mock(FileSystem.class);
3633       Mockito.when(fs.exists((Path) Mockito.anyObject())).thenThrow(new IOException());
3634       HTableDescriptor htd = new HTableDescriptor(tableName);
3635       htd.addFamily(new HColumnDescriptor("cf"));
3636       info = new HRegionInfo(htd.getName(), HConstants.EMPTY_BYTE_ARRAY,
3637           HConstants.EMPTY_BYTE_ARRAY, false);
3638       Path path = new Path(DIR + "testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization");
3639       // no where we are instantiating HStore in this test case so useTableNameGlobally is null. To
3640       // avoid NullPointerException we are setting useTableNameGlobally to false.
3641       SchemaMetrics.setUseTableNameInTest(false);
3642       region = HRegion.newHRegion(path, null, fs, conf, info, htd, null);
3643       // region initialization throws IOException and set task state to ABORTED.
3644       region.initialize();
3645       fail("Region initialization should fail due to IOException");
3646     } catch (IOException io) {
3647       List<MonitoredTask> tasks = TaskMonitor.get().getTasks();
3648       for (MonitoredTask monitoredTask : tasks) {
3649         if (!(monitoredTask instanceof MonitoredRPCHandler)
3650             && monitoredTask.getDescription().contains(region.toString())) {
3651           assertTrue("Region state should be ABORTED.",
3652               monitoredTask.getState().equals(MonitoredTask.State.ABORTED));
3653           break;
3654         }
3655       }
3656     } finally {
3657       HRegion.closeHRegion(region);
3658     }
3659   }
3660 
3661   public void testIndexesScanWithOneDeletedRow() throws IOException {
3662     byte[] tableName = Bytes.toBytes("testIndexesScanWithOneDeletedRow");
3663     byte[] family = Bytes.toBytes("family");
3664 
3665     //Setting up region
3666     String method = "testIndexesScanWithOneDeletedRow";
3667     this.region = initHRegion(tableName, method, conf, family);
3668     try {
3669       Put put = new Put(Bytes.toBytes(1L));
3670       put.add(family, qual1, 1L, Bytes.toBytes(1L));
3671       region.put(put);
3672 
3673       region.flushcache();
3674 
3675       Delete delete = new Delete(Bytes.toBytes(1L), 1L, null);
3676       //delete.deleteColumn(family, qual1);
3677       region.delete(delete, null, true);
3678 
3679       put = new Put(Bytes.toBytes(2L));
3680       put.add(family, qual1, 2L, Bytes.toBytes(2L));
3681       region.put(put);
3682 
3683       Scan idxScan = new Scan();
3684       idxScan.addFamily(family);
3685       idxScan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,
3686           Arrays.<Filter>asList(new SingleColumnValueFilter(family, qual1,
3687               CompareOp.GREATER_OR_EQUAL,
3688               new BinaryComparator(Bytes.toBytes(0L))),
3689               new SingleColumnValueFilter(family, qual1, CompareOp.LESS_OR_EQUAL,
3690                   new BinaryComparator(Bytes.toBytes(3L)))
3691               )));
3692       InternalScanner scanner = region.getScanner(idxScan);
3693       List<KeyValue> res = new ArrayList<KeyValue>();
3694 
3695       //long start = System.nanoTime();
3696       while (scanner.next(res)) ;
3697       //long end = System.nanoTime();
3698       //System.out.println("memStoreEmpty=" + memStoreEmpty + ", time=" + (end - start)/1000000D);
3699       assertEquals(1L, res.size());
3700     } finally {
3701       HRegion.closeHRegion(this.region);
3702       this.region = null;
3703     }
3704   }
3705 
3706   //////////////////////////////////////////////////////////////////////////////
3707   // Bloom filter test
3708   //////////////////////////////////////////////////////////////////////////////
3709   public void testBloomFilterSize() throws IOException {
3710     byte [] tableName = Bytes.toBytes("testBloomFilterSize");
3711     byte [] row1 = Bytes.toBytes("row1");
3712     byte [] fam1 = Bytes.toBytes("fam1");
3713     byte [] qf1  = Bytes.toBytes("col");
3714     byte [] val1  = Bytes.toBytes("value1");
3715     // Create Table
3716     HColumnDescriptor hcd = new HColumnDescriptor(fam1)
3717         .setMaxVersions(Integer.MAX_VALUE)
3718         .setBloomFilterType(BloomType.ROWCOL);
3719 
3720     HTableDescriptor htd = new HTableDescriptor(tableName);
3721     htd.addFamily(hcd);
3722     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
3723     Path path = new Path(DIR + "testBloomFilterSize");
3724     this.region = HRegion.createHRegion(info, path, conf, htd);
3725     try {
3726       int num_unique_rows = 10;
3727       int duplicate_multiplier =2;
3728       int num_storefiles = 4;
3729 
3730       int version = 0;
3731       for (int f =0 ; f < num_storefiles; f++) {
3732         for (int i = 0; i < duplicate_multiplier; i ++) {
3733           for (int j = 0; j < num_unique_rows; j++) {
3734             Put put = new Put(Bytes.toBytes("row" + j));
3735             put.setWriteToWAL(false);
3736             put.add(fam1, qf1, version++, val1);
3737             region.put(put);
3738           }
3739         }
3740         region.flushcache();
3741       }
3742       //before compaction
3743       Store store = region.getStore(fam1);
3744       List<StoreFile> storeFiles = store.getStorefiles();
3745       for (StoreFile storefile : storeFiles) {
3746         StoreFile.Reader reader = storefile.getReader();
3747         reader.loadFileInfo();
3748         reader.loadBloomfilter();
3749         assertEquals(num_unique_rows*duplicate_multiplier, reader.getEntries());
3750         assertEquals(num_unique_rows, reader.getFilterEntries());
3751       }
3752 
3753       region.compactStores(true);
3754 
3755       //after compaction
3756       storeFiles = store.getStorefiles();
3757       for (StoreFile storefile : storeFiles) {
3758         StoreFile.Reader reader = storefile.getReader();
3759         reader.loadFileInfo();
3760         reader.loadBloomfilter();
3761         assertEquals(num_unique_rows*duplicate_multiplier*num_storefiles,
3762             reader.getEntries());
3763         assertEquals(num_unique_rows, reader.getFilterEntries());
3764       }
3765     } finally {
3766       HRegion.closeHRegion(this.region);
3767       this.region = null;
3768     }
3769   }
3770 
3771   public void testAllColumnsWithBloomFilter() throws IOException {
3772     byte [] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
3773     byte [] FAMILY = Bytes.toBytes("family");
3774 
3775     //Create table
3776     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY)
3777         .setMaxVersions(Integer.MAX_VALUE)
3778         .setBloomFilterType(BloomType.ROWCOL);
3779     HTableDescriptor htd = new HTableDescriptor(TABLE);
3780     htd.addFamily(hcd);
3781     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
3782     Path path = new Path(DIR + "testAllColumnsWithBloomFilter");
3783     this.region = HRegion.createHRegion(info, path, conf, htd);
3784     try {
3785       // For row:0, col:0: insert versions 1 through 5.
3786       byte row[] = Bytes.toBytes("row:" + 0);
3787       byte column[] = Bytes.toBytes("column:" + 0);
3788       Put put = new Put(row);
3789       put.setWriteToWAL(false);
3790       for (long idx = 1; idx <= 4; idx++) {
3791         put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
3792       }
3793       region.put(put);
3794 
3795       //Flush
3796       region.flushcache();
3797 
3798       //Get rows
3799       Get get = new Get(row);
3800       get.setMaxVersions();
3801       KeyValue[] kvs = region.get(get, null).raw();
3802 
3803       //Check if rows are correct
3804       assertEquals(4, kvs.length);
3805       checkOneCell(kvs[0], FAMILY, 0, 0, 4);
3806       checkOneCell(kvs[1], FAMILY, 0, 0, 3);
3807       checkOneCell(kvs[2], FAMILY, 0, 0, 2);
3808       checkOneCell(kvs[3], FAMILY, 0, 0, 1);
3809     } finally {
3810       HRegion.closeHRegion(this.region);
3811       this.region = null;
3812     }
3813   }
3814 
3815   /**
3816     * Testcase to cover bug-fix for HBASE-2823
3817     * Ensures correct delete when issuing delete row
3818     * on columns with bloom filter set to row+col (BloomType.ROWCOL)
3819    */
3820   public void testDeleteRowWithBloomFilter() throws IOException {
3821     byte [] tableName = Bytes.toBytes("testDeleteRowWithBloomFilter");
3822     byte [] familyName = Bytes.toBytes("familyName");
3823 
3824     // Create Table
3825     HColumnDescriptor hcd = new HColumnDescriptor(familyName)
3826         .setMaxVersions(Integer.MAX_VALUE)
3827         .setBloomFilterType(BloomType.ROWCOL);
3828 
3829     HTableDescriptor htd = new HTableDescriptor(tableName);
3830     htd.addFamily(hcd);
3831     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
3832     Path path = new Path(DIR + "TestDeleteRowWithBloomFilter");
3833     this.region = HRegion.createHRegion(info, path, conf, htd);
3834     try {
3835       // Insert some data
3836       byte row[] = Bytes.toBytes("row1");
3837       byte col[] = Bytes.toBytes("col1");
3838 
3839       Put put = new Put(row);
3840       put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue"));
3841       region.put(put);
3842       region.flushcache();
3843 
3844       Delete del = new Delete(row);
3845       region.delete(del, null, true);
3846       region.flushcache();
3847 
3848       // Get remaining rows (should have none)
3849       Get get = new Get(row);
3850       get.addColumn(familyName, col);
3851 
3852       KeyValue[] keyValues = region.get(get, null).raw();
3853       assertTrue(keyValues.length == 0);
3854     } finally {
3855       HRegion.closeHRegion(this.region);
3856       this.region = null;
3857     }
3858   }
3859 
3860   @Test public void testgetHDFSBlocksDistribution() throws Exception {
3861     HBaseTestingUtility htu = new HBaseTestingUtility();
3862     final int DEFAULT_BLOCK_SIZE = 1024;
3863     htu.getConfiguration().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
3864     htu.getConfiguration().setInt("dfs.replication", 2);
3865 
3866 
3867     // set up a cluster with 3 nodes
3868     MiniHBaseCluster cluster = null;
3869     String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
3870     int regionServersCount = 3;
3871 
3872     try {
3873       cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
3874       byte [][] families = {fam1, fam2};
3875       HTable ht = htu.createTable(Bytes.toBytes(this.getName()), families);
3876 
3877       //Setting up region
3878       byte row[] = Bytes.toBytes("row1");
3879       byte col[] = Bytes.toBytes("col1");
3880 
3881       Put put = new Put(row);
3882       put.add(fam1, col, 1, Bytes.toBytes("test1"));
3883       put.add(fam2, col, 1, Bytes.toBytes("test2"));
3884       ht.put(put);
3885 
3886       HRegion firstRegion = htu.getHBaseCluster().
3887           getRegions(Bytes.toBytes(this.getName())).get(0);
3888       firstRegion.flushcache();
3889       HDFSBlocksDistribution blocksDistribution1 =
3890           firstRegion.getHDFSBlocksDistribution();
3891 
3892       // given the default replication factor is 2 and we have 2 HFiles,
3893       // we will have total of 4 replica of blocks on 3 datanodes; thus there
3894       // must be at least one host that have replica for 2 HFiles. That host's
3895       // weight will be equal to the unique block weight.
3896       long uniqueBlocksWeight1 =
3897           blocksDistribution1.getUniqueBlocksTotalWeight();
3898 
3899       String topHost = blocksDistribution1.getTopHosts().get(0);
3900       long topHostWeight = blocksDistribution1.getWeight(topHost);
3901       assertTrue(uniqueBlocksWeight1 == topHostWeight);
3902 
3903       // use the static method to compute the value, it should be the same.
3904       // static method is used by load balancer or other components
3905       HDFSBlocksDistribution blocksDistribution2 =
3906         HRegion.computeHDFSBlocksDistribution(htu.getConfiguration(),
3907         firstRegion.getTableDesc(),
3908         firstRegion.getRegionInfo().getEncodedName());
3909       long uniqueBlocksWeight2 =
3910         blocksDistribution2.getUniqueBlocksTotalWeight();
3911 
3912       assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
3913 
3914       ht.close();
3915       } finally {
3916         if (cluster != null) {
3917           htu.shutdownMiniCluster();
3918         }
3919       }
3920   }
3921 
3922   /**
3923    * Test case to check put function with memstore flushing for same row, same ts
3924    * @throws Exception
3925    */
3926   public void testPutWithMemStoreFlush() throws Exception {
3927     Configuration conf = HBaseConfiguration.create();
3928     String method = "testPutWithMemStoreFlush";
3929     byte[] tableName = Bytes.toBytes(method);
3930     byte[] family = Bytes.toBytes("family");;
3931     byte[] qualifier = Bytes.toBytes("qualifier");
3932     byte[] row = Bytes.toBytes("putRow");
3933     byte[] value = null;
3934     this.region = initHRegion(tableName, method, conf, family);
3935     Put put = null;
3936     Get get = null;
3937     List<KeyValue> kvs = null;
3938     Result res = null;
3939 
3940     put = new Put(row);
3941     value = Bytes.toBytes("value0");
3942     put.add(family, qualifier, 1234567l, value);
3943     region.put(put);
3944     get = new Get(row);
3945     get.addColumn(family, qualifier);
3946     get.setMaxVersions();
3947     res = this.region.get(get, null);
3948     kvs = res.getColumn(family, qualifier);
3949     assertEquals(1, kvs.size());
3950     assertEquals(Bytes.toBytes("value0"), kvs.get(0).getValue());
3951 
3952     region.flushcache();
3953     get = new Get(row);
3954     get.addColumn(family, qualifier);
3955     get.setMaxVersions();
3956     res = this.region.get(get, null);
3957     kvs = res.getColumn(family, qualifier);
3958     assertEquals(1, kvs.size());
3959     assertEquals(Bytes.toBytes("value0"), kvs.get(0).getValue());
3960 
3961     put = new Put(row);
3962     value = Bytes.toBytes("value1");
3963     put.add(family, qualifier, 1234567l, value);
3964     region.put(put);
3965     get = new Get(row);
3966     get.addColumn(family, qualifier);
3967     get.setMaxVersions();
3968     res = this.region.get(get, null);
3969     kvs = res.getColumn(family, qualifier);
3970     assertEquals(1, kvs.size());
3971     assertEquals(Bytes.toBytes("value1"), kvs.get(0).getValue());
3972 
3973     region.flushcache();
3974     get = new Get(row);
3975     get.addColumn(family, qualifier);
3976     get.setMaxVersions();
3977     res = this.region.get(get, null);
3978     kvs = res.getColumn(family, qualifier);
3979     assertEquals(1, kvs.size());
3980     assertEquals(Bytes.toBytes("value1"), kvs.get(0).getValue());
3981   }
3982   
3983   /**
3984    * TestCase for increment
3985    *
3986    */
3987   private static class Incrementer implements Runnable {
3988     private HRegion region;
3989     private final static byte[] incRow = Bytes.toBytes("incRow");
3990     private final static byte[] family = Bytes.toBytes("family");
3991     private final static byte[] qualifier = Bytes.toBytes("qualifier");
3992     private final static long ONE = 1l;
3993     private int incCounter;
3994 
3995     public Incrementer(HRegion region, int incCounter) {
3996       this.region = region;
3997       this.incCounter = incCounter;
3998     }
3999 
4000     @Override
4001     public void run() {
4002       int count = 0;
4003       while (count < incCounter) {
4004         Increment inc = new Increment(incRow);
4005         inc.addColumn(family, qualifier, ONE);
4006         count++;
4007         try {
4008           region.increment(inc, null, true);
4009         } catch (IOException e) {
4010           e.printStackTrace();
4011           break;
4012         }
4013       }
4014     }
4015   }
4016 
4017   /**
4018    * TestCase for append
4019    * 
4020    */
4021   private static class Appender implements Runnable {
4022     private HRegion region;
4023     private final static byte[] appendRow = Bytes.toBytes("appendRow");
4024     private final static byte[] family = Bytes.toBytes("family");
4025     private final static byte[] qualifier = Bytes.toBytes("qualifier");
4026     private final static byte[] CHAR = Bytes.toBytes("a");
4027     private int appendCounter;
4028 
4029     public Appender(HRegion region, int appendCounter) {
4030       this.region = region;
4031       this.appendCounter = appendCounter;
4032     }
4033 
4034     @Override
4035     public void run() {
4036       int count = 0;
4037       while (count < appendCounter) {
4038         Append app = new Append(appendRow);
4039         app.add(family, qualifier, CHAR);
4040         count++;
4041         try {
4042           region.append(app, null, true);
4043         } catch (IOException e) {
4044           e.printStackTrace();
4045           break;
4046         }
4047       }
4048     }
4049   }
4050 
4051   /**
4052    * Test case to check append function with memstore flushing
4053    * 
4054    * @throws Exception
4055    */
4056   @Test
4057   public void testParallelAppendWithMemStoreFlush() throws Exception {
4058     Configuration conf = HBaseConfiguration.create();
4059     String method = "testParallelAppendWithMemStoreFlush";
4060     byte[] tableName = Bytes.toBytes(method);
4061     byte[] family = Appender.family;
4062     this.region = initHRegion(tableName, method, conf, family);
4063     final HRegion region = this.region;
4064     final AtomicBoolean appendDone = new AtomicBoolean(false);
4065     Runnable flusher = new Runnable() {
4066       @Override
4067       public void run() {
4068         while (!appendDone.get()) {
4069           try {
4070             region.flushcache();
4071           } catch (Exception e) {
4072             e.printStackTrace();
4073           }
4074         }
4075       }
4076     };
4077 
4078     // after all append finished, the value will append to threadNum * appendCounter Appender.CHAR
4079     int threadNum = 20;
4080     int appendCounter = 100;
4081     byte[] expected = new byte[threadNum * appendCounter];
4082     for (int i = 0; i < threadNum * appendCounter; i++) {
4083       System.arraycopy(Appender.CHAR, 0, expected, i, 1);
4084     }
4085     Thread[] appenders = new Thread[threadNum];
4086     Thread flushThread = new Thread(flusher);
4087     for (int i = 0; i < threadNum; i++) {
4088       appenders[i] = new Thread(new Appender(this.region, appendCounter));
4089       appenders[i].start();
4090     }
4091     flushThread.start();
4092     for (int i = 0; i < threadNum; i++) {
4093       appenders[i].join();
4094     }
4095 
4096     appendDone.set(true);
4097     flushThread.join();
4098 
4099     Get get = new Get(Appender.appendRow);
4100     get.addColumn(Appender.family, Appender.qualifier);
4101     get.setMaxVersions(1);
4102     Result res = this.region.get(get, null);
4103     List<KeyValue> kvs = res.getColumn(Appender.family, Appender.qualifier);
4104 
4105     // we just got the latest version
4106     assertEquals(kvs.size(), 1);
4107     KeyValue kv = kvs.get(0);
4108     byte[] appendResult = new byte[kv.getValueLength()];
4109     System.arraycopy(kv.getBuffer(), kv.getValueOffset(), appendResult, 0, kv.getValueLength());
4110     assertEquals(expected, appendResult);
4111     this.region = null;
4112   }
4113    
4114   /**
4115    * Test case to check increment function with memstore flushing
4116    * @throws Exception
4117    */
4118   @Test
4119   public void testParallelIncrementWithMemStoreFlush() throws Exception {
4120     String method = "testParallelIncrementWithMemStoreFlush";
4121     byte[] tableName = Bytes.toBytes(method);
4122     byte[] family = Incrementer.family;
4123     this.region = initHRegion(tableName, method, conf, family);
4124     final HRegion region = this.region;
4125     final AtomicBoolean incrementDone = new AtomicBoolean(false);
4126     Runnable reader = new Runnable() {
4127       @Override
4128       public void run() {
4129         while (!incrementDone.get()) {
4130           try {
4131             region.flushcache();
4132           } catch (Exception e) {
4133             e.printStackTrace();
4134           }
4135         }
4136       }
4137     };
4138 
4139     //after all increment finished, the row will increment to 20*100 = 2000
4140     int threadNum = 20;
4141     int incCounter = 100;
4142     long expected = threadNum * incCounter;
4143     Thread[] incrementers = new Thread[threadNum];
4144     Thread flushThread = new Thread(reader);
4145     for (int i = 0; i < threadNum; i++) {
4146       incrementers[i] = new Thread(new Incrementer(this.region, incCounter));
4147       incrementers[i].start();
4148     }
4149     flushThread.start();
4150     for (int i = 0; i < threadNum; i++) {
4151       incrementers[i].join();
4152     }
4153 
4154     incrementDone.set(true);
4155     flushThread.join();
4156 
4157     Get get = new Get(Incrementer.incRow);
4158     get.addColumn(Incrementer.family, Incrementer.qualifier);
4159     get.setMaxVersions(1);
4160     Result res = this.region.get(get, null);
4161     List<KeyValue> kvs = res.getColumn(Incrementer.family,
4162         Incrementer.qualifier);
4163     
4164     //we just got the latest version
4165     assertEquals(kvs.size(), 1);
4166     KeyValue kv = kvs.get(0);
4167     assertEquals(expected, Bytes.toLong(kv.getBuffer(), kv.getValueOffset()));
4168     this.region = null;
4169   }
4170 
4171   private void putData(int startRow, int numRows, byte [] qf,
4172       byte [] ...families)
4173   throws IOException {
4174     for(int i=startRow; i<startRow+numRows; i++) {
4175       Put put = new Put(Bytes.toBytes("" + i));
4176       put.setWriteToWAL(false);
4177       for(byte [] family : families) {
4178         put.add(family, qf, null);
4179       }
4180       region.put(put);
4181     }
4182   }
4183 
4184   private void verifyData(HRegion newReg, int startRow, int numRows, byte [] qf,
4185       byte [] ... families)
4186   throws IOException {
4187     for(int i=startRow; i<startRow + numRows; i++) {
4188       byte [] row = Bytes.toBytes("" + i);
4189       Get get = new Get(row);
4190       for(byte [] family : families) {
4191         get.addColumn(family, qf);
4192       }
4193       Result result = newReg.get(get, null);
4194       KeyValue [] raw = result.raw();
4195       assertEquals(families.length, result.size());
4196       for(int j=0; j<families.length; j++) {
4197         assertEquals(0, Bytes.compareTo(row, raw[j].getRow()));
4198         assertEquals(0, Bytes.compareTo(families[j], raw[j].getFamily()));
4199         assertEquals(0, Bytes.compareTo(qf, raw[j].getQualifier()));
4200       }
4201     }
4202   }
4203 
4204   private void assertGet(final HRegion r, final byte [] family, final byte [] k)
4205   throws IOException {
4206     // Now I have k, get values out and assert they are as expected.
4207     Get get = new Get(k).addFamily(family).setMaxVersions();
4208     KeyValue [] results = r.get(get, null).raw();
4209     for (int j = 0; j < results.length; j++) {
4210       byte [] tmp = results[j].getValue();
4211       // Row should be equal to value every time.
4212       assertTrue(Bytes.equals(k, tmp));
4213     }
4214   }
4215 
4216   /*
4217    * Assert first value in the passed region is <code>firstValue</code>.
4218    * @param r
4219    * @param fs
4220    * @param firstValue
4221    * @throws IOException
4222    */
4223   private void assertScan(final HRegion r, final byte [] fs,
4224       final byte [] firstValue)
4225   throws IOException {
4226     byte [][] families = {fs};
4227     Scan scan = new Scan();
4228     for (int i = 0; i < families.length; i++) scan.addFamily(families[i]);
4229     InternalScanner s = r.getScanner(scan);
4230     try {
4231       List<KeyValue> curVals = new ArrayList<KeyValue>();
4232       boolean first = true;
4233       OUTER_LOOP: while(s.next(curVals)) {
4234         for (KeyValue kv: curVals) {
4235           byte [] val = kv.getValue();
4236           byte [] curval = val;
4237           if (first) {
4238             first = false;
4239             assertTrue(Bytes.compareTo(curval, firstValue) == 0);
4240           } else {
4241             // Not asserting anything.  Might as well break.
4242             break OUTER_LOOP;
4243           }
4244         }
4245       }
4246     } finally {
4247       s.close();
4248     }
4249   }
4250 
4251   private Configuration initSplit() {
4252     Configuration conf = HBaseConfiguration.create(this.conf);
4253 
4254     // Always compact if there is more than one store file.
4255     conf.setInt("hbase.hstore.compactionThreshold", 2);
4256 
4257     // Make lease timeout longer, lease checks less frequent
4258     conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
4259 
4260     conf.setInt(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, 10 * 1000);
4261 
4262     // Increase the amount of time between client retries
4263     conf.setLong("hbase.client.pause", 15 * 1000);
4264 
4265     // This size should make it so we always split using the addContent
4266     // below.  After adding all data, the first region is 1.3M
4267     conf.setLong(HConstants.HREGION_MAX_FILESIZE, 1024 * 128);
4268     return conf;
4269   }
4270 
4271   /**
4272    * @param tableName
4273    * @param callingMethod
4274    * @param conf
4275    * @param families
4276    * @throws IOException
4277    * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
4278    */
4279   public static HRegion initHRegion (byte [] tableName, String callingMethod,
4280       Configuration conf, byte [] ... families)
4281     throws IOException{
4282     return initHRegion(tableName, null, null, callingMethod, conf, false, families);
4283   }
4284 
4285   /**
4286    * @param tableName
4287    * @param callingMethod
4288    * @param conf
4289    * @param isReadOnly
4290    * @param families
4291    * @throws IOException
4292    * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
4293    */
4294   public static HRegion initHRegion (byte [] tableName, String callingMethod,
4295       Configuration conf, boolean isReadOnly, byte [] ... families)
4296     throws IOException{
4297     return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
4298   }
4299 
4300   /**
4301    * @param tableName
4302    * @param startKey
4303    * @param stopKey
4304    * @param callingMethod
4305    * @param conf
4306    * @param isReadOnly
4307    * @param families
4308    * @throws IOException
4309    * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
4310    */
4311   private static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
4312       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
4313       throws IOException {
4314     HTableDescriptor htd = new HTableDescriptor(tableName);
4315     htd.setReadOnly(isReadOnly);
4316     for(byte [] family : families) {
4317       htd.addFamily(new HColumnDescriptor(family));
4318     }
4319     HRegionInfo info = new HRegionInfo(htd.getName(), startKey, stopKey, false);
4320     Path path = new Path(DIR + callingMethod);
4321     FileSystem fs = FileSystem.get(conf);
4322     if (fs.exists(path)) {
4323       if (!fs.delete(path, true)) {
4324         throw new IOException("Failed delete of " + path);
4325       }
4326     }
4327     return HRegion.createHRegion(info, path, conf, htd);
4328   }
4329 
4330   /**
4331    * Assert that the passed in KeyValue has expected contents for the
4332    * specified row, column & timestamp.
4333    */
4334   private void checkOneCell(KeyValue kv, byte[] cf,
4335                              int rowIdx, int colIdx, long ts) {
4336     String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
4337     assertEquals("Row mismatch which checking: " + ctx,
4338                  "row:"+ rowIdx, Bytes.toString(kv.getRow()));
4339     assertEquals("ColumnFamily mismatch while checking: " + ctx,
4340                  Bytes.toString(cf), Bytes.toString(kv.getFamily()));
4341     assertEquals("Column qualifier mismatch while checking: " + ctx,
4342                  "column:" + colIdx, Bytes.toString(kv.getQualifier()));
4343     assertEquals("Timestamp mismatch while checking: " + ctx,
4344                  ts, kv.getTimestamp());
4345     assertEquals("Value mismatch while checking: " + ctx,
4346                  "value-version-" + ts, Bytes.toString(kv.getValue()));
4347   }
4348 
4349 
4350   @org.junit.Rule
4351   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
4352     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
4353 }
4354