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