You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/10/03 21:47:09 UTC

svn commit: r1528984 [1/3] - in /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase: ./ io/hfile/ master/handler/ regionserver/ security/access/

Author: jmhsieh
Date: Thu Oct  3 19:47:09 2013
New Revision: 1528984

URL: http://svn.apache.org/r1528984
Log:
HBASE-9686 More temporary test files are being left in /tmp/hbase-<user>

Modified:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionBusyWait.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java Thu Oct  3 19:47:09 2013
@@ -254,7 +254,7 @@ public abstract class HBaseTestCase exte
    * @throws IOException
    * @return count of what we added.
    */
-  protected static long addContent(final Incommon updater,
+  public static long addContent(final Incommon updater,
       final String columnFamily) throws IOException {
     return addContent(updater, columnFamily, START_KEY_BYTES, null);
   }
@@ -275,7 +275,7 @@ public abstract class HBaseTestCase exte
    * @return count of what we added.
    * @throws IOException
    */
-  protected static long addContent(final Incommon updater, final String columnFamily,
+  public static long addContent(final Incommon updater, final String columnFamily,
       final byte [] startKeyBytes, final byte [] endKey)
   throws IOException {
     return addContent(updater, columnFamily, null, startKeyBytes, endKey, -1);
@@ -299,8 +299,8 @@ public abstract class HBaseTestCase exte
    * @return count of what we added.
    * @throws IOException
    */
-  protected static long addContent(final Incommon updater,
-                                   final String columnFamily,
+  public static long addContent(final Incommon updater,
+                                   final String columnFamily, 
                                    final String column,
       final byte [] startKeyBytes, final byte [] endKey, final long ts)
   throws IOException {
@@ -573,26 +573,6 @@ public abstract class HBaseTestCase exte
     }
   }
 
-//  protected void assertCellEquals(final HRegion region, final byte [] row,
-//    final byte [] column, final long timestamp, final String value)
-//  throws IOException {
-//    Map<byte [], Cell> result = region.getFull(row, null, timestamp, 1, null);
-//    Cell cell_value = result.get(column);
-//    if (value == null) {
-//      assertEquals(Bytes.toString(column) + " at timestamp " + timestamp, null,
-//        cell_value);
-//    } else {
-//      if (cell_value == null) {
-//        fail(Bytes.toString(column) + " at timestamp " + timestamp +
-//          "\" was expected to be \"" + value + " but was null");
-//      }
-//      if (cell_value != null) {
-//        assertEquals(Bytes.toString(column) + " at timestamp "
-//            + timestamp, value, new String(cell_value.getValue()));
-//      }
-//    }
-//  }
-
   protected void assertResultEquals(final HRegion region, final byte [] row,
       final byte [] family, final byte [] qualifier, final long timestamp,
       final byte [] value)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Thu Oct  3 19:47:09 2013
@@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.tool.Canary;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -125,7 +126,7 @@ import org.apache.zookeeper.ZooKeeper.St
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class HBaseTestingUtility extends HBaseCommonTestingUtility {
-   private Configuration conf;
+   protected Configuration conf;
    private MiniZooKeeperCluster zkCluster = null;
 
   /**
@@ -1503,9 +1504,10 @@ public class HBaseTestingUtility extends
   public static final byte[][] COLUMNS = {fam1, fam2, fam3};
   private static final int MAXVERSIONS = 3;
   
-  private static final char FIRST_CHAR = 'a';
+  public static final char FIRST_CHAR = 'a';
+  public static final char LAST_CHAR = 'z';
   public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
-
+  public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET);
 
   /**
    * Create a table of name <code>name</code> with {@link COLUMNS} for
@@ -1565,7 +1567,47 @@ public class HBaseTestingUtility extends
   public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc) throws IOException {
     return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc);
   }
+
+  /**
+   * Create an HRegion that writes to the local tmp dirs with specified hlog
+   * @param info regioninfo
+   * @param desc table descriptor
+   * @param hlog hlog for this region.
+   * @return created hregion
+   * @throws IOException
+   */
+  public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, HLog hlog) throws IOException {
+    return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, hlog);
+  }
+
   
+  /**
+   * @param tableName
+   * @param startKey
+   * @param stopKey
+   * @param callingMethod
+   * @param conf
+   * @param isReadOnly
+   * @param families
+   * @throws IOException
+   * @return A region on which you must call
+   *         {@link HRegion#closeHRegion(HRegion)} when done.
+   */
+  public HRegion createLocalHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
+      String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
+      HLog hlog, byte[]... families) throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
+    htd.setReadOnly(isReadOnly);
+    for (byte[] family : families) {
+      HColumnDescriptor hcd = new HColumnDescriptor(family);
+      // Set default to be three versions.
+      hcd.setMaxVersions(Integer.MAX_VALUE);
+      htd.addFamily(hcd);
+    }
+    htd.setDurability(durability);
+    HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false);
+    return createLocalHRegion(info, htd, hlog);
+  }
   //
   // ==========================================================================
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java Thu Oct  3 19:47:09 2013
@@ -53,7 +53,7 @@ import org.junit.runners.Parameterized.P
 @RunWith(Parameterized.class)
 @Category(SmallTests.class)
 public class TestScannerSelectionUsingKeyRange {
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility().createLocalHTU();
   private static TableName TABLE = TableName.valueOf("myTable");
   private static String FAMILY = "myCF";
   private static byte[] FAMILY_BYTES = Bytes.toBytes(FAMILY);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java Thu Oct  3 19:47:09 2013
@@ -61,7 +61,7 @@ public class TestScannerSelectionUsingTT
       LogFactory.getLog(TestScannerSelectionUsingTTL.class);
 
   private static final HBaseTestingUtility TEST_UTIL =
-      new HBaseTestingUtility();
+      new HBaseTestingUtility().createLocalHTU();
   private static TableName TABLE = TableName.valueOf("myTable");
   private static String FAMILY = "myCF";
   private static byte[] FAMILY_BYTES = Bytes.toBytes(FAMILY);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java Thu Oct  3 19:47:09 2013
@@ -20,27 +20,29 @@
 package org.apache.hadoop.hbase.master.handler;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Set;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 
 /**
  * Verify that the HTableDescriptor is updated after
@@ -48,10 +50,10 @@ import org.junit.experimental.categories
  */
 @Category(LargeTests.class)
 public class TestTableDescriptorModification {
-
+  
+  @Rule public TestName name = new TestName();
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final TableName TABLE_NAME =
-      TableName.valueOf("table");
+  private static TableName TABLE_NAME = null;
   private static final byte[] FAMILY_0 = Bytes.toBytes("cf0");
   private static final byte[] FAMILY_1 = Bytes.toBytes("cf1");
 
@@ -65,6 +67,12 @@ public class TestTableDescriptorModifica
     TEST_UTIL.startMiniCluster(1);
   }
 
+  @Before
+  public void setup() {
+    TABLE_NAME = TableName.valueOf(name.getMethodName());
+
+  }
+  
   @AfterClass
   public static void afterAllTests() throws Exception {
     TEST_UTIL.shutdownMiniCluster();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java Thu Oct  3 19:47:09 2013
@@ -38,7 +38,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.Multithre
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.filter.Co
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -79,12 +80,10 @@ public class TestAtomicOperation {
   @Rule public TestName name = new TestName();
 
   HRegion region = null;
-  private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final String DIR = TEST_UTIL.getDataTestDir("TestAtomicOperation").toString();
-
+  private HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   // Test names
-  static final byte[] tableName = Bytes.toBytes("testtable");;
+  static  byte[] tableName;
   static final byte[] qual1 = Bytes.toBytes("qual1");
   static final byte[] qual2 = Bytes.toBytes("qual2");
   static final byte[] qual3 = Bytes.toBytes("qual3");
@@ -93,6 +92,11 @@ public class TestAtomicOperation {
   static final byte [] row = Bytes.toBytes("rowA");
   static final byte [] row2 = Bytes.toBytes("rowB");
 
+  @Before 
+  public void setup() {
+    tableName = Bytes.toBytes(name.getMethodName());
+  }
+  
   //////////////////////////////////////////////////////////////////////////////
   // New tests that doesn't spin up a mini cluster but rather just test the
   // individual code pieces in the HRegion. 
@@ -196,14 +200,7 @@ public class TestAtomicOperation {
       htd.addFamily(hcd);
     }
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-    Path path = new Path(DIR + callingMethod);
-    FileSystem fs = TEST_UTIL.getTestFileSystem();
-    if (fs.exists(path)) {
-      if (!fs.delete(path, true)) {
-        throw new IOException("Failed delete of " + path);
-      }
-    }
-    region = HRegion.createHRegion(info, path, HBaseConfiguration.create(), htd);
+    region = TEST_UTIL.createLocalHRegion(info, htd);
   }
 
   /**
@@ -521,10 +518,10 @@ public class TestAtomicOperation {
   public void testPutAndCheckAndPutInParallel() throws Exception {
 
     final String tableName = "testPutAndCheckAndPut";
-    Configuration conf = HBaseConfiguration.create();
+    Configuration conf = TEST_UTIL.getConfiguration();
     conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
-    final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(
-        Bytes.toBytes(tableName), tableName, conf, Bytes.toBytes(family));
+    final MockHRegion region = (MockHRegion) TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
+        null, null, tableName, conf, false, Durability.SYNC_WAL, null, Bytes.toBytes(family));
 
     Put[] puts = new Put[1];
     Put put = new Put(Bytes.toBytes("r1"));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java Thu Oct  3 19:47:09 2013
@@ -54,8 +54,10 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -69,10 +71,11 @@ import org.junit.runners.Parameterized.P
 public class TestCacheOnWriteInSchema {
 
   private static final Log LOG = LogFactory.getLog(TestCacheOnWriteInSchema.class);
+  @Rule public TestName name = new TestName();
 
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
   private static final String DIR = TEST_UTIL.getDataTestDir("TestCacheOnWriteInSchema").toString();
-  private static final byte [] table = Bytes.toBytes("table");
+  private static byte [] table;
   private static byte [] family = Bytes.toBytes("family");
   private static final int NUM_KV = 25000;
   private static final Random rand = new Random(12983177L);
@@ -141,6 +144,9 @@ public class TestCacheOnWriteInSchema {
 
   @Before
   public void setUp() throws IOException {
+    // parameterized tests add [#] suffix get rid of [ and ].
+    table = Bytes.toBytes(name.getMethodName().replaceAll("[\\[\\]]", "_"));
+
     conf = TEST_UTIL.getConfiguration();
     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
@@ -165,7 +171,7 @@ public class TestCacheOnWriteInSchema {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     hlog = HLogFactory.createHLog(fs, basedir, logName, conf);
 
-    region = new HRegion(basedir, hlog, fs, conf, info, htd, null);
+    region = TEST_UTIL.createLocalHRegion(info, htd, hlog);
     store = new HStore(region, hcd, conf);
   }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java?rev=1528984&r1=1528983&r2=1528984&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java Thu Oct  3 19:47:09 2013
@@ -18,9 +18,20 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.mockito.AdditionalMatchers.aryEq;
-import static org.mockito.Matchers.*;
-import static org.mockito.Mockito.*;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -43,17 +54,17 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
@@ -67,8 +78,13 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
+import org.junit.After;
 import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -78,10 +94,12 @@ import org.mockito.stubbing.Answer;
  * Test compactions
  */
 @Category(MediumTests.class)
-public class TestCompaction extends HBaseTestCase {
+public class TestCompaction {
+  @Rule public TestName name = new TestName();
   static final Log LOG = LogFactory.getLog(TestCompaction.class.getName());
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility().createLocalHTU();
+  protected Configuration conf = UTIL.getConfiguration();
+  
   private HRegion r = null;
   private HTableDescriptor htd = null;
   private Path compactionDir = null;
@@ -113,19 +131,17 @@ public class TestCompaction extends HBas
     col2 = Bytes.toBytes("column2");
   }
 
-  @Override
+  @Before
   public void setUp() throws Exception {
-    super.setUp();
-    this.htd = createTableDescriptor(getName());
-    this.r = createNewHRegion(htd, null, null);
+    this.htd = UTIL.createTableDescriptor(name.getMethodName());
+    this.r = UTIL.createLocalHRegion(htd, null, null);
   }
 
-  @Override
+  @After
   public void tearDown() throws Exception {
     HLog hlog = r.getLog();
     this.r.close();
     hlog.closeAndDelete();
-    super.tearDown();
   }
 
   /**
@@ -134,6 +150,7 @@ public class TestCompaction extends HBas
    * right answer in this case - and that it just basically works.
    * @throws IOException
    */
+  @Test
   public void testMajorCompactingToNoOutput() throws IOException {
     createStoreFile(r);
     for (int i = 0; i < compactionThreshold; i++) {
@@ -168,14 +185,17 @@ public class TestCompaction extends HBas
    * Assert deletes get cleaned up.
    * @throws Exception
    */
+  @Test
   public void testMajorCompaction() throws Exception {
     majorCompaction();
   }
 
+  @Test
   public void testDataBlockEncodingInCacheOnly() throws Exception {
     majorCompactionWithDataBlockEncoding(true);
   }
 
+  @Test
   public void testDataBlockEncodingEverywhere() throws Exception {
     majorCompactionWithDataBlockEncoding(false);
   }
@@ -210,7 +230,7 @@ public class TestCompaction extends HBas
       createStoreFile(r);
     }
     // Add more content.
-    addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY));
+    HBaseTestCase.addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY));
 
     // Now there are about 5 versions of each column.
     // Default is that there only 3 (MAXVERSIONS) versions allowed per column.
@@ -310,6 +330,7 @@ public class TestCompaction extends HBas
     assertEquals("Should not see anything after TTL has expired", 0, count);
   }
 
