You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2012/12/23 21:55:55 UTC

svn commit: r1425526 [5/6] - in /hbase/branches/0.94-test: ./ bin/ conf/ security/src/main/java/org/apache/hadoop/hbase/ipc/ security/src/main/java/org/apache/hadoop/hbase/security/access/ security/src/test/java/org/apache/hadoop/hbase/security/access/...

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java Sun Dec 23 20:55:53 2012
@@ -60,7 +60,6 @@ public class TestClassLoading {
   static final String cpName4 = "TestCP4";
   static final String cpName5 = "TestCP5";
   static final String cpName6 = "TestCP6";
-  static final String cpNameInvalid = "TestCPInvalid";
 
   private static Class<?> regionCoprocessor1 = ColumnAggregationEndpoint.class;
   private static Class<?> regionCoprocessor2 = GenericEndpoint.class;
@@ -69,6 +68,14 @@ public class TestClassLoading {
 
   private static final String[] regionServerSystemCoprocessors =
       new String[]{
+      regionCoprocessor1.getSimpleName(),
+      regionServerCoprocessor.getSimpleName()
+  };
+
+  private static final String[] regionServerSystemAndUserCoprocessors =
+      new String[] {
+      regionCoprocessor1.getSimpleName(),
+      regionCoprocessor2.getSimpleName(),
       regionServerCoprocessor.getSimpleName()
   };
 
@@ -200,18 +207,16 @@ public class TestClassLoading {
       new Path(fs.getUri().toString() + Path.SEPARATOR));
     String jarFileOnHDFS1 = fs.getUri().toString() + Path.SEPARATOR +
       jarFile1.getName();
-    Path pathOnHDFS1 = new Path(jarFileOnHDFS1);
     assertTrue("Copy jar file to HDFS failed.",
-      fs.exists(pathOnHDFS1));
+      fs.exists(new Path(jarFileOnHDFS1)));
     LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS1);
 
     fs.copyFromLocalFile(new Path(jarFile2.getPath()),
         new Path(fs.getUri().toString() + Path.SEPARATOR));
     String jarFileOnHDFS2 = fs.getUri().toString() + Path.SEPARATOR +
       jarFile2.getName();
-    Path pathOnHDFS2 = new Path(jarFileOnHDFS2);
     assertTrue("Copy jar file to HDFS failed.",
-      fs.exists(pathOnHDFS2));
+      fs.exists(new Path(jarFileOnHDFS2)));
     LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS2);
 
     // create a table that references the coprocessors
@@ -223,78 +228,41 @@ public class TestClassLoading {
       // with configuration values
     htd.setValue("COPROCESSOR$2", jarFileOnHDFS2.toString() + "|" + cpName2 +
       "|" + Coprocessor.PRIORITY_USER + "|k1=v1,k2=v2,k3=v3");
-    // same jar but invalid class name (should fail to load this class)
-    htd.setValue("COPROCESSOR$3", jarFileOnHDFS2.toString() + "|" + cpNameInvalid +
-      "|" + Coprocessor.PRIORITY_USER);
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
     }
-    CoprocessorHost.classLoadersCache.clear();
-    byte[] startKey = {10, 63};
-    byte[] endKey = {12, 43};
-    admin.createTable(htd, startKey, endKey, 4);
+    admin.createTable(htd);
     waitForTable(htd.getName());
 
     // verify that the coprocessors were loaded
-    boolean foundTableRegion=false;
-    boolean found_invalid = true, found1 = true, found2 = true, found2_k1 = true,
-        found2_k2 = true, found2_k3 = true;
-    Map<HRegion, Set<ClassLoader>> regionsActiveClassLoaders =
-        new HashMap<HRegion, Set<ClassLoader>>();
+    boolean found1 = false, found2 = false, found2_k1 = false,
+        found2_k2 = false, found2_k3 = false;
     MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster();
     for (HRegion region:
         hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
       if (region.getRegionNameAsString().startsWith(tableName)) {
-        foundTableRegion = true;
         CoprocessorEnvironment env;
         env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName1);
-        found1 = found1 && (env != null);
+        if (env != null) {
+          found1 = true;
+        }
         env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName2);
-        found2 = found2 && (env != null);
         if (env != null) {
+          found2 = true;
           Configuration conf = env.getConfiguration();
-          found2_k1 = found2_k1 && (conf.get("k1") != null);
-          found2_k2 = found2_k2 && (conf.get("k2") != null);
-          found2_k3 = found2_k3 && (conf.get("k3") != null);
-        } else {
-          found2_k1 = found2_k2 = found2_k3 = false;
+          found2_k1 = conf.get("k1") != null;
+          found2_k2 = conf.get("k2") != null;
+          found2_k3 = conf.get("k3") != null;
         }
-        env = region.getCoprocessorHost().findCoprocessorEnvironment(cpNameInvalid);
-        found_invalid = found_invalid && (env != null);
-
-        regionsActiveClassLoaders
-            .put(region, ((CoprocessorHost) region.getCoprocessorHost()).getExternalClassLoaders());
       }
     }
-
-    assertTrue("No region was found for table " + tableName, foundTableRegion);
     assertTrue("Class " + cpName1 + " was missing on a region", found1);
     assertTrue("Class " + cpName2 + " was missing on a region", found2);
-    //an invalid CP class name is defined for this table, validate that it is not loaded
-    assertFalse("Class " + cpNameInvalid + " was found on a region", found_invalid);
     assertTrue("Configuration key 'k1' was missing on a region", found2_k1);
     assertTrue("Configuration key 'k2' was missing on a region", found2_k2);
     assertTrue("Configuration key 'k3' was missing on a region", found2_k3);
-    // check if CP classloaders are cached
-    assertTrue(jarFileOnHDFS1 + " was not cached",
-      CoprocessorHost.classLoadersCache.containsKey(pathOnHDFS1));
-    assertTrue(jarFileOnHDFS2 + " was not cached",
-      CoprocessorHost.classLoadersCache.containsKey(pathOnHDFS2));
-    //two external jar used, should be one classloader per jar
-    assertEquals("The number of cached classloaders should be equal to the number" +
-      " of external jar files",
-      2, CoprocessorHost.classLoadersCache.size());
-    //check if region active classloaders are shared across all RS regions
-    Set<ClassLoader> externalClassLoaders = new HashSet<ClassLoader>(
-        CoprocessorHost.classLoadersCache.values());
-    for (Map.Entry<HRegion, Set<ClassLoader>> regionCP : regionsActiveClassLoaders.entrySet()) {
-      assertTrue("Some CP classloaders for region " + regionCP.getKey() + " are not cached."
-            + " ClassLoader Cache:" + externalClassLoaders
-            + " Region ClassLoaders:" + regionCP.getValue(),
-            externalClassLoaders.containsAll(regionCP.getValue()));
-    }
   }
 
   @Test
