Repository: hbase
Updated Branches:
  refs/heads/HBASE-14614 9464f461b -> a23fcc97d


http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
index 9f23848..eca963d 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.security.User;
 
 /**
  * Base class for all the Table procedures that want to use a 
StateMachineProcedure.
- * It provide some basic helpers like basic locking, sync latch, and basic 
toStringClassDetails().
+ * It provides helpers like basic locking, sync latch, and 
toStringClassDetails().
  */
 @InterfaceAudience.Private
 public abstract class AbstractStateMachineTableProcedure<TState>
@@ -52,9 +52,10 @@ public abstract class 
AbstractStateMachineTableProcedure<TState>
 
   protected AbstractStateMachineTableProcedure(final MasterProcedureEnv env,
       final ProcedurePrepareLatch latch) {
-    this.user = env.getRequestUser();
-    this.setOwner(user);
-
+    if (env != null) {
+      this.user = env.getRequestUser();
+      this.setOwner(user);
+    }
     // used for compatibility with clients without procedures
     // they need a sync TableExistsException, TableNotFoundException, 
TableNotDisabledException, ...
     this.syncLatch = latch;
@@ -110,4 +111,4 @@ public abstract class 
AbstractStateMachineTableProcedure<TState>
       throw new TableNotFoundException(getTableName());
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
index 1478fc7..15ed429 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
@@ -275,7 +275,7 @@ public class DispatchMergingRegionsProcedure
 
   @Override
   public TableOperationType getTableOperationType() {
-    return TableOperationType.MERGE;
+    return TableOperationType.REGION_MERGE;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index bcb0004..61e984c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -572,11 +572,13 @@ public class MasterProcedureScheduler extends 
AbstractProcedureScheduler {
         return false;
       // region operations are using the shared-lock on the table
       // and then they will grab an xlock on the region.
-      case SPLIT:
-      case MERGE:
-      case ASSIGN:
-      case UNASSIGN:
+      case REGION_SPLIT:
+      case REGION_MERGE:
+      case REGION_ASSIGN:
+      case REGION_UNASSIGN:
       case REGION_EDIT:
+      case REGION_GC:
+      case MERGED_REGIONS_GC:
         return false;
       default:
         break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 9e00579..3bd2c9e 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -413,14 +413,8 @@ implements ServerProcedureInterface {
       final HRegionInfo hri = it.next();
       RegionTransitionProcedure rtp = 
am.getRegionStates().getRegionTransitionProcedure(hri);
       if (rtp == null) continue;
-      ServerName rtpServerName = rtp.getServer();
-      if (rtpServerName == null) {
-        LOG.warn("RIT with ServerName null! " + rtp);
-        continue;
-      }
-      if (!rtpServerName.equals(this.serverName)) continue;
       LOG.info("pid=" + getProcId() + " found RIT " + rtp + "; " +
-          rtp.getRegionState(env).toShortString());
+      rtp.getRegionState(env).toShortString());
       // Notify RIT on server crash.
       if (sce == null) {
         sce = new ServerCrashException(getProcId());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
index f74df79..86143ac 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
@@ -31,7 +31,8 @@ import 
org.apache.hadoop.hbase.classification.InterfaceStability;
 public interface TableProcedureInterface {
   public enum TableOperationType {
     CREATE, DELETE, DISABLE, EDIT, ENABLE, READ,
-    REGION_EDIT, SPLIT, MERGE, ASSIGN, UNASSIGN, /* region operations */
+    REGION_EDIT, REGION_SPLIT, REGION_MERGE, REGION_ASSIGN, REGION_UNASSIGN,
+      REGION_GC, MERGED_REGIONS_GC/* region operations */
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 43d3ce5..285b529 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1380,14 +1380,12 @@ public class HRegion implements HeapSize, 
PropagatingConfigurationObserver, Regi
     return !isClosed() && !isClosing();
   }
 
-  /** @return true if region is splittable */
+  @Override
   public boolean isSplittable() {
     return isAvailable() && !hasReferences();
   }
 
-  /**
-   * @return true if region is mergeable
-   */
+  @Override
   public boolean isMergeable() {
     if (!isAvailable()) {
       LOG.debug("Region " + this

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 91b463d..59a0fe5 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -318,7 +318,8 @@ public class HRegionFileSystem {
    * @throws IOException
    */
   public boolean hasReferences(final String familyName) throws IOException {
-    FileStatus[] files = FSUtils.listStatus(fs, getStoreDir(familyName));
+    Path storeDir = getStoreDir(familyName);
+    FileStatus[] files = FSUtils.listStatus(fs, storeDir);
     if (files != null) {
       for(FileStatus stat: files) {
         if(stat.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 0c0e500..edad12a 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1544,6 +1544,8 @@ public class RSRpcServices implements 
HBaseRPCErrorHandler,
       if (request.hasCompactionState() && request.getCompactionState()) {
         builder.setCompactionState(region.getCompactionState());
       }
+      builder.setSplittable(region.isSplittable());
+      builder.setMergeable(region.isMergeable());
       builder.setIsRecovering(region.isRecovering());
       return builder.build();
     } catch (IOException ie) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 295b825..6c4eca9 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -96,6 +96,14 @@ public interface Region extends ConfigurationObserver {
   /** @return True if region is read only */
   boolean isReadOnly();
 
+  /** @return true if region is splittable */
+  boolean isSplittable();
+
+  /**
+   * @return true if region is mergeable
+   */
+  boolean isMergeable();
+
   /**
    * Return the list of Stores managed by this region
    * <p>Use with caution.  Exposed for use of fixup utilities.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 8a46052..b527195 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1744,7 +1744,7 @@ public class HBaseTestingUtility extends 
HBaseCommonTestingUtility {
    */
   public void deleteTable(TableName tableName) throws IOException {
     try {
-      getAdmin().disableTableAsync(tableName);
+      getAdmin().disableTable(tableName);
     } catch (TableNotEnabledException e) {
       LOG.debug("Table: " + tableName + " already disabled, so just deleting 
it.");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index a4fab7a..074a421 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
@@ -39,11 +38,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.NoSuchProcedureException;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.NoSuchProcedureException;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index d542fbe..5c082a4 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -27,8 +27,11 @@ import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.SortedMap;
+import java.util.SortedSet;
 import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,268 +39,140 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaMockingUtil;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
-import 
org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.Reference;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import 
org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
-import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.master.assignment.MockMasterServices;
 import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Triple;
+import org.junit.After;
+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;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, SmallTests.class})
 public class TestCatalogJanitor {
   private static final Log LOG = LogFactory.getLog(TestCatalogJanitor.class);
-
-  @Rule
-  public TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+     withTimeout(this.getClass()).withLookingForStuckThread(true).build();
+  @Rule public final TestName name = new TestName();
+  private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
+  private MockMasterServices masterServices;
+  private CatalogJanitor janitor;
 
   @BeforeClass
-  public static void setup() throws Exception {
+  public static void beforeClass() throws Exception {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
   }
-  /**
-   * Mock MasterServices for tests below.
-   */
-  class MockMasterServices extends MockNoopMasterServices {
-    private final ClusterConnection connection;
-    private final MasterFileSystem mfs;
-    private final AssignmentManager asm;
-    private final ServerManager sm;
-
-    MockMasterServices(final HBaseTestingUtility htu) throws IOException {
-      super(htu.getConfiguration());
-
-      ClientProtos.ClientService.BlockingInterface ri =
-        Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
-      MutateResponse.Builder builder = MutateResponse.newBuilder();
-      builder.setProcessed(true);
-      try {
-        Mockito.when(ri.mutate(
-          (RpcController)Mockito.any(), (MutateRequest)Mockito.any())).
-            thenReturn(builder.build());
-      } catch (ServiceException se) {
-        throw ProtobufUtil.handleRemoteException(se);
-      }
-      try {
-        Mockito.when(ri.multi(
-          (RpcController)Mockito.any(), (MultiRequest)Mockito.any())).
-            thenAnswer(new Answer<MultiResponse>() {
-              @Override
-              public MultiResponse answer(InvocationOnMock invocation) throws 
Throwable {
-                return buildMultiResponse( 
(MultiRequest)invocation.getArguments()[1]);
-              }
-            });
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      }
-      // Mock an ClusterConnection and a AdminProtocol implementation.  Have 
the
-      // ClusterConnection return the HRI.  Have the HRI return a few mocked 
up responses
-      // to make our test work.
-      this.connection =
-        
HConnectionTestingUtility.getMockedConnectionAndDecorate(getConfiguration(),
-          Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri,
-            ServerName.valueOf("example.org,12345,6789"),
-          HRegionInfo.FIRST_META_REGIONINFO);
-      // Set hbase.rootdir into test dir.
-      FileSystem.get(getConfiguration());
-      Path rootdir = FSUtils.getRootDir(getConfiguration());
-      FSUtils.setRootDir(getConfiguration(), rootdir);
-      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
-
-      this.mfs = new MasterFileSystem(this);
-      this.asm = Mockito.mock(AssignmentManager.class);
-      this.sm = Mockito.mock(ServerManager.class);
-    }
-
-    @Override
-    public AssignmentManager getAssignmentManager() {
-      return this.asm;
-    }
 
-    @Override
-    public MasterFileSystem getMasterFileSystem() {
-      return this.mfs;
-    }
-
-    @Override
-    public ClusterConnection getConnection() {
-      return this.connection;
-    }
-
-    @Override
-    public ServerName getServerName() {
-      return ServerName.valueOf("mockserver.example.org", 1234, -1L);
-    }
-
-    @Override
-    public ServerManager getServerManager() {
-      return this.sm;
-    }
-
-    @Override
-    public CoordinatedStateManager getCoordinatedStateManager() {
-      BaseCoordinatedStateManager m = 
Mockito.mock(BaseCoordinatedStateManager.class);
-      SplitLogManagerCoordination c = 
Mockito.mock(SplitLogManagerCoordination.class);
-      Mockito.when(m.getSplitLogManagerCoordination()).thenReturn(c);
-      SplitLogManagerDetails d = Mockito.mock(SplitLogManagerDetails.class);
-      Mockito.when(c.getDetails()).thenReturn(d);
-      return m;
-    }
+  @Before
+  public void setup() throws IOException {
+    setRootDirAndCleanIt(HTU, this.name.getMethodName());
+    NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers =
+        new ConcurrentSkipListMap<ServerName, SortedSet<byte []>>();
+    this.masterServices =
+        new MockMasterServices(HTU.getConfiguration(), regionsToRegionServers);
+    this.masterServices.start(10, null);
+    this.janitor = new CatalogJanitor(masterServices);
+  }
 
-    @Override
-    public TableDescriptors getTableDescriptors() {
-      return new TableDescriptors() {
-        @Override
-        public HTableDescriptor remove(TableName tablename) throws IOException 
{
-          // noop
-          return null;
-        }
-
-        @Override
-        public Map<String, HTableDescriptor> getAll() throws IOException {
-          // noop
-          return null;
-        }
-
-        @Override public Map<String, HTableDescriptor> getAllDescriptors() 
throws IOException {
-          // noop
-          return null;
-        }
-
-        @Override
-        public HTableDescriptor get(TableName tablename)
-            throws IOException {
-          return createHTableDescriptor();
-        }
-
-        @Override
-        public Map<String, HTableDescriptor> getByNamespace(String name) 
throws IOException {
-          return null;
-        }
-
-        @Override
-        public void add(HTableDescriptor htd) throws IOException {
-          // noop
-        }
-
-        @Override
-        public void setCacheOn() throws IOException {
-        }
-
-        @Override
-        public void setCacheOff() throws IOException {
-        }
-      };
-    }
+  @After
+  public void teardown() {
+    this.janitor.cancel(true);
+    this.masterServices.stop("DONE");
   }
 
+  /**
+   * Test clearing a split parent.
+   */
   @Test
   public void testCleanParent() throws IOException, InterruptedException {
-    HBaseTestingUtility htu = new HBaseTestingUtility();
-    setRootDirAndCleanIt(htu, "testCleanParent");
-    MasterServices services = new MockMasterServices(htu);
-    try {
-      CatalogJanitor janitor = new CatalogJanitor(services);
-      // Create regions.
-      HTableDescriptor htd = new 
HTableDescriptor(TableName.valueOf(name.getMethodName()));
-      htd.addFamily(new HColumnDescriptor("f"));
-      HRegionInfo parent =
-        new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
-            Bytes.toBytes("eee"));
-      HRegionInfo splita =
-        new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
-            Bytes.toBytes("ccc"));
-      HRegionInfo splitb =
-        new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
-            Bytes.toBytes("eee"));
-      // Test that when both daughter regions are in place, that we do not
-      // remove the parent.
-      Result r = createResult(parent, splita, splitb);
-      // Add a reference under splitA directory so we don't clear out the 
parent.
-      Path rootdir = services.getMasterFileSystem().getRootDir();
-      Path tabledir =
-        FSUtils.getTableDir(rootdir, htd.getTableName());
-      Path storedir = HStore.getStoreHomedir(tabledir, splita,
-          htd.getColumnFamilies()[0].getName());
-      Reference ref = Reference.createTopReference(Bytes.toBytes("ccc"));
-      long now = System.currentTimeMillis();
-      // Reference name has this format: StoreFile#REF_NAME_PARSER
-      Path p = new Path(storedir, Long.toString(now) + "." + 
parent.getEncodedName());
-      FileSystem fs = services.getMasterFileSystem().getFileSystem();
-      Path path = ref.write(fs, p);
-      assertTrue(fs.exists(path));
-      assertFalse(janitor.cleanParent(parent, r));
-      // Remove the reference file and try again.
-      assertTrue(fs.delete(p, true));
-      assertTrue(janitor.cleanParent(parent, r));
-    } finally {
-      services.stop("shutdown");
+    TableDescriptor td = createTableDescriptorForCurrentMethod();
+    // Create regions.
+    HRegionInfo parent =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), 
Bytes.toBytes("eee"));
+    HRegionInfo splita =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), 
Bytes.toBytes("ccc"));
+    HRegionInfo splitb =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), 
Bytes.toBytes("eee"));
+    // Test that when both daughter regions are in place, that we do not 
remove the parent.
+    Result r = createResult(parent, splita, splitb);
+    // Add a reference under splitA directory so we don't clear out the parent.
+    Path rootdir = this.masterServices.getMasterFileSystem().getRootDir();
+    Path tabledir = FSUtils.getTableDir(rootdir, td.getTableName());
+    Path parentdir = new Path(tabledir, parent.getEncodedName());
+    Path storedir = HStore.getStoreHomedir(tabledir, splita, 
td.getColumnFamilies()[0].getName());
+    Reference ref = Reference.createTopReference(Bytes.toBytes("ccc"));
+    long now = System.currentTimeMillis();
+    // Reference name has this format: StoreFile#REF_NAME_PARSER
+    Path p = new Path(storedir, Long.toString(now) + "." + 
parent.getEncodedName());
+    FileSystem fs = this.masterServices.getMasterFileSystem().getFileSystem();
+    Path path = ref.write(fs, p);
+    assertTrue(fs.exists(path));
+    LOG.info("Created reference " + path);
+    // Add a parentdir for kicks so can check it gets removed by the 
catalogjanitor.
+    fs.mkdirs(parentdir);
+    assertFalse(this.janitor.cleanParent(parent, r));
+    // Remove the reference file and try again.
+    assertTrue(fs.delete(p, true));
+    assertTrue(this.janitor.cleanParent(parent, r));
+    // Parent cleanup is run async as a procedure. Make sure parentdir is 
removed.
+    LOG.info("Waiting on " + parentdir);
+    while (fs.exists(parentdir)) {
+      Threads.sleep(10);
     }
   }
 
   /**
    * Make sure parent gets cleaned up even if daughter is cleaned up before it.
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
   public void testParentCleanedEvenIfDaughterGoneFirst()
   throws IOException, InterruptedException {
-    parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
-      "testParentCleanedEvenIfDaughterGoneFirst", Bytes.toBytes("eee"));
+    
parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(this.name.getMethodName(),
+        Bytes.toBytes("eee"));
   }
 
   /**
    * Make sure last parent with empty end key gets cleaned up even if daughter 
is cleaned up before it.
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
   public void testLastParentCleanedEvenIfDaughterGoneFirst()
   throws IOException, InterruptedException {
-    parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
-      "testLastParentCleanedEvenIfDaughterGoneFirst", new byte[0]);
+    
parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(this.name.getMethodName(),
+        new byte[0]);
+  }
+
+  /**
+   * @return A TableDescriptor with a tableName of current method name and a 
column
+   * family that is MockMasterServices.DEFAULT_COLUMN_FAMILY_NAME)
+   */
+  private TableDescriptor createTableDescriptorForCurrentMethod() {
+    return 
TableDescriptorBuilder.newBuilder(TableName.valueOf(this.name.getMethodName())).
+        addFamily(new 
HColumnDescriptor(MockMasterServices.DEFAULT_COLUMN_FAMILY_NAME)).
+        build();
   }
 
   /**
@@ -305,50 +180,40 @@ public class TestCatalogJanitor {
    *
    * @param rootDir the test case name, used as the HBase testing utility root
    * @param lastEndKey the end key of the split parent
-   * @throws IOException
-   * @throws InterruptedException
    */
   private void parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
   final String rootDir, final byte[] lastEndKey)
   throws IOException, InterruptedException {
-    HBaseTestingUtility htu = new HBaseTestingUtility();
-    setRootDirAndCleanIt(htu, rootDir);
-    MasterServices services = new MockMasterServices(htu);
-    CatalogJanitor janitor = new CatalogJanitor(services);
-    final HTableDescriptor htd = createHTableDescriptor();
-
+    TableDescriptor td = createTableDescriptorForCurrentMethod();
     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
-
-    // Parent
-    HRegionInfo parent = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("aaa"),
-      lastEndKey);
+    HRegionInfo parent = new HRegionInfo(td.getTableName(), 
Bytes.toBytes("aaa"), lastEndKey);
     // Sleep a second else the encoded name on these regions comes out
     // same for all with same start key and made in same second.
     Thread.sleep(1001);
 
     // Daughter a
-    HRegionInfo splita = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("aaa"),
-      Bytes.toBytes("ccc"));
+    HRegionInfo splita =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), 
Bytes.toBytes("ccc"));
     Thread.sleep(1001);
     // Make daughters of daughter a; splitaa and splitab.
-    HRegionInfo splitaa = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("aaa"),
-      Bytes.toBytes("bbb"));
-    HRegionInfo splitab = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("bbb"),
-      Bytes.toBytes("ccc"));
+    HRegionInfo splitaa =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), 
Bytes.toBytes("bbb"));
+    HRegionInfo splitab =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("bbb"), 
Bytes.toBytes("ccc"));
 
     // Daughter b