+  @Test
   public void testTimeBasedMajorCompaction() throws Exception {
     // create 2 storefiles and force a major compaction to reset the time
     int delay = 10 * 1000; // 10 sec
@@ -359,16 +380,21 @@ public class TestCompaction extends HBas
     }
   }
 
+  @Test
   public void testMinorCompactionWithDeleteRow() throws Exception {
     Delete deleteRow = new Delete(secondRowBytes);
     testMinorCompactionWithDelete(deleteRow);
   }
+
+  @Test
   public void testMinorCompactionWithDeleteColumn1() throws Exception {
     Delete dc = new Delete(secondRowBytes);
     /* delete all timestamps in the column */
     dc.deleteColumns(fam2, col2);
     testMinorCompactionWithDelete(dc);
   }
+
+  @Test
   public void testMinorCompactionWithDeleteColumn2() throws Exception {
     Delete dc = new Delete(secondRowBytes);
     dc.deleteColumn(fam2, col2);
@@ -381,11 +407,15 @@ public class TestCompaction extends HBas
     //testMinorCompactionWithDelete(dc, 2);
     testMinorCompactionWithDelete(dc, 3);
   }
+
+  @Test
   public void testMinorCompactionWithDeleteColumnFamily() throws Exception {
     Delete deleteCF = new Delete(secondRowBytes);
     deleteCF.deleteFamily(fam2);
     testMinorCompactionWithDelete(deleteCF);
   }
