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