-    HRegionInfo splitb = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("ccc"),
-      lastEndKey);
+    HRegionInfo splitb =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), lastEndKey);
     Thread.sleep(1001);
     // Make Daughters of daughterb; splitba and splitbb.
-    HRegionInfo splitba = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("ccc"),
-      Bytes.toBytes("ddd"));
-    HRegionInfo splitbb = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("ddd"),
-    lastEndKey);
+    HRegionInfo splitba =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), 
Bytes.toBytes("ddd"));
+    HRegionInfo splitbb =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("ddd"), lastEndKey);
 
     // First test that our Comparator works right up in CatalogJanitor.
-    // Just fo kicks.
-    SortedMap<HRegionInfo, Result> regions = new TreeMap<>(new 
CatalogJanitor.SplitParentFirstComparator());
+    SortedMap<HRegionInfo, Result> regions =
+        new TreeMap<>(new CatalogJanitor.SplitParentFirstComparator());
     // Now make sure that this regions map sorts as we expect it to.
     regions.put(parent, createResult(parent, splita, splitb));
     regions.put(splitb, createResult(splitb, splitba, splitbb));
@@ -366,10 +231,9 @@ public class TestCatalogJanitor {
       index++;
     }
 
-    // Now play around with the cleanParent function.  Create a ref from splita
-    // up to the parent.
+    // Now play around with the cleanParent function. Create a ref from splita 
up to the parent.
     Path splitaRef =