+
+  @Test
   public void testMinorCompactionWithDeleteVersion1() throws Exception {
     Delete deleteVersion = new Delete(secondRowBytes);
     deleteVersion.deleteColumns(fam2, col2, 2);
@@ -394,6 +424,8 @@ public class TestCompaction extends HBas
      */
     testMinorCompactionWithDelete(deleteVersion, 1);
   }
+
+  @Test
   public void testMinorCompactionWithDeleteVersion2() throws Exception {
     Delete deleteVersion = new Delete(secondRowBytes);
     deleteVersion.deleteColumn(fam2, col2, 1);
@@ -417,10 +449,10 @@ public class TestCompaction extends HBas
   private void testMinorCompactionWithDelete(Delete delete, int expectedResultsAfterDelete) throws Exception {
     HRegionIncommon loader = new HRegionIncommon(r);
     for (int i = 0; i < compactionThreshold + 1; i++) {
-      addContent(loader, Bytes.toString(fam1), Bytes.toString(col1), firstRowBytes, thirdRowBytes, i);
-      addContent(loader, Bytes.toString(fam1), Bytes.toString(col2), firstRowBytes, thirdRowBytes, i);
-      addContent(loader, Bytes.toString(fam2), Bytes.toString(col1), firstRowBytes, thirdRowBytes, i);
-      addContent(loader, Bytes.toString(fam2), Bytes.toString(col2), firstRowBytes, thirdRowBytes, i);
+      HBaseTestCase.addContent(loader, Bytes.toString(fam1), Bytes.toString(col1), firstRowBytes, thirdRowBytes, i);
+      HBaseTestCase.addContent(loader, Bytes.toString(fam1), Bytes.toString(col2), firstRowBytes, thirdRowBytes, i);
+      HBaseTestCase.addContent(loader, Bytes.toString(fam2), Bytes.toString(col1), firstRowBytes, thirdRowBytes, i);
+      HBaseTestCase.addContent(loader, Bytes.toString(fam2), Bytes.toString(col2), firstRowBytes, thirdRowBytes, i);
       r.flushcache();
     }
 
@@ -496,6 +528,7 @@ public class TestCompaction extends HBas
    * (used during RS shutdown)
    * @throws Exception
    */
+  @Test
   public void testInterruptCompaction() throws Exception {
     assertEquals(0, count());
 
@@ -514,7 +547,7 @@ public class TestCompaction extends HBas
         for (int j = 0; j < jmax; j++) {
           p.add(COLUMN_FAMILY, Bytes.toBytes(j), pad);
         }
-        addContent(loader, Bytes.toString(COLUMN_FAMILY));
+        HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY));
         loader.put(p);
         loader.flushcache();
       }