@@ -456,8 +424,6 @@ public class TestClassLoading {
     File outerJarFile = new File(TEST_UTIL.getDataTestDir().toString(), "outer.jar");
 
     byte buffer[] = new byte[BUFFER_SIZE];
-    // TODO: code here and elsewhere in this file is duplicated w/TestClassFinder.
-    //       Some refactoring may be in order...
     // Open archive file
     FileOutputStream stream = new FileOutputStream(outerJarFile);
     JarOutputStream out = new JarOutputStream(stream, new Manifest());
@@ -467,7 +433,7 @@ public class TestClassLoading {
       JarEntry jarAdd = new JarEntry("/lib/" + jarFile.getName());
       jarAdd.setTime(jarFile.lastModified());
       out.putNextEntry(jarAdd);
-
+  
       // Write file to archive
       FileInputStream in = new FileInputStream(jarFile);
       while (true) {
@@ -539,12 +505,82 @@ public class TestClassLoading {
 
   @Test
   public void testRegionServerCoprocessorsReported() throws Exception {
-    // This was a test for HBASE-4070.
-    // We are removing coprocessors from region load in HBASE-5258.
-    // Therefore, this test now only checks system coprocessors.
+    // HBASE 4070: Improve region server metrics to report loaded coprocessors
+    // to master: verify that each regionserver is reporting the correct set of
+    // loaded coprocessors.
+
+    // We rely on the fact that getCoprocessors() will return a sorted
+    // display of the coprocessors' names, so for example, regionCoprocessor1's
+    // name "ColumnAggregationEndpoint" will appear before regionCoprocessor2's
+    // name "GenericEndpoint" because "C" is before "G" lexicographically.
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+
+    // disable all user tables, if any are loaded.
+    for (HTableDescriptor htd: admin.listTables()) {
+      if (!htd.isMetaTable()) {
+        String tableName = htd.getNameAsString();
+        if (admin.isTableEnabled(tableName)) {
+          try {
+            admin.disableTable(htd.getNameAsString());
+          } catch (TableNotEnabledException e) {
+            // ignoring this exception for now : not sure why it's happening.
+          }
+        }
+      }
+    }
+
+    // should only be system coprocessors loaded at this point.
     assertAllRegionServers(regionServerSystemCoprocessors,null);
+
+    // The next two tests enable and disable user tables to see if coprocessor
+    // load reporting changes as coprocessors are loaded and unloaded.
+    //
+
+    // Create a table.
+    // should cause regionCoprocessor2 to be loaded, since we've specified it
+    // for loading on any user table with USER_REGION_COPROCESSOR_CONF_KEY
+    // in setUpBeforeClass().
+    String userTable1 = "userTable1";
+    HTableDescriptor userTD1 = new HTableDescriptor(userTable1);
+    admin.createTable(userTD1);
+    waitForTable(userTD1.getName());
+
+    // table should be enabled now.
+    assertTrue(admin.isTableEnabled(userTable1));
+    assertAllRegionServers(regionServerSystemAndUserCoprocessors, userTable1);
+
+    // unload and make sure we're back to only system coprocessors again.
+    admin.disableTable(userTable1);
+    assertAllRegionServers(regionServerSystemCoprocessors,null);
+
+    // create another table, with its own specified coprocessor.
+    String userTable2 = "userTable2";
+    HTableDescriptor htd2 = new HTableDescriptor(userTable2);
+
+    String userTableCP = "userTableCP";
+    File jarFile1 = buildCoprocessorJar(userTableCP);
+    htd2.addFamily(new HColumnDescriptor("myfamily"));
+    htd2.setValue("COPROCESSOR$1", jarFile1.toString() + "|" + userTableCP +
+      "|" + Coprocessor.PRIORITY_USER);
+    admin.createTable(htd2);
+    waitForTable(htd2.getName());
+    // table should be enabled now.
+    assertTrue(admin.isTableEnabled(userTable2));
+
+    ArrayList<String> existingCPsPlusNew =
+        new ArrayList<String>(Arrays.asList(regionServerSystemAndUserCoprocessors));
+    existingCPsPlusNew.add(userTableCP);
+    String[] existingCPsPlusNewArray = new String[existingCPsPlusNew.size()];
+    assertAllRegionServers(existingCPsPlusNew.toArray(existingCPsPlusNewArray),
+        userTable2);
+
+    admin.disableTable(userTable2);
+    assertTrue(admin.isTableDisabled(userTable2));
+
+    // we should be back to only system coprocessors again.
+    assertAllRegionServers(regionServerSystemCoprocessors, null);
+
   }
 
   /**
@@ -591,7 +627,7 @@ public class TestClassLoading {
       }
       boolean any_failed = false;
       for(Map.Entry<ServerName,HServerLoad> server: servers.entrySet()) {
-        actualCoprocessors = server.getValue().getRsCoprocessors();
+        actualCoprocessors = server.getValue().getCoprocessors();
         if (!Arrays.equals(actualCoprocessors, expectedCoprocessors)) {
           LOG.debug("failed comparison: actual: " +
               Arrays.toString(actualCoprocessors) +

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java Sun Dec 23 20:55:53 2012
@@ -85,18 +85,6 @@ public class TestCoprocessorInterface ex
     }
 
     @Override
-    public boolean nextRaw(List<KeyValue> result, int limit, String metric) 
-        throws IOException {
-      return delegate.nextRaw(result, limit, metric);
-    }
-
-    @Override
-    public boolean nextRaw(List<KeyValue> result, String metric) 
-        throws IOException {
-      return delegate.nextRaw(result, metric);
-    }
-
-    @Override
     public void close() throws IOException {
       delegate.close();
     }
@@ -116,10 +104,6 @@ public class TestCoprocessorInterface ex
       return false;
     }
 
-    @Override
-    public long getMvccReadPoint() {
-      return delegate.getMvccReadPoint();
-    }
   }
 
   public static class CoprocessorImpl extends BaseRegionObserver {

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java Sun Dec 23 20:55:53 2012
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.apache.hadoop.io.compress.Compressor;
 
 import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.*;
@@ -718,8 +719,7 @@ public class TestHFileBlock {
       }
       BlockType bt = BlockType.values()[blockTypeOrdinal];
       DataOutputStream dos = hbw.startWriting(bt);
-      int size = rand.nextInt(500);
-      for (int j = 0; j < size; ++j) {
+      for (int j = 0; j < rand.nextInt(500); ++j) {
         // This might compress well.
         dos.writeShort(i + 1);
         dos.writeInt(j + 1);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java Sun Dec 23 20:55:53 2012
@@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.client.Get;
@@ -61,12 +61,12 @@ import org.apache.hadoop.hbase.util.Thre
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKTable.TableState;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKTable.TableState;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -74,7 +74,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
-import org.mockito.internal.util.reflection.Whitebox;
 
 import com.google.protobuf.ServiceException;
 
@@ -82,7 +81,7 @@ import com.google.protobuf.ServiceExcept
 /**
  * Test {@link AssignmentManager}
  */
-@Category(MediumTests.class)
+@Category(SmallTests.class)
 public class TestAssignmentManager {
   private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
   private static final ServerName SERVERNAME_A =
@@ -92,10 +91,6 @@ public class TestAssignmentManager {
   private static final HRegionInfo REGIONINFO =
     new HRegionInfo(Bytes.toBytes("t"),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-  private static final HRegionInfo REGIONINFO_2 = new HRegionInfo(Bytes.toBytes("t"),
-      Bytes.toBytes("a"),Bytes.toBytes( "b"));
-  private static int assignmentCount;
-  private static boolean enabling = false;  
 
   // Mocked objects or; get redone for each test.
   private Server server;
@@ -162,7 +157,7 @@ public class TestAssignmentManager {
 
   /**
    * Test a balance going on at same time as a master failover
-   *
+   * 
    * @throws IOException
    * @throws KeeperException
    * @throws InterruptedException
@@ -184,8 +179,10 @@ public class TestAssignmentManager {
       int versionid =
         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
       assertNotSame(versionid, -1);
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
+      while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
+          EventType.M_ZK_REGION_OFFLINE)) {
+        Threads.sleep(1);
+      }
       // Get current versionid else will fail on transition from OFFLINE to
       // OPENING below
       versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
@@ -226,8 +223,10 @@ public class TestAssignmentManager {
         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
       assertNotSame(versionid, -1);
       am.gate.set(false);
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
+      while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
+          EventType.M_ZK_REGION_OFFLINE)) {
+        Threads.sleep(1);
+      }
       // Get current versionid else will fail on transition from OFFLINE to
       // OPENING below
       versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
@@ -267,8 +266,10 @@ public class TestAssignmentManager {
       int versionid =
         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
       assertNotSame(versionid, -1);
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
+      while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
+          EventType.M_ZK_REGION_OFFLINE)) {
+        Threads.sleep(1);
+      }
       am.gate.set(false);
       // Get current versionid else will fail on transition from OFFLINE to
       // OPENING below
@@ -307,11 +308,10 @@ public class TestAssignmentManager {
    * from one server to another mocking regionserver responding over zk.
    * @throws IOException
    * @throws KeeperException
-   * @throws InterruptedException
    */
-  @Test(timeout = 10000)
+  @Test
   public void testBalance()
-  throws IOException, KeeperException, InterruptedException {
+  throws IOException, KeeperException {
     // Create and startup an executor.  This is used by AssignmentManager
     // handling zk callbacks.
     ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
@@ -345,9 +345,11 @@ public class TestAssignmentManager {
       // AM is going to notice above CLOSED and queue up a new assign.  The
       // assign will go to open the region in the new location set by the
       // balancer.  The zk node will be OFFLINE waiting for regionserver to
-      // transition it through OPENING, OPENED.  Wait till we see the RIT
-      // before we proceed.
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
+      // transition it through OPENING, OPENED.  Wait till we see the OFFLINE
+      // zk node before we proceed.
+      while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
+        Threads.sleep(1);
+      }
       // Get current versionid else will fail on transition from OFFLINE to OPENING below
       versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
       assertNotSame(-1, versionid);
@@ -400,7 +402,7 @@ public class TestAssignmentManager {
 
   /**
    * To test closed region handler to remove rit and delete corresponding znode if region in pending
-   * close or closing while processing shutdown of a region server.(HBASE-5927).
+   * close or closing while processing shutdown of a region server.(HBASE-5927). 
    * @throws KeeperException
    * @throws IOException
    */
@@ -410,7 +412,7 @@ public class TestAssignmentManager {
     testCaseWithPartiallyDisabledState(TableState.DISABLING);
     testCaseWithPartiallyDisabledState(TableState.DISABLED);
   }
-
+  
   /**
    * To test if the split region is removed from RIT if the region was in SPLITTING state
    * but the RS has actually completed the splitting in META but went down. See HBASE-6070
@@ -444,7 +446,7 @@ public class TestAssignmentManager {
     am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO,
         State.SPLITTING, System.currentTimeMillis(), SERVERNAME_A));
     am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
-
+    
     RegionTransitionData data = new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
         REGIONINFO.getRegionName(), SERVERNAME_A);
     String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
@@ -452,11 +454,11 @@ public class TestAssignmentManager {
     ZKUtil.createAndWatch(this.watcher, node, data.getBytes());
 
     try {
-
+      
       processServerShutdownHandler(ct, am, regionSplitDone);
       // check znode deleted or not.
       // In both cases the znode should be deleted.
-
+      
       if(regionSplitDone){
         assertTrue("Region state of region in SPLITTING should be removed from rit.",
             am.regionsInTransition.isEmpty());
@@ -499,7 +501,7 @@ public class TestAssignmentManager {
     } else {
       am.getZKTable().setDisabledTable(REGIONINFO.getTableNameAsString());
     }
-
+    
     RegionTransitionData data = new RegionTransitionData(EventType.M_ZK_REGION_CLOSING,
         REGIONINFO.getRegionName(), SERVERNAME_A);
     String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
@@ -574,7 +576,7 @@ public class TestAssignmentManager {
    * @param hri Region to serialize into HRegionInfo
    * @return A mocked up Result that fakes a Get on a row in the
    * <code>.META.</code> table.
-   * @throws IOException
+   * @throws IOException 
    */
   private Result getMetaTableRowResult(final HRegionInfo hri,
       final ServerName sn)
@@ -593,13 +595,13 @@ public class TestAssignmentManager {
       Bytes.toBytes(sn.getStartcode())));
     return new Result(kvs);
   }
-
+  
   /**
    * @param sn ServerName to use making startcode and server in meta
    * @param hri Region to serialize into HRegionInfo
    * @return A mocked up Result that fakes a Get on a row in the
    * <code>.META.</code> table.
-   * @throws IOException
+   * @throws IOException 
    */
   private Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri, final ServerName sn)
       throws IOException {
@@ -661,12 +663,12 @@ public class TestAssignmentManager {
       am.shutdown();
     }
   }
-
+  
   /**
    * Tests the processDeadServersAndRegionsInTransition should not fail with NPE
    * when it failed to get the children. Let's abort the system in this
    * situation
-   * @throws ServiceException
+   * @throws ServiceException 
    */
   @Test(timeout = 5000)
   public void testProcessDeadServersAndRegionsInTransitionShouldNotFailWithNPE()
@@ -706,8 +708,8 @@ public class TestAssignmentManager {
    * @param region region to be created as offline
    * @param serverName server event originates from
    * @return Version of znode created.
-   * @throws KeeperException
-   * @throws IOException
+   * @throws KeeperException 
+   * @throws IOException 
    */
   // Copied from SplitTransaction rather than open the method over there in
   // the regionserver package.
@@ -766,27 +768,14 @@ public class TestAssignmentManager {
     // with an encoded name by doing a Get on .META.
     HRegionInterface ri = Mockito.mock(HRegionInterface.class);
     // Get a meta row result that has region up on SERVERNAME_A for REGIONINFO
-    Result[] result = null;
-    if (enabling) {
-      result = new Result[2];
-      result[0] = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
-      result[1] = getMetaTableRowResult(REGIONINFO_2, SERVERNAME_A);
-    }
     Result r = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
     Mockito.when(ri .openScanner((byte[]) Mockito.any(), (Scan) Mockito.any())).
       thenReturn(System.currentTimeMillis());
-   if (enabling) {
-      Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(result, result, result,
-          (Result[]) null);
-      // If a get, return the above result too for REGIONINFO_2
-      Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).thenReturn(
-          getMetaTableRowResult(REGIONINFO_2, SERVERNAME_A));
-    } else {
-      // Return good result 'r' first and then return null to indicate end of scan
-      Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(new Result[] { r });
-      // If a get, return the above result too for REGIONINFO
-      Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).thenReturn(r);
-    }
+    // Return good result 'r' first and then return null to indicate end of scan
+    Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(new Result[] { r });
+    // If a get, return the above result too for REGIONINFO
+    Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).
+      thenReturn(r);
     // Get a connection w/ mocked up common methods.
     HConnection connection = HConnectionTestingUtility.
       getMockedConnectionAndDecorate(HTU.getConfiguration(), ri, SERVERNAME_B,
@@ -800,9 +789,9 @@ public class TestAssignmentManager {
         server, manager, ct, balancer, executor);
     return am;
   }
-
+  
   /**
-   * TestCase verifies that the regionPlan is updated whenever a region fails to open
+   * TestCase verifies that the regionPlan is updated whenever a region fails to open 
    * and the master tries to process RS_ZK_FAILED_OPEN state.(HBASE-5546).
    */
   @Test
@@ -850,18 +839,17 @@ public class TestAssignmentManager {
       assertNotSame("Same region plan should not come", regionPlan, newRegionPlan);
       assertTrue("Destnation servers should be different.", !(regionPlan.getDestination().equals(
         newRegionPlan.getDestination())));
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
     } finally {
       this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
         DefaultLoadBalancer.class, LoadBalancer.class);
       am.shutdown();
     }
   }
-
+  
   /**
    * Test verifies whether assignment is skipped for regions of tables in DISABLING state during
    * clean cluster startup. See HBASE-6281.
-   *
+   * 
    * @throws KeeperException
    * @throws IOException
    * @throws Exception
@@ -904,53 +892,6 @@ public class TestAssignmentManager {
   }
 
   /**
-   * Test verifies whether all the enabling table regions assigned only once during master startup.
-   * 
-   * @throws KeeperException
-   * @throws IOException
-   * @throws Exception
-   */
-  @Test
-  public void testMasterRestartWhenTableInEnabling() throws KeeperException, IOException, Exception {
-    enabling = true;
-    this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
-        DefaultLoadBalancer.class, LoadBalancer.class);
-    Map<ServerName, HServerLoad> serverAndLoad = new HashMap<ServerName, HServerLoad>();
-    serverAndLoad.put(SERVERNAME_A, null);
-    Mockito.when(this.serverManager.getOnlineServers()).thenReturn(serverAndLoad);
-    Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B)).thenReturn(false);
-    Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
-    HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
-    Server server = new HMaster(HTU.getConfiguration());
-    Whitebox.setInternalState(server, "serverManager", this.serverManager);
-    assignmentCount = 0;
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
-        this.serverManager);
-    am.regionOnline(new HRegionInfo("t1".getBytes(), HConstants.EMPTY_START_ROW,
-        HConstants.EMPTY_END_ROW), SERVERNAME_A);
-    am.gate.set(false);
-    try {
-      // set table in enabling state.
-      am.getZKTable().setEnablingTable(REGIONINFO.getTableNameAsString());
-      ZKAssign.createNodeOffline(this.watcher, REGIONINFO_2, SERVERNAME_B);
-
-      am.joinCluster();
-      while (!am.getZKTable().isEnabledTable(REGIONINFO.getTableNameAsString())) {
-        Thread.sleep(10);
-      }
-      assertEquals("Number of assignments should be equal.", 2, assignmentCount);
-      assertTrue("Table should be enabled.",
-          am.getZKTable().isEnabledTable(REGIONINFO.getTableNameAsString()));
-    } finally {
-      enabling = false;
-      am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
-      am.shutdown();
-      ZKAssign.deleteAllNodes(this.watcher);
-      assignmentCount = 0;
-    }
-  }
-
-  /**
    * Mocked load balancer class used in the testcase to make sure that the testcase waits until
    * random assignment is called and the gate variable is set to true.
    */
@@ -967,7 +908,7 @@ public class TestAssignmentManager {
       this.gate.set(true);
       return randomServerName;
     }
-
+    
     @Override
     public Map<ServerName, List<HRegionInfo>> retainAssignment(
         Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
@@ -1019,13 +960,8 @@ public class TestAssignmentManager {
     @Override
     public void assign(HRegionInfo region, boolean setOfflineInZK, boolean forceNewPlan,
         boolean hijack) {
-      if (enabling) {
-        assignmentCount++;
-        this.regionOnline(region, SERVERNAME_A);
-      } else {
-        assignInvoked = true;
-        super.assign(region, setOfflineInZK, forceNewPlan, hijack);
-      }
+      assignInvoked = true;
+      super.assign(region, setOfflineInZK, forceNewPlan, hijack);
     }
     
     @Override

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Sun Dec 23 20:55:53 2012
@@ -241,11 +241,13 @@ public class TestCatalogJanitor {
       return new TableDescriptors() {
         @Override
         public HTableDescriptor remove(String tablename) throws IOException {
+          // TODO Auto-generated method stub
           return null;
         }
         
         @Override
         public Map<String, HTableDescriptor> getAll() throws IOException {
+          // TODO Auto-generated method stub
           return null;
         }
         
@@ -263,6 +265,8 @@ public class TestCatalogJanitor {
         
         @Override
         public void add(HTableDescriptor htd) throws IOException {
+          // TODO Auto-generated method stub
+          
         }
       };
     }
@@ -281,34 +285,6 @@ public class TestCatalogJanitor {
     public <T extends CoprocessorProtocol> boolean registerProtocol(Class<T> protocol, T handler) {
       return false;
     }
-
-    @Override
-    public void deleteTable(byte[] tableName) throws IOException {
-    }
-
-    @Override
-    public void modifyTable(byte[] tableName, HTableDescriptor descriptor) throws IOException {
-    }
-
-    @Override
-    public void enableTable(byte[] tableName) throws IOException {
-    }
-
-    @Override
-    public void disableTable(byte[] tableName) throws IOException {
-    }
-
-    @Override
-    public void addColumn(byte[] tableName, HColumnDescriptor column) throws IOException {
-    }
-
-    @Override
-    public void modifyColumn(byte[] tableName, HColumnDescriptor descriptor) throws IOException {
-    }
-
-    @Override
-    public void deleteColumn(byte[] tableName, byte[] columnName) throws IOException {
-    }
   }
 
   @Test

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java Sun Dec 23 20:55:53 2012
@@ -88,7 +88,7 @@ public class TestHFileCleaner {
         + status.getAccessTime();
   }
 
-  @Test(timeout = 60 *1000)
+  @Test
   public void testHFileCleaning() throws Exception {
     final EnvironmentEdge originalEdge = EnvironmentEdgeManager.getDelegate();
     String prefix = "someHFileThatWouldBeAUUID";

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java Sun Dec 23 20:55:53 2012
@@ -587,10 +587,8 @@ public class TestCompaction extends HBas
 
     List<StoreFile> storeFiles = store.getStorefiles();
     long maxId = StoreFile.getMaxSequenceIdInList(storeFiles);
-    Compactor tool = new Compactor(this.conf);
 
-    StoreFile.Writer compactedFile =
-      tool.compact(store, storeFiles, false, maxId);
+    StoreFile.Writer compactedFile = store.compactStore(storeFiles, false, maxId);
 
     // Now lets corrupt the compacted file.
     FileSystem fs = FileSystem.get(conf);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java Sun Dec 23 20:55:53 2012
@@ -16,9 +16,6 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SmallTests;
-import org.apache.hadoop.hbase.MultithreadedTestUtil;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
@@ -33,32 +30,19 @@ import org.junit.experimental.categories
 
 import com.google.common.collect.Lists;
 
-/**
- * Test of HBASE-7051; that checkAndPuts and puts behave atomically with respect to each other.
- * Rather than perform a bunch of trials to verify atomicity, this test recreates a race condition
- * that causes the test to fail if checkAndPut doesn't wait for outstanding put transactions
- * to complete.  It does this by invasively overriding HRegion function to affect the timing of
- * the operations.
- */
 @Category(SmallTests.class)
 public class TestHBase7051 {
 
+  private static volatile boolean putCompleted = false;
   private static CountDownLatch latch = new CountDownLatch(1);
-  private enum TestStep {
-    INIT,                  // initial put of 10 to set value of the cell
-    PUT_STARTED,           // began doing a put of 50 to cell
-    PUT_COMPLETED,         // put complete (released RowLock, but may not have advanced MVCC).
-    CHECKANDPUT_STARTED,   // began checkAndPut: if 10 -> 11
-    CHECKANDPUT_COMPLETED  // completed checkAndPut
-    // NOTE: at the end of these steps, the value of the cell should be 50, not 11!
-  }
-  private static volatile TestStep testStep = TestStep.INIT;
-  private final String family = "f1";
-  	 
+  private boolean checkAndPutCompleted = false;
+  private static int count = 0;
+
   @Test
   public void testPutAndCheckAndPutInParallel() throws Exception {
 
     final String tableName = "testPutAndCheckAndPut";
+    final String family = "f1";
     Configuration conf = HBaseConfiguration.create();
     conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
     final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(Bytes.toBytes(tableName),
@@ -73,16 +57,14 @@ public class TestHBase7051 {
 
     putsAndLocks.add(pair);
 
+    count++;
     region.batchMutate(putsAndLocks.toArray(new Pair[0]));
-    MultithreadedTestUtil.TestContext ctx =
-      new MultithreadedTestUtil.TestContext(conf);
-    ctx.addThread(new PutThread(ctx, region));
-    ctx.addThread(new CheckAndPutThread(ctx, region));
-    ctx.startThreads();
-    while (testStep != TestStep.CHECKANDPUT_COMPLETED) {
+    makeCheckAndPut(family, region);
+
+    makePut(family, region);
+    while (!checkAndPutCompleted) {
       Thread.sleep(100);
     }
-    ctx.stop();
     Scan s = new Scan();
     RegionScanner scanner = region.getScanner(s);
     List<KeyValue> results = new ArrayList<KeyValue>();
@@ -93,46 +75,54 @@ public class TestHBase7051 {
 
   }
 
-  private class PutThread extends TestThread {
-    private MockHRegion region;
-    PutThread(TestContext ctx, MockHRegion region) {
-      super(ctx);
-      this.region = region;
-    }
-
-    public void doWork() throws Exception {
-      List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
-      Put[] puts = new Put[1];
-      Put put = new Put(Bytes.toBytes("r1"));
-      put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
-      puts[0] = put;
-      Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
-      putsAndLocks.add(pair);
-      testStep = TestStep.PUT_STARTED;
-      region.batchMutate(putsAndLocks.toArray(new Pair[0]));
-    }
+  private void makePut(final String family, final MockHRegion region) {
+    new Thread() {
+      public void run() {
+        List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
+        Put[] puts = new Put[1];
+        Put put = new Put(Bytes.toBytes("r1"));
+        put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
+        puts[0] = put;
+        try {
+          Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
+          putsAndLocks.add(pair);
+          count++;
+          region.batchMutate(putsAndLocks.toArray(new Pair[0]));
+        } catch (IOException e) {
+          // TODO Auto-generated catch block
+          e.printStackTrace();
+        }
+      }
+    }.start();
   }
 
-  private class CheckAndPutThread extends TestThread {
-    private MockHRegion region;
-    CheckAndPutThread(TestContext ctx, MockHRegion region) {
-      super(ctx);
-      this.region = region;
-   }
-
-    public void doWork() throws Exception {
-      Put[] puts = new Put[1];
-      Put put = new Put(Bytes.toBytes("r1"));
-      put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("11"));
-      puts[0] = put;
-      while (testStep != TestStep.PUT_COMPLETED) {
-        Thread.sleep(100);
-      }
-      testStep = TestStep.CHECKANDPUT_STARTED;
-      region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
-        CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
-      testStep = TestStep.CHECKANDPUT_COMPLETED;
-    }
+  private void makeCheckAndPut(final String family, final MockHRegion region) {
+    new Thread() {
+
+      public void run() {
+        Put[] puts = new Put[1];
+        Put put = new Put(Bytes.toBytes("r1"));
+        put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("11"));
+        puts[0] = put;
+        try {
+          while (putCompleted == false) {
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException e) {
+              // TODO Auto-generated catch block
+              e.printStackTrace();
+            }
+          }
+          count++;
+          region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
+              CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
+          checkAndPutCompleted = true;
+        } catch (IOException e) {
+          // TODO Auto-generated catch block
+          e.printStackTrace();
+        }
+      }
+    }.start();
   }
 
   public static class MockHRegion extends HRegion {
@@ -144,39 +134,36 @@ public class TestHBase7051 {
 
     @Override
     public void releaseRowLock(Integer lockId) {
-      if (testStep == TestStep.INIT) {
+      if (count == 1) {
         super.releaseRowLock(lockId);
         return;
       }
 
-      if (testStep == TestStep.PUT_STARTED) {
+      if (count == 2) {
         try {
-          testStep = TestStep.PUT_COMPLETED;
+          putCompleted = true;
           super.releaseRowLock(lockId);
-          // put has been written to the memstore and the row lock has been released, but the
-          // MVCC has not been advanced.  Prior to fixing HBASE-7051, the following order of
-          // operations would cause the non-atomicity to show up:
-          // 1) Put releases row lock (where we are now)
-          // 2) CheckAndPut grabs row lock and reads the value prior to the put (10)
-          //    because the MVCC has not advanced
-          // 3) Put advances MVCC
-          // So, in order to recreate this order, we wait for the checkAndPut to grab the rowLock
-          // (see below), and then wait some more to give the checkAndPut time to read the old
-          // value.
           latch.await();
-          Thread.sleep(1000);
         } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
+          // TODO Auto-generated catch block
+          e.printStackTrace();
         }
       }
-      else if (testStep == TestStep.CHECKANDPUT_STARTED) {
+      if (count == 3) {
         super.releaseRowLock(lockId);
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          // TODO Auto-generated catch block
+          e.printStackTrace();
+        }
+        latch.countDown();
       }
     }
 
     @Override
     public Integer getLock(Integer lockid, byte[] row, boolean waitForLock) throws IOException {
-      if (testStep == TestStep.CHECKANDPUT_STARTED) {
+      if (count == 3) {
         latch.countDown();
       }
       return super.getLock(lockid, row, waitForLock);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Sun Dec 23 20:55:53 2012
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.regionse
 
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -52,15 +51,17 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.MultithreadedTestUtil;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
+import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
@@ -70,6 +71,8 @@ import org.apache.hadoop.hbase.filter.Fi
 import org.apache.hadoop.hbase.filter.NullComparator;
 import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -102,7 +105,6 @@ import com.google.common.collect.Lists;
  * HRegions or in the HBaseMaster, so only basic testing is possible.
  */
 @Category(MediumTests.class)
-@SuppressWarnings("deprecation")
 public class TestHRegion extends HBaseTestCase {
   // Do not spin up clusters in here.  If you need to spin up a cluster, do it
   // over in TestHRegionOnCluster.
@@ -155,6 +157,7 @@ public class TestHRegion extends HBaseTe
     String method = "testCompactionAffectedByScanners";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
+    Configuration conf = HBaseConfiguration.create();
     this.region = initHRegion(tableName, method, conf, family);
 
     Put put = new Put(Bytes.toBytes("r1"));
@@ -206,6 +209,7 @@ public class TestHRegion extends HBaseTe
     String method = "testToShowNPEOnRegionScannerReseek";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
+    Configuration conf = HBaseConfiguration.create();
     this.region = initHRegion(tableName, method, conf, family);
 
     Put put = new Put(Bytes.toBytes("r1"));
@@ -238,6 +242,7 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplay";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
+    Configuration conf = HBaseConfiguration.create();
     this.region = initHRegion(tableName, method, conf, family);
     try {
       Path regiondir = region.getRegionDir();
@@ -283,7 +288,7 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplaySomeIgnored";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
-    this.region = initHRegion(tableName, method, conf, family);
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
     try {
       Path regiondir = region.getRegionDir();
       FileSystem fs = region.getFilesystem();
@@ -333,7 +338,7 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplayAllIgnored";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
-    this.region = initHRegion(tableName, method, conf, family);
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
     try {
       Path regiondir = region.getRegionDir();
       FileSystem fs = region.getFilesystem();
@@ -459,7 +464,7 @@ public class TestHRegion extends HBaseTe
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
-    this.region = initHRegion(TABLE, getName(), conf, FAMILIES);
+    this.region = initHRegion(TABLE, getName(), FAMILIES);
     try {
       String value = "this is the value";
       String value2 = "this is some other value";
@@ -580,7 +585,7 @@ public class TestHRegion extends HBaseTe
   public void testFamilyWithAndWithoutColon() throws Exception {
     byte [] b = Bytes.toBytes(getName());
     byte [] cf = Bytes.toBytes(COLUMN_FAMILY);
-    this.region = initHRegion(b, getName(), conf, cf);
+    this.region = initHRegion(b, getName(), cf);
     try {
       Put p = new Put(b);
       byte [] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
@@ -604,7 +609,7 @@ public class TestHRegion extends HBaseTe
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
-    this.region = initHRegion(b, getName(), conf, cf);
+    this.region = initHRegion(b, getName(), cf);
     try {
       HLog.getSyncTime(); // clear counter from prior tests
       assertEquals(0, HLog.getSyncTime().count);
@@ -638,7 +643,7 @@ public class TestHRegion extends HBaseTe
       Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
 
       MultithreadedTestUtil.TestContext ctx =
-        new MultithreadedTestUtil.TestContext(conf);
+        new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
       final AtomicReference<OperationStatus[]> retFromThread =
         new AtomicReference<OperationStatus[]>();
       TestThread putter = new TestThread(ctx) {
@@ -705,7 +710,9 @@ public class TestHRegion extends HBaseTe
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
-    Configuration conf = HBaseConfiguration.create(this.conf);
+
+    HBaseConfiguration conf = new HBaseConfiguration();
+
 
     // add data with a timestamp that is too recent for range. Ensure assert
     conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
@@ -752,7 +759,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting empty data in key
       Put put = new Put(row1);
@@ -827,7 +834,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in key
       Put put = new Put(row1);
@@ -861,7 +868,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in key
       Put put = new Put(row1);
@@ -899,7 +906,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in the key to check
       Put put = new Put(row1);
@@ -938,7 +945,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testCheckAndPut_wrongRowInPut() throws IOException {
-    this.region = initHRegion(tableName, this.getName(), conf, COLUMNS);
+    this.region = initHRegion(tableName, this.getName(), COLUMNS);
     try {
       Put put = new Put(row2);
       put.add(fam1, qual1, value1);
@@ -973,7 +980,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Put content
       Put put = new Put(row1);
@@ -1048,7 +1055,7 @@ public class TestHRegion extends HBaseTe
     put.add(fam1, qual, 2, value);
 
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       region.put(put);
 
@@ -1078,7 +1085,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1, fam2, fam3);
+    this.region = initHRegion(tableName, method, fam1, fam2, fam3);
     try {
       List<KeyValue> kvs  = new ArrayList<KeyValue>();
       kvs.add(new KeyValue(row1, fam4, null, null));
@@ -1116,7 +1123,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
 
@@ -1184,7 +1191,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       byte [] row = Bytes.toBytes("table_name");
       // column names
@@ -1227,7 +1234,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       byte [] row = Bytes.toBytes("row1");
       // column names
@@ -1280,7 +1287,7 @@ public class TestHRegion extends HBaseTe
     byte[] fam = Bytes.toBytes("info");
     byte[][] families = { fam };
     String method = this.getName();
-    Configuration conf = HBaseConfiguration.create(this.conf);
+    HBaseConfiguration conf = new HBaseConfiguration();
 
     // add data with a timestamp that is too recent for range. Ensure assert
     conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
@@ -1311,7 +1318,7 @@ public class TestHRegion extends HBaseTe
     byte [] tableName = Bytes.toBytes("test_table");
     byte [] fam1 = Bytes.toBytes("columnA");
     byte [] fam2 = Bytes.toBytes("columnB");
-    this.region = initHRegion(tableName, getName(), conf, fam1, fam2);
+    this.region = initHRegion(tableName, getName(), fam1, fam2);
     try {
       byte [] rowA = Bytes.toBytes("rowA");
       byte [] rowB = Bytes.toBytes("rowB");
@@ -1364,7 +1371,7 @@ public class TestHRegion extends HBaseTe
 
   public void doTestDelete_AndPostInsert(Delete delete)
       throws IOException, InterruptedException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
       Put put = new Put(row);
@@ -1417,7 +1424,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Building checkerList
       List<KeyValue> kvs  = new ArrayList<KeyValue>();
@@ -1457,7 +1464,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       Get get = new Get(row1);
       get.addColumn(fam2, col1);
@@ -1488,7 +1495,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Add to memstore
       Put put = new Put(row1);
@@ -1538,7 +1545,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("fam");
 
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam);
+    this.region = initHRegion(tableName, method, fam);
     try {
       Get get = new Get(row);
       get.addFamily(fam);
@@ -1558,8 +1565,7 @@ public class TestHRegion extends HBaseTe
   public void stestGet_Root() throws IOException {
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(HConstants.ROOT_TABLE_NAME,
-      method, conf, HConstants.CATALOG_FAMILY);
+    this.region = initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
     try {
       //Add to memstore
       Put put = new Put(HConstants.EMPTY_START_ROW);
@@ -1791,7 +1797,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       Scan scan = new Scan();
       scan.addFamily(fam1);
@@ -1816,7 +1822,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       Scan scan = new Scan();
       scan.addFamily(fam2);
@@ -1845,7 +1851,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
 
       //Putting data in Region
@@ -1893,7 +1899,7 @@ public class TestHRegion extends HBaseTe
     //Setting up region
     String method = this.getName();
     try {
-      this.region = initHRegion(tableName, method, conf, families);
+      this.region = initHRegion(tableName, method, families);
     } catch (IOException e) {
       e.printStackTrace();
       fail("Got IOException during initHRegion, " + e.getMessage());
@@ -1929,7 +1935,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -1996,7 +2002,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -2056,7 +2062,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -2121,7 +2127,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
@@ -2203,7 +2209,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -2264,7 +2270,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in Region
       Put put = null;
@@ -2315,7 +2321,7 @@ public class TestHRegion extends HBaseTe
   public void testScanner_StopRow1542() throws IOException {
     byte [] tableName = Bytes.toBytes("test_table");
     byte [] family = Bytes.toBytes("testFamily");
-    this.region = initHRegion(tableName, getName(), conf, family);
+    this.region = initHRegion(tableName, getName(), family);
     try {
       byte [] row1 = Bytes.toBytes("row111");
       byte [] row2 = Bytes.toBytes("row222");
@@ -2362,7 +2368,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2390,7 +2396,7 @@ public class TestHRegion extends HBaseTe
   public void testIncrementColumnValue_BumpSnapshot() throws IOException {
     ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
     EnvironmentEdgeManagerTestHelper.injectEdge(mee);
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 42L;
       long incr = 44L;
@@ -2429,7 +2435,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_ConcurrentFlush() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2463,7 +2469,7 @@ public class TestHRegion extends HBaseTe
   public void testIncrementColumnValue_heapSize() throws IOException {
     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
 
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long byAmount = 1L;
       long size;
@@ -2482,7 +2488,7 @@ public class TestHRegion extends HBaseTe
 
   public void testIncrementColumnValue_UpdatingInPlace_Negative()
     throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 3L;
       long amount = -1L;
@@ -2503,7 +2509,7 @@ public class TestHRegion extends HBaseTe
 
   public void testIncrementColumnValue_AddingNew()
     throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2532,7 +2538,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2560,7 +2566,7 @@ public class TestHRegion extends HBaseTe
 
   public void testIncrementColumnValue_AddingNewAfterSFCheck()
     throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2599,7 +2605,7 @@ public class TestHRegion extends HBaseTe
    * @throws IOException
    */
   public void testIncrementColumnValue_UpdatingInPlace_TimestampClobber() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2647,7 +2653,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_WrongInitialSize() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
       int row1Field1 = 0;
@@ -2675,7 +2681,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrement_WrongInitialSize() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
       long row1Field1 = 0;
@@ -2751,7 +2757,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in Region
       KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
@@ -2966,7 +2972,7 @@ public class TestHRegion extends HBaseTe
     int compactInterval = 10 * flushAndScanInterval;
 
     String method = "testFlushCacheWhileScanning";
-    this.region = initHRegion(tableName,method, conf, family);
+    this.region = initHRegion(tableName,method, family);
     try {
       FlushThread flushThread = new FlushThread();
       flushThread.start();
@@ -3097,7 +3103,7 @@ public class TestHRegion extends HBaseTe
     }
 
     String method = "testWritesWhileScanning";
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       PutThread putThread = new PutThread(numRows, families, qualifiers);
       putThread.start();
@@ -3219,8 +3225,6 @@ public class TestHRegion extends HBaseTe
             }
             numPutsFinished++;
           }
-        } catch (InterruptedIOException e) {
-          // This is fine. It means we are done, or didn't get the lock on time
         } catch (IOException e) {
           LOG.error("error while putting records", e);
           error = e;
@@ -3257,9 +3261,8 @@ public class TestHRegion extends HBaseTe
       qualifiers[i] = Bytes.toBytes("qual" + i);
     }
 
-    Configuration conf = HBaseConfiguration.create(this.conf);
-
     String method = "testWritesWhileGetting";
+    Configuration conf = HBaseConfiguration.create();
     // This test flushes constantly and can cause many files to be created, possibly
     // extending over the ulimit.  Make sure compactions are aggressive in reducing
     // the number of HFiles created.
@@ -3268,7 +3271,7 @@ public class TestHRegion extends HBaseTe
     this.region = initHRegion(tableName, method, conf, families);
     PutThread putThread = null;
     MultithreadedTestUtil.TestContext ctx =
-      new MultithreadedTestUtil.TestContext(conf);
+      new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
     try {
       putThread = new PutThread(numRows, families, qualifiers);
       putThread.start();
@@ -3354,7 +3357,7 @@ public class TestHRegion extends HBaseTe
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method,
-        conf, family);
+        HBaseConfiguration.create(), family);
     try {
       byte[] rowNotServed = Bytes.toBytes("a");
       Get g = new Get(rowNotServed);
@@ -3418,7 +3421,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = "testIndexesScanWithOneDeletedRow";
-    this.region = initHRegion(tableName, method, conf, family);
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
     try {
       Put put = new Put(Bytes.toBytes(1L));
       put.add(family, qual1, 1L, Bytes.toBytes(1L));
@@ -3871,6 +3874,7 @@ public class TestHRegion extends HBaseTe
    */
   @Test
   public void testParallelIncrementWithMemStoreFlush() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
     String method = "testParallelIncrementWithMemStoreFlush";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Incrementer.family;
@@ -4003,8 +4007,7 @@ public class TestHRegion extends HBaseTe
   }
 
   private Configuration initSplit() {
-    Configuration conf = HBaseConfiguration.create(this.conf);
-
+    Configuration conf = HBaseConfiguration.create();
     // Always compact if there is more than one store file.
     conf.setInt("hbase.hstore.compactionThreshold", 2);
 
@@ -4025,6 +4028,19 @@ public class TestHRegion extends HBaseTe
   /**
    * @param tableName
    * @param callingMethod
+   * @param families
+   * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
+   * @throws IOException
+   */
+  private static HRegion initHRegion (byte [] tableName, String callingMethod,
+    byte[] ... families)
+  throws IOException {
+    return initHRegion(tableName, callingMethod, HBaseConfiguration.create(), families);
+  }
+
+  /**
+   * @param tableName
+   * @param callingMethod
    * @param conf
    * @param families
    * @throws IOException

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java Sun Dec 23 20:55:53 2012
@@ -113,8 +113,7 @@ public class TestQueryMatcher extends HB
     memstore.add(new KeyValue(row2, fam1, col1, data));
 
     List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
-    KeyValue k = memstore.get(0);
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(memstore.get(0).getRow());
 
     for (KeyValue kv : memstore){
       actual.add(qm.match(kv));
@@ -159,8 +158,7 @@ public class TestQueryMatcher extends HB
 
     List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
 
-    KeyValue k = memstore.get(0);
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(memstore.get(0).getRow());
 
     for(KeyValue kv : memstore) {
       actual.add(qm.match(kv));
@@ -212,8 +210,7 @@ public class TestQueryMatcher extends HB
         new KeyValue(row2, fam1, col1, now-10, data)
     };
 
-    KeyValue k = kvs[0];
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(kvs[0].getRow());
 
     List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
     for (KeyValue kv : kvs) {
@@ -265,8 +262,7 @@ public class TestQueryMatcher extends HB
         new KeyValue(row1, fam2, col5, now-10000, data),
         new KeyValue(row2, fam1, col1, now-10, data)
     };
-    KeyValue k = kvs[0];
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(kvs[0].getRow());
 
     List<ScanQueryMatcher.MatchCode> actual =
         new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java Sun Dec 23 20:55:53 2012
@@ -36,7 +36,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -728,44 +727,6 @@ public class TestSplitTransactionOnClust
     }
   }
 
-  @Test(timeout = 20000)
-  public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
-    final byte[] tableName = 
-        Bytes.toBytes("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
-    HRegionServer regionServer = null;
-    List<HRegion> regions = null;
-    HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
-    try {
-      // Create table then get the single region for our new table.
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      htd.addFamily(new HColumnDescriptor("cf"));
-      admin.createTable(htd);
-      HTable t = new HTable(cluster.getConfiguration(), tableName);
-      regions = cluster.getRegions(tableName);
-      int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
-      regionServer = cluster.getRegionServer(regionServerIndex);
-      insertData(tableName, admin, t);
-      // Turn off balancer so it doesn't cut in and mess up our placements.
-      cluster.getMaster().setCatalogJanitorEnabled(false);
-      boolean tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
-          Bytes.toString(tableName));
-      assertEquals("The specified table should present.", true, tableExists);
-      SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("row2"));
-      try {
-        st.prepare();
-        st.createDaughters(regionServer, regionServer);
-      } catch (IOException e) {
-
-      }
-      tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
-          Bytes.toString(tableName));
-      assertEquals("The specified table should present.", true, tableExists);
-    } finally {
-      cluster.getMaster().setCatalogJanitorEnabled(true);
-      admin.close();
-    }
-  }
-  
   private void insertData(final byte[] tableName, HBaseAdmin admin, HTable t) throws IOException,
       InterruptedException {
     Put p = new Put(Bytes.toBytes("row1"));

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java Sun Dec 23 20:55:53 2012
@@ -49,9 +49,6 @@ public class FaultySequenceFileLogReader
         HLogKey key = HLog.newKey(conf);
         WALEdit val = new WALEdit();
         HLog.Entry e = new HLog.Entry(key, val);
-        if (compressionContext != null) {
-          e.setCompressionContext(compressionContext);
-        }
         b = this.reader.next(e.getKey(), e.getEdit());
         nextQueue.offer(e);
         numberOfFileEntries++;

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Sun Dec 23 20:55:53 2012
@@ -475,13 +475,15 @@ public class TestHLog  {
       throw t.exception;
 
     // Make sure you can read all the content
-    HLog.Reader reader = HLog.getReader(this.fs, walPath, this.conf);
+    SequenceFile.Reader reader
+      = new SequenceFile.Reader(this.fs, walPath, this.conf);
     int count = 0;
-    HLog.Entry entry = new HLog.Entry();
-    while (reader.next(entry) != null) {
+    HLogKey key = HLog.newKey(conf);
+    WALEdit val = new WALEdit();
+    while (reader.next(key, val)) {
       count++;
       assertTrue("Should be one KeyValue per WALEdit",
-                 entry.getEdit().getKeyValues().size() == 1);
+                 val.getKeyValues().size() == 1);
     }
     assertEquals(total, count);
     reader.close();

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java Sun Dec 23 20:55:53 2012
@@ -86,7 +86,7 @@ public class TestHLogSplit {
   private Configuration conf;
   private FileSystem fs;
 
-  protected final static HBaseTestingUtility
+  private final static HBaseTestingUtility
           TEST_UTIL = new HBaseTestingUtility();
 
 

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java Sun Dec 23 20:55:53 2012
@@ -61,7 +61,7 @@ public class TestReplication {
 
   private static final Log LOG = LogFactory.getLog(TestReplication.class);
 
-  protected static Configuration conf1 = HBaseConfiguration.create();
+  private static Configuration conf1;
   private static Configuration conf2;
   private static Configuration CONF_WITH_LOCALFS;
 
@@ -78,8 +78,8 @@ public class TestReplication {
   private static final int NB_ROWS_IN_BATCH = 100;
   private static final int NB_ROWS_IN_BIG_BATCH =
       NB_ROWS_IN_BATCH * 10;
-  private static final long SLEEP_TIME = 1500;
-  private static final int NB_RETRIES = 15;
+  private static final long SLEEP_TIME = 500;
+  private static final int NB_RETRIES = 10;
 
   private static final byte[] tableName = Bytes.toBytes("test");
   private static final byte[] famName = Bytes.toBytes("f");
@@ -91,6 +91,7 @@ public class TestReplication {
    */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    conf1 = HBaseConfiguration.create();
     conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
     // smaller block size and capacity to trigger more operations
     // and test them
@@ -519,7 +520,7 @@ public class TestReplication {
 
     // disable and start the peer
     admin.disablePeer("2");
-    utility2.startMiniHBaseCluster(1, 2);
+    utility2.startMiniHBaseCluster(1, 1);
     Get get = new Get(rowkey);
     for (int i = 0; i < NB_RETRIES; i++) {
       Result res = htable2.get(get);
@@ -716,7 +717,7 @@ public class TestReplication {
    */
   @Test(timeout=300000)
   public void queueFailover() throws Exception {
-    utility1.createMultiRegions(htable1, famName, false);
+    utility1.createMultiRegions(htable1, famName);
 
     // killing the RS with .META. can result into failed puts until we solve
     // IO fencing
@@ -759,8 +760,7 @@ public class TestReplication {
     int lastCount = 0;
 
     final long start = System.currentTimeMillis();
-    int i = 0;
-    while (true) {
+    for (int i = 0; i < NB_RETRIES; i++) {
       if (i==NB_RETRIES-1) {
         fail("Waited too much time for queueFailover replication. " +
           "Waited "+(System.currentTimeMillis() - start)+"ms.");
@@ -772,8 +772,6 @@ public class TestReplication {
       if (res2.length < initialCount) {
         if (lastCount < res2.length) {
           i--; // Don't increment timeout if we make progress
-        } else {
-          i++;
         }
         lastCount = res2.length;
         LOG.info("Only got " + lastCount + " rows instead of " +
@@ -793,7 +791,7 @@ public class TestReplication {
           Thread.sleep(timeout);
           utility.expireRegionServerSession(rs);
         } catch (Exception e) {
-          LOG.error("Couldn't kill a region server", e);
+          LOG.error(e);
         }
       }
     };

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java Sun Dec 23 20:55:53 2012
@@ -114,6 +114,7 @@ public class PerformanceEvaluation  {
 
   protected Map<String, CmdDescriptor> commands = new TreeMap<String, CmdDescriptor>();
   protected static Cluster cluster = new Cluster();
+  protected static String accessToken = null;
 
   volatile Configuration conf;
   private boolean nomapred = false;
@@ -448,7 +449,8 @@ public class PerformanceEvaluation  {
    */
   private boolean checkTable() throws IOException {
     HTableDescriptor tableDescriptor = getTableDescriptor();
-    RemoteAdmin admin = new RemoteAdmin(new Client(cluster), conf);
+    RemoteAdmin admin =
+      new RemoteAdmin(new Client(cluster), conf, accessToken);
     if (!admin.isTableAvailable(tableDescriptor.getName())) {
       admin.createTable(tableDescriptor);
       return true;
@@ -712,7 +714,8 @@ public class PerformanceEvaluation  {
     }
     
     void testSetup() throws IOException {
-      this.table = new RemoteHTable(new Client(cluster), conf, tableName);
+      this.table = new RemoteHTable(new Client(cluster), conf, tableName,
+        accessToken);
     }
 
     void testTakedown()  throws IOException {
@@ -1130,6 +1133,7 @@ public class PerformanceEvaluation  {
     System.err.println();
     System.err.println("Options:");
     System.err.println(" host          String. Specify Stargate endpoint.");
+    System.err.println(" token         String. API access token.");
     System.err.println(" rows          Integer. Rows each client runs. Default: One million");
     System.err.println(" rowsPerPut    Integer. Rows each Stargate (multi)Put. Default: 100");
     System.err.println(" nomapred      (Flag) Run multiple clients using threads " +
@@ -1204,6 +1208,12 @@ public class PerformanceEvaluation  {
           continue;
         }
 
+        final String token = "--token=";
+        if (cmd.startsWith(token)) {
+          accessToken = cmd.substring(token.length());
+          continue;
+        }
+
         Class<? extends Test> cmdClass = determineCommandClass(cmd);
         if (cmdClass != null) {
           getArgs(i + 1, args);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java?rev=1425526&r1=1425525&r2=1425526&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java Sun Dec 23 20:55:53 2012
@@ -101,7 +101,7 @@ public class TestRemoteTable {
     remoteTable = new RemoteHTable(
       new Client(new Cluster().add("localhost", 
           REST_TEST_UTIL.getServletPort())),
-        TEST_UTIL.getConfiguration(), TABLE);
+        TEST_UTIL.getConfiguration(), TABLE, null);
   }
 
   @AfterClass
@@ -222,45 +222,6 @@ public class TestRemoteTable {
   }
 
   @Test
-  public void testMultiGet() throws Exception {
-    ArrayList<Get> gets = new ArrayList<Get>();
-    gets.add(new Get(ROW_1));
-    gets.add(new Get(ROW_2));
-    Result[] results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(2, results.length);
-    assertEquals(1, results[0].size());
-    assertEquals(2, results[1].size());
-
-    //Test Versions
-    gets = new ArrayList<Get>();
-    Get g = new Get(ROW_1);
-    g.setMaxVersions(3);
-    gets.add(g);
-    gets.add(new Get(ROW_2));
-    results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(2, results.length);
-    assertEquals(1, results[0].size());
-    assertEquals(3, results[1].size());
-
-    //404
-    gets = new ArrayList<Get>();
-    gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
-    results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(0, results.length);
-
-    gets = new ArrayList<Get>();
-    gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
-    gets.add(new Get(ROW_1));
-    gets.add(new Get(ROW_2));
-    results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(0, results.length);
-  }
-
-  @Test
   public void testPut() throws IOException {
     Put put = new Put(ROW_3);
     put.add(COLUMN_1, QUALIFIER_1, VALUE_1);