diff --git src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 6a1b3f9..1ecf64c 100644
--- src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -61,13 +61,13 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.UnknownScannerException;
-import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
@@ -78,11 +78,10 @@ import org.apache.hadoop.hbase.client.RowLock;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.coprocessor.Exec;
import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
-import org.apache.hadoop.hbase.filter.NullComparator;
import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -3080,7 +3079,11 @@ public class HRegion implements HeapSize { // , Writable{
* bootstrap code in the HMaster constructor.
* Note, this method creates an {@link HLog} for the created region. It
* needs to be closed explicitly. Use {@link HRegion#getLog()} to get
- * access.
+ * access. When done with a region created using this method, you will
+ * need to explicitly close the {@link HLog} it created too; it will not be
+ * done for you. Not closing the log will leave at least a daemon thread
+ * running. Call {@link #closeHRegion(HRegion)} and it will do
+ * necessary cleanup for you.
* @param info Info for region to create.
* @param rootDir Root directory for HBase instance
* @param conf
@@ -3096,6 +3099,23 @@ public class HRegion implements HeapSize { // , Writable{
}
/**
+ * This will do the necessary cleanup a call to {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)}
+ * requires. This method will close the region and then close its
+ * associated {@link HLog} file. You use it if you call the other createHRegion,
+ * the one that takes an {@link HLog} instance but don't be surprised by the
+ * call to the {@link HLog#closeAndDelete()} on the {@link HLog} the
+ * HRegion was carrying.
+ * @param r
+ * @throws IOException
+ */
+ public static void closeHRegion(final HRegion r) throws IOException {
+ if (r == null) return;
+ r.close();
+ if (r.getLog() == null) return;
+ r.getLog().closeAndDelete();
+ }
+
+ /**
* Convenience method creating new HRegions. Used by createTable.
* The {@link HLog} for the created region needs to be closed explicitly.
* Use {@link HRegion#getLog()} to get access.
diff --git src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
index 7313fbe..f1261a7 100644
--- src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
+++ src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
@@ -170,10 +170,19 @@ public abstract class HBaseTestCase extends TestCase {
);
}
+ /**
+ * You must call close on the returned region and then close on the log file
+ * it created. Do {@link HRegion#close()} followed by {@link HRegion#getLog()}
+ * and on it call close.
+ * @param desc
+ * @param startKey
+ * @param endKey
+ * @return An {@link HRegion}
+ * @throws IOException
+ */
protected HRegion createNewHRegion(HTableDescriptor desc, byte [] startKey,
byte [] endKey)
throws IOException {
- FileSystem filesystem = FileSystem.get(conf);
HRegionInfo hri = new HRegionInfo(desc.getName(), startKey, endKey);
return HRegion.createHRegion(hri, testDir, conf, desc);
}
@@ -679,6 +688,11 @@ public abstract class HBaseTestCase extends TestCase {
}
}
+ /**
+ * You must call {@link #closeRootAndMeta()} when done after calling this
+ * method. It does cleanup.
+ * @throws IOException
+ */
protected void createRootAndMetaRegions() throws IOException {
root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir,
conf, HTableDescriptor.ROOT_TABLEDESC);
diff --git src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
index 7d2b300..4068369 100644
--- src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
+++ src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
@@ -53,52 +53,56 @@ public class TestColumnPrefixFilter {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, TEST_UTIL.
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
-
- List rows = generateRandomWords(100, "row");
- List columns = generateRandomWords(10000, "column");
- long maxTimestamp = 2;
-
- List kvList = new ArrayList();
-
- Map> prefixMap = new HashMap>();
-
- prefixMap.put("p", new ArrayList());
- prefixMap.put("s", new ArrayList());
-
- String valueString = "ValueString";
-
- for (String row: rows) {
- Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
- for (String column: columns) {
- for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
- KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
- valueString);
- p.add(kv);
- kvList.add(kv);
- for (String s: prefixMap.keySet()) {
- if (column.startsWith(s)) {
- prefixMap.get(s).add(kv);
+ try {
+ List rows = generateRandomWords(100, "row");
+ List columns = generateRandomWords(10000, "column");
+ long maxTimestamp = 2;
+
+ List kvList = new ArrayList();
+
+ Map> prefixMap = new HashMap>();
+
+ prefixMap.put("p", new ArrayList());
+ prefixMap.put("s", new ArrayList());
+
+ String valueString = "ValueString";
+
+ for (String row: rows) {
+ Put p = new Put(Bytes.toBytes(row));
+ p.setWriteToWAL(false);
+ for (String column: columns) {
+ for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+ KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
+ valueString);
+ p.add(kv);
+ kvList.add(kv);
+ for (String s: prefixMap.keySet()) {
+ if (column.startsWith(s)) {
+ prefixMap.get(s).add(kv);
+ }
}
}
}
+ region.put(p);
}
- region.put(p);
- }
- ColumnPrefixFilter filter;
- Scan scan = new Scan();
- scan.setMaxVersions();
- for (String s: prefixMap.keySet()) {
- filter = new ColumnPrefixFilter(Bytes.toBytes(s));
+ ColumnPrefixFilter filter;
+ Scan scan = new Scan();
+ scan.setMaxVersions();
+ for (String s: prefixMap.keySet()) {
+ filter = new ColumnPrefixFilter(Bytes.toBytes(s));
- scan.setFilter(filter);
+ scan.setFilter(filter);
- InternalScanner scanner = region.getScanner(scan);
- List results = new ArrayList();
- while(scanner.next(results));
- assertEquals(prefixMap.get(s).size(), results.size());
+ InternalScanner scanner = region.getScanner(scan);
+ List results = new ArrayList();
+ while(scanner.next(results));
+ assertEquals(prefixMap.get(s).size(), results.size());
+ }
+ } finally {
+ region.close();
+ region.getLog().closeAndDelete();
}
}
@@ -110,55 +114,59 @@ public class TestColumnPrefixFilter {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, TEST_UTIL.
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
-
- List rows = generateRandomWords(100, "row");
- List columns = generateRandomWords(10000, "column");
- long maxTimestamp = 2;
-
- List kvList = new ArrayList();
-
- Map> prefixMap = new HashMap>();
-
- prefixMap.put("p", new ArrayList());
- prefixMap.put("s", new ArrayList());
-
- String valueString = "ValueString";
-
- for (String row: rows) {
- Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
- for (String column: columns) {
- for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
- KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
- valueString);
- p.add(kv);
- kvList.add(kv);
- for (String s: prefixMap.keySet()) {
- if (column.startsWith(s)) {
- prefixMap.get(s).add(kv);
+ try {
+ List rows = generateRandomWords(100, "row");
+ List columns = generateRandomWords(10000, "column");
+ long maxTimestamp = 2;
+
+ List kvList = new ArrayList();
+
+ Map> prefixMap = new HashMap>();
+
+ prefixMap.put("p", new ArrayList());
+ prefixMap.put("s", new ArrayList());
+
+ String valueString = "ValueString";
+
+ for (String row: rows) {
+ Put p = new Put(Bytes.toBytes(row));
+ p.setWriteToWAL(false);
+ for (String column: columns) {
+ for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+ KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
+ valueString);
+ p.add(kv);
+ kvList.add(kv);
+ for (String s: prefixMap.keySet()) {
+ if (column.startsWith(s)) {
+ prefixMap.get(s).add(kv);
+ }
}
}
}
+ region.put(p);
}
- region.put(p);
- }
- ColumnPrefixFilter filter;
- Scan scan = new Scan();
- scan.setMaxVersions();
- for (String s: prefixMap.keySet()) {
- filter = new ColumnPrefixFilter(Bytes.toBytes(s));
-
- //this is how this test differs from the one above
- FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
- filterList.addFilter(filter);
- scan.setFilter(filterList);
-
- InternalScanner scanner = region.getScanner(scan);
- List results = new ArrayList();
- while(scanner.next(results));
- assertEquals(prefixMap.get(s).size(), results.size());
+ ColumnPrefixFilter filter;
+ Scan scan = new Scan();
+ scan.setMaxVersions();
+ for (String s: prefixMap.keySet()) {
+ filter = new ColumnPrefixFilter(Bytes.toBytes(s));
+
+ //this is how this test differs from the one above
+ FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
+ filterList.addFilter(filter);
+ scan.setFilter(filterList);
+
+ InternalScanner scanner = region.getScanner(scan);
+ List results = new ArrayList();
+ while(scanner.next(results));
+ assertEquals(prefixMap.get(s).size(), results.size());
+ }
+ } finally {
+ region.close();
+ region.getLog().closeAndDelete();
}
}
diff --git src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
index 954be6b..cf017a9 100644
--- src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
+++ src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
@@ -85,6 +85,7 @@ public class TestDependentColumnFilter extends TestCase {
protected void tearDown() throws Exception {
super.tearDown();
this.region.close();
+ this.region.getLog().closeAndDelete();
}
private void addData() throws IOException {
diff --git src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index 2c513d5..ba7c97b 100644
--- src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -167,6 +167,7 @@ public class TestFilter extends HBaseTestCase {
protected void tearDown() throws Exception {
this.region.close();
+ this.region.getLog().closeAndDelete();
super.tearDown();
}
diff --git src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
index 424dfb0..8c13253 100644
--- src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
+++ src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
@@ -54,54 +54,58 @@ public class TestMultipleColumnPrefixFilter {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, TEST_UTIL.
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+ try {
+ List rows = generateRandomWords(100, "row");
+ List columns = generateRandomWords(10000, "column");
+ long maxTimestamp = 2;
- List rows = generateRandomWords(100, "row");
- List columns = generateRandomWords(10000, "column");
- long maxTimestamp = 2;
-
- List kvList = new ArrayList();
-
- Map> prefixMap = new HashMap>();
-
- prefixMap.put("p", new ArrayList());
- prefixMap.put("q", new ArrayList());
- prefixMap.put("s", new ArrayList());
-
- String valueString = "ValueString";
-
- for (String row: rows) {
- Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
- for (String column: columns) {
- for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
- KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
- valueString);
- p.add(kv);
- kvList.add(kv);
- for (String s: prefixMap.keySet()) {
- if (column.startsWith(s)) {
- prefixMap.get(s).add(kv);
+ List kvList = new ArrayList();
+
+ Map> prefixMap = new HashMap>();
+
+ prefixMap.put("p", new ArrayList());
+ prefixMap.put("q", new ArrayList());
+ prefixMap.put("s", new ArrayList());
+
+ String valueString = "ValueString";
+
+ for (String row: rows) {
+ Put p = new Put(Bytes.toBytes(row));
+ p.setWriteToWAL(false);
+ for (String column: columns) {
+ for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+ KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
+ valueString);
+ p.add(kv);
+ kvList.add(kv);
+ for (String s: prefixMap.keySet()) {
+ if (column.startsWith(s)) {
+ prefixMap.get(s).add(kv);
+ }
}
}
}
+ region.put(p);
}
- region.put(p);
- }
- MultipleColumnPrefixFilter filter;
- Scan scan = new Scan();
- scan.setMaxVersions();
- byte [][] filter_prefix = new byte [2][];
- filter_prefix[0] = new byte [] {'p'};
- filter_prefix[1] = new byte [] {'q'};
-
- filter = new MultipleColumnPrefixFilter(filter_prefix);
- scan.setFilter(filter);
- List results = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
- while(scanner.next(results));
- assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
+ MultipleColumnPrefixFilter filter;
+ Scan scan = new Scan();
+ scan.setMaxVersions();
+ byte [][] filter_prefix = new byte [2][];
+ filter_prefix[0] = new byte [] {'p'};
+ filter_prefix[1] = new byte [] {'q'};
+
+ filter = new MultipleColumnPrefixFilter(filter_prefix);
+ scan.setFilter(filter);
+ List results = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
+ while(scanner.next(results));
+ assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
+ } finally {
+ region.close();
+ region.getLog().closeAndDelete();
+ }
}
@Test
@@ -114,60 +118,64 @@ public class TestMultipleColumnPrefixFilter {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, TEST_UTIL.
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+ try {
+ List rows = generateRandomWords(100, "row");
+ List columns = generateRandomWords(10000, "column");
+ long maxTimestamp = 3;
- List rows = generateRandomWords(100, "row");
- List columns = generateRandomWords(10000, "column");
- long maxTimestamp = 3;
-
- List kvList = new ArrayList();
+ List kvList = new ArrayList();
- Map> prefixMap = new HashMap>();
+ Map> prefixMap = new HashMap>();
- prefixMap.put("p", new ArrayList());
- prefixMap.put("q", new ArrayList());
- prefixMap.put("s", new ArrayList());
+ prefixMap.put("p", new ArrayList());
+ prefixMap.put("q", new ArrayList());
+ prefixMap.put("s", new ArrayList());
- String valueString = "ValueString";
+ String valueString = "ValueString";
- for (String row: rows) {
- Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
- for (String column: columns) {
- for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
- double rand = Math.random();
- KeyValue kv;
- if (rand < 0.5)
- kv = KeyValueTestUtil.create(row, family1, column, timestamp,
- valueString);
- else
- kv = KeyValueTestUtil.create(row, family2, column, timestamp,
- valueString);
- p.add(kv);
- kvList.add(kv);
- for (String s: prefixMap.keySet()) {
- if (column.startsWith(s)) {
- prefixMap.get(s).add(kv);
+ for (String row: rows) {
+ Put p = new Put(Bytes.toBytes(row));
+ p.setWriteToWAL(false);
+ for (String column: columns) {
+ for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+ double rand = Math.random();
+ KeyValue kv;
+ if (rand < 0.5)
+ kv = KeyValueTestUtil.create(row, family1, column, timestamp,
+ valueString);
+ else
+ kv = KeyValueTestUtil.create(row, family2, column, timestamp,
+ valueString);
+ p.add(kv);
+ kvList.add(kv);
+ for (String s: prefixMap.keySet()) {
+ if (column.startsWith(s)) {
+ prefixMap.get(s).add(kv);
+ }
}
}
}
+ region.put(p);
}
- region.put(p);
- }
- MultipleColumnPrefixFilter filter;
- Scan scan = new Scan();
- scan.setMaxVersions();
- byte [][] filter_prefix = new byte [2][];
- filter_prefix[0] = new byte [] {'p'};
- filter_prefix[1] = new byte [] {'q'};
-
- filter = new MultipleColumnPrefixFilter(filter_prefix);
- scan.setFilter(filter);
- List results = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
- while(scanner.next(results));
- assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
+ MultipleColumnPrefixFilter filter;
+ Scan scan = new Scan();
+ scan.setMaxVersions();
+ byte [][] filter_prefix = new byte [2][];
+ filter_prefix[0] = new byte [] {'p'};
+ filter_prefix[1] = new byte [] {'q'};
+
+ filter = new MultipleColumnPrefixFilter(filter_prefix);
+ scan.setFilter(filter);
+ List results = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
+ while(scanner.next(results));
+ assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
+ } finally {
+ region.close();
+ region.getLog().closeAndDelete();
+ }
}
@Test
@@ -178,49 +186,53 @@ public class TestMultipleColumnPrefixFilter {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, TEST_UTIL.
getDataTestDir(), TEST_UTIL.getConfiguration(),htd);
+ try {
+ List rows = generateRandomWords(100, "row");
+ List columns = generateRandomWords(10000, "column");
+ long maxTimestamp = 2;
- List rows = generateRandomWords(100, "row");
- List columns = generateRandomWords(10000, "column");
- long maxTimestamp = 2;
-
- String valueString = "ValueString";
+ String valueString = "ValueString";
- for (String row: rows) {
- Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
- for (String column: columns) {
- for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
- KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
- valueString);
- p.add(kv);
+ for (String row: rows) {
+ Put p = new Put(Bytes.toBytes(row));
+ p.setWriteToWAL(false);
+ for (String column: columns) {
+ for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+ KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
+ valueString);
+ p.add(kv);
+ }
}
+ region.put(p);
}
- region.put(p);
- }
- MultipleColumnPrefixFilter multiplePrefixFilter;
- Scan scan1 = new Scan();
- scan1.setMaxVersions();
- byte [][] filter_prefix = new byte [1][];
- filter_prefix[0] = new byte [] {'p'};
-
- multiplePrefixFilter = new MultipleColumnPrefixFilter(filter_prefix);
- scan1.setFilter(multiplePrefixFilter);
- List results1 = new ArrayList();
- InternalScanner scanner1 = region.getScanner(scan1);
- while(scanner1.next(results1));
-
- ColumnPrefixFilter singlePrefixFilter;
- Scan scan2 = new Scan();
- scan2.setMaxVersions();
- singlePrefixFilter = new ColumnPrefixFilter(Bytes.toBytes("p"));
-
- scan2.setFilter(singlePrefixFilter);
- List results2 = new ArrayList();
- InternalScanner scanner2 = region.getScanner(scan1);
- while(scanner2.next(results2));
-
- assertEquals(results1.size(), results2.size());
+ MultipleColumnPrefixFilter multiplePrefixFilter;
+ Scan scan1 = new Scan();
+ scan1.setMaxVersions();
+ byte [][] filter_prefix = new byte [1][];
+ filter_prefix[0] = new byte [] {'p'};
+
+ multiplePrefixFilter = new MultipleColumnPrefixFilter(filter_prefix);
+ scan1.setFilter(multiplePrefixFilter);
+ List results1 = new ArrayList();
+ InternalScanner scanner1 = region.getScanner(scan1);
+ while(scanner1.next(results1));
+
+ ColumnPrefixFilter singlePrefixFilter;
+ Scan scan2 = new Scan();
+ scan2.setMaxVersions();
+ singlePrefixFilter = new ColumnPrefixFilter(Bytes.toBytes("p"));
+
+ scan2.setFilter(singlePrefixFilter);
+ List results2 = new ArrayList();
+ InternalScanner scanner2 = region.getScanner(scan1);
+ while(scanner2.next(results2));
+
+ assertEquals(results1.size(), results2.size());
+ } finally {
+ region.close();
+ region.getLog().closeAndDelete();
+ }
}
List generateRandomWords(int numberOfWords, String suffix) {
diff --git src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
index 6783888..8b60723 100644
--- src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
+++ src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
@@ -388,7 +389,7 @@ public class TestMasterFailover {
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(), null, null);
- HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
+ createRegion(hriEnabled, rootdir, conf, htdEnabled);
List enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
@@ -399,7 +400,7 @@ public class TestMasterFailover {
// Write the .tableinfo
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdDisabled);
HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
- HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
+ createRegion(hriDisabled, rootdir, conf, htdDisabled);
List disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
@@ -701,7 +702,7 @@ public class TestMasterFailover {
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(),
null, null);
- HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
+ createRegion(hriEnabled, rootdir, conf, htdEnabled);
List enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
@@ -712,7 +713,7 @@ public class TestMasterFailover {
// Write the .tableinfo
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdDisabled);
HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
- HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
+ createRegion(hriDisabled, rootdir, conf, htdDisabled);
List disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
@@ -1028,6 +1029,19 @@ public class TestMasterFailover {
TEST_UTIL.shutdownMiniCluster();
}
+ HRegion createRegion(final HRegionInfo hri, final Path rootdir, final Configuration c,
+ final HTableDescriptor htd)
+ throws IOException {
+ HRegion r = HRegion.createHRegion(hri, rootdir, c, htd);
+ // The above call to create a region will create an hlog file. Each
+ // log file create will also create a running thread to do syncing. We need
+ // to close out this log else we will have a running thread trying to sync
+ // the file system continuously which is ugly when dfs is taken away at the
+ // end of the test.
+ r.getLog().closeAndDelete();
+ return r;
+ }
+
// TODO: Next test to add is with testing permutations of the RIT or the RS
// killed are hosting ROOT and META regions.
diff --git src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
index dcfc2ae..b1ef310 100644
--- src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
+++ src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
@@ -116,6 +116,7 @@ public class TestOpenedRegionHandler {
@Test
public void testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches()
throws Exception {
+ HRegion region = null;
try {
int testIndex = 0;
TEST_UTIL.startMiniZKCluster();
@@ -124,8 +125,7 @@ public class TestOpenedRegionHandler {
"testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches");
HRegionInfo hri = new HRegionInfo(htd.getName(),
Bytes.toBytes(testIndex), Bytes.toBytes(testIndex + 1));
- HRegion region = HRegion.createHRegion(hri, TEST_UTIL
- .getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+ region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
assertNotNull(region);
AssignmentManager am = Mockito.mock(AssignmentManager.class);
when(am.isRegionInTransition(hri)).thenReturn(
@@ -164,6 +164,8 @@ public class TestOpenedRegionHandler {
assertEquals("The region should not be opened successfully.", regionName,
region.getRegionInfo().getEncodedName());
} finally {
+ region.close();
+ region.getLog().closeAndDelete();
TEST_UTIL.shutdownMiniZKCluster();
}
}
diff --git src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
index 3a6d819..115cb15 100644
--- src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
+++ src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
@@ -78,7 +78,16 @@ public class TestBlocksRead extends HBaseTestCase {
EnvironmentEdgeManagerTestHelper.reset();
}
- private void initHRegion (byte [] tableName, String callingMethod,
+ /**
+ * Callers must afterward call {@link HRegion#closeHRegion(HRegion)}
+ * @param tableName
+ * @param callingMethod
+ * @param conf
+ * @param families
+ * @throws IOException
+ * @return created and initialized region.
+ */
+ private HRegion initHRegion (byte [] tableName, String callingMethod,
HBaseConfiguration conf, byte [] ... families)
throws IOException {
HTableDescriptor htd = new HTableDescriptor(tableName);
@@ -97,8 +106,9 @@ public class TestBlocksRead extends HBaseTestCase {
}
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + callingMethod);
- region = HRegion.createHRegion(info, path, conf, htd);
+ HRegion r = HRegion.createHRegion(info, path, conf, htd);
blockCache = new CacheConfig(conf).getBlockCache();
+ return r;
}
private void putData(byte[] cf, String row, String col, long version)
@@ -195,38 +205,41 @@ public class TestBlocksRead extends HBaseTestCase {
KeyValue kvs[];
HBaseConfiguration conf = getConf();
- initHRegion(TABLE, getName(), conf, FAMILIES);
-
- putData(FAMILY, "row", "col1", 1);
- putData(FAMILY, "row", "col2", 2);
- putData(FAMILY, "row", "col3", 3);
- putData(FAMILY, "row", "col4", 4);
- putData(FAMILY, "row", "col5", 5);
- putData(FAMILY, "row", "col6", 6);
- putData(FAMILY, "row", "col7", 7);
- region.flushcache();
-
- // Expected block reads: 1
- kvs = getData(FAMILY, "row", "col1", 1);
- assertEquals(1, kvs.length);
- verifyData(kvs[0], "row", "col1", 1);
-
- // Expected block reads: 2
- kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
- assertEquals(2, kvs.length);
- verifyData(kvs[0], "row", "col1", 1);
- verifyData(kvs[1], "row", "col2", 2);
-
- // Expected block reads: 3
- kvs = getData(FAMILY, "row", Arrays.asList("col2", "col3"), 3);
- assertEquals(2, kvs.length);
- verifyData(kvs[0], "row", "col2", 2);
- verifyData(kvs[1], "row", "col3", 3);
-
- // Expected block reads: 3
- kvs = getData(FAMILY, "row", Arrays.asList("col5"), 3);
- assertEquals(1, kvs.length);
- verifyData(kvs[0], "row", "col5", 5);
+ this.region = initHRegion(TABLE, getName(), conf, FAMILIES);
+ try {
+ putData(FAMILY, "row", "col1", 1);
+ putData(FAMILY, "row", "col2", 2);
+ putData(FAMILY, "row", "col3", 3);
+ putData(FAMILY, "row", "col4", 4);
+ putData(FAMILY, "row", "col5", 5);
+ putData(FAMILY, "row", "col6", 6);
+ putData(FAMILY, "row", "col7", 7);
+ region.flushcache();
+
+ // Expected block reads: 1
+ kvs = getData(FAMILY, "row", "col1", 1);
+ assertEquals(1, kvs.length);
+ verifyData(kvs[0], "row", "col1", 1);
+
+ // Expected block reads: 2
+ kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
+ assertEquals(2, kvs.length);
+ verifyData(kvs[0], "row", "col1", 1);
+ verifyData(kvs[1], "row", "col2", 2);
+
+ // Expected block reads: 3
+ kvs = getData(FAMILY, "row", Arrays.asList("col2", "col3"), 3);
+ assertEquals(2, kvs.length);
+ verifyData(kvs[0], "row", "col2", 2);
+ verifyData(kvs[1], "row", "col3", 3);
+
+ // Expected block reads: 3
+ kvs = getData(FAMILY, "row", Arrays.asList("col5"), 3);
+ assertEquals(1, kvs.length);
+ verifyData(kvs[0], "row", "col5", 5);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ }
}
/**
@@ -241,85 +254,88 @@ public class TestBlocksRead extends HBaseTestCase {
KeyValue kvs[];
HBaseConfiguration conf = getConf();
- initHRegion(TABLE, getName(), conf, FAMILIES);
-
- // File 1
- putData(FAMILY, "row", "col1", 1);
- putData(FAMILY, "row", "col2", 2);
- region.flushcache();
-
- // File 2
- putData(FAMILY, "row", "col1", 3);
- putData(FAMILY, "row", "col2", 4);
- region.flushcache();
-
- // Baseline expected blocks read: 2
- kvs = getData(FAMILY, "row", Arrays.asList("col1"), 2);
- assertEquals(1, kvs.length);
- verifyData(kvs[0], "row", "col1", 3);
-
- // Baseline expected blocks read: 4
- kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 4);
- assertEquals(2, kvs.length);
- verifyData(kvs[0], "row", "col1", 3);
- verifyData(kvs[1], "row", "col2", 4);
-
- // File 3: Add another column
- putData(FAMILY, "row", "col3", 5);
- region.flushcache();
-
- // Baseline expected blocks read: 5
- kvs = getData(FAMILY, "row", "col3", 5);
- assertEquals(1, kvs.length);
- verifyData(kvs[0], "row", "col3", 5);
-
- // Get a column from older file.
- // Baseline expected blocks read: 3
- kvs = getData(FAMILY, "row", Arrays.asList("col1"), 3);
- assertEquals(1, kvs.length);
- verifyData(kvs[0], "row", "col1", 3);
-
- // File 4: Delete the entire row.
- deleteFamily(FAMILY, "row", 6);
- region.flushcache();
-
- // Baseline expected blocks read: 6.
- kvs = getData(FAMILY, "row", "col1", 6);
- assertEquals(0, kvs.length);
- kvs = getData(FAMILY, "row", "col2", 6);
- assertEquals(0, kvs.length);
- kvs = getData(FAMILY, "row", "col3", 6);
- assertEquals(0, kvs.length);
- kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 6);
- assertEquals(0, kvs.length);
-
- // File 5: Delete
- deleteFamily(FAMILY, "row", 10);
- region.flushcache();
-
- // File 6: some more puts, but with timestamps older than the
- // previous delete.
- putData(FAMILY, "row", "col1", 7);
- putData(FAMILY, "row", "col2", 8);
- putData(FAMILY, "row", "col3", 9);
- region.flushcache();
-
- // Baseline expected blocks read: 10
- kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 10);
- assertEquals(0, kvs.length);
-
- // File 7: Put back new data
- putData(FAMILY, "row", "col1", 11);
- putData(FAMILY, "row", "col2", 12);
- putData(FAMILY, "row", "col3", 13);
- region.flushcache();
-
- // Baseline expected blocks read: 13
- kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 13);
- assertEquals(3, kvs.length);
- verifyData(kvs[0], "row", "col1", 11);
- verifyData(kvs[1], "row", "col2", 12);
- verifyData(kvs[2], "row", "col3", 13);
+ this.region = initHRegion(TABLE, getName(), conf, FAMILIES);
+ try {
+ // File 1
+ putData(FAMILY, "row", "col1", 1);
+ putData(FAMILY, "row", "col2", 2);
+ region.flushcache();
+
+ // File 2
+ putData(FAMILY, "row", "col1", 3);
+ putData(FAMILY, "row", "col2", 4);
+ region.flushcache();
+
+ // Baseline expected blocks read: 2
+ kvs = getData(FAMILY, "row", Arrays.asList("col1"), 2);
+ assertEquals(1, kvs.length);
+ verifyData(kvs[0], "row", "col1", 3);
+
+ // Baseline expected blocks read: 4
+ kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 4);
+ assertEquals(2, kvs.length);
+ verifyData(kvs[0], "row", "col1", 3);
+ verifyData(kvs[1], "row", "col2", 4);
+
+ // File 3: Add another column
+ putData(FAMILY, "row", "col3", 5);
+ region.flushcache();
+
+ // Baseline expected blocks read: 5
+ kvs = getData(FAMILY, "row", "col3", 5);
+ assertEquals(1, kvs.length);
+ verifyData(kvs[0], "row", "col3", 5);
+
+ // Get a column from older file.
+ // Baseline expected blocks read: 3
+ kvs = getData(FAMILY, "row", Arrays.asList("col1"), 3);
+ assertEquals(1, kvs.length);
+ verifyData(kvs[0], "row", "col1", 3);
+
+ // File 4: Delete the entire row.
+ deleteFamily(FAMILY, "row", 6);
+ region.flushcache();
+
+ // Baseline expected blocks read: 6.
+ kvs = getData(FAMILY, "row", "col1", 6);
+ assertEquals(0, kvs.length);
+ kvs = getData(FAMILY, "row", "col2", 6);
+ assertEquals(0, kvs.length);
+ kvs = getData(FAMILY, "row", "col3", 6);
+ assertEquals(0, kvs.length);
+ kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 6);
+ assertEquals(0, kvs.length);
+
+ // File 5: Delete
+ deleteFamily(FAMILY, "row", 10);
+ region.flushcache();
+
+ // File 6: some more puts, but with timestamps older than the
+ // previous delete.
+ putData(FAMILY, "row", "col1", 7);
+ putData(FAMILY, "row", "col2", 8);
+ putData(FAMILY, "row", "col3", 9);
+ region.flushcache();
+
+ // Baseline expected blocks read: 10
+ kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 10);
+ assertEquals(0, kvs.length);
+
+ // File 7: Put back new data
+ putData(FAMILY, "row", "col1", 11);
+ putData(FAMILY, "row", "col2", 12);
+ putData(FAMILY, "row", "col3", 13);
+ region.flushcache();
+
+ // Baseline expected blocks read: 13
+ kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 13);
+ assertEquals(3, kvs.length);
+ verifyData(kvs[0], "row", "col1", 11);
+ verifyData(kvs[1], "row", "col2", 12);
+ verifyData(kvs[2], "row", "col3", 13);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ }
}
/**
@@ -333,37 +349,40 @@ public class TestBlocksRead extends HBaseTestCase {
byte [][] FAMILIES = new byte[][] { FAMILY };
HBaseConfiguration conf = getConf();
- initHRegion(TABLE, getName(), conf, FAMILIES);
-
- putData(FAMILY, "row", "col1", 1);
- putData(FAMILY, "row", "col2", 2);
- region.flushcache();
-
- // Execute a scan with caching turned off
- // Expected blocks stored: 0
- long blocksStart = getBlkCount();
- Scan scan = new Scan();
- scan.setCacheBlocks(false);
- RegionScanner rs = region.getScanner(scan);
- List result = new ArrayList(2);
- rs.next(result);
- assertEquals(2, result.size());
- rs.close();
- long blocksEnd = getBlkCount();
-
- assertEquals(blocksStart, blocksEnd);
-
- // Execute with caching turned on
- // Expected blocks stored: 2
- blocksStart = blocksEnd;
- scan.setCacheBlocks(true);
- rs = region.getScanner(scan);
- result = new ArrayList(2);
- rs.next(result);
- assertEquals(2, result.size());
- rs.close();
- blocksEnd = getBlkCount();
-
- assertEquals(2, blocksEnd - blocksStart);
+ this.region = initHRegion(TABLE, getName(), conf, FAMILIES);
+ try {
+ putData(FAMILY, "row", "col1", 1);
+ putData(FAMILY, "row", "col2", 2);
+ region.flushcache();
+
+ // Execute a scan with caching turned off
+ // Expected blocks stored: 0
+ long blocksStart = getBlkCount();
+ Scan scan = new Scan();
+ scan.setCacheBlocks(false);
+ RegionScanner rs = region.getScanner(scan);
+ List result = new ArrayList(2);
+ rs.next(result);
+ assertEquals(2, result.size());
+ rs.close();
+ long blocksEnd = getBlkCount();
+
+ assertEquals(blocksStart, blocksEnd);
+
+ // Execute with caching turned on
+ // Expected blocks stored: 2
+ blocksStart = blocksEnd;
+ scan.setCacheBlocks(true);
+ rs = region.getScanner(scan);
+ result = new ArrayList(2);
+ rs.next(result);
+ assertEquals(2, result.size());
+ rs.close();
+ blocksEnd = getBlkCount();
+
+ assertEquals(2, blocksEnd - blocksStart);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ }
}
}
diff --git src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
index d2945fa..68d6a6b 100644
--- src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
+++ src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
@@ -70,95 +70,98 @@ public class TestColumnSeeking {
HRegion region =
HRegion.createHRegion(info, TEST_UTIL.getDataTestDir(), TEST_UTIL
.getConfiguration(), htd);
-
- List rows = generateRandomWords(10, "row");
- List allColumns = generateRandomWords(10, "column");
- List values = generateRandomWords(100, "value");
-
- long maxTimestamp = 2;
- double selectPercent = 0.5;
- int numberOfTests = 5;
- double flushPercentage = 0.2;
- double minorPercentage = 0.2;
- double majorPercentage = 0.2;
- double putPercentage = 0.2;
-
- HashMap allKVMap = new HashMap();
-
- HashMap[] kvMaps = new HashMap[numberOfTests];
- ArrayList[] columnLists = new ArrayList[numberOfTests];
-
- for (int i = 0; i < numberOfTests; i++) {
- kvMaps[i] = new HashMap();
- columnLists[i] = new ArrayList();
- for (String column : allColumns) {
- if (Math.random() < selectPercent) {
- columnLists[i].add(column);
+ try {
+ List rows = generateRandomWords(10, "row");
+ List allColumns = generateRandomWords(10, "column");
+ List values = generateRandomWords(100, "value");
+
+ long maxTimestamp = 2;
+ double selectPercent = 0.5;
+ int numberOfTests = 5;
+ double flushPercentage = 0.2;
+ double minorPercentage = 0.2;
+ double majorPercentage = 0.2;
+ double putPercentage = 0.2;
+
+ HashMap allKVMap = new HashMap();
+
+ HashMap[] kvMaps = new HashMap[numberOfTests];
+ ArrayList[] columnLists = new ArrayList[numberOfTests];
+
+ for (int i = 0; i < numberOfTests; i++) {
+ kvMaps[i] = new HashMap();
+ columnLists[i] = new ArrayList();
+ for (String column : allColumns) {
+ if (Math.random() < selectPercent) {
+ columnLists[i].add(column);
+ }
}
}
- }
- for (String value : values) {
- for (String row : rows) {
- Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
- for (String column : allColumns) {
- for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
- KeyValue kv =
- KeyValueTestUtil.create(row, family, column, timestamp, value);
- if (Math.random() < putPercentage) {
- p.add(kv);
- allKVMap.put(kv.getKeyString(), kv);
- for (int i = 0; i < numberOfTests; i++) {
- if (columnLists[i].contains(column)) {
- kvMaps[i].put(kv.getKeyString(), kv);
+ for (String value : values) {
+ for (String row : rows) {
+ Put p = new Put(Bytes.toBytes(row));
+ p.setWriteToWAL(false);
+ for (String column : allColumns) {
+ for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+ KeyValue kv =
+ KeyValueTestUtil.create(row, family, column, timestamp, value);
+ if (Math.random() < putPercentage) {
+ p.add(kv);
+ allKVMap.put(kv.getKeyString(), kv);
+ for (int i = 0; i < numberOfTests; i++) {
+ if (columnLists[i].contains(column)) {
+ kvMaps[i].put(kv.getKeyString(), kv);
+ }
}
}
}
}
- }
- region.put(p);
- if (Math.random() < flushPercentage) {
- LOG.info("Flushing... ");
- region.flushcache();
- }
+ region.put(p);
+ if (Math.random() < flushPercentage) {
+ LOG.info("Flushing... ");
+ region.flushcache();
+ }
- if (Math.random() < minorPercentage) {
- LOG.info("Minor compacting... ");
- region.compactStores(false);
- }
+ if (Math.random() < minorPercentage) {
+ LOG.info("Minor compacting... ");
+ region.compactStores(false);
+ }
- if (Math.random() < majorPercentage) {
- LOG.info("Major compacting... ");
- region.compactStores(true);
+ if (Math.random() < majorPercentage) {
+ LOG.info("Major compacting... ");
+ region.compactStores(true);
+ }
}
}
- }
- for (int i = 0; i < numberOfTests + 1; i++) {
- Collection kvSet;
- Scan scan = new Scan();
- scan.setMaxVersions();
- if (i < numberOfTests) {
- kvSet = kvMaps[i].values();
- for (String column : columnLists[i]) {
- scan.addColumn(familyBytes, Bytes.toBytes(column));
- }
- LOG.info("ExplicitColumns scanner");
- LOG.info("Columns: " + columnLists[i].size() + " Keys: "
- + kvSet.size());
- } else {
- kvSet = allKVMap.values();
- LOG.info("Wildcard scanner");
- LOG.info("Columns: " + allColumns.size() + " Keys: " + kvSet.size());
+ for (int i = 0; i < numberOfTests + 1; i++) {
+ Collection kvSet;
+ Scan scan = new Scan();
+ scan.setMaxVersions();
+ if (i < numberOfTests) {
+ kvSet = kvMaps[i].values();
+ for (String column : columnLists[i]) {
+ scan.addColumn(familyBytes, Bytes.toBytes(column));
+ }
+ LOG.info("ExplicitColumns scanner");
+ LOG.info("Columns: " + columnLists[i].size() + " Keys: "
+ + kvSet.size());
+ } else {
+ kvSet = allKVMap.values();
+ LOG.info("Wildcard scanner");
+ LOG.info("Columns: " + allColumns.size() + " Keys: " + kvSet.size());
+ }
+ InternalScanner scanner = region.getScanner(scan);
+ List results = new ArrayList();
+ while (scanner.next(results))
+ ;
+ assertEquals(kvSet.size(), results.size());
+ assertTrue(results.containsAll(kvSet));
}
- InternalScanner scanner = region.getScanner(scan);
- List results = new ArrayList();
- while (scanner.next(results))
- ;
- assertEquals(kvSet.size(), results.size());
- assertTrue(results.containsAll(kvSet));
+ } finally {
+ HRegion.closeHRegion(region);
}
}
diff --git src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
index 7882fe9..375d378 100644
--- src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
+++ src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
@@ -84,9 +84,10 @@ public class TestCompactSelection extends TestCase {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
- HRegion.createHRegion(info, basedir, conf, htd);
+ HRegion region = HRegion.createHRegion(info, basedir, conf, htd);
+ HRegion.closeHRegion(region);
Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
- HRegion region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
+ region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
store = new Store(basedir, region, hcd, fs, conf);
TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());
diff --git src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 3bd9558..6f5d20f 100644
--- src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -94,8 +94,8 @@ public class TestHRegion extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestHRegion.class);
HRegion region = null;
- private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
- private final String DIR = TEST_UTIL.getDataTestDir("TestHRegion").toString();
+ private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+ private static final String DIR = TEST_UTIL.getDataTestDir("TestHRegion").toString();
private final int MAX_VERSIONS = 2;
@@ -136,39 +136,44 @@ public class TestHRegion extends HBaseTestCase {
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
Configuration conf = HBaseConfiguration.create();
- initHRegion(tableName, method, conf, family);
- Path regiondir = region.getRegionDir();
- FileSystem fs = region.getFilesystem();
- byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
+ this.region = initHRegion(tableName, method, conf, family);
+ try {
+ Path regiondir = region.getRegionDir();
+ FileSystem fs = region.getFilesystem();
+ byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
- Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
+ Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
- long maxSeqId = 1050;
- long minSeqId = 1000;
+ long maxSeqId = 1050;
+ long minSeqId = 1000;
- for (long i = minSeqId; i <= maxSeqId; i += 10) {
- Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
- fs.create(recoveredEdits);
- HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
+ for (long i = minSeqId; i <= maxSeqId; i += 10) {
+ Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
+ fs.create(recoveredEdits);
+ HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
- long time = System.nanoTime();
- WALEdit edit = new WALEdit();
- edit.add(new KeyValue(row, family, Bytes.toBytes(i),
- time, KeyValue.Type.Put, Bytes.toBytes(i)));
- writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
- i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
+ long time = System.nanoTime();
+ WALEdit edit = new WALEdit();
+ edit.add(new KeyValue(row, family, Bytes.toBytes(i),
+ time, KeyValue.Type.Put, Bytes.toBytes(i)));
+ writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
+ i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
- writer.close();
- }
- MonitoredTask status = TaskMonitor.get().createStatus(method);
- long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId-1, null, status);
- assertEquals(maxSeqId, seqId);
- Get get = new Get(row);
- Result result = region.get(get, null);
- for (long i = minSeqId; i <= maxSeqId; i += 10) {
- List kvs = result.getColumn(family, Bytes.toBytes(i));
- assertEquals(1, kvs.size());
- assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+ writer.close();
+ }
+ MonitoredTask status = TaskMonitor.get().createStatus(method);
+ long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId-1, null, status);
+ assertEquals(maxSeqId, seqId);
+ Get get = new Get(row);
+ Result result = region.get(get, null);
+ for (long i = minSeqId; i <= maxSeqId; i += 10) {
+ List kvs = result.getColumn(family, Bytes.toBytes(i));
+ assertEquals(1, kvs.size());
+ assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+ }
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
}
@@ -176,44 +181,49 @@ public class TestHRegion extends HBaseTestCase {
String method = "testSkipRecoveredEditsReplaySomeIgnored";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
- initHRegion(tableName, method, HBaseConfiguration.create(), family);
- Path regiondir = region.getRegionDir();
- FileSystem fs = region.getFilesystem();
- byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
-
- Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
-
- long maxSeqId = 1050;
- long minSeqId = 1000;
-
- for (long i = minSeqId; i <= maxSeqId; i += 10) {
- Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
- fs.create(recoveredEdits);
- HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
-
- long time = System.nanoTime();
- WALEdit edit = new WALEdit();
- edit.add(new KeyValue(row, family, Bytes.toBytes(i),
- time, KeyValue.Type.Put, Bytes.toBytes(i)));
- writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
- i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
-
- writer.close();
- }
- long recoverSeqId = 1030;
- MonitoredTask status = TaskMonitor.get().createStatus(method);
- long seqId = region.replayRecoveredEditsIfAny(regiondir, recoverSeqId-1, null, status);
- assertEquals(maxSeqId, seqId);
- Get get = new Get(row);
- Result result = region.get(get, null);
- for (long i = minSeqId; i <= maxSeqId; i += 10) {
- List kvs = result.getColumn(family, Bytes.toBytes(i));
- if (i < recoverSeqId) {
- assertEquals(0, kvs.size());
- } else {
- assertEquals(1, kvs.size());
- assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+ this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
+ try {
+ Path regiondir = region.getRegionDir();
+ FileSystem fs = region.getFilesystem();
+ byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
+
+ Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
+
+ long maxSeqId = 1050;
+ long minSeqId = 1000;
+
+ for (long i = minSeqId; i <= maxSeqId; i += 10) {
+ Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
+ fs.create(recoveredEdits);
+ HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
+
+ long time = System.nanoTime();
+ WALEdit edit = new WALEdit();
+ edit.add(new KeyValue(row, family, Bytes.toBytes(i),
+ time, KeyValue.Type.Put, Bytes.toBytes(i)));
+ writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
+ i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
+
+ writer.close();
}
+ long recoverSeqId = 1030;
+ MonitoredTask status = TaskMonitor.get().createStatus(method);
+ long seqId = region.replayRecoveredEditsIfAny(regiondir, recoverSeqId-1, null, status);
+ assertEquals(maxSeqId, seqId);
+ Get get = new Get(row);
+ Result result = region.get(get, null);
+ for (long i = minSeqId; i <= maxSeqId; i += 10) {
+ List kvs = result.getColumn(family, Bytes.toBytes(i));
+ if (i < recoverSeqId) {
+ assertEquals(0, kvs.size());
+ } else {
+ assertEquals(1, kvs.size());
+ assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+ }
+ }
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
}
@@ -221,25 +231,30 @@ public class TestHRegion extends HBaseTestCase {
String method = "testSkipRecoveredEditsReplayAllIgnored";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
- initHRegion(tableName, method, HBaseConfiguration.create(), family);
- Path regiondir = region.getRegionDir();
- FileSystem fs = region.getFilesystem();
-
- Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
- for (int i = 1000; i < 1050; i += 10) {
+ this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
+ try {
+ Path regiondir = region.getRegionDir();
+ FileSystem fs = region.getFilesystem();
+
+ Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
+ for (int i = 1000; i < 1050; i += 10) {
+ Path recoveredEdits = new Path(
+ recoveredEditsDir, String.format("%019d", i));
+ FSDataOutputStream dos= fs.create(recoveredEdits);
+ dos.writeInt(i);
+ dos.close();
+ }
+ long minSeqId = 2000;
Path recoveredEdits = new Path(
- recoveredEditsDir, String.format("%019d", i));
+ recoveredEditsDir, String.format("%019d", minSeqId-1));
FSDataOutputStream dos= fs.create(recoveredEdits);
- dos.writeInt(i);
dos.close();
+ long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId, null, null);
+ assertEquals(minSeqId, seqId);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
- long minSeqId = 2000;
- Path recoveredEdits = new Path(
- recoveredEditsDir, String.format("%019d", minSeqId-1));
- FSDataOutputStream dos= fs.create(recoveredEdits);
- dos.close();
- long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId, null, null);
- assertEquals(minSeqId, seqId);
}
public void testGetWhileRegionClose() throws IOException {
@@ -249,52 +264,56 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, hc, families);
-
- // Put data in region
- final int startRow = 100;
- putData(startRow, numRows, qual1, families);
- putData(startRow, numRows, qual2, families);
- putData(startRow, numRows, qual3, families);
- // this.region.flushcache();
- final AtomicBoolean done = new AtomicBoolean(false);
- final AtomicInteger gets = new AtomicInteger(0);
- GetTillDoneOrException [] threads = new GetTillDoneOrException[10];
+ this.region = initHRegion(tableName, method, hc, families);
try {
- // Set ten threads running concurrently getting from the region.
- for (int i = 0; i < threads.length / 2; i++) {
- threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
- done, gets);
- threads[i].setDaemon(true);
- threads[i].start();
- }
- // Artificially make the condition by setting closing flag explicitly.
- // I can't make the issue happen with a call to region.close().
- this.region.closing.set(true);
- for (int i = threads.length / 2; i < threads.length; i++) {
- threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
- done, gets);
- threads[i].setDaemon(true);
- threads[i].start();
- }
- } finally {
- if (this.region != null) {
- this.region.close();
- this.region.getLog().closeAndDelete();
- }
- }
- done.set(true);
- for (GetTillDoneOrException t: threads) {
+ // Put data in region
+ final int startRow = 100;
+ putData(startRow, numRows, qual1, families);
+ putData(startRow, numRows, qual2, families);
+ putData(startRow, numRows, qual3, families);
+ // this.region.flushcache();
+ final AtomicBoolean done = new AtomicBoolean(false);
+ final AtomicInteger gets = new AtomicInteger(0);
+ GetTillDoneOrException [] threads = new GetTillDoneOrException[10];
try {
- t.join();
- } catch (InterruptedException e) {
- e.printStackTrace();
+ // Set ten threads running concurrently getting from the region.
+ for (int i = 0; i < threads.length / 2; i++) {
+ threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
+ done, gets);
+ threads[i].setDaemon(true);
+ threads[i].start();
+ }
+ // Artificially make the condition by setting closing flag explicitly.
+ // I can't make the issue happen with a call to region.close().
+ this.region.closing.set(true);
+ for (int i = threads.length / 2; i < threads.length; i++) {
+ threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
+ done, gets);
+ threads[i].setDaemon(true);
+ threads[i].start();
+ }
+ } finally {
+ if (this.region != null) {
+ this.region.close();
+ this.region.getLog().closeAndDelete();
+ }
}
- if (t.e != null) {
- LOG.info("Exception=" + t.e);
- assertFalse("Found a NPE in " + t.getName(),
- t.e instanceof NullPointerException);
+ done.set(true);
+ for (GetTillDoneOrException t: threads) {
+ try {
+ t.join();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ if (t.e != null) {
+ LOG.info("Exception=" + t.e);
+ assertFalse("Found a NPE in " + t.getName(),
+ t.e instanceof NullPointerException);
+ }
}
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
}
@@ -338,38 +357,43 @@ public class TestHRegion extends HBaseTestCase {
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
- initHRegion(TABLE, getName(), FAMILIES);
- String value = "this is the value";
- String value2 = "this is some other value";
- String keyPrefix1 = "prefix1"; // UUID.randomUUID().toString();
- String keyPrefix2 = "prefix2"; // UUID.randomUUID().toString();
- String keyPrefix3 = "prefix3"; // UUID.randomUUID().toString();
- putRows(this.region, 3, value, keyPrefix1);
- putRows(this.region, 3, value, keyPrefix2);
- putRows(this.region, 3, value, keyPrefix3);
- // this.region.flushCommits();
- putRows(this.region, 3, value2, keyPrefix1);
- putRows(this.region, 3, value2, keyPrefix2);
- putRows(this.region, 3, value2, keyPrefix3);
- System.out.println("Checking values for key: " + keyPrefix1);
- assertEquals("Got back incorrect number of rows from scan", 3,
- getNumberOfRows(keyPrefix1, value2, this.region));
- System.out.println("Checking values for key: " + keyPrefix2);
- assertEquals("Got back incorrect number of rows from scan", 3,
- getNumberOfRows(keyPrefix2, value2, this.region));
- System.out.println("Checking values for key: " + keyPrefix3);
- assertEquals("Got back incorrect number of rows from scan", 3,
- getNumberOfRows(keyPrefix3, value2, this.region));
- deleteColumns(this.region, value2, keyPrefix1);
- deleteColumns(this.region, value2, keyPrefix2);
- deleteColumns(this.region, value2, keyPrefix3);
- System.out.println("Starting important checks.....");
- assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
- 0, getNumberOfRows(keyPrefix1, value2, this.region));
- assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
- 0, getNumberOfRows(keyPrefix2, value2, this.region));
- assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
- 0, getNumberOfRows(keyPrefix3, value2, this.region));
+ this.region = initHRegion(TABLE, getName(), FAMILIES);
+ try {
+ String value = "this is the value";
+ String value2 = "this is some other value";
+ String keyPrefix1 = "prefix1"; // UUID.randomUUID().toString();
+ String keyPrefix2 = "prefix2"; // UUID.randomUUID().toString();
+ String keyPrefix3 = "prefix3"; // UUID.randomUUID().toString();
+ putRows(this.region, 3, value, keyPrefix1);
+ putRows(this.region, 3, value, keyPrefix2);
+ putRows(this.region, 3, value, keyPrefix3);
+ // this.region.flushCommits();
+ putRows(this.region, 3, value2, keyPrefix1);
+ putRows(this.region, 3, value2, keyPrefix2);
+ putRows(this.region, 3, value2, keyPrefix3);
+ System.out.println("Checking values for key: " + keyPrefix1);
+ assertEquals("Got back incorrect number of rows from scan", 3,
+ getNumberOfRows(keyPrefix1, value2, this.region));
+ System.out.println("Checking values for key: " + keyPrefix2);
+ assertEquals("Got back incorrect number of rows from scan", 3,
+ getNumberOfRows(keyPrefix2, value2, this.region));
+ System.out.println("Checking values for key: " + keyPrefix3);
+ assertEquals("Got back incorrect number of rows from scan", 3,
+ getNumberOfRows(keyPrefix3, value2, this.region));
+ deleteColumns(this.region, value2, keyPrefix1);
+ deleteColumns(this.region, value2, keyPrefix2);
+ deleteColumns(this.region, value2, keyPrefix3);
+ System.out.println("Starting important checks.....");
+ assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
+ 0, getNumberOfRows(keyPrefix1, value2, this.region));
+ assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
+ 0, getNumberOfRows(keyPrefix2, value2, this.region));
+ assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
+ 0, getNumberOfRows(keyPrefix3, value2, this.region));
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
private void deleteColumns(HRegion r, String value, String keyPrefix)
@@ -454,17 +478,22 @@ public class TestHRegion extends HBaseTestCase {
public void testFamilyWithAndWithoutColon() throws Exception {
byte [] b = Bytes.toBytes(getName());
byte [] cf = Bytes.toBytes("cf");
- initHRegion(b, getName(), cf);
- Put p = new Put(b);
- byte [] cfwithcolon = Bytes.toBytes("cf:");
- p.add(cfwithcolon, cfwithcolon, cfwithcolon);
- boolean exception = false;
+ this.region = initHRegion(b, getName(), cf);
try {
- this.region.put(p);
- } catch (NoSuchColumnFamilyException e) {
- exception = true;
+ Put p = new Put(b);
+ byte [] cfwithcolon = Bytes.toBytes("cf:");
+ p.add(cfwithcolon, cfwithcolon, cfwithcolon);
+ boolean exception = false;
+ try {
+ this.region.put(p);
+ } catch (NoSuchColumnFamilyException e) {
+ exception = true;
+ }
+ assertTrue(exception);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
- assertTrue(exception);
}
@SuppressWarnings("unchecked")
@@ -473,96 +502,100 @@ public class TestHRegion extends HBaseTestCase {
byte[] cf = Bytes.toBytes("cf");
byte[] qual = Bytes.toBytes("qual");
byte[] val = Bytes.toBytes("val");
- initHRegion(b, getName(), cf);
-
- HLog.getSyncOps(); // clear counter from prior tests
- assertEquals(0, HLog.getSyncOps());
-
- LOG.info("First a batch put with all valid puts");
- final Put[] puts = new Put[10];
- for (int i = 0; i < 10; i++) {
- puts[i] = new Put(Bytes.toBytes("row_" + i));
- puts[i].add(cf, qual, val);
- }
-
- OperationStatus[] codes = this.region.put(puts);
- assertEquals(10, codes.length);
- for (int i = 0; i < 10; i++) {
- assertEquals(OperationStatusCode.SUCCESS, codes[i]
- .getOperationStatusCode());
- }
- assertEquals(1, HLog.getSyncOps());
-
- LOG.info("Next a batch put with one invalid family");
- puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
- codes = this.region.put(puts);
- assertEquals(10, codes.length);
- for (int i = 0; i < 10; i++) {
- assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
- OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
- }
- assertEquals(1, HLog.getSyncOps());
-
- LOG.info("Next a batch put that has to break into two batches to avoid a lock");
- Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
-
- MultithreadedTestUtil.TestContext ctx =
- new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
- final AtomicReference retFromThread =
- new AtomicReference();
- TestThread putter = new TestThread(ctx) {
- @Override
- public void doWork() throws IOException {
- retFromThread.set(region.put(puts));
+ this.region = initHRegion(b, getName(), cf);
+ try {
+ HLog.getSyncOps(); // clear counter from prior tests
+ assertEquals(0, HLog.getSyncOps());
+
+ LOG.info("First a batch put with all valid puts");
+ final Put[] puts = new Put[10];
+ for (int i = 0; i < 10; i++) {
+ puts[i] = new Put(Bytes.toBytes("row_" + i));
+ puts[i].add(cf, qual, val);
}
- };
- LOG.info("...starting put thread while holding lock");
- ctx.addThread(putter);
- ctx.startThreads();
-
- LOG.info("...waiting for put thread to sync first time");
- long startWait = System.currentTimeMillis();
- while (HLog.getSyncOps() == 0) {
- Thread.sleep(100);
- if (System.currentTimeMillis() - startWait > 10000) {
- fail("Timed out waiting for thread to sync first minibatch");
+
+ OperationStatus[] codes = this.region.put(puts);
+ assertEquals(10, codes.length);
+ for (int i = 0; i < 10; i++) {
+ assertEquals(OperationStatusCode.SUCCESS, codes[i]
+ .getOperationStatusCode());
}
- }
- LOG.info("...releasing row lock, which should let put thread continue");
- region.releaseRowLock(lockedRow);
- LOG.info("...joining on thread");
- ctx.stop();
- LOG.info("...checking that next batch was synced");
- assertEquals(1, HLog.getSyncOps());
- codes = retFromThread.get();
- for (int i = 0; i < 10; i++) {
- assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
- OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
- }
+ assertEquals(1, HLog.getSyncOps());
+
+ LOG.info("Next a batch put with one invalid family");
+ puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
+ codes = this.region.put(puts);
+ assertEquals(10, codes.length);
+ for (int i = 0; i < 10; i++) {
+ assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
+ OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
+ }
+ assertEquals(1, HLog.getSyncOps());
+
+ LOG.info("Next a batch put that has to break into two batches to avoid a lock");
+ Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
+
+ MultithreadedTestUtil.TestContext ctx =
+ new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
+ final AtomicReference retFromThread =
+ new AtomicReference();
+ TestThread putter = new TestThread(ctx) {
+ @Override
+ public void doWork() throws IOException {
+ retFromThread.set(region.put(puts));
+ }
+ };
+ LOG.info("...starting put thread while holding lock");
+ ctx.addThread(putter);
+ ctx.startThreads();
+
+ LOG.info("...waiting for put thread to sync first time");
+ long startWait = System.currentTimeMillis();
+ while (HLog.getSyncOps() == 0) {
+ Thread.sleep(100);
+ if (System.currentTimeMillis() - startWait > 10000) {
+ fail("Timed out waiting for thread to sync first minibatch");
+ }
+ }
+ LOG.info("...releasing row lock, which should let put thread continue");
+ region.releaseRowLock(lockedRow);
+ LOG.info("...joining on thread");
+ ctx.stop();
+ LOG.info("...checking that next batch was synced");
+ assertEquals(1, HLog.getSyncOps());
+ codes = retFromThread.get();
+ for (int i = 0; i < 10; i++) {
+ assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
+ OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
+ }
- LOG.info("Nexta, a batch put which uses an already-held lock");
- lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
- LOG.info("...obtained row lock");
- List> putsAndLocks = Lists.newArrayList();
- for (int i = 0; i < 10; i++) {
- Pair pair = new Pair(puts[i], null);
- if (i == 2) pair.setSecond(lockedRow);
- putsAndLocks.add(pair);
- }
+ LOG.info("Nexta, a batch put which uses an already-held lock");
+ lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
+ LOG.info("...obtained row lock");
+ List> putsAndLocks = Lists.newArrayList();
+ for (int i = 0; i < 10; i++) {
+ Pair pair = new Pair(puts[i], null);
+ if (i == 2) pair.setSecond(lockedRow);
+ putsAndLocks.add(pair);
+ }
- codes = region.put(putsAndLocks.toArray(new Pair[0]));
- LOG.info("...performed put");
- for (int i = 0; i < 10; i++) {
- assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
- OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
- }
- // Make sure we didn't do an extra batch
- assertEquals(1, HLog.getSyncOps());
+ codes = region.put(putsAndLocks.toArray(new Pair[0]));
+ LOG.info("...performed put");
+ for (int i = 0; i < 10; i++) {
+ assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
+ OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
+ }
+ // Make sure we didn't do an extra batch
+ assertEquals(1, HLog.getSyncOps());
- // Make sure we still hold lock
- assertTrue(region.isRowLocked(lockedRow));
- LOG.info("...releasing lock");
- region.releaseRowLock(lockedRow);
+ // Make sure we still hold lock
+ assertTrue(region.isRowLocked(lockedRow));
+ LOG.info("...releasing lock");
+ region.releaseRowLock(lockedRow);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
//////////////////////////////////////////////////////////////////////////////
@@ -580,65 +613,68 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, fam1);
-
- //Putting empty data in key
- Put put = new Put(row1);
- put.add(fam1, qf1, emptyVal);
-
- //checkAndPut with empty value
- boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(emptyVal), put, lockId, true);
- assertTrue(res);
-
- //Putting data in key
- put = new Put(row1);
- put.add(fam1, qf1, val1);
-
- //checkAndPut with correct value
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(emptyVal), put, lockId, true);
- assertTrue(res);
-
- // not empty anymore
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(emptyVal), put, lockId, true);
- assertFalse(res);
-
- Delete delete = new Delete(row1);
- delete.deleteColumn(fam1, qf1);
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(emptyVal), delete, lockId, true);
- assertFalse(res);
-
- put = new Put(row1);
- put.add(fam1, qf1, val2);
- //checkAndPut with correct value
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(val1), put, lockId, true);
- assertTrue(res);
-
- //checkAndDelete with correct value
- delete = new Delete(row1);
- delete.deleteColumn(fam1, qf1);
- delete.deleteColumn(fam1, qf1);
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(val2), delete, lockId, true);
- assertTrue(res);
-
- delete = new Delete(row1);
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(emptyVal), delete, lockId, true);
- assertTrue(res);
-
- //checkAndPut looking for a null value
- put = new Put(row1);
- put.add(fam1, qf1, val1);
-
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new NullComparator(), put, lockId, true);
- assertTrue(res);
-
+ this.region = initHRegion(tableName, method, fam1);
+ try {
+ //Putting empty data in key
+ Put put = new Put(row1);
+ put.add(fam1, qf1, emptyVal);
+
+ //checkAndPut with empty value
+ boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(emptyVal), put, lockId, true);
+ assertTrue(res);
+
+ //Putting data in key
+ put = new Put(row1);
+ put.add(fam1, qf1, val1);
+
+ //checkAndPut with correct value
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(emptyVal), put, lockId, true);
+ assertTrue(res);
+
+ // not empty anymore
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(emptyVal), put, lockId, true);
+ assertFalse(res);
+
+ Delete delete = new Delete(row1);
+ delete.deleteColumn(fam1, qf1);
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(emptyVal), delete, lockId, true);
+ assertFalse(res);
+
+ put = new Put(row1);
+ put.add(fam1, qf1, val2);
+ //checkAndPut with correct value
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(val1), put, lockId, true);
+ assertTrue(res);
+
+ //checkAndDelete with correct value
+ delete = new Delete(row1);
+ delete.deleteColumn(fam1, qf1);
+ delete.deleteColumn(fam1, qf1);
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(val2), delete, lockId, true);
+ assertTrue(res);
+
+ delete = new Delete(row1);
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(emptyVal), delete, lockId, true);
+ assertTrue(res);
+
+ //checkAndPut looking for a null value
+ put = new Put(row1);
+ put.add(fam1, qf1, val1);
+
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new NullComparator(), put, lockId, true);
+ assertTrue(res);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testCheckAndMutate_WithWrongValue() throws IOException{
@@ -652,24 +688,28 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, fam1);
-
- //Putting data in key
- Put put = new Put(row1);
- put.add(fam1, qf1, val1);
- region.put(put);
-
- //checkAndPut with wrong value
- boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(val2), put, lockId, true);
- assertEquals(false, res);
+ this.region = initHRegion(tableName, method, fam1);
+ try {
+ //Putting data in key
+ Put put = new Put(row1);
+ put.add(fam1, qf1, val1);
+ region.put(put);
- //checkAndDelete with wrong value
- Delete delete = new Delete(row1);
- delete.deleteFamily(fam1);
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(val2), delete, lockId, true);
- assertEquals(false, res);
+ //checkAndPut with wrong value
+ boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(val2), put, lockId, true);
+ assertEquals(false, res);
+
+ //checkAndDelete with wrong value
+ Delete delete = new Delete(row1);
+ delete.deleteFamily(fam1);
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(val2), delete, lockId, true);
+ assertEquals(false, res);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testCheckAndMutate_WithCorrectValue() throws IOException{
@@ -682,24 +722,28 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, fam1);
-
- //Putting data in key
- Put put = new Put(row1);
- put.add(fam1, qf1, val1);
- region.put(put);
-
- //checkAndPut with correct value
- boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(val1), put, lockId, true);
- assertEquals(true, res);
+ this.region = initHRegion(tableName, method, fam1);
+ try {
+ //Putting data in key
+ Put put = new Put(row1);
+ put.add(fam1, qf1, val1);
+ region.put(put);
- //checkAndDelete with correct value
- Delete delete = new Delete(row1);
- delete.deleteColumn(fam1, qf1);
- res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(val1), put, lockId, true);
- assertEquals(true, res);
+ //checkAndPut with correct value
+ boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(val1), put, lockId, true);
+ assertEquals(true, res);
+
+ //checkAndDelete with correct value
+ Delete delete = new Delete(row1);
+ delete.deleteColumn(fam1, qf1);
+ res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(val1), put, lockId, true);
+ assertEquals(true, res);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testCheckAndPut_ThatPutWasWritten() throws IOException{
@@ -716,52 +760,59 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, families);
-
- //Putting data in the key to check
- Put put = new Put(row1);
- put.add(fam1, qf1, val1);
- region.put(put);
+ this.region = initHRegion(tableName, method, families);
+ try {
+ //Putting data in the key to check
+ Put put = new Put(row1);
+ put.add(fam1, qf1, val1);
+ region.put(put);
- //Creating put to add
- long ts = System.currentTimeMillis();
- KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
- put = new Put(row1);
- put.add(kv);
+ //Creating put to add
+ long ts = System.currentTimeMillis();
+ KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
+ put = new Put(row1);
+ put.add(kv);
- //checkAndPut with wrong value
- Store store = region.getStore(fam1);
- store.memstore.kvset.size();
+ //checkAndPut with wrong value
+ Store store = region.getStore(fam1);
+ store.memstore.kvset.size();
- boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
- new BinaryComparator(val1), put, lockId, true);
- assertEquals(true, res);
- store.memstore.kvset.size();
+ boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+ new BinaryComparator(val1), put, lockId, true);
+ assertEquals(true, res);
+ store.memstore.kvset.size();
- Get get = new Get(row1);
- get.addColumn(fam2, qf1);
- KeyValue [] actual = region.get(get, null).raw();
+ Get get = new Get(row1);
+ get.addColumn(fam2, qf1);
+ KeyValue [] actual = region.get(get, null).raw();
- KeyValue [] expected = {kv};
+ KeyValue [] expected = {kv};
- assertEquals(expected.length, actual.length);
- for(int i=0; i kvs = new ArrayList();
- kvs.add(new KeyValue(row1, fam4, null, null));
+ this.region = initHRegion(tableName, method, fam1, fam2, fam3);
+ try {
+ List kvs = new ArrayList();
+ kvs.add(new KeyValue(row1, fam4, null, null));
- //testing existing family
- byte [] family = fam2;
- try {
- Map> deleteMap = new HashMap>();
- deleteMap.put(family, kvs);
- region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
- } catch (Exception e) {
- assertTrue("Family " +new String(family)+ " does not exist", false);
- }
+ //testing existing family
+ byte [] family = fam2;
+ try {
+ Map> deleteMap = new HashMap>();
+ deleteMap.put(family, kvs);
+ region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+ } catch (Exception e) {
+ assertTrue("Family " +new String(family)+ " does not exist", false);
+ }
- //testing non existing family
- boolean ok = false;
- family = fam4;
- try {
- Map> deleteMap = new HashMap>();
- deleteMap.put(family, kvs);
- region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
- } catch (Exception e) {
- ok = true;
+ //testing non existing family
+ boolean ok = false;
+ family = fam4;
+ try {
+ Map> deleteMap = new HashMap>();
+ deleteMap.put(family, kvs);
+ region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+ } catch (Exception e) {
+ ok = true;
+ }
+ assertEquals("Family " +new String(family)+ " does exist", true, ok);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
- assertEquals("Family " +new String(family)+ " does exist", true, ok);
}
public void testDelete_mixed() throws IOException, InterruptedException {
@@ -914,62 +977,67 @@ public class TestHRegion extends HBaseTestCase {
byte [] fam = Bytes.toBytes("info");
byte [][] families = {fam};
String method = this.getName();
- initHRegion(tableName, method, families);
- EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
+ this.region = initHRegion(tableName, method, families);
+ try {
+ EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
- byte [] row = Bytes.toBytes("table_name");
- // column names
- byte [] serverinfo = Bytes.toBytes("serverinfo");
- byte [] splitA = Bytes.toBytes("splitA");
- byte [] splitB = Bytes.toBytes("splitB");
+ byte [] row = Bytes.toBytes("table_name");
+ // column names
+ byte [] serverinfo = Bytes.toBytes("serverinfo");
+ byte [] splitA = Bytes.toBytes("splitA");
+ byte [] splitB = Bytes.toBytes("splitB");
- // add some data:
- Put put = new Put(row);
- put.add(fam, splitA, Bytes.toBytes("reference_A"));
- region.put(put);
+ // add some data:
+ Put put = new Put(row);
+ put.add(fam, splitA, Bytes.toBytes("reference_A"));
+ region.put(put);
- put = new Put(row);
- put.add(fam, splitB, Bytes.toBytes("reference_B"));
- region.put(put);
+ put = new Put(row);
+ put.add(fam, splitB, Bytes.toBytes("reference_B"));
+ region.put(put);
- put = new Put(row);
- put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
- region.put(put);
+ put = new Put(row);
+ put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
+ region.put(put);
- // ok now delete a split:
- Delete delete = new Delete(row);
- delete.deleteColumns(fam, splitA);
- region.delete(delete, null, true);
+ // ok now delete a split:
+ Delete delete = new Delete(row);
+ delete.deleteColumns(fam, splitA);
+ region.delete(delete, null, true);
- // assert some things:
- Get get = new Get(row).addColumn(fam, serverinfo);
- Result result = region.get(get, null);
- assertEquals(1, result.size());
+ // assert some things:
+ Get get = new Get(row).addColumn(fam, serverinfo);
+ Result result = region.get(get, null);
+ assertEquals(1, result.size());
- get = new Get(row).addColumn(fam, splitA);
- result = region.get(get, null);
- assertEquals(0, result.size());
+ get = new Get(row).addColumn(fam, splitA);
+ result = region.get(get, null);
+ assertEquals(0, result.size());
- get = new Get(row).addColumn(fam, splitB);
- result = region.get(get, null);
- assertEquals(1, result.size());
+ get = new Get(row).addColumn(fam, splitB);
+ result = region.get(get, null);
+ assertEquals(1, result.size());
- // Assert that after a delete, I can put.
- put = new Put(row);
- put.add(fam, splitA, Bytes.toBytes("reference_A"));
- region.put(put);
- get = new Get(row);
- result = region.get(get, null);
- assertEquals(3, result.size());
-
- // Now delete all... then test I can add stuff back
- delete = new Delete(row);
- region.delete(delete, null, false);
- assertEquals(0, region.get(get, null).size());
-
- region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
- result = region.get(get, null);
- assertEquals(1, result.size());
+ // Assert that after a delete, I can put.
+ put = new Put(row);
+ put.add(fam, splitA, Bytes.toBytes("reference_A"));
+ region.put(put);
+ get = new Get(row);
+ result = region.get(get, null);
+ assertEquals(3, result.size());
+
+ // Now delete all... then test I can add stuff back
+ delete = new Delete(row);
+ region.delete(delete, null, false);
+ assertEquals(0, region.get(get, null).size());
+
+ region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
+ result = region.get(get, null);
+ assertEquals(1, result.size());
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testDeleteRowWithFutureTs() throws IOException {
@@ -977,34 +1045,38 @@ public class TestHRegion extends HBaseTestCase {
byte [] fam = Bytes.toBytes("info");
byte [][] families = {fam};
String method = this.getName();
- initHRegion(tableName, method, families);
-
- byte [] row = Bytes.toBytes("table_name");
- // column names
- byte [] serverinfo = Bytes.toBytes("serverinfo");
+ this.region = initHRegion(tableName, method, families);
+ try {
+ byte [] row = Bytes.toBytes("table_name");
+ // column names
+ byte [] serverinfo = Bytes.toBytes("serverinfo");
- // add data in the far future
- Put put = new Put(row);
- put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP-5,Bytes.toBytes("value"));
- region.put(put);
+ // add data in the far future
+ Put put = new Put(row);
+ put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP-5,Bytes.toBytes("value"));
+ region.put(put);
- // now delete something in the present
- Delete delete = new Delete(row);
- region.delete(delete, null, true);
+ // now delete something in the present
+ Delete delete = new Delete(row);
+ region.delete(delete, null, true);
- // make sure we still see our data
- Get get = new Get(row).addColumn(fam, serverinfo);
- Result result = region.get(get, null);
- assertEquals(1, result.size());
+ // make sure we still see our data
+ Get get = new Get(row).addColumn(fam, serverinfo);
+ Result result = region.get(get, null);
+ assertEquals(1, result.size());
- // delete the future row
- delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
- region.delete(delete, null, true);
+ // delete the future row
+ delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
+ region.delete(delete, null, true);
- // make sure it is gone
- get = new Get(row).addColumn(fam, serverinfo);
- result = region.get(get, null);
- assertEquals(0, result.size());
+ // make sure it is gone
+ get = new Get(row).addColumn(fam, serverinfo);
+ result = region.get(get, null);
+ assertEquals(0, result.size());
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
/**
@@ -1016,41 +1088,45 @@ public class TestHRegion extends HBaseTestCase {
byte [] fam = Bytes.toBytes("info");
byte [][] families = {fam};
String method = this.getName();
- initHRegion(tableName, method, families);
-
- byte [] row = Bytes.toBytes("row1");
- // column names
- byte [] qual = Bytes.toBytes("qual");
-
- // add data with LATEST_TIMESTAMP, put without WAL
- Put put = new Put(row);
- put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
- region.put(put, false);
+ this.region = initHRegion(tableName, method, families);
+ try {
+ byte [] row = Bytes.toBytes("row1");
+ // column names
+ byte [] qual = Bytes.toBytes("qual");
- // Make sure it shows up with an actual timestamp
- Get get = new Get(row).addColumn(fam, qual);
- Result result = region.get(get, null);
- assertEquals(1, result.size());
- KeyValue kv = result.raw()[0];
- LOG.info("Got: " + kv);
- assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
- kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
-
- // Check same with WAL enabled (historically these took different
- // code paths, so check both)
- row = Bytes.toBytes("row2");
- put = new Put(row);
- put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
- region.put(put, true);
-
- // Make sure it shows up with an actual timestamp
- get = new Get(row).addColumn(fam, qual);
- result = region.get(get, null);
- assertEquals(1, result.size());
- kv = result.raw()[0];
- LOG.info("Got: " + kv);
- assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
- kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
+ // add data with LATEST_TIMESTAMP, put without WAL
+ Put put = new Put(row);
+ put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
+ region.put(put, false);
+
+ // Make sure it shows up with an actual timestamp
+ Get get = new Get(row).addColumn(fam, qual);
+ Result result = region.get(get, null);
+ assertEquals(1, result.size());
+ KeyValue kv = result.raw()[0];
+ LOG.info("Got: " + kv);
+ assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
+ kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
+
+ // Check same with WAL enabled (historically these took different
+ // code paths, so check both)
+ row = Bytes.toBytes("row2");
+ put = new Put(row);
+ put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
+ region.put(put, true);
+
+ // Make sure it shows up with an actual timestamp
+ get = new Get(row).addColumn(fam, qual);
+ result = region.get(get, null);
+ assertEquals(1, result.size());
+ kv = result.raw()[0];
+ LOG.info("Got: " + kv);
+ assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
+ kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
@@ -1058,39 +1134,42 @@ public class TestHRegion extends HBaseTestCase {
byte [] tableName = Bytes.toBytes("test_table");
byte [] fam1 = Bytes.toBytes("columnA");
byte [] fam2 = Bytes.toBytes("columnB");
- initHRegion(tableName, getName(), fam1, fam2);
-
- byte [] rowA = Bytes.toBytes("rowA");
- byte [] rowB = Bytes.toBytes("rowB");
-
- byte [] value = Bytes.toBytes("value");
+ this.region = initHRegion(tableName, getName(), fam1, fam2);
+ try {
+ byte [] rowA = Bytes.toBytes("rowA");
+ byte [] rowB = Bytes.toBytes("rowB");
- Delete delete = new Delete(rowA);
- delete.deleteFamily(fam1);
+ byte [] value = Bytes.toBytes("value");
- region.delete(delete, null, true);
+ Delete delete = new Delete(rowA);
+ delete.deleteFamily(fam1);
- // now create data.
- Put put = new Put(rowA);
- put.add(fam2, null, value);
- region.put(put);
+ region.delete(delete, null, true);
- put = new Put(rowB);
- put.add(fam1, null, value);
- put.add(fam2, null, value);
- region.put(put);
+ // now create data.
+ Put put = new Put(rowA);
+ put.add(fam2, null, value);
+ region.put(put);
- Scan scan = new Scan();
- scan.addFamily(fam1).addFamily(fam2);
- InternalScanner s = region.getScanner(scan);
- List results = new ArrayList();
- s.next(results);
- assertTrue(Bytes.equals(rowA, results.get(0).getRow()));
+ put = new Put(rowB);
+ put.add(fam1, null, value);
+ put.add(fam2, null, value);
+ region.put(put);
- results.clear();
- s.next(results);
- assertTrue(Bytes.equals(rowB, results.get(0).getRow()));
+ Scan scan = new Scan();
+ scan.addFamily(fam1).addFamily(fam2);
+ InternalScanner s = region.getScanner(scan);
+ List results = new ArrayList();
+ s.next(results);
+ assertTrue(Bytes.equals(rowA, results.get(0).getRow()));
+ results.clear();
+ s.next(results);
+ assertTrue(Bytes.equals(rowB, results.get(0).getRow()));
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testDeleteColumns_PostInsert() throws IOException,
@@ -1108,47 +1187,50 @@ public class TestHRegion extends HBaseTestCase {
public void doTestDelete_AndPostInsert(Delete delete)
throws IOException, InterruptedException {
- initHRegion(tableName, getName(), fam1);
- EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
- Put put = new Put(row);
- put.add(fam1, qual1, value1);
- region.put(put);
-
- // now delete the value:
- region.delete(delete, null, true);
-
-
- // ok put data:
- put = new Put(row);
- put.add(fam1, qual1, value2);
- region.put(put);
-
- // ok get:
- Get get = new Get(row);
- get.addColumn(fam1, qual1);
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
+ Put put = new Put(row);
+ put.add(fam1, qual1, value1);
+ region.put(put);
- Result r = region.get(get, null);
- assertEquals(1, r.size());
- assertByteEquals(value2, r.getValue(fam1, qual1));
+ // now delete the value:
+ region.delete(delete, null, true);
- // next:
- Scan scan = new Scan(row);
- scan.addColumn(fam1, qual1);
- InternalScanner s = region.getScanner(scan);
- List results = new ArrayList();
- assertEquals(false, s.next(results));
- assertEquals(1, results.size());
- KeyValue kv = results.get(0);
+ // ok put data:
+ put = new Put(row);
+ put.add(fam1, qual1, value2);
+ region.put(put);
- assertByteEquals(value2, kv.getValue());
- assertByteEquals(fam1, kv.getFamily());
- assertByteEquals(qual1, kv.getQualifier());
- assertByteEquals(row, kv.getRow());
+ // ok get:
+ Get get = new Get(row);
+ get.addColumn(fam1, qual1);
+
+ Result r = region.get(get, null);
+ assertEquals(1, r.size());
+ assertByteEquals(value2, r.getValue(fam1, qual1));
+
+ // next:
+ Scan scan = new Scan(row);
+ scan.addColumn(fam1, qual1);
+ InternalScanner s = region.getScanner(scan);
+
+ List results = new ArrayList();
+ assertEquals(false, s.next(results));
+ assertEquals(1, results.size());
+ KeyValue kv = results.get(0);
+
+ assertByteEquals(value2, kv.getValue());
+ assertByteEquals(fam1, kv.getFamily());
+ assertByteEquals(qual1, kv.getQualifier());
+ assertByteEquals(row, kv.getRow());
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
-
-
public void testDelete_CheckTimestampUpdated()
throws IOException {
byte [] row1 = Bytes.toBytes("row1");
@@ -1158,27 +1240,31 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, fam1);
-
- //Building checkerList
- List kvs = new ArrayList();
- kvs.add(new KeyValue(row1, fam1, col1, null));
- kvs.add(new KeyValue(row1, fam1, col2, null));
- kvs.add(new KeyValue(row1, fam1, col3, null));
+ this.region = initHRegion(tableName, method, fam1);
+ try {
+ //Building checkerList
+ List kvs = new ArrayList();
+ kvs.add(new KeyValue(row1, fam1, col1, null));
+ kvs.add(new KeyValue(row1, fam1, col2, null));
+ kvs.add(new KeyValue(row1, fam1, col3, null));
- Map> deleteMap = new HashMap>();
- deleteMap.put(fam1, kvs);
- region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+ Map> deleteMap = new HashMap>();
+ deleteMap.put(fam1, kvs);
+ region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
- // extract the key values out the memstore:
- // This is kinda hacky, but better than nothing...
- long now = System.currentTimeMillis();
- KeyValue firstKv = region.getStore(fam1).memstore.kvset.first();
- assertTrue(firstKv.getTimestamp() <= now);
- now = firstKv.getTimestamp();
- for (KeyValue kv: region.getStore(fam1).memstore.kvset) {
- assertTrue(kv.getTimestamp() <= now);
- now = kv.getTimestamp();
+ // extract the key values out the memstore:
+ // This is kinda hacky, but better than nothing...
+ long now = System.currentTimeMillis();
+ KeyValue firstKv = region.getStore(fam1).memstore.kvset.first();
+ assertTrue(firstKv.getTimestamp() <= now);
+ now = firstKv.getTimestamp();
+ for (KeyValue kv: region.getStore(fam1).memstore.kvset) {
+ assertTrue(kv.getTimestamp() <= now);
+ now = kv.getTimestamp();
+ }
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
}
@@ -1194,19 +1280,23 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, fam1);
-
- Get get = new Get(row1);
- get.addColumn(fam2, col1);
-
- //Test
+ this.region = initHRegion(tableName, method, fam1);
try {
- region.get(get, null);
- } catch (NoSuchColumnFamilyException e){
- assertFalse(false);
- return;
+ Get get = new Get(row1);
+ get.addColumn(fam2, col1);
+
+ //Test
+ try {
+ region.get(get, null);
+ } catch (NoSuchColumnFamilyException e){
+ assertFalse(false);
+ return;
+ }
+ assertFalse(true);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
- assertFalse(true);
}
public void testGet_Basic() throws IOException {
@@ -1221,44 +1311,48 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, fam1);
+ this.region = initHRegion(tableName, method, fam1);
+ try {
+ //Add to memstore
+ Put put = new Put(row1);
+ put.add(fam1, col1, null);
+ put.add(fam1, col2, null);
+ put.add(fam1, col3, null);
+ put.add(fam1, col4, null);
+ put.add(fam1, col5, null);
+ region.put(put);
- //Add to memstore
- Put put = new Put(row1);
- put.add(fam1, col1, null);
- put.add(fam1, col2, null);
- put.add(fam1, col3, null);
- put.add(fam1, col4, null);
- put.add(fam1, col5, null);
- region.put(put);
-
- Get get = new Get(row1);
- get.addColumn(fam1, col2);
- get.addColumn(fam1, col4);
- //Expected result
- KeyValue kv1 = new KeyValue(row1, fam1, col2);
- KeyValue kv2 = new KeyValue(row1, fam1, col4);
- KeyValue [] expected = {kv1, kv2};
-
- //Test
- Result res = region.get(get, null);
- assertEquals(expected.length, res.size());
- for(int i=0; i result = new ArrayList();
- s.next(result);
+ //test2
+ res = region.get(get, null);
+
+ assertEquals(expected.length, res.size());
+ for(int i=0; i result = new ArrayList();
+ s.next(result);
+
+ assertEquals(expected.length, result.size());
+ for(int i=0; ithreads = new ArrayList(threadCount);
- for (int i = 0; i < threadCount; i++) {
- threads.add(new Thread(Integer.toString(i)) {
- @Override
- public void run() {
- Integer [] lockids = new Integer[lockCount];
- // Get locks.
- for (int i = 0; i < lockCount; i++) {
- try {
- byte [] rowid = Bytes.toBytes(Integer.toString(i));
- lockids[i] = region.obtainRowLock(rowid);
- assertEquals(rowid, region.getRowFromLock(lockids[i]));
- LOG.debug(getName() + " locked " + Bytes.toString(rowid));
- } catch (IOException e) {
- e.printStackTrace();
+ this.region = initHRegion(tableName, method, hc, families);
+ try {
+ final int threadCount = 10;
+ final int lockCount = 10;
+
+ Listthreads = new ArrayList(threadCount);
+ for (int i = 0; i < threadCount; i++) {
+ threads.add(new Thread(Integer.toString(i)) {
+ @Override
+ public void run() {
+ Integer [] lockids = new Integer[lockCount];
+ // Get locks.
+ for (int i = 0; i < lockCount; i++) {
+ try {
+ byte [] rowid = Bytes.toBytes(Integer.toString(i));
+ lockids[i] = region.obtainRowLock(rowid);
+ assertEquals(rowid, region.getRowFromLock(lockids[i]));
+ LOG.debug(getName() + " locked " + Bytes.toString(rowid));
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
}
- }
- LOG.debug(getName() + " set " +
- Integer.toString(lockCount) + " locks");
+ LOG.debug(getName() + " set " +
+ Integer.toString(lockCount) + " locks");
- // Abort outstanding locks.
- for (int i = lockCount - 1; i >= 0; i--) {
- region.releaseRowLock(lockids[i]);
- LOG.debug(getName() + " unlocked " + i);
+ // Abort outstanding locks.
+ for (int i = lockCount - 1; i >= 0; i--) {
+ region.releaseRowLock(lockids[i]);
+ LOG.debug(getName() + " unlocked " + i);
+ }
+ LOG.debug(getName() + " released " +
+ Integer.toString(lockCount) + " locks");
}
- LOG.debug(getName() + " released " +
- Integer.toString(lockCount) + " locks");
- }
- });
- }
+ });
+ }
- // Startup all our threads.
- for (Thread t : threads) {
- t.start();
- }
+ // Startup all our threads.
+ for (Thread t : threads) {
+ t.start();
+ }
- // Now wait around till all are done.
- for (Thread t: threads) {
- while (t.isAlive()) {
- try {
- Thread.sleep(1);
- } catch (InterruptedException e) {
- // Go around again.
+ // Now wait around till all are done.
+ for (Thread t: threads) {
+ while (t.isAlive()) {
+ try {
+ Thread.sleep(1);
+ } catch (InterruptedException e) {
+ // Go around again.
+ }
}
}
+ LOG.info("locks completed.");
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
- LOG.info("locks completed.");
}
//////////////////////////////////////////////////////////////////////////////
@@ -1421,46 +1527,51 @@ public class TestHRegion extends HBaseTestCase {
Configuration hc = initSplit();
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, hc, families);
+ this.region = initHRegion(tableName, method, hc, families);
try {
- LOG.info("" + addContent(region, fam3));
- region.flushcache();
- region.compactStores();
- byte [] splitRow = region.checkSplit();
- assertNotNull(splitRow);
- LOG.info("SplitRow: " + Bytes.toString(splitRow));
- HRegion [] subregions = splitRegion(region, splitRow);
try {
- // Need to open the regions.
- for (int i = 0; i < subregions.length; i++) {
- openClosedRegion(subregions[i]);
- subregions[i].compactStores();
+ LOG.info("" + addContent(region, fam3));
+ region.flushcache();
+ region.compactStores();
+ byte [] splitRow = region.checkSplit();
+ assertNotNull(splitRow);
+ LOG.info("SplitRow: " + Bytes.toString(splitRow));
+ HRegion [] subregions = splitRegion(region, splitRow);
+ try {
+ // Need to open the regions.
+ for (int i = 0; i < subregions.length; i++) {
+ openClosedRegion(subregions[i]);
+ subregions[i].compactStores();
+ }
+ Path oldRegionPath = region.getRegionDir();
+ Path oldRegion1 = subregions[0].getRegionDir();
+ Path oldRegion2 = subregions[1].getRegionDir();
+ long startTime = System.currentTimeMillis();
+ region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
+ LOG.info("Merge regions elapsed time: " +
+ ((System.currentTimeMillis() - startTime) / 1000.0));
+ fs.delete(oldRegion1, true);
+ fs.delete(oldRegion2, true);
+ fs.delete(oldRegionPath, true);
+ LOG.info("splitAndMerge completed.");
+ } finally {
+ for (int i = 0; i < subregions.length; i++) {
+ try {
+ subregions[i].close();
+ } catch (IOException e) {
+ // Ignore.
+ }
+ }
}
- Path oldRegionPath = region.getRegionDir();
- Path oldRegion1 = subregions[0].getRegionDir();
- Path oldRegion2 = subregions[1].getRegionDir();
- long startTime = System.currentTimeMillis();
- region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
- LOG.info("Merge regions elapsed time: " +
- ((System.currentTimeMillis() - startTime) / 1000.0));
- fs.delete(oldRegion1, true);
- fs.delete(oldRegion2, true);
- fs.delete(oldRegionPath, true);
- LOG.info("splitAndMerge completed.");
} finally {
- for (int i = 0; i < subregions.length; i++) {
- try {
- subregions[i].close();
- } catch (IOException e) {
- // Ignore.
- }
+ if (region != null) {
+ region.close();
+ region.getLog().closeAndDelete();
}
}
} finally {
- if (region != null) {
- region.close();
- region.getLog().closeAndDelete();
- }
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
}
@@ -1529,17 +1640,21 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, families);
-
- Scan scan = new Scan();
- scan.addFamily(fam2);
- boolean ok = false;
+ this.region = initHRegion(tableName, method, families);
try {
- region.getScanner(scan);
- } catch (Exception e) {
- ok = true;
+ Scan scan = new Scan();
+ scan.addFamily(fam2);
+ boolean ok = false;
+ try {
+ region.getScanner(scan);
+ } catch (Exception e) {
+ ok = true;
+ }
+ assertTrue("Families could not be found in Region", ok);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
- assertTrue("Families could not be found in Region", ok);
}
public void testGetScanner_WithNoFamilies() throws IOException {
@@ -1554,40 +1669,45 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, families);
-
+ this.region = initHRegion(tableName, method, families);
+ try {
- //Putting data in Region
- Put put = new Put(row1);
- put.add(fam1, null, null);
- put.add(fam2, null, null);
- put.add(fam3, null, null);
- put.add(fam4, null, null);
- region.put(put);
-
- Scan scan = null;
- HRegion.RegionScannerImpl is = null;
-
- //Testing to see how many scanners that is produced by getScanner, starting
- //with known number, 2 - current = 1
- scan = new Scan();
- scan.addFamily(fam2);
- scan.addFamily(fam4);
- is = (RegionScannerImpl) region.getScanner(scan);
- MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
- assertEquals(1, ((RegionScannerImpl)is).storeHeap.getHeap().size());
+ //Putting data in Region
+ Put put = new Put(row1);
+ put.add(fam1, null, null);
+ put.add(fam2, null, null);
+ put.add(fam3, null, null);
+ put.add(fam4, null, null);
+ region.put(put);
- scan = new Scan();
- is = (RegionScannerImpl) region.getScanner(scan);
- MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
- assertEquals(families.length -1,
- ((RegionScannerImpl)is).storeHeap.getHeap().size());
+ Scan scan = null;
+ HRegion.RegionScannerImpl is = null;
+
+ //Testing to see how many scanners that is produced by getScanner, starting
+ //with known number, 2 - current = 1
+ scan = new Scan();
+ scan.addFamily(fam2);
+ scan.addFamily(fam4);
+ is = (RegionScannerImpl) region.getScanner(scan);
+ MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
+ assertEquals(1, ((RegionScannerImpl)is).storeHeap.getHeap().size());
+
+ scan = new Scan();
+ is = (RegionScannerImpl) region.getScanner(scan);
+ MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
+ assertEquals(families.length -1,
+ ((RegionScannerImpl)is).storeHeap.getHeap().size());
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
/**
* This method tests https://issues.apache.org/jira/browse/HBASE-2516.
+ * @throws IOException
*/
- public void testGetScanner_WithRegionClosed() {
+ public void testGetScanner_WithRegionClosed() throws IOException {
byte[] tableName = Bytes.toBytes("testtable");
byte[] fam1 = Bytes.toBytes("fam1");
byte[] fam2 = Bytes.toBytes("fam2");
@@ -1597,20 +1717,25 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
try {
- initHRegion(tableName, method, families);
+ this.region = initHRegion(tableName, method, families);
} catch (IOException e) {
e.printStackTrace();
fail("Got IOException during initHRegion, " + e.getMessage());
}
- region.closed.set(true);
try {
- region.getScanner(null);
- fail("Expected to get an exception during getScanner on a region that is closed");
- } catch (org.apache.hadoop.hbase.NotServingRegionException e) {
- //this is the correct exception that is expected
- } catch (IOException e) {
- fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
- + e.getMessage());
+ region.closed.set(true);
+ try {
+ region.getScanner(null);
+ fail("Expected to get an exception during getScanner on a region that is closed");
+ } catch (org.apache.hadoop.hbase.NotServingRegionException e) {
+ //this is the correct exception that is expected
+ } catch (IOException e) {
+ fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
+ + e.getMessage());
+ }
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
}
@@ -1628,53 +1753,56 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, families);
+ this.region = initHRegion(tableName, method, families);
+ try {
+ //Putting data in Region
+ Put put = null;
+ put = new Put(row1);
+ put.add(fam1, null, ts, null);
+ put.add(fam2, null, ts, null);
+ put.add(fam3, null, ts, null);
+ put.add(fam4, null, ts, null);
+ region.put(put);
- //Putting data in Region
- Put put = null;
- put = new Put(row1);
- put.add(fam1, null, ts, null);
- put.add(fam2, null, ts, null);
- put.add(fam3, null, ts, null);
- put.add(fam4, null, ts, null);
- region.put(put);
-
- put = new Put(row2);
- put.add(fam1, null, ts, null);
- put.add(fam2, null, ts, null);
- put.add(fam3, null, ts, null);
- put.add(fam4, null, ts, null);
- region.put(put);
+ put = new Put(row2);
+ put.add(fam1, null, ts, null);
+ put.add(fam2, null, ts, null);
+ put.add(fam3, null, ts, null);
+ put.add(fam4, null, ts, null);
+ region.put(put);
- Scan scan = new Scan();
- scan.addFamily(fam2);
- scan.addFamily(fam4);
- InternalScanner is = region.getScanner(scan);
+ Scan scan = new Scan();
+ scan.addFamily(fam2);
+ scan.addFamily(fam4);
+ InternalScanner is = region.getScanner(scan);
- List res = null;
+ List res = null;
- //Result 1
- List expected1 = new ArrayList();
- expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
- expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
+ //Result 1
+ List expected1 = new ArrayList();
+ expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
+ expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
- res = new ArrayList();
- is.next(res);
- for(int i=0; i();
+ is.next(res);
+ for(int i=0; i expected2 = new ArrayList();
- expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
- expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
+ //Result 2
+ List expected2 = new ArrayList();
+ expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
+ expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
- res = new ArrayList();
- is.next(res);
- for(int i=0; i();
+ is.next(res);
+ for(int i=0; i expected = new ArrayList();
- expected.add(kv13);
- expected.add(kv12);
+ //Expected
+ List expected = new ArrayList();
+ expected.add(kv13);
+ expected.add(kv12);
- Scan scan = new Scan(row1);
- scan.addColumn(fam1, qf1);
- scan.setMaxVersions(MAX_VERSIONS);
- List actual = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
+ Scan scan = new Scan(row1);
+ scan.addColumn(fam1, qf1);
+ scan.setMaxVersions(MAX_VERSIONS);
+ List actual = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
- boolean hasNext = scanner.next(actual);
- assertEquals(false, hasNext);
+ boolean hasNext = scanner.next(actual);
+ assertEquals(false, hasNext);
- //Verify result
- for(int i=0; i expected = new ArrayList();
- expected.add(kv13);
- expected.add(kv12);
- expected.add(kv23);
- expected.add(kv22);
+ this.region = initHRegion(tableName, method, families);
+ try {
+ //Putting data in Region
+ Put put = null;
+ KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
+ KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
+ KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
+
+ KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
+ KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
+ KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
+
+ put = new Put(row1);
+ put.add(kv13);
+ put.add(kv12);
+ put.add(kv11);
+ put.add(kv23);
+ put.add(kv22);
+ put.add(kv21);
+ region.put(put);
+ region.flushcache();
- Scan scan = new Scan(row1);
- scan.addColumn(fam1, qf1);
- scan.addColumn(fam1, qf2);
- scan.setMaxVersions(MAX_VERSIONS);
- List actual = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
+ //Expected
+ List expected = new ArrayList();
+ expected.add(kv13);
+ expected.add(kv12);
+ expected.add(kv23);
+ expected.add(kv22);
+
+ Scan scan = new Scan(row1);
+ scan.addColumn(fam1, qf1);
+ scan.addColumn(fam1, qf2);
+ scan.setMaxVersions(MAX_VERSIONS);
+ List actual = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
- boolean hasNext = scanner.next(actual);
- assertEquals(false, hasNext);
+ boolean hasNext = scanner.next(actual);
+ assertEquals(false, hasNext);
- //Verify result
- for(int i=0; i expected = new ArrayList();
- expected.add(kv14);
- expected.add(kv13);
- expected.add(kv12);
- expected.add(kv24);
- expected.add(kv23);
- expected.add(kv22);
+ put = new Put(row1);
+ put.add(kv21);
+ put.add(kv11);
+ region.put(put);
- Scan scan = new Scan(row1);
- scan.addColumn(fam1, qf1);
- scan.addColumn(fam1, qf2);
- int versions = 3;
- scan.setMaxVersions(versions);
- List actual = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
+ //Expected
+ List expected = new ArrayList();
+ expected.add(kv14);
+ expected.add(kv13);
+ expected.add(kv12);
+ expected.add(kv24);
+ expected.add(kv23);
+ expected.add(kv22);
+
+ Scan scan = new Scan(row1);
+ scan.addColumn(fam1, qf1);
+ scan.addColumn(fam1, qf2);
+ int versions = 3;
+ scan.setMaxVersions(versions);
+ List actual = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
- boolean hasNext = scanner.next(actual);
- assertEquals(false, hasNext);
+ boolean hasNext = scanner.next(actual);
+ assertEquals(false, hasNext);
- //Verify result
- for(int i=0; i expected = new ArrayList();
- expected.add(kv13);
- expected.add(kv12);
- expected.add(kv23);
- expected.add(kv22);
-
- Scan scan = new Scan(row1);
- scan.addFamily(fam1);
- scan.setMaxVersions(MAX_VERSIONS);
- List actual = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
+ //Expected
+ List expected = new ArrayList();
+ expected.add(kv13);
+ expected.add(kv12);
+ expected.add(kv23);
+ expected.add(kv22);
+
+ Scan scan = new Scan(row1);
+ scan.addFamily(fam1);
+ scan.setMaxVersions(MAX_VERSIONS);
+ List actual = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
- boolean hasNext = scanner.next(actual);
- assertEquals(false, hasNext);
+ boolean hasNext = scanner.next(actual);
+ assertEquals(false, hasNext);
- //Verify result
- for(int i=0; i expected = new ArrayList();
- expected.add(kv13);
- expected.add(kv12);
- expected.add(kv23);
- expected.add(kv22);
+ this.region = initHRegion(tableName, method, fam1);
+ try {
+ //Putting data in Region
+ Put put = null;
+ KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
+ KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
+ KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
+
+ KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
+ KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
+ KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
+
+ put = new Put(row1);
+ put.add(kv13);
+ put.add(kv12);
+ put.add(kv11);
+ put.add(kv23);
+ put.add(kv22);
+ put.add(kv21);
+ region.put(put);
+ region.flushcache();
- Scan scan = new Scan(row1);
- scan.addFamily(fam1);
- scan.setMaxVersions(MAX_VERSIONS);
- List actual = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
+ //Expected
+ List expected = new ArrayList();
+ expected.add(kv13);
+ expected.add(kv12);
+ expected.add(kv23);
+ expected.add(kv22);
+
+ Scan scan = new Scan(row1);
+ scan.addFamily(fam1);
+ scan.setMaxVersions(MAX_VERSIONS);
+ List actual = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
- boolean hasNext = scanner.next(actual);
- assertEquals(false, hasNext);
+ boolean hasNext = scanner.next(actual);
+ assertEquals(false, hasNext);
- //Verify result
- for(int i=0; i results = new ArrayList();
- assertEquals(false, s.next(results));
- assertEquals(0, results.size());
+ List results = new ArrayList();
+ assertEquals(false, s.next(results));
+ assertEquals(0, results.size());
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- long value = 1L;
- long amount = 3L;
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 1L;
+ long amount = 3L;
- Put put = new Put(row);
- put.add(fam1, qual1, Bytes.toBytes(value));
- region.put(put);
+ Put put = new Put(row);
+ put.add(fam1, qual1, Bytes.toBytes(value));
+ region.put(put);
- long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
+ long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
- assertEquals(value+amount, result);
+ assertEquals(value+amount, result);
- Store store = region.getStore(fam1);
- // ICV removes any extra values floating around in there.
- assertEquals(1, store.memstore.kvset.size());
- assertTrue(store.memstore.snapshot.isEmpty());
+ Store store = region.getStore(fam1);
+ // ICV removes any extra values floating around in there.
+ assertEquals(1, store.memstore.kvset.size());
+ assertTrue(store.memstore.snapshot.isEmpty());
- assertICV(row, fam1, qual1, value+amount);
+ assertICV(row, fam1, qual1, value+amount);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_BumpSnapshot() throws IOException {
ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
EnvironmentEdgeManagerTestHelper.injectEdge(mee);
- initHRegion(tableName, getName(), fam1);
-
- long value = 42L;
- long incr = 44L;
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 42L;
+ long incr = 44L;
- // first put something in kvset, then snapshot it.
- Put put = new Put(row);
- put.add(fam1, qual1, Bytes.toBytes(value));
- region.put(put);
+ // first put something in kvset, then snapshot it.
+ Put put = new Put(row);
+ put.add(fam1, qual1, Bytes.toBytes(value));
+ region.put(put);
- // get the store in question:
- Store s = region.getStore(fam1);
- s.snapshot(); //bam
+ // get the store in question:
+ Store s = region.getStore(fam1);
+ s.snapshot(); //bam
- // now increment:
- long newVal = region.incrementColumnValue(row, fam1, qual1,
- incr, false);
+ // now increment:
+ long newVal = region.incrementColumnValue(row, fam1, qual1,
+ incr, false);
- assertEquals(value+incr, newVal);
+ assertEquals(value+incr, newVal);
- // get both versions:
- Get get = new Get(row);
- get.setMaxVersions();
- get.addColumn(fam1,qual1);
+ // get both versions:
+ Get get = new Get(row);
+ get.setMaxVersions();
+ get.addColumn(fam1,qual1);
- Result r = region.get(get, null);
- assertEquals(2, r.size());
- KeyValue first = r.raw()[0];
- KeyValue second = r.raw()[1];
+ Result r = region.get(get, null);
+ assertEquals(2, r.size());
+ KeyValue first = r.raw()[0];
+ KeyValue second = r.raw()[1];
- assertTrue("ICV failed to upgrade timestamp",
- first.getTimestamp() != second.getTimestamp());
+ assertTrue("ICV failed to upgrade timestamp",
+ first.getTimestamp() != second.getTimestamp());
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_ConcurrentFlush() throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- long value = 1L;
- long amount = 3L;
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 1L;
+ long amount = 3L;
- Put put = new Put(row);
- put.add(fam1, qual1, Bytes.toBytes(value));
- region.put(put);
+ Put put = new Put(row);
+ put.add(fam1, qual1, Bytes.toBytes(value));
+ region.put(put);
- // now increment during a flush
- Thread t = new Thread() {
- public void run() {
- try {
- region.flushcache();
- } catch (IOException e) {
- LOG.info("test ICV, got IOE during flushcache()");
+ // now increment during a flush
+ Thread t = new Thread() {
+ public void run() {
+ try {
+ region.flushcache();
+ } catch (IOException e) {
+ LOG.info("test ICV, got IOE during flushcache()");
+ }
}
- }
- };
- t.start();
- long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
- assertEquals(value+amount, r);
+ };
+ t.start();
+ long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
+ assertEquals(value+amount, r);
- // this also asserts there is only 1 KeyValue in the set.
- assertICV(row, fam1, qual1, value+amount);
+ // this also asserts there is only 1 KeyValue in the set.
+ assertICV(row, fam1, qual1, value+amount);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_heapSize() throws IOException {
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
- initHRegion(tableName, getName(), fam1);
-
- long byAmount = 1L;
- long size;
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long byAmount = 1L;
+ long size;
- for( int i = 0; i < 1000 ; i++) {
- region.incrementColumnValue(row, fam1, qual1, byAmount, true);
+ for( int i = 0; i < 1000 ; i++) {
+ region.incrementColumnValue(row, fam1, qual1, byAmount, true);
- size = region.memstoreSize.get();
- assertTrue("memstore size: " + size, size >= 0);
+ size = region.memstoreSize.get();
+ assertTrue("memstore size: " + size, size >= 0);
+ }
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
}
}
public void testIncrementColumnValue_UpdatingInPlace_Negative()
throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- long value = 3L;
- long amount = -1L;
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 3L;
+ long amount = -1L;
- Put put = new Put(row);
- put.add(fam1, qual1, Bytes.toBytes(value));
- region.put(put);
+ Put put = new Put(row);
+ put.add(fam1, qual1, Bytes.toBytes(value));
+ region.put(put);
- long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
- assertEquals(value+amount, result);
+ long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
+ assertEquals(value+amount, result);
- assertICV(row, fam1, qual1, value+amount);
+ assertICV(row, fam1, qual1, value+amount);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_AddingNew()
throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- long value = 1L;
- long amount = 3L;
-
- Put put = new Put(row);
- put.add(fam1, qual1, Bytes.toBytes(value));
- put.add(fam1, qual2, Bytes.toBytes(value));
- region.put(put);
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 1L;
+ long amount = 3L;
- long result = region.incrementColumnValue(row, fam1, qual3, amount, true);
- assertEquals(amount, result);
+ Put put = new Put(row);
+ put.add(fam1, qual1, Bytes.toBytes(value));
+ put.add(fam1, qual2, Bytes.toBytes(value));
+ region.put(put);
- Get get = new Get(row);
- get.addColumn(fam1, qual3);
- Result rr = region.get(get, null);
- assertEquals(1, rr.size());
+ long result = region.incrementColumnValue(row, fam1, qual3, amount, true);
+ assertEquals(amount, result);
- // ensure none of the other cols were incremented.
- assertICV(row, fam1, qual1, value);
- assertICV(row, fam1, qual2, value);
- assertICV(row, fam1, qual3, amount);
+ Get get = new Get(row);
+ get.addColumn(fam1, qual3);
+ Result rr = region.get(get, null);
+ assertEquals(1, rr.size());
+
+ // ensure none of the other cols were incremented.
+ assertICV(row, fam1, qual1, value);
+ assertICV(row, fam1, qual2, value);
+ assertICV(row, fam1, qual3, amount);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- long value = 1L;
- long amount = 3L;
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 1L;
+ long amount = 3L;
- Put put = new Put(row);
- put.add(fam1, qual1, Bytes.toBytes(value));
- put.add(fam1, qual2, Bytes.toBytes(value));
- region.put(put);
+ Put put = new Put(row);
+ put.add(fam1, qual1, Bytes.toBytes(value));
+ put.add(fam1, qual2, Bytes.toBytes(value));
+ region.put(put);
- // flush to disk.
- region.flushcache();
+ // flush to disk.
+ region.flushcache();
- Store store = region.getStore(fam1);
- assertEquals(0, store.memstore.kvset.size());
+ Store store = region.getStore(fam1);
+ assertEquals(0, store.memstore.kvset.size());
- long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
- assertEquals(value+amount, r);
+ long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
+ assertEquals(value+amount, r);
- assertICV(row, fam1, qual1, value+amount);
+ assertICV(row, fam1, qual1, value+amount);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_AddingNewAfterSFCheck()
throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- long value = 1L;
- long amount = 3L;
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 1L;
+ long amount = 3L;
- Put put = new Put(row);
- put.add(fam1, qual1, Bytes.toBytes(value));
- put.add(fam1, qual2, Bytes.toBytes(value));
- region.put(put);
- region.flushcache();
+ Put put = new Put(row);
+ put.add(fam1, qual1, Bytes.toBytes(value));
+ put.add(fam1, qual2, Bytes.toBytes(value));
+ region.put(put);
+ region.flushcache();
- Store store = region.getStore(fam1);
- assertEquals(0, store.memstore.kvset.size());
+ Store store = region.getStore(fam1);
+ assertEquals(0, store.memstore.kvset.size());
- long r = region.incrementColumnValue(row, fam1, qual3, amount, true);
- assertEquals(amount, r);
+ long r = region.incrementColumnValue(row, fam1, qual3, amount, true);
+ assertEquals(amount, r);
- assertICV(row, fam1, qual3, amount);
+ assertICV(row, fam1, qual3, amount);
- region.flushcache();
+ region.flushcache();
- // ensure that this gets to disk.
- assertICV(row, fam1, qual3, amount);
+ // ensure that this gets to disk.
+ assertICV(row, fam1, qual3, amount);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
/**
@@ -2239,71 +2423,79 @@ public class TestHRegion extends HBaseTestCase {
* @throws IOException
*/
public void testIncrementColumnValue_UpdatingInPlace_TimestampClobber() throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- long value = 1L;
- long amount = 3L;
- long now = EnvironmentEdgeManager.currentTimeMillis();
- ManualEnvironmentEdge mock = new ManualEnvironmentEdge();
- mock.setValue(now);
- EnvironmentEdgeManagerTestHelper.injectEdge(mock);
-
- // verify we catch an ICV on a put with the same timestamp
- Put put = new Put(row);
- put.add(fam1, qual1, now, Bytes.toBytes(value));
- region.put(put);
+ this.region = initHRegion(tableName, getName(), fam1);
+ try {
+ long value = 1L;
+ long amount = 3L;
+ long now = EnvironmentEdgeManager.currentTimeMillis();
+ ManualEnvironmentEdge mock = new ManualEnvironmentEdge();
+ mock.setValue(now);
+ EnvironmentEdgeManagerTestHelper.injectEdge(mock);
+
+ // verify we catch an ICV on a put with the same timestamp
+ Put put = new Put(row);
+ put.add(fam1, qual1, now, Bytes.toBytes(value));
+ region.put(put);
- long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
+ long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
- assertEquals(value+amount, result);
+ assertEquals(value+amount, result);
- Store store = region.getStore(fam1);
- // ICV should update the existing Put with the same timestamp
- assertEquals(1, store.memstore.kvset.size());
- assertTrue(store.memstore.snapshot.isEmpty());
+ Store store = region.getStore(fam1);
+ // ICV should update the existing Put with the same timestamp
+ assertEquals(1, store.memstore.kvset.size());
+ assertTrue(store.memstore.snapshot.isEmpty());
- assertICV(row, fam1, qual1, value+amount);
+ assertICV(row, fam1, qual1, value+amount);
- // verify we catch an ICV even when the put ts > now
- put = new Put(row);
- put.add(fam1, qual2, now+1, Bytes.toBytes(value));
- region.put(put);
+ // verify we catch an ICV even when the put ts > now
+ put = new Put(row);
+ put.add(fam1, qual2, now+1, Bytes.toBytes(value));
+ region.put(put);
- result = region.incrementColumnValue(row, fam1, qual2, amount, true);
+ result = region.incrementColumnValue(row, fam1, qual2, amount, true);
- assertEquals(value+amount, result);
+ assertEquals(value+amount, result);
- store = region.getStore(fam1);
- // ICV should update the existing Put with the same timestamp
- assertEquals(2, store.memstore.kvset.size());
- assertTrue(store.memstore.snapshot.isEmpty());
+ store = region.getStore(fam1);
+ // ICV should update the existing Put with the same timestamp
+ assertEquals(2, store.memstore.kvset.size());
+ assertTrue(store.memstore.snapshot.isEmpty());
- assertICV(row, fam1, qual2, value+amount);
- EnvironmentEdgeManagerTestHelper.reset();
+ assertICV(row, fam1, qual2, value+amount);
+ EnvironmentEdgeManagerTestHelper.reset();
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
public void testIncrementColumnValue_WrongInitialSize() throws IOException {
- initHRegion(tableName, getName(), fam1);
-
- byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
- int row1Field1 = 0;
- int row1Field2 = 1;
- Put put1 = new Put(row1);
- put1.add(fam1, qual1, Bytes.toBytes(row1Field1));
- put1.add(fam1, qual2, Bytes.toBytes(row1Field2));
- region.put(put1);
-
- long result;
+ this.region = initHRegion(tableName, getName(), fam1);
try {
+ byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
+ int row1Field1 = 0;
+ int row1Field2 = 1;
+ Put put1 = new Put(row1);
+ put1.add(fam1, qual1, Bytes.toBytes(row1Field1));
+ put1.add(fam1, qual2, Bytes.toBytes(row1Field2));
+ region.put(put1);
+
+ long result;
+ try {
result = region.incrementColumnValue(row1, fam1, qual1, 1, true);
fail("Expected to fail here");
- } catch (Exception exception) {
+ } catch (Exception exception) {
// Expected.
- }
+ }
- assertICV(row1, fam1, qual1, row1Field1);
- assertICV(row1, fam1, qual2, row1Field2);
+ assertICV(row1, fam1, qual1, row1Field1);
+ assertICV(row1, fam1, qual2, row1Field2);
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
private void assertICV(byte [] row,
@@ -2351,64 +2543,68 @@ public class TestHRegion extends HBaseTestCase {
//Setting up region
String method = this.getName();
- initHRegion(tableName, method, fam1);
-
- //Putting data in Region
- KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
- KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
- KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
- KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
-
- KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
- KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
- KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
- KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
-
- Put put = null;
- put = new Put(row1);
- put.add(kv14);
- put.add(kv24);
- region.put(put);
- region.flushcache();
-
- put = new Put(row1);
- put.add(kv23);
- put.add(kv13);
- region.put(put);
- region.flushcache();
+ this.region = initHRegion(tableName, method, fam1);
+ try {
+ //Putting data in Region
+ KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
+ KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
+ KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
+ KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
+
+ KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
+ KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
+ KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
+ KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
+
+ Put put = null;
+ put = new Put(row1);
+ put.add(kv14);
+ put.add(kv24);
+ region.put(put);
+ region.flushcache();
- put = new Put(row1);
- put.add(kv22);
- put.add(kv12);
- region.put(put);
- region.flushcache();
+ put = new Put(row1);
+ put.add(kv23);
+ put.add(kv13);
+ region.put(put);
+ region.flushcache();
- put = new Put(row1);
- put.add(kv21);
- put.add(kv11);
- region.put(put);
+ put = new Put(row1);
+ put.add(kv22);
+ put.add(kv12);
+ region.put(put);
+ region.flushcache();
- //Expected
- List expected = new ArrayList();
- expected.add(kv14);
- expected.add(kv13);
- expected.add(kv12);
- expected.add(kv24);
- expected.add(kv23);
- expected.add(kv22);
+ put = new Put(row1);
+ put.add(kv21);
+ put.add(kv11);
+ region.put(put);
- Scan scan = new Scan(row1);
- int versions = 3;
- scan.setMaxVersions(versions);
- List actual = new ArrayList();
- InternalScanner scanner = region.getScanner(scan);
+ //Expected
+ List expected = new ArrayList();
+ expected.add(kv14);
+ expected.add(kv13);
+ expected.add(kv12);
+ expected.add(kv24);
+ expected.add(kv23);
+ expected.add(kv22);
+
+ Scan scan = new Scan(row1);
+ int versions = 3;
+ scan.setMaxVersions(versions);
+ List actual = new ArrayList();
+ InternalScanner scanner = region.getScanner(scan);
- boolean hasNext = scanner.next(actual);
- assertEquals(false, hasNext);
+ boolean hasNext = scanner.next(actual);
+ assertEquals(false, hasNext);
- //Verify result
- for(int i=0; i res = new ArrayList();
-
- boolean toggle=true;
- for (long i = 0; i < numRows; i++) {
- Put put = new Put(Bytes.toBytes(i));
- put.setWriteToWAL(false);
- put.add(family, qual1, Bytes.toBytes(i % 10));
- region.put(put);
-
- if (i != 0 && i % compactInterval == 0) {
- //System.out.println("iteration = " + i);
- region.compactStores(true);
- }
-
- if (i % 10 == 5L) {
- expectedCount++;
- }
+ this.region = initHRegion(tableName,method, family);
+ try {
+ FlushThread flushThread = new FlushThread();
+ flushThread.start();
+
+ Scan scan = new Scan();
+ scan.addFamily(family);
+ scan.setFilter(new SingleColumnValueFilter(family, qual1,
+ CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes(5L))));
+
+ int expectedCount = 0;
+ List res = new ArrayList();
+
+ boolean toggle=true;
+ for (long i = 0; i < numRows; i++) {
+ Put put = new Put(Bytes.toBytes(i));
+ put.setWriteToWAL(false);
+ put.add(family, qual1, Bytes.toBytes(i % 10));
+ region.put(put);
+
+ if (i != 0 && i % compactInterval == 0) {
+ //System.out.println("iteration = " + i);
+ region.compactStores(true);
+ }
- if (i != 0 && i % flushAndScanInterval == 0) {
- res.clear();
- InternalScanner scanner = region.getScanner(scan);
- if (toggle) {
- flushThread.flush();
+ if (i % 10 == 5L) {
+ expectedCount++;
}
- while (scanner.next(res)) ;
- if (!toggle) {
- flushThread.flush();
+
+ if (i != 0 && i % flushAndScanInterval == 0) {
+ res.clear();
+ InternalScanner scanner = region.getScanner(scan);
+ if (toggle) {
+ flushThread.flush();
+ }
+ while (scanner.next(res)) ;
+ if (!toggle) {
+ flushThread.flush();
+ }
+ assertEquals("i=" + i, expectedCount, res.size());
+ toggle = !toggle;
}
- assertEquals("i=" + i, expectedCount, res.size());
- toggle = !toggle;
}
- }
- flushThread.done();
- flushThread.join();
- flushThread.checkNoError();
+ flushThread.done();
+ flushThread.join();
+ flushThread.checkNoError();
+ } finally {
+ HRegion.closeHRegion(this.region);
+ this.region = null;
+ }
}
protected class FlushThread extends Thread {
@@ -2692,56 +2889,61 @@ public class TestHRegion extends HBaseTestCase {
}
String method = "testWritesWhileScanning";
- initHRegion(tableName, method, families);
- PutThread putThread = new PutThread(numRows, families, qualifiers);
- putThread.start();
- putThread.waitForFirstPut();
+ this.region = initHRegion(tableName, method, families);
+ try {
+ PutThread putThread = new PutThread(numRows, families, qualifiers);
+ putThread.start();
+ putThread.waitForFirstPut();
- FlushThread flushThread = new FlushThread();
- flushThread.start();
+ FlushThread flushThread = new FlushThread();
+ flushThread.start();
- Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
-// scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
-// new BinaryComparator(Bytes.toBytes("row0"))));
+ Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
+ // scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
+ // new BinaryComparator(Bytes.toBytes("row0"))));
- int expectedCount = numFamilies * numQualifiers;
- List res = new ArrayList();
+ int expectedCount = numFamilies * numQualifiers;
+ List res = new ArrayList