-      createReferences(services, htd, parent, splita, Bytes.toBytes("ccc"), 
false);
+        createReferences(this.masterServices, td, parent, splita, 
Bytes.toBytes("ccc"), false);
     // Make sure actual super parent sticks around because splita has a ref.
     assertFalse(janitor.cleanParent(parent, regions.get(parent)));
 
@@ -381,13 +245,13 @@ public class TestCatalogJanitor {
     // the daughter splita can be split (can't split if still references).
     // BUT make the timing such that the daughter gets cleaned up before we
     // can get a chance to let go of the parent.
-    FileSystem fs = FileSystem.get(htu.getConfiguration());
+    FileSystem fs = FileSystem.get(HTU.getConfiguration());
     assertTrue(fs.delete(splitaRef, true));
     // Create the refs from daughters of splita.
     Path splitaaRef =
-      createReferences(services, htd, splita, splitaa, Bytes.toBytes("bbb"), 
false);
+      createReferences(this.masterServices, td, splita, splitaa, 
Bytes.toBytes("bbb"), false);
     Path splitabRef =
-      createReferences(services, htd, splita, splitab, Bytes.toBytes("bbb"), 
true);
+      createReferences(this.masterServices, td, splita, splitab, 
Bytes.toBytes("bbb"), true);
 
     // Test splita.  It should stick around because references from splitab, 
etc.
     assertFalse(janitor.cleanParent(splita, regions.get(splita)));
@@ -399,74 +263,67 @@ public class TestCatalogJanitor {
 
     // Super parent should get cleaned up now both splita and splitb are gone.
     assertTrue(janitor.cleanParent(parent, regions.get(parent)));
-
-    services.stop("test finished");
-    janitor.cancel(true);
   }
 
   /**
    * CatalogJanitor.scan() should not clean parent regions if their own
-   * parents are still referencing them. This ensures that grandfather regions
+   * parents are still referencing them. This ensures that grandparent regions
    * do not point to deleted parent regions.
    */
   @Test
   public void testScanDoesNotCleanRegionsWithExistingParents() throws 
Exception {
-    HBaseTestingUtility htu = new HBaseTestingUtility();
-    setRootDirAndCleanIt(htu, 
"testScanDoesNotCleanRegionsWithExistingParents");
-    MasterServices services = new MockMasterServices(htu);
-
-    final HTableDescriptor htd = createHTableDescriptor();
-
+    TableDescriptor td = createTableDescriptorForCurrentMethod();
     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
 
     // Parent
-    HRegionInfo parent = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("aaa"),
-      new byte[0], true);
+    HRegionInfo parent = new HRegionInfo(td.getTableName(), 
Bytes.toBytes("aaa"),
+            HConstants.EMPTY_BYTE_ARRAY, true);
     // Sleep a second else the encoded name on these regions comes out
     // same for all with same start key and made in same second.
     Thread.sleep(1001);
 
     // Daughter a
-    HRegionInfo splita = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("aaa"),
-      Bytes.toBytes("ccc"), true);
+    HRegionInfo splita =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), 
Bytes.toBytes("ccc"), true);
     Thread.sleep(1001);