@@ -591,17 +624,18 @@ public class TestCompaction extends HBas
 
   private void createStoreFile(final HRegion region, String family) throws IOException {
     HRegionIncommon loader = new HRegionIncommon(region);
-    addContent(loader, family);
+    HBaseTestCase.addContent(loader, family);
     loader.flushcache();
   }
 
   private void createSmallerStoreFile(final HRegion region) throws IOException {
     HRegionIncommon loader = new HRegionIncommon(region);
-    addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" +
+    HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" +
     		"bbb").getBytes(), null);
     loader.flushcache();
   }
 
+  @Test
   public void testCompactionWithCorruptResult() throws Exception {
     int nfiles = 10;
     for (int i = 0; i < nfiles; i++) {
@@ -641,6 +675,7 @@ public class TestCompaction extends HBas
   /**
    * Test for HBASE-5920 - Test user requested major compactions always occurring
    */
+  @Test
   public void testNonUserMajorCompactionRequest() throws Exception {
     Store store = r.getStore(COLUMN_FAMILY);
     createStoreFile(r);
@@ -660,6 +695,7 @@ public class TestCompaction extends HBas
   /**
    * Test for HBASE-5920
    */
+  @Test
   public void testUserMajorCompactionRequest() throws IOException{
     Store store = r.getStore(COLUMN_FAMILY);
     createStoreFile(r);
@@ -679,6 +715,7 @@ public class TestCompaction extends HBas
    * Create a custom compaction request and be sure that we can track it through the queue, knowing
    * when the compaction is completed.
    */
+  @Test
   public void testTrackingCompactionRequest() throws Exception {
     // setup a compact/split thread on a mock server
     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
@@ -707,6 +744,7 @@ public class TestCompaction extends HBas
    * {@link CompactSplitThread}
    * @throws Exception on failure
    */
+  @Test
   public void testMultipleCustomCompactionRequests() throws Exception {
     // setup a compact/split thread on a mock server
     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
@@ -883,6 +921,7 @@ public class TestCompaction extends HBas
   }
 
   /** Test compaction priority management and multiple compactions per store (HBASE-8665). */
+  @Test
   public void testCompactionQueuePriorities() throws Exception {
     // Setup a compact/split thread on a mock server.
     final Configuration conf = HBaseConfiguration.create();