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