+
     // Make daughters of daughter a; splitaa and splitab.
-    HRegionInfo splitaa = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("aaa"),
-      Bytes.toBytes("bbb"), false);
-    HRegionInfo splitab = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("bbb"),
-      Bytes.toBytes("ccc"), false);
+    HRegionInfo splitaa =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), 
Bytes.toBytes("bbb"), false);
+    HRegionInfo splitab =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("bbb"), 
Bytes.toBytes("ccc"), false);
 
     // Daughter b
-    HRegionInfo splitb = new HRegionInfo(htd.getTableName(), 
Bytes.toBytes("ccc"),
-        new byte[0]);
+    HRegionInfo splitb =
+        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), 
HConstants.EMPTY_BYTE_ARRAY);
     Thread.sleep(1001);
 
+    // Parent has daughters splita and splitb. Splita has daughters splitaa 
and splitab.
     final Map<HRegionInfo, Result> splitParents = new TreeMap<>(new 
SplitParentFirstComparator());
     splitParents.put(parent, createResult(parent, splita, splitb));
     splita.setOffline(true); //simulate that splita goes offline when it is 
split
-    splitParents.put(splita, createResult(splita, splitaa,splitab));
+    splitParents.put(splita, createResult(splita, splitaa, splitab));
 
     final Map<HRegionInfo, Result> mergedRegions = new TreeMap<>();
-    CatalogJanitor janitor = spy(new CatalogJanitor(services));
-    doReturn(new Triple<>(10, mergedRegions, splitParents)).when(janitor)
-        .getMergedRegionsAndSplitParents();
+    CatalogJanitor spy = spy(this.janitor);
+    doReturn(new Triple<>(10, mergedRegions, splitParents)).when(spy).
+      getMergedRegionsAndSplitParents();
 
-    //create ref from splita to parent
+    // Create ref from splita to parent
+    LOG.info("parent=" + parent.getShortNameToLog() + ", splita=" + 
splita.getShortNameToLog());
     Path splitaRef =
-        createReferences(services, htd, parent, splita, Bytes.toBytes("ccc"), 
false);
+        createReferences(this.masterServices, td, parent, splita, 
Bytes.toBytes("ccc"), false);
+    LOG.info("Created reference " + splitaRef);
 
-    //parent and A should not be removed
-    assertEquals(0, janitor.scan());
+    // Parent and splita should not be removed because a reference from splita 
to parent.
+    assertEquals(0, spy.scan());
 
-    //now delete the ref
-    FileSystem fs = FileSystem.get(htu.getConfiguration());
+    // Now delete the ref
+    FileSystem fs = FileSystem.get(HTU.getConfiguration());
     assertTrue(fs.delete(splitaRef, true));
 
     //now, both parent, and splita can be deleted
-    assertEquals(2, janitor.scan());
-
-    services.stop("test finished");
-    janitor.cancel(true);
+    assertEquals(2, spy.scan());
   }
 
   /**
@@ -476,7 +333,7 @@ public class TestCatalogJanitor {
   @Test
   public void testSplitParentFirstComparator() {
     SplitParentFirstComparator comp = new SplitParentFirstComparator();
-    final HTableDescriptor htd = createHTableDescriptor();
+    TableDescriptor td = createTableDescriptorForCurrentMethod();
 
     /*  Region splits:
      *
@@ -497,15 +354,12 @@ public class TestCatalogJanitor {
      */
 
     // root region
-    HRegionInfo rootRegion = new HRegionInfo(htd.getTableName(),
-      HConstants.EMPTY_START_ROW,
-      HConstants.EMPTY_END_ROW, true);
-    HRegionInfo firstRegion = new HRegionInfo(htd.getTableName(),
-      HConstants.EMPTY_START_ROW,
-      Bytes.toBytes("bbb"), true);
-    HRegionInfo lastRegion = new HRegionInfo(htd.getTableName(),
-      Bytes.toBytes("bbb"),
-      HConstants.EMPTY_END_ROW, true);
+    HRegionInfo rootRegion = new HRegionInfo(td.getTableName(),
+      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, true);
+    HRegionInfo firstRegion = new HRegionInfo(td.getTableName(),
+      HConstants.EMPTY_START_ROW, Bytes.toBytes("bbb"), true);
+    HRegionInfo lastRegion = new HRegionInfo(td.getTableName(),
+      Bytes.toBytes("bbb"), HConstants.EMPTY_END_ROW, true);
 
     assertTrue(comp.compare(rootRegion, rootRegion) == 0);
     assertTrue(comp.compare(firstRegion, firstRegion) == 0);
@@ -515,19 +369,15 @@ public class TestCatalogJanitor {
     assertTrue(comp.compare(firstRegion, lastRegion) < 0);
 
     //first region split into a, b
-    HRegionInfo firstRegiona = new HRegionInfo(htd.getTableName(),
-      HConstants.EMPTY_START_ROW,
-      Bytes.toBytes("aaa"), true);
-    HRegionInfo firstRegionb = new HRegionInfo(htd.getTableName(),
-        Bytes.toBytes("aaa"),
-      Bytes.toBytes("bbb"), true);
+    HRegionInfo firstRegiona = new HRegionInfo(td.getTableName(),
+      HConstants.EMPTY_START_ROW, Bytes.toBytes("aaa"), true);
+    HRegionInfo firstRegionb = new HRegionInfo(td.getTableName(),
+        Bytes.toBytes("aaa"), Bytes.toBytes("bbb"), true);
     //last region split into a, b
-    HRegionInfo lastRegiona = new HRegionInfo(htd.getTableName(),
-      Bytes.toBytes("bbb"),
-      Bytes.toBytes("ddd"), true);
-    HRegionInfo lastRegionb = new HRegionInfo(htd.getTableName(),
-      Bytes.toBytes("ddd"),
-      HConstants.EMPTY_END_ROW, true);
+    HRegionInfo lastRegiona = new HRegionInfo(td.getTableName(),
+      Bytes.toBytes("bbb"), Bytes.toBytes("ddd"), true);
+    HRegionInfo lastRegionb = new HRegionInfo(td.getTableName(),
+      Bytes.toBytes("ddd"), HConstants.EMPTY_END_ROW, true);
 
     assertTrue(comp.compare(firstRegiona, firstRegiona) == 0);
     assertTrue(comp.compare(firstRegionb, firstRegionb) == 0);
@@ -550,58 +400,47 @@ public class TestCatalogJanitor {
     assertTrue(comp.compare(firstRegionb, lastRegiona) < 0);
     assertTrue(comp.compare(firstRegionb, lastRegionb) < 0);
 
-    HRegionInfo lastRegionaa = new HRegionInfo(htd.getTableName(),
-      Bytes.toBytes("bbb"),
-      Bytes.toBytes("ccc"), false);
-    HRegionInfo lastRegionab = new HRegionInfo(htd.getTableName(),
-      Bytes.toBytes("ccc"),
-      Bytes.toBytes("ddd"), false);
+    HRegionInfo lastRegionaa = new HRegionInfo(td.getTableName(),
+      Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), false);
+    HRegionInfo lastRegionab = new HRegionInfo(td.getTableName(),
+      Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), false);
 
     assertTrue(comp.compare(lastRegiona, lastRegionaa) < 0);
     assertTrue(comp.compare(lastRegiona, lastRegionab) < 0);
     assertTrue(comp.compare(lastRegionaa, lastRegionab) < 0);
-
   }
 
   @Test
   public void testArchiveOldRegion() throws Exception {
-    HBaseTestingUtility htu = new HBaseTestingUtility();
-    setRootDirAndCleanIt(htu, "testCleanParent");
-    MasterServices services = new MockMasterServices(htu);
-
-    // create the janitor
-    CatalogJanitor janitor = new CatalogJanitor(services);
-
     // Create regions.
-    HTableDescriptor htd = new 
HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    htd.addFamily(new HColumnDescriptor("f"));
-    HRegionInfo parent = new HRegionInfo(htd.getTableName(),
+    TableDescriptor td = createTableDescriptorForCurrentMethod();
+    HRegionInfo parent = new HRegionInfo(td.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
-    HRegionInfo splita = new HRegionInfo(htd.getTableName(),
+    HRegionInfo splita = new HRegionInfo(td.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
-    HRegionInfo splitb = new HRegionInfo(htd.getTableName(),
-        Bytes.toBytes("ccc"),
-        Bytes.toBytes("eee"));
+    HRegionInfo splitb = new HRegionInfo(td.getTableName(),
+        Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
 
     // Test that when both daughter regions are in place, that we do not
     // remove the parent.
     Result parentMetaRow = createResult(parent, splita, splitb);
-    FileSystem fs = FileSystem.get(htu.getConfiguration());
-    Path rootdir = services.getMasterFileSystem().getRootDir();
+    FileSystem fs = FileSystem.get(HTU.getConfiguration());
+    Path rootdir = this.masterServices.getMasterFileSystem().getRootDir();
     // have to set the root directory since we use it in HFileDisposer to 
figure out to get to the
     // archive directory. Otherwise, it just seems to pick the first root 
directory it can find (so
     // the single test passes, but when the full suite is run, things get 
borked).
     FSUtils.setRootDir(fs.getConf(), rootdir);
-    Path tabledir = FSUtils.getTableDir(rootdir, htd.getTableName());
-    Path storedir = HStore.getStoreHomedir(tabledir, parent, 
htd.getColumnFamilies()[0].getName());
-    Path storeArchive = 
HFileArchiveUtil.getStoreArchivePath(services.getConfiguration(), parent,
-      tabledir, htd.getColumnFamilies()[0].getName());
+    Path tabledir = FSUtils.getTableDir(rootdir, td.getTableName());
+    Path storedir = HStore.getStoreHomedir(tabledir, parent, 
td.getColumnFamilies()[0].getName());
+    Path storeArchive =
+        
HFileArchiveUtil.getStoreArchivePath(this.masterServices.getConfiguration(), 
parent,
+            tabledir, td.getColumnFamilies()[0].getName());
     LOG.debug("Table dir:" + tabledir);
     LOG.debug("Store dir:" + storedir);
     LOG.debug("Store archive dir:" + storeArchive);
 
     // add a couple of store files that we can check for
-    FileStatus[] mockFiles = addMockStoreFiles(2, services, storedir);
+    FileStatus[] mockFiles = addMockStoreFiles(2, this.masterServices, 
storedir);
     // get the current store files for comparison
     FileStatus[] storeFiles = fs.listStatus(storedir);
     int index = 0;
@@ -614,6 +453,12 @@ public class TestCatalogJanitor {
 
     // do the cleaning of the parent
     assertTrue(janitor.cleanParent(parent, parentMetaRow));
+    Path parentDir = new Path(tabledir, parent.getEncodedName());
+    // Cleanup procedure runs async. Wait till it done.
+    LOG.info("Waiting on parent " + parentDir);
+    while (fs.exists(parentDir)) {
+      Threads.sleep(10);
+    }
     LOG.debug("Finished cleanup of parent region");
 
     // and now check to make sure that the files have actually been archived
@@ -625,8 +470,6 @@ public class TestCatalogJanitor {
 
     // cleanup
     FSUtils.delete(fs, rootdir, true);
-    services.stop("Test finished");
-    janitor.cancel(true);
   }
 
   /**
@@ -646,69 +489,67 @@ public class TestCatalogJanitor {
    */
   @Test
   public void testDuplicateHFileResolution() throws Exception {
-    HBaseTestingUtility htu = new HBaseTestingUtility();
-    setRootDirAndCleanIt(htu, "testCleanParent");
-    MasterServices services = new MockMasterServices(htu);
-
-    // create the janitor
-
-    CatalogJanitor janitor = new CatalogJanitor(services);
+   TableDescriptor td = createTableDescriptorForCurrentMethod();
 
     // Create regions.
-    HTableDescriptor htd = new 
HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    htd.addFamily(new HColumnDescriptor("f"));
-    HRegionInfo parent = new HRegionInfo(htd.getTableName(),
+    HRegionInfo parent = new HRegionInfo(td.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
-    HRegionInfo splita = new HRegionInfo(htd.getTableName(),
+    HRegionInfo splita = new HRegionInfo(td.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
-    HRegionInfo splitb = new HRegionInfo(htd.getTableName(),
+    HRegionInfo splitb = new HRegionInfo(td.getTableName(),
         Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
     // Test that when both daughter regions are in place, that we do not
     // remove the parent.
     Result r = createResult(parent, splita, splitb);
-
-    FileSystem fs = FileSystem.get(htu.getConfiguration());
-
-    Path rootdir = services.getMasterFileSystem().getRootDir();
-    // have to set the root directory since we use it in HFileDisposer to 
figure out to get to the
+    FileSystem fs = FileSystem.get(HTU.getConfiguration());
+    Path rootdir = this.masterServices.getMasterFileSystem().getRootDir();
+    // Have to set the root directory since we use it in HFileDisposer to 
figure out to get to the
     // archive directory. Otherwise, it just seems to pick the first root 
directory it can find (so
     // the single test passes, but when the full suite is run, things get 
borked).
     FSUtils.setRootDir(fs.getConf(), rootdir);
     Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
-    Path storedir = HStore.getStoreHomedir(tabledir, parent, 
htd.getColumnFamilies()[0].getName());
+    Path storedir = HStore.getStoreHomedir(tabledir, parent, 
td.getColumnFamilies()[0].getName());
     System.out.println("Old root:" + rootdir);
     System.out.println("Old table:" + tabledir);
     System.out.println("Old store:" + storedir);
 
-    Path storeArchive = 
HFileArchiveUtil.getStoreArchivePath(services.getConfiguration(), parent,
-      tabledir, htd.getColumnFamilies()[0].getName());
+    Path storeArchive =
+        
HFileArchiveUtil.getStoreArchivePath(this.masterServices.getConfiguration(), 
parent,
+      tabledir, td.getColumnFamilies()[0].getName());
     System.out.println("Old archive:" + storeArchive);
 
     // enable archiving, make sure that files get archived
-    addMockStoreFiles(2, services, storedir);
+    addMockStoreFiles(2, this.masterServices, storedir);
     // get the current store files for comparison
     FileStatus[] storeFiles = fs.listStatus(storedir);
-    // do the cleaning of the parent
+    // Do the cleaning of the parent
     assertTrue(janitor.cleanParent(parent, r));
+    Path parentDir = new Path(tabledir, parent.getEncodedName());
+    // Cleanup procedure runs async. Wait till it done.
+    LOG.info("Waiting on parent " + parentDir);
+    while (fs.exists(parentDir)) {
+      Threads.sleep(10);
+    }
 
-    // and now check to make sure that the files have actually been archived
+    // And now check to make sure that the files have actually been archived
     FileStatus[] archivedStoreFiles = fs.listStatus(storeArchive);
     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs);
 
     // now add store files with the same names as before to check backup
     // enable archiving, make sure that files get archived
-    addMockStoreFiles(2, services, storedir);
+    addMockStoreFiles(2, this.masterServices, storedir);
 
-    // do the cleaning of the parent
+    // Do the cleaning of the parent
     assertTrue(janitor.cleanParent(parent, r));
+    // Cleanup procedure runs async. Wait till it done.
+    LOG.info("Waiting on parent " + parentDir);
+    while (fs.exists(parentDir)) {
+      Threads.sleep(10);
+    }
 
     // and now check to make sure that the files have actually been archived
     archivedStoreFiles = fs.listStatus(storeArchive);
     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs, true);
-
-    // cleanup
-    services.stop("Test finished");
-    janitor.cancel(true);
   }
 
   private FileStatus[] addMockStoreFiles(int count, MasterServices services, 
Path storedir)
@@ -730,8 +571,7 @@ public class TestCatalogJanitor {
     return storeFiles;
   }
 
-  private String setRootDirAndCleanIt(final HBaseTestingUtility htu,
-      final String subdir)
+  private String setRootDirAndCleanIt(final HBaseTestingUtility htu, final 
String subdir)
   throws IOException {
     Path testdir = htu.getDataTestDir(subdir);
     FileSystem fs = FileSystem.get(htu.getConfiguration());
@@ -740,24 +580,14 @@ public class TestCatalogJanitor {
     return FSUtils.getRootDir(htu.getConfiguration()).toString();
   }
 
-  /**
-   * @param services Master services instance.
-   * @param htd
-   * @param parent
-   * @param daughter
-   * @param midkey
-   * @param top True if we are to write a 'top' reference.
-   * @return Path to reference we created.
-   * @throws IOException
-   */
   private Path createReferences(final MasterServices services,
-      final HTableDescriptor htd, final HRegionInfo parent,
+      final TableDescriptor td, final HRegionInfo parent,
       final HRegionInfo daughter, final byte [] midkey, final boolean top)
   throws IOException {
     Path rootdir = services.getMasterFileSystem().getRootDir();
     Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
     Path storedir = HStore.getStoreHomedir(tabledir, daughter,
-      htd.getColumnFamilies()[0].getName());
+      td.getColumnFamilies()[0].getName());
     Reference ref =
       top? Reference.createTopReference(midkey): 
Reference.createBottomReference(midkey);
     long now = System.currentTimeMillis();
@@ -773,30 +603,4 @@ public class TestCatalogJanitor {
   throws IOException {
     return MetaMockingUtil.getMetaTableRowResult(parent, null, a, b);
   }
-
-  private HTableDescriptor createHTableDescriptor() {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("t"));
-    htd.addFamily(new HColumnDescriptor("f"));
-    return htd;
-  }
-
-  private MultiResponse buildMultiResponse(MultiRequest req) {
-    MultiResponse.Builder builder = MultiResponse.newBuilder();
-    RegionActionResult.Builder regionActionResultBuilder =
-        RegionActionResult.newBuilder();
-    ResultOrException.Builder roeBuilder = ResultOrException.newBuilder();
-    for (RegionAction regionAction: req.getRegionActionList()) {
-      regionActionResultBuilder.clear();
-      for (ClientProtos.Action action: regionAction.getActionList()) {
-        roeBuilder.clear();
-        roeBuilder.setResult(ClientProtos.Result.getDefaultInstance());
-        roeBuilder.setIndex(action.getIndex());
-        regionActionResultBuilder.addResultOrException(roeBuilder.build());
-      }
-      builder.addRegionActionResult(regionActionResultBuilder.build());
-    }
-    return builder.build();
-  }
-
-}
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
index b3ed2d4..d558aaf 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -19,16 +19,23 @@ package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.NavigableMap;
 import java.util.SortedSet;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
@@ -41,11 +48,32 @@ import 
org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
 import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
+/**
+ * A mocked master services.
+ * Tries to fake it. May not always work.
+ */
 public class MockMasterServices extends MockNoopMasterServices {
   private final MasterFileSystem fileSystemManager;
   private final MasterWalManager walManager;
@@ -54,12 +82,17 @@ public class MockMasterServices extends 
MockNoopMasterServices {
   private MasterProcedureEnv procedureEnv;
   private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
   private ProcedureStore procedureStore;
-
-  private LoadBalancer balancer;
-  private ServerManager serverManager;
+  private final ClusterConnection connection;
+  private final LoadBalancer balancer;
+  private final ServerManager serverManager;
   // Set of regions on a 'server'. Populated externally. Used in below faking 
'cluster'.
   private final NavigableMap<ServerName, SortedSet<byte []>> 
regionsToRegionServers;
 
+  private final ProcedureEvent initialized = new ProcedureEvent("master 
initialized");
+  public static final String DEFAULT_COLUMN_FAMILY_NAME = "cf";
+  public static final ServerName MOCK_MASTER_SERVERNAME =
+      ServerName.valueOf("mockmaster.example.org", 1234, -1L);
+
   public MockMasterServices(Configuration conf,
       NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers)
   throws IOException {
@@ -68,7 +101,7 @@ public class MockMasterServices extends 
MockNoopMasterServices {
     Superusers.initialize(conf);
     this.fileSystemManager = new MasterFileSystem(this);
     this.walManager = new MasterWalManager(this);
-    
+    // Mock an AM.
     this.assignmentManager = new AssignmentManager(this, new 
MockRegionStateStore(this)) {
       public boolean isTableEnabled(final TableName tableName) {
         return true;
@@ -89,16 +122,51 @@ public class MockMasterServices extends 
MockNoopMasterServices {
     };
     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
     this.serverManager = new ServerManager(this);
+
+    // Mock up a Client Interface
+    ClientProtos.ClientService.BlockingInterface ri =
+        Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
+    MutateResponse.Builder builder = MutateResponse.newBuilder();
+    builder.setProcessed(true);
+    try {
+      Mockito.when(ri.mutate((RpcController)Mockito.any(), 
(MutateRequest)Mockito.any())).
+        thenReturn(builder.build());
+    } catch (ServiceException se) {
+      throw ProtobufUtil.handleRemoteException(se);
+    }
+    try {
+      Mockito.when(ri.multi((RpcController)Mockito.any(), 
(MultiRequest)Mockito.any())).
+        thenAnswer(new Answer<MultiResponse>() {
+          @Override
+          public MultiResponse answer(InvocationOnMock invocation) throws 
Throwable {
+            return buildMultiResponse( 
(MultiRequest)invocation.getArguments()[1]);
+          }
+        });
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
+    // Mock n ClusterConnection and an AdminProtocol implementation. Have the
+    // ClusterConnection return the HRI.  Have the HRI return a few mocked up 
responses
+    // to make our test work.
+    this.connection =
+        
HConnectionTestingUtility.getMockedConnectionAndDecorate(getConfiguration(),
+          Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri, 
MOCK_MASTER_SERVERNAME,
+          HRegionInfo.FIRST_META_REGIONINFO);
+    // Set hbase.rootdir into test dir.
+    Path rootdir = FSUtils.getRootDir(getConfiguration());
+    FSUtils.setRootDir(getConfiguration(), rootdir);
+    Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
   }
 
   public void start(final int numServes, final RSProcedureDispatcher 
remoteDispatcher)
       throws IOException {
     startProcedureExecutor(remoteDispatcher);
-    assignmentManager.start();
+    this.assignmentManager.start();
     for (int i = 0; i < numServes; ++i) {
       serverManager.regionServerReport(
         ServerName.valueOf("localhost", 100 + i, 1), 
ServerLoad.EMPTY_SERVERLOAD);
     }
+    this.procedureExecutor.getEnvironment().setEventReady(initialized, true);
   }
 
   @Override
@@ -115,13 +183,13 @@ public class MockMasterServices extends 
MockNoopMasterServices {
 
     //procedureStore = new WALProcedureStore(conf, 
fileSystemManager.getFileSystem(), logDir,
     //    new MasterProcedureEnv.WALStoreLeaseRecovery(this));
-    procedureStore = new NoopProcedureStore();
-    procedureStore.registerListener(new 
MasterProcedureEnv.MasterProcedureStoreListener(this));
+    this.procedureStore = new NoopProcedureStore();
+    this.procedureStore.registerListener(new 
MasterProcedureEnv.MasterProcedureStoreListener(this));
 
-    procedureEnv = new MasterProcedureEnv(this,
+    this.procedureEnv = new MasterProcedureEnv(this,
        remoteDispatcher != null ? remoteDispatcher : new 
RSProcedureDispatcher(this));
 
-    procedureExecutor = new ProcedureExecutor(conf, procedureEnv, 
procedureStore,
+    this.procedureExecutor = new ProcedureExecutor(conf, procedureEnv, 
procedureStore,
         procedureEnv.getProcedureScheduler());
 
     final int numThreads = 
conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
@@ -130,22 +198,22 @@ public class MockMasterServices extends 
MockNoopMasterServices {
     final boolean abortOnCorruption = conf.getBoolean(
         MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
         MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
-    procedureStore.start(numThreads);
-    procedureExecutor.start(numThreads, abortOnCorruption);
-    procedureEnv.getRemoteDispatcher().start();
+    this.procedureStore.start(numThreads);
+    this.procedureExecutor.start(numThreads, abortOnCorruption);
+    this.procedureEnv.getRemoteDispatcher().start();
   }
 
   private void stopProcedureExecutor() {
-    if (procedureEnv != null) {
-      procedureEnv.getRemoteDispatcher().stop();
+    if (this.procedureEnv != null) {
+      this.procedureEnv.getRemoteDispatcher().stop();
     }
 
-    if (procedureExecutor != null) {
-      procedureExecutor.stop();
+    if (this.procedureExecutor != null) {
+      this.procedureExecutor.stop();
     }
 
-    if (procedureStore != null) {
-      procedureStore.stop(isAborted());
+    if (this.procedureStore != null) {
+      this.procedureStore.stop(isAborted());
     }
   }
 
@@ -155,6 +223,11 @@ public class MockMasterServices extends 
MockNoopMasterServices {
   }
 
   @Override
+  public ProcedureEvent getInitializedEvent() {
+    return this.initialized;
+  }
+
+  @Override
   public MasterFileSystem getMasterFileSystem() {
     return fileSystemManager;
   }
@@ -185,6 +258,16 @@ public class MockMasterServices extends 
MockNoopMasterServices {
   }
 
   @Override
+  public ClusterConnection getConnection() {
+    return this.connection;
+  }
+
+  @Override
+  public ServerName getServerName() {
+    return MOCK_MASTER_SERVERNAME;
+  }
+
+  @Override
   public CoordinatedStateManager getCoordinatedStateManager() {
     return super.getCoordinatedStateManager();
   }
@@ -207,4 +290,69 @@ public class MockMasterServices extends 
MockNoopMasterServices {
         ServerName lastHost, long openSeqNum, long pid) throws IOException {
     }
   }
+
+  @Override
+  public TableDescriptors getTableDescriptors() {
+    return new TableDescriptors() {
+      @Override
+      public HTableDescriptor remove(TableName tablename) throws IOException {
+        // noop
+        return null;
+      }
+
+      @Override
+      public Map<String, HTableDescriptor> getAll() throws IOException {
+        // noop
+        return null;
+      }
+
+      @Override public Map<String, HTableDescriptor> getAllDescriptors() 
throws IOException {
+        // noop
+        return null;
+      }
+
+      @Override
+      public HTableDescriptor get(TableName tablename) throws IOException {
+        HTableDescriptor htd = new HTableDescriptor(tablename);
+        htd.addFamily(new HColumnDescriptor(DEFAULT_COLUMN_FAMILY_NAME));
+        return htd;
+      }
+
+      @Override
+      public Map<String, HTableDescriptor> getByNamespace(String name) throws 
IOException {
+        return null;
+      }
+
+      @Override
+      public void add(HTableDescriptor htd) throws IOException {
+        // noop
+      }
+
+      @Override
+      public void setCacheOn() throws IOException {
+      }
+
+      @Override
+      public void setCacheOff() throws IOException {
+      }
+    };
+  }
+
+  private static MultiResponse buildMultiResponse(MultiRequest req) {
+    MultiResponse.Builder builder = MultiResponse.newBuilder();
+    RegionActionResult.Builder regionActionResultBuilder =
+        RegionActionResult.newBuilder();
+    ResultOrException.Builder roeBuilder = ResultOrException.newBuilder();
+    for (RegionAction regionAction: req.getRegionActionList()) {
+      regionActionResultBuilder.clear();
+      for (ClientProtos.Action action: regionAction.getActionList()) {
+        roeBuilder.clear();
+        roeBuilder.setResult(ClientProtos.Result.getDefaultInstance());
+        roeBuilder.setIndex(action.getIndex());
+        regionActionResultBuilder.addResultOrException(roeBuilder.build());
+      }
+      builder.addRegionActionResult(regionActionResultBuilder.build());
+    }
+    return builder.build();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
index 8be1be9..44fd575 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
@@ -26,11 +26,13 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
@@ -41,18 +43,23 @@ import 
org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
-@Ignore // Fix for AMv2.
 public class TestMergeTableRegionsProcedure {
   private static final Log LOG = 
LogFactory.getLog(TestMergeTableRegionsProcedure.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+      withTimeout(this.getClass()).withLookingForStuckThread(true).build();
+  @Rule public final TestName name = new TestName();
 
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static long nonceGroup = HConstants.NO_NONCE;
@@ -68,7 +75,6 @@ public class TestMergeTableRegionsProcedure {
     conf.setInt("hbase.assignment.maximum.attempts", 3);
     conf.setInt("hbase.master.maximum.ping.server.attempts", 3);
     conf.setInt("hbase.master.ping.server.retry.sleep.interval", 1);
-
     conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
   }
 
@@ -119,9 +125,9 @@ public class TestMergeTableRegionsProcedure {
   /**
    * This tests two region merges
    */
-  @Test(timeout=60000)
+  @Test
   public void testMergeTwoRegions() throws Exception {
-    final TableName tableName = TableName.valueOf("testMergeTwoRegions");
+    final TableName tableName = TableName.valueOf(this.name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
 
     List<HRegionInfo> tableRegions = createTable(tableName);
@@ -129,19 +135,33 @@ public class TestMergeTableRegionsProcedure {
     HRegionInfo[] regionsToMerge = new HRegionInfo[2];
     regionsToMerge[0] = tableRegions.get(0);
     regionsToMerge[1] = tableRegions.get(1);
-
-    long procId = procExec.submitProcedure(new MergeTableRegionsProcedure(
-      procExec.getEnvironment(), regionsToMerge, true));
+    MergeTableRegionsProcedure proc =
+        new MergeTableRegionsProcedure(procExec.getEnvironment(), 
regionsToMerge, true);
+    long procId = procExec.submitProcedure(proc);
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
     assertRegionCount(tableName, initialRegionCount - 1);
+    Pair<HRegionInfo, HRegionInfo> pair =
+      MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
+        proc.getMergedRegion().getRegionName());
+    assertTrue(pair.getFirst() != null && pair.getSecond() != null);
+
+    // Can I purge the merged regions from hbase:meta? Check that all went
+    // well by looking at the merged row up in hbase:meta. It should have no
+    // more mention of the merged regions; they are purged as last step in
+    // the merged regions cleanup.
+    UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
+    UTIL.getHBaseCluster().getMaster().getCatalogJanitor().triggerNow();
+    while (pair != null && pair.getFirst() != null && pair.getSecond() != 
null) {
+      pair = 
MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
+          proc.getMergedRegion().getRegionName());
+    }
   }
 
   /**
    * This tests two concurrent region merges
    */
-  @Test(timeout=60000)
+  @Test
   public void testMergeRegionsConcurrently() throws Exception {
     final TableName tableName = 
TableName.valueOf("testMergeRegionsConcurrently");
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -166,7 +186,7 @@ public class TestMergeTableRegionsProcedure {
     assertRegionCount(tableName, initialRegionCount - 2);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testRecoveryAndDoubleExecution() throws Exception {
     final TableName tableName = 
TableName.valueOf("testRecoveryAndDoubleExecution");
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -190,7 +210,7 @@ public class TestMergeTableRegionsProcedure {
     assertRegionCount(tableName, initialRegionCount - 1);
   }
 
-  @Test(timeout = 60000)
+  @Test
   public void testRollbackAndDoubleExecution() throws Exception {
     final TableName tableName = 
TableName.valueOf("testRollbackAndDoubleExecution");
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
index 78836e8..6824597 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
@@ -67,8 +67,8 @@ import org.junit.rules.TestRule;
 @Category({MasterTests.class, MediumTests.class})
 public class TestSplitTableRegionProcedure {
   private static final Log LOG = 
LogFactory.getLog(TestSplitTableRegionProcedure.class);
-  @Rule public final TestRule timeout = 
CategoryBasedTimeout.builder().withTimeout(this.getClass()).
-      withLookingForStuckThread(true).build();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+      withTimeout(this.getClass()).withLookingForStuckThread(true).build();
 
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
@@ -143,7 +143,7 @@ public class TestSplitTableRegionProcedure {
     verify(tableName, splitRowNum);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionNoStoreFile() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -167,7 +167,7 @@ public class TestSplitTableRegionProcedure {
     assertTrue(UTIL.countRows(tableName) == 0);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionUnevenDaughter() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -192,7 +192,7 @@ public class TestSplitTableRegionProcedure {
     verify(tableName, splitRowNum);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionEmptyDaughter() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -221,7 +221,7 @@ public class TestSplitTableRegionProcedure {
     assertTrue(UTIL.countRows(daughters.get(0)) == 0 || 
UTIL.countRows(daughters.get(1)) == 0);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionDeletedRowsDaughter() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -261,7 +261,7 @@ public class TestSplitTableRegionProcedure {
     assertTrue(UTIL.countRows(daughters.get(0)) == 0 || 
UTIL.countRows(daughters.get(1)) == 0);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testInvalidSplitKey() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -284,7 +284,7 @@ public class TestSplitTableRegionProcedure {
     }
   }
 
-  @Test(timeout = 60000)
+  @Test
   public void testRollbackAndDoubleExecution() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -318,7 +318,7 @@ public class TestSplitTableRegionProcedure {
     Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes(ColumnFamilyName2));
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testRecoveryAndDoubleExecution() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
@@ -425,4 +425,4 @@ public class TestSplitTableRegionProcedure {
   private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
     return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
index 2b28c9f..822e57c 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
@@ -80,7 +80,7 @@ public class MasterProcedureSchedulerPerformanceEvaluation 
extends AbstractHBase
 
   private class RegionProcedure extends 
TestMasterProcedureScheduler.TestRegionProcedure {
     RegionProcedure(long procId, HRegionInfo hri) {
-      super(procId, hri.getTable(), TableOperationType.UNASSIGN, hri);
+      super(procId, hri.getTable(), TableOperationType.REGION_UNASSIGN, hri);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
index e23c90a..5f20c7f 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
@@ -343,11 +343,11 @@ public class TestMasterProcedureScheduler {
     final HRegionInfo regionA = new HRegionInfo(tableName, Bytes.toBytes("a"), 
Bytes.toBytes("b"));
 
     queue.addBack(new TestRegionProcedure(1, tableName,
-        TableProcedureInterface.TableOperationType.ASSIGN, regionA));
+        TableProcedureInterface.TableOperationType.REGION_ASSIGN, regionA));
     queue.addBack(new TestTableProcedure(2, tableName,
           TableProcedureInterface.TableOperationType.EDIT));
     queue.addBack(new TestRegionProcedure(3, tableName,
-        TableProcedureInterface.TableOperationType.UNASSIGN, regionA));
+        TableProcedureInterface.TableOperationType.REGION_UNASSIGN, regionA));
 
     // Fetch the 1st item and take the shared lock
     Procedure proc = queue.poll();
@@ -391,13 +391,13 @@ public class TestMasterProcedureScheduler {
     queue.addBack(new TestTableProcedure(1, tableName,
           TableProcedureInterface.TableOperationType.EDIT));
     queue.addBack(new TestRegionProcedure(2, tableName,
-        TableProcedureInterface.TableOperationType.MERGE, regionA, regionB));
+        TableProcedureInterface.TableOperationType.REGION_MERGE, regionA, 
regionB));
     queue.addBack(new TestRegionProcedure(3, tableName,
-        TableProcedureInterface.TableOperationType.SPLIT, regionA));
+        TableProcedureInterface.TableOperationType.REGION_SPLIT, regionA));
     queue.addBack(new TestRegionProcedure(4, tableName,
-        TableProcedureInterface.TableOperationType.SPLIT, regionB));
+        TableProcedureInterface.TableOperationType.REGION_SPLIT, regionB));
     queue.addBack(new TestRegionProcedure(5, tableName,
-        TableProcedureInterface.TableOperationType.UNASSIGN, regionC));
+        TableProcedureInterface.TableOperationType.REGION_UNASSIGN, regionC));
 
     // Fetch the 1st item and take the write lock
     Procedure proc = queue.poll();
@@ -520,9 +520,9 @@ public class TestMasterProcedureScheduler {
     final HRegionInfo region = new HRegionInfo(tableName, Bytes.toBytes("a"), 
Bytes.toBytes("b"));
 
     queue.addBack(new TestRegionProcedure(1, tableName,
-        TableProcedureInterface.TableOperationType.SPLIT, region));
+        TableProcedureInterface.TableOperationType.REGION_SPLIT, region));
     queue.addBack(new TestRegionProcedure(1, 2, tableName,
-        TableProcedureInterface.TableOperationType.UNASSIGN, region));
+        TableProcedureInterface.TableOperationType.REGION_UNASSIGN, region));
     queue.addBack(new TestRegionProcedure(3, tableName,
         TableProcedureInterface.TableOperationType.REGION_EDIT, region));
 
@@ -600,7 +600,7 @@ public class TestMasterProcedureScheduler {
     final TestRegionProcedure[] childProcs = new 
TestRegionProcedure[regions.length];
     for (int i = 0; i < regions.length; ++i) {
       childProcs[i] = new TestRegionProcedure(1, 2 + i, tableName,
-          TableProcedureInterface.TableOperationType.ASSIGN, regions[i]);
+          TableProcedureInterface.TableOperationType.REGION_ASSIGN, 
regions[i]);
     }
     testInheritedXLockAndChildrenSharedLock(tableName,
       new TestTableProcedure(1, tableName, 
TableProcedureInterface.TableOperationType.CREATE),
@@ -615,7 +615,7 @@ public class TestMasterProcedureScheduler {
     final TestRegionProcedure[] childProcs = new 
TestRegionProcedure[regions.length];
     for (int i = 0; i < regions.length; ++i) {
       childProcs[i] = new TestRegionProcedure(1, 2, 3 + i, tableName,
-          TableProcedureInterface.TableOperationType.ASSIGN, regions[i]);
+          TableProcedureInterface.TableOperationType.REGION_ASSIGN, 
regions[i]);
     }
     testInheritedXLockAndChildrenSharedLock(tableName,
       new TestTableProcedure(1, tableName, 
TableProcedureInterface.TableOperationType.CREATE),

Reply via email to