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

[hbase] branch master updated: HBASE-25005 Refactor CatalogJanitor (#2373)

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 23713f4  HBASE-25005 Refactor CatalogJanitor (#2373)
23713f4 is described below

commit 23713f4693286795e599bf1b0ae94f880af3c7f5
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Sep 10 11:59:15 2020 +0800

    HBASE-25005 Refactor CatalogJanitor (#2373)
    
     Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../hadoop/hbase/client/RegionInfoBuilder.java     |   6 +-
 .../apache/hadoop/hbase/master/CatalogJanitor.java | 819 ---------------------
 .../org/apache/hadoop/hbase/master/HMaster.java    |   4 +-
 .../hadoop/hbase/master/MasterRpcServices.java     |   1 +
 .../apache/hadoop/hbase/master/MasterServices.java |   1 +
 .../hbase/master/janitor/CatalogJanitor.java       | 456 ++++++++++++
 .../hbase/master/{ => janitor}/MetaFixer.java      |  24 +-
 .../apache/hadoop/hbase/master/janitor/Report.java | 128 ++++
 .../hbase/master/janitor/ReportMakingVisitor.java  | 294 ++++++++
 .../main/resources/hbase-webapps/master/hbck.jsp   |   6 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java |   2 +-
 .../hbase/client/TestAsyncRegionAdminApi2.java     |   2 +-
 .../hbase/master/MockNoopMasterServices.java       |   1 +
 .../hbase/master/TestMasterChoreScheduled.java     |   1 +
 .../master/{ => janitor}/TestCatalogJanitor.java   |  44 +-
 .../{ => janitor}/TestCatalogJanitorCluster.java   | 123 ++--
 .../TestCatalogJanitorInMemoryStates.java          |  66 +-
 .../hbase/master/{ => janitor}/TestMetaFixer.java  |  33 +-
 .../{ => janitor}/TestMetaFixerNoCluster.java      |  58 +-
 19 files changed, 1057 insertions(+), 1012 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
index 8f045e0..a9e7806 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -20,12 +20,9 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class RegionInfoBuilder {
-  private static final Logger LOG = LoggerFactory.getLogger(RegionInfoBuilder.class);
 
   /** A non-capture group so that this can be embedded. */
   public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
@@ -33,6 +30,9 @@ public class RegionInfoBuilder {
   //TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
   public static final String NO_HASH = null;
 
+  public static final RegionInfo UNDEFINED =
+    RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
+
   /**
    * RegionInfo for first meta region
    * You cannot use this builder to make an instance of the {@link #FIRST_META_REGIONINFO}.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
deleted file mode 100644
index de1dfff..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ /dev/null
@@ -1,819 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CatalogFamilyFormat;
-import org.apache.hadoop.hbase.ClientMetaTableAccessor;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.GCMultipleMergedRegionsProcedure;
-import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.PairOfSameType;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-/**
- * A janitor for the catalog tables. Scans the <code>hbase:meta</code> catalog
- * table on a period. Makes a lastReport on state of hbase:meta. Looks for unused
- * regions to garbage collect. Scan of hbase:meta runs if we are NOT in maintenance
- * mode, if we are NOT shutting down, AND if the assignmentmanager is loaded.
- * Playing it safe, we will garbage collect no-longer needed region references
- * only if there are no regions-in-transition (RIT).
- */
-// TODO: Only works with single hbase:meta region currently.  Fix.
-// TODO: Should it start over every time? Could it continue if runs into problem? Only if
-// problem does not mess up 'results'.
-// TODO: Do more by way of 'repair'; see note on unknownServers below.
-@InterfaceAudience.Private
-public class CatalogJanitor extends ScheduledChore {
-  private static final Logger LOG = LoggerFactory.getLogger(CatalogJanitor.class.getName());
-  private final AtomicBoolean alreadyRunning = new AtomicBoolean(false);
-  private final AtomicBoolean enabled = new AtomicBoolean(true);
-  private final MasterServices services;
-
-  /**
-   * Saved report from last hbase:meta scan to completion. May be stale if having trouble
-   * completing scan. Check its date.
-   */
-  private volatile Report lastReport;
-
-  CatalogJanitor(final MasterServices services) {
-    super("CatalogJanitor-" + services.getServerName().toShortString(), services,
-      services.getConfiguration().getInt("hbase.catalogjanitor.interval", 300000));
-    this.services = services;
-  }
-
-  @Override
-  protected boolean initialChore() {
-    try {
-      if (getEnabled()) {
-        scan();
-      }
-    } catch (IOException e) {
-      LOG.warn("Failed initial janitorial scan of hbase:meta table", e);
-      return false;
-    }
-    return true;
-  }
-
-  boolean setEnabled(final boolean enabled) {
-    boolean alreadyEnabled = this.enabled.getAndSet(enabled);
-    // If disabling is requested on an already enabled chore, we could have an active
-    // scan still going on, callers might not be aware of that and do further action thinkng
-    // that no action would be from this chore.  In this case, the right action is to wait for
-    // the active scan to complete before exiting this function.
-    if (!enabled && alreadyEnabled) {
-      while (alreadyRunning.get()) {
-        Threads.sleepWithoutInterrupt(100);
-      }
-    }
-    return alreadyEnabled;
-  }
-
-  boolean getEnabled() {
-    return this.enabled.get();
-  }
-
-  @Override
-  protected void chore() {
-    try {
-      AssignmentManager am = this.services.getAssignmentManager();
-      if (getEnabled() && !this.services.isInMaintenanceMode() &&
-          !this.services.getServerManager().isClusterShutdown() &&
-          isMetaLoaded(am)) {
-        scan();
-      } else {
-        LOG.warn("CatalogJanitor is disabled! Enabled=" + getEnabled() + 
-          ", maintenanceMode=" + this.services.isInMaintenanceMode() + ", am=" + am +
-          ", metaLoaded=" + isMetaLoaded(am) + ", hasRIT=" + isRIT(am) +
-          " clusterShutDown=" + this.services.getServerManager().isClusterShutdown());
-      }
-    } catch (IOException e) {
-      LOG.warn("Failed janitorial scan of hbase:meta table", e);
-    }
-  }
-
-  private static boolean isMetaLoaded(AssignmentManager am) {
-    return am != null && am.isMetaLoaded();
-  }
-
-  private static boolean isRIT(AssignmentManager am) {
-    return isMetaLoaded(am) && am.hasRegionsInTransition();
-  }
-
-  /**
-   * Run janitorial scan of catalog <code>hbase:meta</code> table looking for
-   * garbage to collect.
-   * @return How many items gc'd whether for merge or split.
-   *   Returns -1 if previous scan is in progress.
-   */
-  @VisibleForTesting
-  public int scan() throws IOException {
-    int gcs = 0;
-    try {
-      if (!alreadyRunning.compareAndSet(false, true)) {
-        LOG.debug("CatalogJanitor already running");
-        // -1 indicates previous scan is in progress
-        return -1;
-      }
-      this.lastReport = scanForReport();
-      if (!this.lastReport.isEmpty()) {
-        LOG.warn(this.lastReport.toString());
-      }
-
-      if (isRIT(this.services.getAssignmentManager())) {
-        LOG.warn("Playing-it-safe skipping merge/split gc'ing of regions from hbase:meta while " +
-            "regions-in-transition (RIT)");
-      }
-      Map<RegionInfo, Result> mergedRegions = this.lastReport.mergedRegions;
-      for (Map.Entry<RegionInfo, Result> e : mergedRegions.entrySet()) {
-        if (this.services.isInMaintenanceMode()) {
-          // Stop cleaning if the master is in maintenance mode
-          break;
-        }
-
-        List<RegionInfo> parents = CatalogFamilyFormat.getMergeRegions(e.getValue().rawCells());
-        if (parents != null && cleanMergeRegion(e.getKey(), parents)) {
-          gcs++;
-        }
-      }
-      // Clean split parents
-      Map<RegionInfo, Result> splitParents = this.lastReport.splitParents;
-
-      // Now work on our list of found parents. See if any we can clean up.
-      HashSet<String> parentNotCleaned = new HashSet<>();
-      for (Map.Entry<RegionInfo, Result> e : splitParents.entrySet()) {
-        if (this.services.isInMaintenanceMode()) {
-          // Stop cleaning if the master is in maintenance mode
-          break;
-        }
-
-        if (!parentNotCleaned.contains(e.getKey().getEncodedName()) &&
-            cleanParent(e.getKey(), e.getValue())) {
-          gcs++;
-        } else {
-          // We could not clean the parent, so it's daughters should not be
-          // cleaned either (HBASE-6160)
-          PairOfSameType<RegionInfo> daughters =
-              MetaTableAccessor.getDaughterRegions(e.getValue());
-          parentNotCleaned.add(daughters.getFirst().getEncodedName());
-          parentNotCleaned.add(daughters.getSecond().getEncodedName());
-        }
-      }
-      return gcs;
-    } finally {
-      alreadyRunning.set(false);
-    }
-  }
-
-  /**
-   * Scan hbase:meta.
-   * @return Return generated {@link Report}
-   */
-  Report scanForReport() throws IOException {
-    ReportMakingVisitor visitor = new ReportMakingVisitor(this.services);
-    // Null tablename means scan all of meta.
-    MetaTableAccessor.scanMetaForTableRegions(this.services.getConnection(), visitor, null);
-    return visitor.getReport();
-  }
-
-  /**
-   * @return Returns last published Report that comes of last successful scan
-   *   of hbase:meta.
-   */
-  public Report getLastReport() {
-    return this.lastReport;
-  }
-
-  /**
-   * If merged region no longer holds reference to the merge regions, archive
-   * merge region on hdfs and perform deleting references in hbase:meta
-   * @return true if we delete references in merged region on hbase:meta and archive
-   *   the files on the file system
-   */
-  private boolean cleanMergeRegion(final RegionInfo mergedRegion, List<RegionInfo> parents)
-      throws IOException {
-    FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
-    Path rootdir = this.services.getMasterFileSystem().getRootDir();
-    Path tabledir = CommonFSUtils.getTableDir(rootdir, mergedRegion.getTable());
-    TableDescriptor htd = getDescriptor(mergedRegion.getTable());
-    HRegionFileSystem regionFs = null;
-    try {
-      regionFs = HRegionFileSystem.openRegionFromFileSystem(
-          this.services.getConfiguration(), fs, tabledir, mergedRegion, true);
-    } catch (IOException e) {
-      LOG.warn("Merged region does not exist: " + mergedRegion.getEncodedName());
-    }
-    if (regionFs == null || !regionFs.hasReferences(htd)) {
-      LOG.debug("Deleting parents ({}) from fs; merged child {} no longer holds references",
-           parents.stream().map(r -> RegionInfo.getShortNameToLog(r)).
-              collect(Collectors.joining(", ")),
-          mergedRegion);
-      ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
-      pe.submitProcedure(new GCMultipleMergedRegionsProcedure(pe.getEnvironment(),
-          mergedRegion,  parents));
-      for (RegionInfo ri:  parents) {
-        // The above scheduled GCMultipleMergedRegionsProcedure does the below.
-        // Do we need this?
-        this.services.getAssignmentManager().getRegionStates().deleteRegion(ri);
-        this.services.getServerManager().removeRegion(ri);
-      }
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Compare HRegionInfos in a way that has split parents sort BEFORE their daughters.
-   */
-  static class SplitParentFirstComparator implements Comparator<RegionInfo> {
-    Comparator<byte[]> rowEndKeyComparator = new Bytes.RowEndKeyComparator();
-    @Override
-    public int compare(RegionInfo left, RegionInfo right) {
-      // This comparator differs from the one RegionInfo in that it sorts
-      // parent before daughters.
-      if (left == null) {
-        return -1;
-      }
-      if (right == null) {
-        return 1;
-      }
-      // Same table name.
-      int result = left.getTable().compareTo(right.getTable());
-      if (result != 0) {
-        return result;
-      }
-      // Compare start keys.
-      result = Bytes.compareTo(left.getStartKey(), right.getStartKey());
-      if (result != 0) {
-        return result;
-      }
-      // Compare end keys, but flip the operands so parent comes first
-      result = rowEndKeyComparator.compare(right.getEndKey(), left.getEndKey());
-
-      return result;
-    }
-  }
-
-  /**
-   * If daughters no longer hold reference to the parents, delete the parent.
-   * @param parent RegionInfo of split offlined parent
-   * @param rowContent Content of <code>parent</code> row in
-   * <code>metaRegionName</code>
-   * @return True if we removed <code>parent</code> from meta table and from
-   * the filesystem.
-   */
-  boolean cleanParent(final RegionInfo parent, Result rowContent)
-  throws IOException {
-    // Check whether it is a merged region and if it is clean of references.
-    if (CatalogFamilyFormat.hasMergeRegions(rowContent.rawCells())) {
-      // Wait until clean of merge parent regions first
-      return false;
-    }
-    // Run checks on each daughter split.
-    PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
-    Pair<Boolean, Boolean> a = checkDaughterInFs(parent, daughters.getFirst());
-    Pair<Boolean, Boolean> b = checkDaughterInFs(parent, daughters.getSecond());
-    if (hasNoReferences(a) && hasNoReferences(b)) {
-      String daughterA = daughters.getFirst() != null?
-          daughters.getFirst().getShortNameToLog(): "null";
-      String daughterB = daughters.getSecond() != null?
-          daughters.getSecond().getShortNameToLog(): "null";
-      LOG.debug("Deleting region " + parent.getShortNameToLog() +
-        " because daughters -- " + daughterA + ", " + daughterB +
-        " -- no longer hold references");
-      ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
-      pe.submitProcedure(new GCRegionProcedure(pe.getEnvironment(), parent));
-      // Remove from in-memory states
-      this.services.getAssignmentManager().getRegionStates().deleteRegion(parent);
-      this.services.getServerManager().removeRegion(parent);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * @param p A pair where the first boolean says whether or not the daughter
-   * region directory exists in the filesystem and then the second boolean says
-   * whether the daughter has references to the parent.
-   * @return True the passed <code>p</code> signifies no references.
-   */
-  private boolean hasNoReferences(final Pair<Boolean, Boolean> p) {
-    return !p.getFirst() || !p.getSecond();
-  }
-
-  /**
-   * Checks if a daughter region -- either splitA or splitB -- still holds
-   * references to parent.
-   * @param parent Parent region
-   * @param daughter Daughter region
-   * @return A pair where the first boolean says whether or not the daughter
-   *   region directory exists in the filesystem and then the second boolean says
-   *   whether the daughter has references to the parent.
-   */
-  private Pair<Boolean, Boolean> checkDaughterInFs(final RegionInfo parent,
-    final RegionInfo daughter)
-  throws IOException {
-    if (daughter == null)  {
-      return new Pair<>(Boolean.FALSE, Boolean.FALSE);
-    }
-
-    FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
-    Path rootdir = this.services.getMasterFileSystem().getRootDir();
-    Path tabledir = CommonFSUtils.getTableDir(rootdir, daughter.getTable());
-
-    Path daughterRegionDir = new Path(tabledir, daughter.getEncodedName());
-
-    HRegionFileSystem regionFs;
-
-    try {
-      if (!CommonFSUtils.isExists(fs, daughterRegionDir)) {
-        return new Pair<>(Boolean.FALSE, Boolean.FALSE);
-      }
-    } catch (IOException ioe) {
-      LOG.error("Error trying to determine if daughter region exists, " +
-               "assuming exists and has references", ioe);
-      return new Pair<>(Boolean.TRUE, Boolean.TRUE);
-    }
-
-    boolean references = false;
-    TableDescriptor parentDescriptor = getDescriptor(parent.getTable());
-    try {
-      regionFs = HRegionFileSystem.openRegionFromFileSystem(
-          this.services.getConfiguration(), fs, tabledir, daughter, true);
-
-      for (ColumnFamilyDescriptor family: parentDescriptor.getColumnFamilies()) {
-        if ((references = regionFs.hasReferences(family.getNameAsString()))) {
-          break;
-        }
-      }
-    } catch (IOException e) {
-      LOG.error("Error trying to determine referenced files from : " + daughter.getEncodedName()
-          + ", to: " + parent.getEncodedName() + " assuming has references", e);
-      return new Pair<>(Boolean.TRUE, Boolean.TRUE);
-    }
-    return new Pair<>(Boolean.TRUE, references);
-  }
-
-  private TableDescriptor getDescriptor(final TableName tableName) throws IOException {
-    return this.services.getTableDescriptors().get(tableName);
-  }
-
-  /**
-   * Report made by ReportMakingVisitor
-   */
-  public static class Report {
-    private final long now = EnvironmentEdgeManager.currentTime();
-
-    // Keep Map of found split parents. These are candidates for cleanup.
-    // Use a comparator that has split parents come before its daughters.
-    final Map<RegionInfo, Result> splitParents = new TreeMap<>(new SplitParentFirstComparator());
-    final Map<RegionInfo, Result> mergedRegions = new TreeMap<>(RegionInfo.COMPARATOR);
-    int count = 0;
-
-    private final List<Pair<RegionInfo, RegionInfo>> holes = new ArrayList<>();
-    private final List<Pair<RegionInfo, RegionInfo>> overlaps = new ArrayList<>();
-
-    /**
-     * TODO: If CatalogJanitor finds an 'Unknown Server', it should 'fix' it by queuing
-     * a {@link org.apache.hadoop.hbase.master.procedure.HBCKServerCrashProcedure} for
-     * found server for it to clean up meta.
-     */
-    private final List<Pair<RegionInfo, ServerName>> unknownServers = new ArrayList<>();
-
-    private final List<byte []> emptyRegionInfo = new ArrayList<>();
-
-    @VisibleForTesting
-    Report() {}
-
-    public long getCreateTime() {
-      return this.now;
-    }
-
-    public List<Pair<RegionInfo, RegionInfo>> getHoles() {
-      return this.holes;
-    }
-
-    /**
-     * @return Overlap pairs found as we scanned hbase:meta; ordered by hbase:meta
-     *   table sort. Pairs of overlaps may have overlap with subsequent pairs.
-     * @see MetaFixer#calculateMerges(int, List) where we aggregate overlaps
-     *   for a single 'merge' call.
-     */
-    public List<Pair<RegionInfo, RegionInfo>> getOverlaps() {
-      return this.overlaps;
-    }
-
-    public Map<RegionInfo, Result> getMergedRegions() {
-      return this.mergedRegions;
-    }
-
-    public List<Pair<RegionInfo, ServerName>> getUnknownServers() {
-      return unknownServers;
-    }
-
-    public List<byte[]> getEmptyRegionInfo() {
-      return emptyRegionInfo;
-    }
-
-    /**
-     * @return True if an 'empty' lastReport -- no problems found.
-     */
-    public boolean isEmpty() {
-      return this.holes.isEmpty() && this.overlaps.isEmpty() && this.unknownServers.isEmpty() &&
-          this.emptyRegionInfo.isEmpty();
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder();
-      for (Pair<RegionInfo, RegionInfo> p: this.holes) {
-        if (sb.length() > 0) {
-          sb.append(", ");
-        }
-        sb.append("hole=").append(p.getFirst().getRegionNameAsString()).append("/").
-            append(p.getSecond().getRegionNameAsString());
-      }
-      for (Pair<RegionInfo, RegionInfo> p: this.overlaps) {
-        if (sb.length() > 0) {
-          sb.append(", ");
-        }
-        sb.append("overlap=").append(p.getFirst().getRegionNameAsString()).append("/").
-            append(p.getSecond().getRegionNameAsString());
-      }
-      for (byte [] r: this.emptyRegionInfo) {
-        if (sb.length() > 0) {
-          sb.append(", ");
-        }
-        sb.append("empty=").append(Bytes.toStringBinary(r));
-      }
-      for (Pair<RegionInfo, ServerName> p: this.unknownServers) {
-        if (sb.length() > 0) {
-          sb.append(", ");
-        }
-        sb.append("unknown_server=").append(p.getSecond()).append("/").
-            append(p.getFirst().getRegionNameAsString());
-      }
-      return sb.toString();
-    }
-  }
-
-  /**
-   * Visitor we use in here in CatalogJanitor to go against hbase:meta table.
-   * Generates a Report made of a collection of split parents and counts of rows
-   * in the hbase:meta table. Also runs hbase:meta consistency checks to
-   * generate more report. Report is NOT ready until after this visitor has been
-   * {@link #close()}'d.
-   */
-  static class ReportMakingVisitor implements ClientMetaTableAccessor.CloseableVisitor {
-    private final MasterServices services;
-    private volatile boolean closed;
-
-    /**
-     * Report is not done until after the close has been called.
-     * @see #close()
-     * @see #getReport()
-     */
-    private Report report = new Report();
-
-    /**
-     * RegionInfo from previous row.
-     */
-    private RegionInfo previous = null;
-
-    /**
-     * Keep account of the highest end key seen as we move through hbase:meta.
-     * Usually, the current RegionInfo has the highest end key but if an overlap,
-     * this may no longer hold. An overlap may be a region with startkey 'd' and
-     * endkey 'g'. The next region in meta may be 'e' to 'f' and then 'f' to 'g'.
-     * Looking at previous and current meta row, we won't know about the 'd' to 'g'
-     * overlap unless we keep a running 'highest-endpoint-seen'.
-     */
-    private RegionInfo highestEndKeyRegionInfo = null;
-
-    ReportMakingVisitor(MasterServices services) {
-      this.services = services;
-    }
-
-    /**
-     * Do not call until after {@link #close()}.
-     * Will throw a {@link RuntimeException} if you do.
-     */
-    Report getReport() {
-      if (!this.closed) {
-        throw new RuntimeException("Report not ready until after close()");
-      }
-      return this.report;
-    }
-
-    @Override
-    public boolean visit(Result r) {
-      if (r == null || r.isEmpty()) {
-        return true;
-      }
-      this.report.count++;
-      RegionInfo regionInfo = null;
-      try {
-        regionInfo = metaTableConsistencyCheck(r);
-      } catch(Throwable t) {
-        LOG.warn("Failed consistency check on {}", Bytes.toStringBinary(r.getRow()), t);
-      }
-      if (regionInfo != null) {
-        LOG.trace(regionInfo.toString());
-        if (regionInfo.isSplitParent()) { // splitParent means split and offline.
-          this.report.splitParents.put(regionInfo, r);
-        }
-        if (CatalogFamilyFormat.hasMergeRegions(r.rawCells())) {
-          this.report.mergedRegions.put(regionInfo, r);
-        }
-      }
-      // Returning true means "keep scanning"
-      return true;
-    }
-
-    /**
-     * Check row.
-     * @param metaTableRow Row from hbase:meta table.
-     * @return Returns default regioninfo found in row parse as a convenience to save
-     *   on having to do a double-parse of Result.
-     */
-    private RegionInfo metaTableConsistencyCheck(Result metaTableRow) {
-      RegionInfo ri;
-      // Locations comes back null if the RegionInfo field is empty.
-      // If locations is null, ensure the regioninfo is for sure empty before progressing.
-      // If really empty, report as missing regioninfo!  Otherwise, can run server check
-      // and get RegionInfo from locations.
-      RegionLocations locations = CatalogFamilyFormat.getRegionLocations(metaTableRow);
-      if (locations == null) {
-        ri = CatalogFamilyFormat.getRegionInfo(metaTableRow,
-            HConstants.REGIONINFO_QUALIFIER);
-      } else {
-        ri = locations.getDefaultRegionLocation().getRegion();
-        checkServer(locations);
-      }
-
-      if (ri == null) {
-        this.report.emptyRegionInfo.add(metaTableRow.getRow());
-        return ri;
-      }
-
-      if (!Bytes.equals(metaTableRow.getRow(), ri.getRegionName())) {
-        LOG.warn("INCONSISTENCY: Row name is not equal to serialized info:regioninfo content; " +
-                "row={} {}; See if RegionInfo is referenced in another hbase:meta row? Delete?",
-            Bytes.toStringBinary(metaTableRow.getRow()), ri.getRegionNameAsString());
-        return null;
-      }
-      // Skip split parent region
-      if (ri.isSplitParent()) {
-        return ri;
-      }
-      // If table is disabled, skip integrity check.
-      if (!isTableDisabled(ri)) {
-        if (isTableTransition(ri)) {
-          // HBCK1 used to have a special category for missing start or end keys.
-          // We'll just lump them in as 'holes'.
-
-          // This is a table transition. If this region is not first region, report a hole.
-          if (!ri.isFirst()) {
-            addHole(RegionInfo.UNDEFINED, ri);
-          }
-          // This is a table transition. If last region was not last region of previous table,
-          // report a hole
-          if (this.previous != null && !this.previous.isLast()) {
-            addHole(this.previous, RegionInfo.UNDEFINED);
-          }
-        } else {
-          if (!this.previous.isNext(ri)) {
-            if (this.previous.isOverlap(ri)) {
-              addOverlap(this.previous, ri);
-            } else if (ri.isOverlap(this.highestEndKeyRegionInfo)) {
-              // We may have seen a region a few rows back that overlaps this one.
-              addOverlap(this.highestEndKeyRegionInfo, ri);
-            } else if (!this.highestEndKeyRegionInfo.isNext(ri)) {
-              // Need to check the case if this.highestEndKeyRegionInfo.isNext(ri). If no,
-              // report a hole, otherwise, it is ok. For an example,
-              // previous: [aa, bb), ri: [cc, dd), highestEndKeyRegionInfo: [a, cc)
-              // In this case, it should not report a hole, as highestEndKeyRegionInfo covers
-              // the hole between previous and ri.
-              addHole(this.previous, ri);
-            }
-          } else if (ri.isOverlap(this.highestEndKeyRegionInfo)) {
-            // We may have seen a region a few rows back that overlaps this one
-            // even though it properly 'follows' the region just before.
-            addOverlap(this.highestEndKeyRegionInfo, ri);
-          }
-        }
-      }
-      this.previous = ri;
-      this.highestEndKeyRegionInfo =
-          MetaFixer.getRegionInfoWithLargestEndKey(this.highestEndKeyRegionInfo, ri);
-      return ri;
-    }
-
-    private void addOverlap(RegionInfo a, RegionInfo b) {
-      this.report.overlaps.add(new Pair<>(a, b));
-    }
-
-    private void addHole(RegionInfo a, RegionInfo b) {
-      this.report.holes.add(new Pair<>(a, b));
-    }
-
-    /**
-     * @return True if table is disabled or disabling; defaults false!
-     */
-    boolean isTableDisabled(RegionInfo ri) {
-      if (ri == null) {
-        return false;
-      }
-      if (this.services == null) {
-        return false;
-      }
-      if (this.services.getTableStateManager() == null) {
-        return false;
-      }
-      TableState state = null;
-      try {
-        state = this.services.getTableStateManager().getTableState(ri.getTable());
-      } catch (IOException e) {
-        LOG.warn("Failed getting table state", e);
-      }
-      return state != null && state.isDisabledOrDisabling();
-    }
-
-    /**
-     * Run through referenced servers and save off unknown and the dead.
-     */
-    private void checkServer(RegionLocations locations) {
-      if (this.services == null) {
-        // Can't do this test if no services.
-        return;
-      }
-      if (locations == null) {
-        return;
-      }
-      if (locations.getRegionLocations() == null) {
-        return;
-      }
-      // Check referenced servers are known/online. Here we are looking
-      // at both the default replica -- the main replica -- and then replica
-      // locations too.
-      for (HRegionLocation location: locations.getRegionLocations()) {
-        if (location == null) {
-          continue;
-        }
-        ServerName sn = location.getServerName();
-        if (sn == null) {
-          continue;
-        }
-        if (location.getRegion() == null) {
-          LOG.warn("Empty RegionInfo in {}", location);
-          // This should never happen but if it does, will mess up below.
-          continue;
-        }
-        RegionInfo ri = location.getRegion();
-        // Skip split parent region
-        if (ri.isSplitParent()) {
-          continue;
-        }
-        // skip the offline regions which belong to disabled table.
-        if (isTableDisabled(ri)) {
-          continue;
-        }
-        RegionState rs = this.services.getAssignmentManager().getRegionStates().getRegionState(ri);
-        if (rs == null || rs.isClosedOrAbnormallyClosed()) {
-          // If closed against an 'Unknown Server', that is should be fine.
-          continue;
-        }
-        ServerManager.ServerLiveState state = this.services.getServerManager().
-            isServerKnownAndOnline(sn);
-        switch (state) {
-          case UNKNOWN:
-            this.report.unknownServers.add(new Pair<>(ri, sn));
-            break;
-
-          default:
-            break;
-        }
-      }
-    }
-
-    /**
-     * @return True iff first row in hbase:meta or if we've broached a new table in hbase:meta
-     */
-    private boolean isTableTransition(RegionInfo ri) {
-      return this.previous == null ||
-          !this.previous.getTable().equals(ri.getTable());
-    }
-
-    @Override
-    public void close() throws IOException {
-      // This is a table transition... after the last region. Check previous.
-      // Should be last region. If not, its a hole on end of laster table.
-      if (this.previous != null && !this.previous.isLast()) {
-        addHole(this.previous, RegionInfo.UNDEFINED);
-      }
-      this.closed = true;
-    }
-  }
-
-  private static void checkLog4jProperties() {
-    String filename = "log4j.properties";
-    try {
-      final InputStream inStream =
-          CatalogJanitor.class.getClassLoader().getResourceAsStream(filename);
-      if (inStream != null) {
-        new Properties().load(inStream);
-      } else {
-        System.out.println("No " + filename + " on classpath; Add one else no logging output!");
-      }
-    } catch (IOException e) {
-      LOG.error("Log4j check failed", e);
-    }
-  }
-
-  /**
-   * For testing against a cluster.
-   * Doesn't have a MasterServices context so does not report on good vs bad servers.
-   */
-  public static void main(String [] args) throws IOException {
-    checkLog4jProperties();
-    ReportMakingVisitor visitor = new ReportMakingVisitor(null);
-    Configuration configuration = HBaseConfiguration.create();
-    configuration.setBoolean("hbase.defaults.for.version.skip", true);
-    try (Connection connection = ConnectionFactory.createConnection(configuration)) {
-      /* Used to generate an overlap.
-      */
-      Get g = new Get(Bytes.toBytes("t2,40,1564119846424.1db8c57d64e0733e0f027aaeae7a0bf0."));
-      g.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-      try (Table t = connection.getTable(TableName.META_TABLE_NAME)) {
-        Result r = t.get(g);
-        byte [] row = g.getRow();
-        row[row.length - 2] <<= row[row.length - 2];
-        Put p = new Put(g.getRow());
-        p.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
-            r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
-        t.put(p);
-      }
-      MetaTableAccessor.scanMetaForTableRegions(connection, visitor, null);
-      Report report = visitor.getReport();
-      LOG.info(report != null? report.toString(): "empty");
-    }
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 022f292..520d9d5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -112,6 +112,7 @@ import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
 import org.apache.hadoop.hbase.master.cleaner.SnapshotCleanerChore;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -1384,8 +1385,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   boolean isCatalogJanitorEnabled() {
-    return catalogJanitorChore != null ?
-      catalogJanitorChore.getEnabled() : false;
+    return catalogJanitorChore != null ? catalogJanitorChore.getEnabled() : false;
   }
 
   boolean isCleanerChoreEnabled() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index dde3f29..46c887b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
+import org.apache.hadoop.hbase.master.janitor.MetaFixer;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 02e9dce..908d212 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java
new file mode 100644
index 0000000..6123f6b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.janitor;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CatalogFamilyFormat;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.GCMultipleMergedRegionsProcedure;
+import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * A janitor for the catalog tables. Scans the <code>hbase:meta</code> catalog table on a period.
+ * Makes a lastReport on state of hbase:meta. Looks for unused regions to garbage collect. Scan of
+ * hbase:meta runs if we are NOT in maintenance mode, if we are NOT shutting down, AND if the
+ * assignmentmanager is loaded. Playing it safe, we will garbage collect no-longer needed region
+ * references only if there are no regions-in-transition (RIT).
+ */
+// TODO: Only works with single hbase:meta region currently. Fix.
+// TODO: Should it start over every time? Could it continue if runs into problem? Only if
+// problem does not mess up 'results'.
+// TODO: Do more by way of 'repair'; see note on unknownServers below.
+@InterfaceAudience.Private
+public class CatalogJanitor extends ScheduledChore {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogJanitor.class.getName());
+
+  private final AtomicBoolean alreadyRunning = new AtomicBoolean(false);
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private final MasterServices services;
+
+  /**
+   * Saved report from last hbase:meta scan to completion. May be stale if having trouble completing
+   * scan. Check its date.
+   */
+  private volatile Report lastReport;
+
+  public CatalogJanitor(final MasterServices services) {
+    super("CatalogJanitor-" + services.getServerName().toShortString(), services,
+      services.getConfiguration().getInt("hbase.catalogjanitor.interval", 300000));
+    this.services = services;
+  }
+
+  @Override
+  protected boolean initialChore() {
+    try {
+      if (getEnabled()) {
+        scan();
+      }
+    } catch (IOException e) {
+      LOG.warn("Failed initial janitorial scan of hbase:meta table", e);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    boolean alreadyEnabled = this.enabled.getAndSet(enabled);
+    // If disabling is requested on an already enabled chore, we could have an active
+    // scan still going on, callers might not be aware of that and do further action thinkng
+    // that no action would be from this chore. In this case, the right action is to wait for
+    // the active scan to complete before exiting this function.
+    if (!enabled && alreadyEnabled) {
+      while (alreadyRunning.get()) {
+        Threads.sleepWithoutInterrupt(100);
+      }
+    }
+    return alreadyEnabled;
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @Override
+  protected void chore() {
+    try {
+      AssignmentManager am = this.services.getAssignmentManager();
+      if (getEnabled() && !this.services.isInMaintenanceMode() &&
+        !this.services.getServerManager().isClusterShutdown() && isMetaLoaded(am)) {
+        scan();
+      } else {
+        LOG.warn("CatalogJanitor is disabled! Enabled=" + getEnabled() + ", maintenanceMode=" +
+          this.services.isInMaintenanceMode() + ", am=" + am + ", metaLoaded=" + isMetaLoaded(am) +
+          ", hasRIT=" + isRIT(am) + " clusterShutDown=" +
+          this.services.getServerManager().isClusterShutdown());
+      }
+    } catch (IOException e) {
+      LOG.warn("Failed janitorial scan of hbase:meta table", e);
+    }
+  }
+
+  private static boolean isMetaLoaded(AssignmentManager am) {
+    return am != null && am.isMetaLoaded();
+  }
+
+  private static boolean isRIT(AssignmentManager am) {
+    return isMetaLoaded(am) && am.hasRegionsInTransition();
+  }
+
+  /**
+   * Run janitorial scan of catalog <code>hbase:meta</code> table looking for garbage to collect.
+   * @return How many items gc'd whether for merge or split. Returns -1 if previous scan is in
+   *         progress.
+   */
+  public int scan() throws IOException {
+    int gcs = 0;
+    try {
+      if (!alreadyRunning.compareAndSet(false, true)) {
+        LOG.debug("CatalogJanitor already running");
+        // -1 indicates previous scan is in progress
+        return -1;
+      }
+      this.lastReport = scanForReport();
+      if (!this.lastReport.isEmpty()) {
+        LOG.warn(this.lastReport.toString());
+      }
+
+      if (isRIT(this.services.getAssignmentManager())) {
+        LOG.warn("Playing-it-safe skipping merge/split gc'ing of regions from hbase:meta while " +
+          "regions-in-transition (RIT)");
+      }
+      Map<RegionInfo, Result> mergedRegions = this.lastReport.mergedRegions;
+      for (Map.Entry<RegionInfo, Result> e : mergedRegions.entrySet()) {
+        if (this.services.isInMaintenanceMode()) {
+          // Stop cleaning if the master is in maintenance mode
+          break;
+        }
+
+        List<RegionInfo> parents = CatalogFamilyFormat.getMergeRegions(e.getValue().rawCells());
+        if (parents != null && cleanMergeRegion(e.getKey(), parents)) {
+          gcs++;
+        }
+      }
+      // Clean split parents
+      Map<RegionInfo, Result> splitParents = this.lastReport.splitParents;
+
+      // Now work on our list of found parents. See if any we can clean up.
+      HashSet<String> parentNotCleaned = new HashSet<>();
+      for (Map.Entry<RegionInfo, Result> e : splitParents.entrySet()) {
+        if (this.services.isInMaintenanceMode()) {
+          // Stop cleaning if the master is in maintenance mode
+          break;
+        }
+
+        if (!parentNotCleaned.contains(e.getKey().getEncodedName()) &&
+          cleanParent(e.getKey(), e.getValue())) {
+          gcs++;
+        } else {
+          // We could not clean the parent, so it's daughters should not be
+          // cleaned either (HBASE-6160)
+          PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(e.getValue());
+          parentNotCleaned.add(daughters.getFirst().getEncodedName());
+          parentNotCleaned.add(daughters.getSecond().getEncodedName());
+        }
+      }
+      return gcs;
+    } finally {
+      alreadyRunning.set(false);
+    }
+  }
+
+  /**
+   * Scan hbase:meta.
+   * @return Return generated {@link Report}
+   */
+  // will be override in tests.
+  @VisibleForTesting
+  protected Report scanForReport() throws IOException {
+    ReportMakingVisitor visitor = new ReportMakingVisitor(this.services);
+    // Null tablename means scan all of meta.
+    MetaTableAccessor.scanMetaForTableRegions(this.services.getConnection(), visitor, null);
+    return visitor.getReport();
+  }
+
+  /**
+   * @return Returns last published Report that comes of last successful scan of hbase:meta.
+   */
+  public Report getLastReport() {
+    return this.lastReport;
+  }
+
+  /**
+   * If merged region no longer holds reference to the merge regions, archive merge region on hdfs
+   * and perform deleting references in hbase:meta
+   * @return true if we delete references in merged region on hbase:meta and archive the files on
+   *         the file system
+   */
+  private boolean cleanMergeRegion(final RegionInfo mergedRegion, List<RegionInfo> parents)
+    throws IOException {
+    FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
+    Path rootdir = this.services.getMasterFileSystem().getRootDir();
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, mergedRegion.getTable());
+    TableDescriptor htd = getDescriptor(mergedRegion.getTable());
+    HRegionFileSystem regionFs = null;
+    try {
+      regionFs = HRegionFileSystem.openRegionFromFileSystem(this.services.getConfiguration(), fs,
+        tabledir, mergedRegion, true);
+    } catch (IOException e) {
+      LOG.warn("Merged region does not exist: " + mergedRegion.getEncodedName());
+    }
+    if (regionFs == null || !regionFs.hasReferences(htd)) {
+      LOG.debug(
+        "Deleting parents ({}) from fs; merged child {} no longer holds references", parents
+          .stream().map(r -> RegionInfo.getShortNameToLog(r)).collect(Collectors.joining(", ")),
+        mergedRegion);
+      ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
+      pe.submitProcedure(
+        new GCMultipleMergedRegionsProcedure(pe.getEnvironment(), mergedRegion, parents));
+      for (RegionInfo ri : parents) {
+        // The above scheduled GCMultipleMergedRegionsProcedure does the below.
+        // Do we need this?
+        this.services.getAssignmentManager().getRegionStates().deleteRegion(ri);
+        this.services.getServerManager().removeRegion(ri);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Compare HRegionInfos in a way that has split parents sort BEFORE their daughters.
+   */
+  static class SplitParentFirstComparator implements Comparator<RegionInfo> {
+    Comparator<byte[]> rowEndKeyComparator = new Bytes.RowEndKeyComparator();
+
+    @Override
+    public int compare(RegionInfo left, RegionInfo right) {
+      // This comparator differs from the one RegionInfo in that it sorts
+      // parent before daughters.
+      if (left == null) {
+        return -1;
+      }
+      if (right == null) {
+        return 1;
+      }
+      // Same table name.
+      int result = left.getTable().compareTo(right.getTable());
+      if (result != 0) {
+        return result;
+      }
+      // Compare start keys.
+      result = Bytes.compareTo(left.getStartKey(), right.getStartKey());
+      if (result != 0) {
+        return result;
+      }
+      // Compare end keys, but flip the operands so parent comes first
+      result = rowEndKeyComparator.compare(right.getEndKey(), left.getEndKey());
+
+      return result;
+    }
+  }
+
+  @VisibleForTesting
+  static boolean cleanParent(MasterServices services, RegionInfo parent, Result rowContent)
+    throws IOException {
+    // Check whether it is a merged region and if it is clean of references.
+    if (CatalogFamilyFormat.hasMergeRegions(rowContent.rawCells())) {
+      // Wait until clean of merge parent regions first
+      return false;
+    }
+    // Run checks on each daughter split.
+    PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
+    Pair<Boolean, Boolean> a = checkDaughterInFs(services, parent, daughters.getFirst());
+    Pair<Boolean, Boolean> b = checkDaughterInFs(services, parent, daughters.getSecond());
+    if (hasNoReferences(a) && hasNoReferences(b)) {
+      String daughterA =
+        daughters.getFirst() != null ? daughters.getFirst().getShortNameToLog() : "null";
+      String daughterB =
+        daughters.getSecond() != null ? daughters.getSecond().getShortNameToLog() : "null";
+      LOG.debug("Deleting region " + parent.getShortNameToLog() + " because daughters -- " +
+        daughterA + ", " + daughterB + " -- no longer hold references");
+      ProcedureExecutor<MasterProcedureEnv> pe = services.getMasterProcedureExecutor();
+      pe.submitProcedure(new GCRegionProcedure(pe.getEnvironment(), parent));
+      // Remove from in-memory states
+      services.getAssignmentManager().getRegionStates().deleteRegion(parent);
+      services.getServerManager().removeRegion(parent);
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * If daughters no longer hold reference to the parents, delete the parent.
+   * @param parent RegionInfo of split offlined parent
+   * @param rowContent Content of <code>parent</code> row in <code>metaRegionName</code>
+   * @return True if we removed <code>parent</code> from meta table and from the filesystem.
+   */
+  private boolean cleanParent(final RegionInfo parent, Result rowContent) throws IOException {
+    return cleanParent(services, parent, rowContent);
+  }
+
+  /**
+   * @param p A pair where the first boolean says whether or not the daughter region directory
+   *          exists in the filesystem and then the second boolean says whether the daughter has
+   *          references to the parent.
+   * @return True the passed <code>p</code> signifies no references.
+   */
+  private static boolean hasNoReferences(final Pair<Boolean, Boolean> p) {
+    return !p.getFirst() || !p.getSecond();
+  }
+
+  /**
+   * Checks if a daughter region -- either splitA or splitB -- still holds references to parent.
+   * @param parent Parent region
+   * @param daughter Daughter region
+   * @return A pair where the first boolean says whether or not the daughter region directory exists
+   *         in the filesystem and then the second boolean says whether the daughter has references
+   *         to the parent.
+   */
+  private static Pair<Boolean, Boolean> checkDaughterInFs(MasterServices services,
+    final RegionInfo parent, final RegionInfo daughter) throws IOException {
+    if (daughter == null) {
+      return new Pair<>(Boolean.FALSE, Boolean.FALSE);
+    }
+
+    FileSystem fs = services.getMasterFileSystem().getFileSystem();
+    Path rootdir = services.getMasterFileSystem().getRootDir();
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, daughter.getTable());
+
+    Path daughterRegionDir = new Path(tabledir, daughter.getEncodedName());
+
+    HRegionFileSystem regionFs;
+
+    try {
+      if (!CommonFSUtils.isExists(fs, daughterRegionDir)) {
+        return new Pair<>(Boolean.FALSE, Boolean.FALSE);
+      }
+    } catch (IOException ioe) {
+      LOG.error("Error trying to determine if daughter region exists, " +
+        "assuming exists and has references", ioe);
+      return new Pair<>(Boolean.TRUE, Boolean.TRUE);
+    }
+
+    boolean references = false;
+    TableDescriptor parentDescriptor = services.getTableDescriptors().get(parent.getTable());
+    try {
+      regionFs = HRegionFileSystem.openRegionFromFileSystem(services.getConfiguration(), fs,
+        tabledir, daughter, true);
+
+      for (ColumnFamilyDescriptor family : parentDescriptor.getColumnFamilies()) {
+        references = regionFs.hasReferences(family.getNameAsString());
+        if (references) {
+          break;
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Error trying to determine referenced files from : " + daughter.getEncodedName() +
+        ", to: " + parent.getEncodedName() + " assuming has references", e);
+      return new Pair<>(Boolean.TRUE, Boolean.TRUE);
+    }
+    return new Pair<>(Boolean.TRUE, references);
+  }
+
+  private TableDescriptor getDescriptor(final TableName tableName) throws IOException {
+    return this.services.getTableDescriptors().get(tableName);
+  }
+
+  private static void checkLog4jProperties() {
+    String filename = "log4j.properties";
+    try {
+      final InputStream inStream =
+        CatalogJanitor.class.getClassLoader().getResourceAsStream(filename);
+      if (inStream != null) {
+        new Properties().load(inStream);
+      } else {
+        System.out.println("No " + filename + " on classpath; Add one else no logging output!");
+      }
+    } catch (IOException e) {
+      LOG.error("Log4j check failed", e);
+    }
+  }
+
+  /**
+   * For testing against a cluster. Doesn't have a MasterServices context so does not report on good
+   * vs bad servers.
+   */
+  public static void main(String[] args) throws IOException {
+    checkLog4jProperties();
+    ReportMakingVisitor visitor = new ReportMakingVisitor(null);
+    Configuration configuration = HBaseConfiguration.create();
+    configuration.setBoolean("hbase.defaults.for.version.skip", true);
+    try (Connection connection = ConnectionFactory.createConnection(configuration)) {
+      /*
+       * Used to generate an overlap.
+       */
+      Get g = new Get(Bytes.toBytes("t2,40,1564119846424.1db8c57d64e0733e0f027aaeae7a0bf0."));
+      g.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+      try (Table t = connection.getTable(TableName.META_TABLE_NAME)) {
+        Result r = t.get(g);
+        byte[] row = g.getRow();
+        row[row.length - 2] <<= row[row.length - 2];
+        Put p = new Put(g.getRow());
+        p.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
+          r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
+        t.put(p);
+      }
+      MetaTableAccessor.scanMetaForTableRegions(connection, visitor, null);
+      Report report = visitor.getReport();
+      LOG.info(report != null ? report.toString() : "empty");
+    }
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java
similarity index 96%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java
index ed1fd27..6f51627 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.master.janitor;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -37,16 +37,18 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
-import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
 
 
 /**
@@ -57,7 +59,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  * and encapsulates their fixing on behalf of the Master.
  */
 @InterfaceAudience.Private
-class MetaFixer {
+public class MetaFixer {
   private static final Logger LOG = LoggerFactory.getLogger(MetaFixer.class);
   private static final String MAX_MERGE_COUNT_KEY = "hbase.master.metafixer.max.merge.count";
   private static final int MAX_MERGE_COUNT_DEFAULT = 64;
@@ -68,14 +70,14 @@ class MetaFixer {
    */
   private final int maxMergeCount;
 
-  MetaFixer(MasterServices masterServices) {
+  public MetaFixer(MasterServices masterServices) {
     this.masterServices = masterServices;
     this.maxMergeCount = this.masterServices.getConfiguration().
       getInt(MAX_MERGE_COUNT_KEY, MAX_MERGE_COUNT_DEFAULT);
   }
 
-  void fix() throws IOException {
-    CatalogJanitor.Report report = this.masterServices.getCatalogJanitor().getLastReport();
+  public void fix() throws IOException {
+    Report report = this.masterServices.getCatalogJanitor().getLastReport();
     if (report == null) {
       LOG.info("CatalogJanitor has not generated a report yet; run 'catalogjanitor_run' in " +
           "shell or wait until CatalogJanitor chore runs.");
@@ -92,7 +94,7 @@ class MetaFixer {
    * If hole, it papers it over by adding a region in the filesystem and to hbase:meta.
    * Does not assign.
    */
-  void fixHoles(CatalogJanitor.Report report) {
+  void fixHoles(Report report) {
     final List<Pair<RegionInfo, RegionInfo>> holes = report.getHoles();
     if (holes.isEmpty()) {
       LOG.info("CatalogJanitor Report contains no holes to fix. Skipping.");
@@ -146,8 +148,8 @@ class MetaFixer {
       return Optional.of(buildRegionInfo(left.getTable(), left.getEndKey(), right.getStartKey()));
     }
 
-    final boolean leftUndefined = left.equals(RegionInfo.UNDEFINED);
-    final boolean rightUndefined = right.equals(RegionInfo.UNDEFINED);
+    final boolean leftUndefined = left.equals(RegionInfoBuilder.UNDEFINED);
+    final boolean rightUndefined = right.equals(RegionInfoBuilder.UNDEFINED);
     final boolean last = left.isLast();
     final boolean first = right.isFirst();
     if (leftUndefined && rightUndefined) {
@@ -235,7 +237,7 @@ class MetaFixer {
   /**
    * Fix overlaps noted in CJ consistency report.
    */
-  List<Long> fixOverlaps(CatalogJanitor.Report report) throws IOException {
+  List<Long> fixOverlaps(Report report) throws IOException {
     List<Long> pidList = new ArrayList<>();
     for (Set<RegionInfo> regions: calculateMerges(maxMergeCount, report.getOverlaps())) {
       RegionInfo [] regionsArray = regions.toArray(new RegionInfo [] {});
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/Report.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/Report.java
new file mode 100644
index 0000000..d38a099
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/Report.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.janitor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor.SplitParentFirstComparator;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Report made by ReportMakingVisitor
+ */
+@InterfaceAudience.Private
+public class Report {
+  private final long now = EnvironmentEdgeManager.currentTime();
+
+  // Keep Map of found split parents. These are candidates for cleanup.
+  // Use a comparator that has split parents come before its daughters.
+  final Map<RegionInfo, Result> splitParents = new TreeMap<>(new SplitParentFirstComparator());
+  final Map<RegionInfo, Result> mergedRegions = new TreeMap<>(RegionInfo.COMPARATOR);
+  int count = 0;
+
+  final List<Pair<RegionInfo, RegionInfo>> holes = new ArrayList<>();
+  final List<Pair<RegionInfo, RegionInfo>> overlaps = new ArrayList<>();
+
+  /**
+   * TODO: If CatalogJanitor finds an 'Unknown Server', it should 'fix' it by queuing a
+   * {@link org.apache.hadoop.hbase.master.procedure.HBCKServerCrashProcedure} for found server for
+   * it to clean up meta.
+   */
+  final List<Pair<RegionInfo, ServerName>> unknownServers = new ArrayList<>();
+
+  final List<byte[]> emptyRegionInfo = new ArrayList<>();
+
+  public long getCreateTime() {
+    return this.now;
+  }
+
+  public List<Pair<RegionInfo, RegionInfo>> getHoles() {
+    return this.holes;
+  }
+
+  /**
+   * @return Overlap pairs found as we scanned hbase:meta; ordered by hbase:meta table sort. Pairs
+   *         of overlaps may have overlap with subsequent pairs.
+   * @see MetaFixer#calculateMerges(int, List) where we aggregate overlaps for a single 'merge'
+   *      call.
+   */
+  public List<Pair<RegionInfo, RegionInfo>> getOverlaps() {
+    return this.overlaps;
+  }
+
+  public Map<RegionInfo, Result> getMergedRegions() {
+    return this.mergedRegions;
+  }
+
+  public List<Pair<RegionInfo, ServerName>> getUnknownServers() {
+    return unknownServers;
+  }
+
+  public List<byte[]> getEmptyRegionInfo() {
+    return emptyRegionInfo;
+  }
+
+  /**
+   * @return True if an 'empty' lastReport -- no problems found.
+   */
+  public boolean isEmpty() {
+    return this.holes.isEmpty() && this.overlaps.isEmpty() && this.unknownServers.isEmpty() &&
+      this.emptyRegionInfo.isEmpty();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for (Pair<RegionInfo, RegionInfo> p : this.holes) {
+      if (sb.length() > 0) {
+        sb.append(", ");
+      }
+      sb.append("hole=").append(p.getFirst().getRegionNameAsString()).append("/")
+        .append(p.getSecond().getRegionNameAsString());
+    }
+    for (Pair<RegionInfo, RegionInfo> p : this.overlaps) {
+      if (sb.length() > 0) {
+        sb.append(", ");
+      }
+      sb.append("overlap=").append(p.getFirst().getRegionNameAsString()).append("/")
+        .append(p.getSecond().getRegionNameAsString());
+    }
+    for (byte[] r : this.emptyRegionInfo) {
+      if (sb.length() > 0) {
+        sb.append(", ");
+      }
+      sb.append("empty=").append(Bytes.toStringBinary(r));
+    }
+    for (Pair<RegionInfo, ServerName> p : this.unknownServers) {
+      if (sb.length() > 0) {
+        sb.append(", ");
+      }
+      sb.append("unknown_server=").append(p.getSecond()).append("/")
+        .append(p.getFirst().getRegionNameAsString());
+    }
+    return sb.toString();
+  }
+}
\ No newline at end of file
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java
new file mode 100644
index 0000000..4dd514e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.janitor;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.CatalogFamilyFormat;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Visitor we use in here in CatalogJanitor to go against hbase:meta table. Generates a Report made
+ * of a collection of split parents and counts of rows in the hbase:meta table. Also runs hbase:meta
+ * consistency checks to generate more report. Report is NOT ready until after this visitor has been
+ * {@link #close()}'d.
+ */
+@InterfaceAudience.Private
+class ReportMakingVisitor implements ClientMetaTableAccessor.CloseableVisitor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReportMakingVisitor.class);
+
+  private final MasterServices services;
+  private volatile boolean closed;
+
+  /**
+   * Report is not done until after the close has been called.
+   */
+  private Report report = new Report();
+
+  /**
+   * RegionInfo from previous row.
+   */
+  private RegionInfo previous = null;
+
+  /**
+   * Keep account of the highest end key seen as we move through hbase:meta. Usually, the current
+   * RegionInfo has the highest end key but if an overlap, this may no longer hold. An overlap may
+   * be a region with startkey 'd' and endkey 'g'. The next region in meta may be 'e' to 'f' and
+   * then 'f' to 'g'. Looking at previous and current meta row, we won't know about the 'd' to 'g'
+   * overlap unless we keep a running 'highest-endpoint-seen'.
+   */
+  private RegionInfo highestEndKeyRegionInfo = null;
+
+  ReportMakingVisitor(MasterServices services) {
+    this.services = services;
+  }
+
+  /**
+   * Do not call until after {@link #close()}. Will throw a {@link RuntimeException} if you do.
+   */
+  Report getReport() {
+    if (!this.closed) {
+      throw new RuntimeException("Report not ready until after close()");
+    }
+    return this.report;
+  }
+
+  @Override
+  public boolean visit(Result r) {
+    if (r == null || r.isEmpty()) {
+      return true;
+    }
+    this.report.count++;
+    RegionInfo regionInfo = null;
+    try {
+      regionInfo = metaTableConsistencyCheck(r);
+    } catch (Throwable t) {
+      LOG.warn("Failed consistency check on {}", Bytes.toStringBinary(r.getRow()), t);
+    }
+    if (regionInfo != null) {
+      LOG.trace(regionInfo.toString());
+      if (regionInfo.isSplitParent()) { // splitParent means split and offline.
+        this.report.splitParents.put(regionInfo, r);
+      }
+      if (CatalogFamilyFormat.hasMergeRegions(r.rawCells())) {
+        this.report.mergedRegions.put(regionInfo, r);
+      }
+    }
+    // Returning true means "keep scanning"
+    return true;
+  }
+
+  /**
+   * Check row.
+   * @param metaTableRow Row from hbase:meta table.
+   * @return Returns default regioninfo found in row parse as a convenience to save on having to do
+   *         a double-parse of Result.
+   */
+  private RegionInfo metaTableConsistencyCheck(Result metaTableRow) {
+    RegionInfo ri;
+    // Locations comes back null if the RegionInfo field is empty.
+    // If locations is null, ensure the regioninfo is for sure empty before progressing.
+    // If really empty, report as missing regioninfo! Otherwise, can run server check
+    // and get RegionInfo from locations.
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(metaTableRow);
+    if (locations == null) {
+      ri = CatalogFamilyFormat.getRegionInfo(metaTableRow, HConstants.REGIONINFO_QUALIFIER);
+    } else {
+      ri = locations.getDefaultRegionLocation().getRegion();
+      checkServer(locations);
+    }
+
+    if (ri == null) {
+      this.report.emptyRegionInfo.add(metaTableRow.getRow());
+      return ri;
+    }
+
+    if (!Bytes.equals(metaTableRow.getRow(), ri.getRegionName())) {
+      LOG.warn(
+        "INCONSISTENCY: Row name is not equal to serialized info:regioninfo content; " +
+          "row={} {}; See if RegionInfo is referenced in another hbase:meta row? Delete?",
+        Bytes.toStringBinary(metaTableRow.getRow()), ri.getRegionNameAsString());
+      return null;
+    }
+    // Skip split parent region
+    if (ri.isSplitParent()) {
+      return ri;
+    }
+    // If table is disabled, skip integrity check.
+    if (!isTableDisabled(ri)) {
+      if (isTableTransition(ri)) {
+        // HBCK1 used to have a special category for missing start or end keys.
+        // We'll just lump them in as 'holes'.
+
+        // This is a table transition. If this region is not first region, report a hole.
+        if (!ri.isFirst()) {
+          addHole(RegionInfoBuilder.UNDEFINED, ri);
+        }
+        // This is a table transition. If last region was not last region of previous table,
+        // report a hole
+        if (this.previous != null && !this.previous.isLast()) {
+          addHole(this.previous, RegionInfoBuilder.UNDEFINED);
+        }
+      } else {
+        if (!this.previous.isNext(ri)) {
+          if (this.previous.isOverlap(ri)) {
+            addOverlap(this.previous, ri);
+          } else if (ri.isOverlap(this.highestEndKeyRegionInfo)) {
+            // We may have seen a region a few rows back that overlaps this one.
+            addOverlap(this.highestEndKeyRegionInfo, ri);
+          } else if (!this.highestEndKeyRegionInfo.isNext(ri)) {
+            // Need to check the case if this.highestEndKeyRegionInfo.isNext(ri). If no,
+            // report a hole, otherwise, it is ok. For an example,
+            // previous: [aa, bb), ri: [cc, dd), highestEndKeyRegionInfo: [a, cc)
+            // In this case, it should not report a hole, as highestEndKeyRegionInfo covers
+            // the hole between previous and ri.
+            addHole(this.previous, ri);
+          }
+        } else if (ri.isOverlap(this.highestEndKeyRegionInfo)) {
+          // We may have seen a region a few rows back that overlaps this one
+          // even though it properly 'follows' the region just before.
+          addOverlap(this.highestEndKeyRegionInfo, ri);
+        }
+      }
+    }
+    this.previous = ri;
+    this.highestEndKeyRegionInfo =
+      MetaFixer.getRegionInfoWithLargestEndKey(this.highestEndKeyRegionInfo, ri);
+    return ri;
+  }
+
+  private void addOverlap(RegionInfo a, RegionInfo b) {
+    this.report.overlaps.add(new Pair<>(a, b));
+  }
+
+  private void addHole(RegionInfo a, RegionInfo b) {
+    this.report.holes.add(new Pair<>(a, b));
+  }
+
+  /**
+   * @return True if table is disabled or disabling; defaults false!
+   */
+  boolean isTableDisabled(RegionInfo ri) {
+    if (ri == null) {
+      return false;
+    }
+    if (this.services == null) {
+      return false;
+    }
+    if (this.services.getTableStateManager() == null) {
+      return false;
+    }
+    TableState state = null;
+    try {
+      state = this.services.getTableStateManager().getTableState(ri.getTable());
+    } catch (IOException e) {
+      LOG.warn("Failed getting table state", e);
+    }
+    return state != null && state.isDisabledOrDisabling();
+  }
+
+  /**
+   * Run through referenced servers and save off unknown and the dead.
+   */
+  private void checkServer(RegionLocations locations) {
+    if (this.services == null) {
+      // Can't do this test if no services.
+      return;
+    }
+    if (locations == null) {
+      return;
+    }
+    if (locations.getRegionLocations() == null) {
+      return;
+    }
+    // Check referenced servers are known/online. Here we are looking
+    // at both the default replica -- the main replica -- and then replica
+    // locations too.
+    for (HRegionLocation location : locations.getRegionLocations()) {
+      if (location == null) {
+        continue;
+      }
+      ServerName sn = location.getServerName();
+      if (sn == null) {
+        continue;
+      }
+      if (location.getRegion() == null) {
+        LOG.warn("Empty RegionInfo in {}", location);
+        // This should never happen but if it does, will mess up below.
+        continue;
+      }
+      RegionInfo ri = location.getRegion();
+      // Skip split parent region
+      if (ri.isSplitParent()) {
+        continue;
+      }
+      // skip the offline regions which belong to disabled table.
+      if (isTableDisabled(ri)) {
+        continue;
+      }
+      RegionState rs = this.services.getAssignmentManager().getRegionStates().getRegionState(ri);
+      if (rs == null || rs.isClosedOrAbnormallyClosed()) {
+        // If closed against an 'Unknown Server', that is should be fine.
+        continue;
+      }
+      ServerManager.ServerLiveState state =
+        this.services.getServerManager().isServerKnownAndOnline(sn);
+      switch (state) {
+        case UNKNOWN:
+          this.report.unknownServers.add(new Pair<>(ri, sn));
+          break;
+
+        default:
+          break;
+      }
+    }
+  }
+
+  /**
+   * @return True iff first row in hbase:meta or if we've broached a new table in hbase:meta
+   */
+  private boolean isTableTransition(RegionInfo ri) {
+    return this.previous == null || !this.previous.getTable().equals(ri.getTable());
+  }
+
+  @Override
+  public void close() throws IOException {
+    // This is a table transition... after the last region. Check previous.
+    // Should be last region. If not, its a hole on end of laster table.
+    if (this.previous != null && !this.previous.isLast()) {
+      addHole(this.previous, RegionInfoBuilder.UNDEFINED);
+    }
+    this.closed = true;
+  }
+}
\ No newline at end of file
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp b/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp
index 87d0018..12757c6 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp
@@ -35,8 +35,8 @@
 <%@ page import="org.apache.hadoop.hbase.ServerName" %>
 <%@ page import="org.apache.hadoop.hbase.util.Bytes" %>
 <%@ page import="org.apache.hadoop.hbase.util.Pair" %>
-<%@ page import="org.apache.hadoop.hbase.master.CatalogJanitor" %>
-<%@ page import="org.apache.hadoop.hbase.master.CatalogJanitor.Report" %>
+<%@ page import="org.apache.hadoop.hbase.master.janitor.CatalogJanitor" %>
+<%@ page import="org.apache.hadoop.hbase.master.janitor.Report" %>
 <%
   final String cacheParameterValue = request.getParameter("cache");
   final HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER);
@@ -74,7 +74,7 @@
     ZoneId.systemDefault());
   String iso8601end = startTimestamp == 0? "-1": zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
   CatalogJanitor cj = master.getCatalogJanitor();
-  CatalogJanitor.Report report = cj == null? null: cj.getLastReport();
+  Report report = cj == null? null: cj.getLastReport();
   final ServerManager serverManager = master.getServerManager();
 %>
 <jsp:include page="header.jsp">
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index c5425a2..a0ed836 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.master.CatalogJanitor;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
 import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java
index cc68026..56c1047 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java
@@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.master.CatalogJanitor;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 50851c1..7c65005 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java
index bbf9a0b..5aec49b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
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/janitor/TestCatalogJanitor.java
similarity index 95%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitor.java
index 695b8b9..2522768 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/janitor/TestCatalogJanitor.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.master.janitor;
 
 import static org.apache.hadoop.hbase.util.HFileArchiveTestingUtil.assertArchiveEqualToOriginal;
 import static org.junit.Assert.assertEquals;
@@ -52,8 +52,9 @@ import org.apache.hadoop.hbase.client.Result;
 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.CatalogJanitor.SplitParentFirstComparator;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.MockMasterServices;
+import org.apache.hadoop.hbase.master.janitor.CatalogJanitor.SplitParentFirstComparator;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -83,16 +84,19 @@ public class TestCatalogJanitor {
     HBaseClassTestRule.forClass(TestCatalogJanitor.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestCatalogJanitor.class);
+
+  private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
+
   @Rule
   public final TestName name = new TestName();
-  private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
+
   private MockMasterServices masterServices;
   private CatalogJanitor janitor;
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0,
-      0, null, MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
+    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
+      MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
   }
 
   @Before
@@ -151,12 +155,12 @@ public class TestCatalogJanitor {
     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));
+    assertFalse(CatalogJanitor.cleanParent(masterServices, parent, r));
     ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
     assertTrue(fs.exists(parentdir));
     // Remove the reference file and try again.
     assertTrue(fs.delete(p, true));
-    assertTrue(this.janitor.cleanParent(parent, r));
+    assertTrue(CatalogJanitor.cleanParent(masterServices, parent, r));
     // Parent cleanup is run async as a procedure. Make sure parentdir is removed.
     ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
     assertTrue(!fs.exists(parentdir));
@@ -249,11 +253,11 @@ public class TestCatalogJanitor {
     Path splitaRef =
       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)));
+    assertFalse(CatalogJanitor.cleanParent(masterServices, parent, regions.get(parent)));
 
     // splitba, and split bb, do not have dirs in fs. That means that if
     // we test splitb, it should get cleaned up.
-    assertTrue(janitor.cleanParent(splitb, regions.get(splitb)));
+    assertTrue(CatalogJanitor.cleanParent(masterServices, splitb, regions.get(splitb)));
 
     // Now remove ref from splita to parent... so parent can be let go and so
     // the daughter splita can be split (can't split if still references).
@@ -268,15 +272,15 @@ public class TestCatalogJanitor {
       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)));
+    assertFalse(CatalogJanitor.cleanParent(masterServices, splita, regions.get(splita)));
 
     // Now clean up parent daughter first. Remove references from its daughters.
     assertTrue(fs.delete(splitaaRef, true));
     assertTrue(fs.delete(splitabRef, true));
-    assertTrue(janitor.cleanParent(splita, regions.get(splita)));
+    assertTrue(CatalogJanitor.cleanParent(masterServices, splita, regions.get(splita)));
 
     // Super parent should get cleaned up now both splita and splitb are gone.
-    assertTrue(janitor.cleanParent(parent, regions.get(parent)));
+    assertTrue(CatalogJanitor.cleanParent(masterServices, parent, regions.get(parent)));
   }
 
   /**
@@ -321,7 +325,7 @@ public class TestCatalogJanitor {
     final Map<RegionInfo, Result> mergedRegions = new TreeMap<>();
     CatalogJanitor spy = spy(this.janitor);
 
-    CatalogJanitor.Report report = new CatalogJanitor.Report();
+    Report report = new Report();
     report.count = 10;
     report.mergedRegions.putAll(mergedRegions);
     report.splitParents.putAll(splitParents);
@@ -461,7 +465,7 @@ public class TestCatalogJanitor {
     }
 
     // do the cleaning of the parent
-    assertTrue(janitor.cleanParent(parent, parentMetaRow));
+    assertTrue(CatalogJanitor.cleanParent(masterServices, parent, parentMetaRow));
     Path parentDir = new Path(tabledir, parent.getEncodedName());
     // Cleanup procedure runs async. Wait till it done.
     ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
@@ -516,20 +520,20 @@ public class TestCatalogJanitor {
     CommonFSUtils.setRootDir(fs.getConf(), rootdir);
     Path tabledir = CommonFSUtils.getTableDir(rootdir, parent.getTable());
     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);
+    LOG.info("Old root:" + rootdir);
+    LOG.info("Old table:" + tabledir);
+    LOG.info("Old store:" + storedir);
 
     Path storeArchive = HFileArchiveUtil.getStoreArchivePath(this.masterServices.getConfiguration(),
       parent, tabledir, td.getColumnFamilies()[0].getName());
-    System.out.println("Old archive:" + storeArchive);
+    LOG.info("Old archive:" + storeArchive);
 
     // enable archiving, make sure that files get archived
     addMockStoreFiles(2, this.masterServices, storedir);
     // get the current store files for comparison
     FileStatus[] storeFiles = fs.listStatus(storedir);
     // Do the cleaning of the parent
-    assertTrue(janitor.cleanParent(parent, r));
+    assertTrue(CatalogJanitor.cleanParent(masterServices, parent, r));
     Path parentDir = new Path(tabledir, parent.getEncodedName());
     ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
     assertTrue(!fs.exists(parentDir));
@@ -543,7 +547,7 @@ public class TestCatalogJanitor {
     addMockStoreFiles(2, this.masterServices, storedir);
 
     // Do the cleaning of the parent
-    assertTrue(janitor.cleanParent(parent, r));
+    assertTrue(CatalogJanitor.cleanParent(masterServices, parent, r));
     // Cleanup procedure runs async. Wait till it done.
     ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
     assertTrue(!fs.exists(parentDir));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorCluster.java
similarity index 78%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorCluster.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorCluster.java
index 0d92c7f..3841d8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorCluster.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.master.janitor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -50,12 +50,13 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category({MasterTests.class, LargeTests.class})
+@Category({ MasterTests.class, LargeTests.class })
 public class TestCatalogJanitorCluster {
   private static final Logger LOG = LoggerFactory.getLogger(TestCatalogJanitorCluster.class);
+
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestCatalogJanitorCluster.class);
+    HBaseClassTestRule.forClass(TestCatalogJanitorCluster.class);
 
   @Rule
   public final TestName name = new TestName();
@@ -70,24 +71,16 @@ public class TestCatalogJanitorCluster {
   @Before
   public void before() throws Exception {
     TEST_UTIL.startMiniCluster();
-    TEST_UTIL.createMultiRegionTable(T1, new byte [][] {HConstants.CATALOG_FAMILY});
-    TEST_UTIL.createMultiRegionTable(T2, new byte [][] {HConstants.CATALOG_FAMILY});
-    TEST_UTIL.createMultiRegionTable(T3, new byte [][] {HConstants.CATALOG_FAMILY});
+    TEST_UTIL.createMultiRegionTable(T1, new byte[][] { HConstants.CATALOG_FAMILY });
+    TEST_UTIL.createMultiRegionTable(T2, new byte[][] { HConstants.CATALOG_FAMILY });
+    TEST_UTIL.createMultiRegionTable(T3, new byte[][] { HConstants.CATALOG_FAMILY });
 
-    final byte[][] keysForT4 = {
-      Bytes.toBytes("aa"),
-      Bytes.toBytes("bb"),
-      Bytes.toBytes("cc"),
-      Bytes.toBytes("dd")
-    };
+    final byte[][] keysForT4 =
+      { Bytes.toBytes("aa"), Bytes.toBytes("bb"), Bytes.toBytes("cc"), Bytes.toBytes("dd") };
 
     TEST_UTIL.createTable(T4, HConstants.CATALOG_FAMILY, keysForT4);
 
-    final byte[][] keysForT5 = {
-      Bytes.toBytes("bb"),
-      Bytes.toBytes("cc"),
-      Bytes.toBytes("dd")
-    };
+    final byte[][] keysForT5 = { Bytes.toBytes("bb"), Bytes.toBytes("cc"), Bytes.toBytes("dd") };
 
     TEST_UTIL.createTable(T5, HConstants.CATALOG_FAMILY, keysForT5);
   }
@@ -98,11 +91,10 @@ public class TestCatalogJanitorCluster {
   }
 
   /**
-   * Fat method where we start with a fat hbase:meta and then gradually intro
-   * problems running catalogjanitor for each to ensure it triggers complaint.
-   * Do one big method because takes a while to build up the context we need.
-   * We create three tables and then make holes, overlaps, add unknown servers
-   * and empty out regioninfo columns. Each should up counts in the
+   * Fat method where we start with a fat hbase:meta and then gradually intro problems running
+   * catalogjanitor for each to ensure it triggers complaint. Do one big method because takes a
+   * while to build up the context we need. We create three tables and then make holes, overlaps,
+   * add unknown servers and empty out regioninfo columns. Each should up counts in the
    * CatalogJanitor.Report produced.
    */
   @Test
@@ -110,34 +102,33 @@ public class TestCatalogJanitorCluster {
     CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
     RegionStateStore regionStateStore =
       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
-    int gc = janitor.scan();
-    CatalogJanitor.Report report = janitor.getLastReport();
+    janitor.scan();
+    Report report = janitor.getLastReport();
     // Assert no problems.
     assertTrue(report.isEmpty());
     // Now remove first region in table t2 to see if catalogjanitor scan notices.
     List<RegionInfo> t2Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T2);
     regionStateStore.deleteRegion(t2Ris.get(0));
-    gc = janitor.scan();
+    janitor.scan();
     report = janitor.getLastReport();
     assertFalse(report.isEmpty());
     assertEquals(1, report.getHoles().size());
-    assertTrue(
-      report.getHoles().get(0).getFirst().getTable().equals(RegionInfo.UNDEFINED.getTable()));
+    assertTrue(report.getHoles().get(0).getFirst().getTable()
+      .equals(RegionInfoBuilder.UNDEFINED.getTable()));
     assertTrue(report.getHoles().get(0).getSecond().getTable().equals(T2));
     assertEquals(0, report.getOverlaps().size());
     // Next, add overlaps to first row in t3
     List<RegionInfo> t3Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T3);
     RegionInfo ri = t3Ris.get(0);
-    RegionInfo newRi1 = RegionInfoBuilder.newBuilder(ri.getTable()).
-        setStartKey(incrementRow(ri.getStartKey())).
-        setEndKey(incrementRow(ri.getEndKey())).build();
+    RegionInfo newRi1 = RegionInfoBuilder.newBuilder(ri.getTable())
+      .setStartKey(incrementRow(ri.getStartKey())).setEndKey(incrementRow(ri.getEndKey())).build();
     Put p1 = MetaTableAccessor.makePutFromRegionInfo(newRi1, System.currentTimeMillis());
-    RegionInfo newRi2 = RegionInfoBuilder.newBuilder(newRi1.getTable()).
-        setStartKey(incrementRow(newRi1.getStartKey())).
-        setEndKey(incrementRow(newRi1.getEndKey())).build();
+    RegionInfo newRi2 = RegionInfoBuilder.newBuilder(newRi1.getTable())
+      .setStartKey(incrementRow(newRi1.getStartKey())).setEndKey(incrementRow(newRi1.getEndKey()))
+      .build();
     Put p2 = MetaTableAccessor.makePutFromRegionInfo(newRi2, System.currentTimeMillis());
     MetaTableAccessor.putsToMetaTable(TEST_UTIL.getConnection(), Arrays.asList(p1, p2));
-    gc = janitor.scan();
+    janitor.scan();
     report = janitor.getLastReport();
     assertFalse(report.isEmpty());
     // We added two overlaps so total three.
@@ -151,10 +142,10 @@ public class TestCatalogJanitorCluster {
     List<RegionInfo> t1Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T1);
     RegionInfo t1Ri1 = t1Ris.get(1);
     Put pServer = new Put(t1Ri1.getRegionName());
-    pServer.addColumn(HConstants.CATALOG_FAMILY,
-        CatalogFamilyFormat.getServerColumn(0), Bytes.toBytes("bad.server.example.org:1234"));
+    pServer.addColumn(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerColumn(0),
+      Bytes.toBytes("bad.server.example.org:1234"));
     MetaTableAccessor.putsToMetaTable(TEST_UTIL.getConnection(), Arrays.asList(pServer));
-    gc = janitor.scan();
+    janitor.scan();
     report = janitor.getLastReport();
     assertFalse(report.isEmpty());
     assertEquals(1, report.getUnknownServers().size());
@@ -163,8 +154,8 @@ public class TestCatalogJanitorCluster {
     // break if this happens.
     LOG.info("Make null info:server");
     Put emptyInfoServerPut = new Put(t1Ri1.getRegionName());
-    emptyInfoServerPut.addColumn(HConstants.CATALOG_FAMILY,
-        CatalogFamilyFormat.getServerColumn(0), Bytes.toBytes(""));
+    emptyInfoServerPut.addColumn(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerColumn(0),
+      Bytes.toBytes(""));
     MetaTableAccessor.putsToMetaTable(TEST_UTIL.getConnection(), Arrays.asList(emptyInfoServerPut));
     janitor.scan();
     report = janitor.getLastReport();
@@ -172,8 +163,8 @@ public class TestCatalogJanitorCluster {
     // Mke an empty regioninfo in t1.
     RegionInfo t1Ri2 = t1Ris.get(2);
     Put pEmptyRI = new Put(t1Ri2.getRegionName());
-    pEmptyRI.addColumn(HConstants.CATALOG_FAMILY,
-        HConstants.REGIONINFO_QUALIFIER, HConstants.EMPTY_BYTE_ARRAY);
+    pEmptyRI.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
+      HConstants.EMPTY_BYTE_ARRAY);
     MetaTableAccessor.putsToMetaTable(TEST_UTIL.getConnection(), Arrays.asList(pEmptyRI));
     janitor.scan();
     report = janitor.getLastReport();
@@ -183,16 +174,15 @@ public class TestCatalogJanitorCluster {
     int overlapsReported = report.getOverlaps().size();
 
     // Test the case for T4
-    //    r1: [aa, bb), r2: [cc, dd), r3: [a, cc)
+    // r1: [aa, bb), r2: [cc, dd), r3: [a, cc)
     // Make sure only overlaps and no holes are reported.
     List<RegionInfo> t4Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T4);
     // delete the region [bb, cc)
     regionStateStore.deleteRegion(t4Ris.get(2));
 
     // add a new region [a, cc)
-    RegionInfo newRiT4 = RegionInfoBuilder.newBuilder(T4).
-      setStartKey("a".getBytes()).
-      setEndKey("cc".getBytes()).build();
+    RegionInfo newRiT4 = RegionInfoBuilder.newBuilder(T4).setStartKey("a".getBytes())
+      .setEndKey("cc".getBytes()).build();
     Put putForT4 = MetaTableAccessor.makePutFromRegionInfo(newRiT4, System.currentTimeMillis());
     MetaTableAccessor.putsToMetaTable(TEST_UTIL.getConnection(), Arrays.asList(putForT4));
 
@@ -206,16 +196,15 @@ public class TestCatalogJanitorCluster {
     overlapsReported = report.getOverlaps().size();
 
     // Test the case for T5
-    //    r0: [, bb), r1: [a, g), r2: [bb, cc), r3: [dd, )
+    // r0: [, bb), r1: [a, g), r2: [bb, cc), r3: [dd, )
     // Make sure only overlaps and no holes are reported.
     List<RegionInfo> t5Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T5);
     // delete the region [cc, dd)
     regionStateStore.deleteRegion(t5Ris.get(2));
 
     // add a new region [a, g)
-    RegionInfo newRiT5 = RegionInfoBuilder.newBuilder(T5).
-      setStartKey("a".getBytes()).
-      setEndKey("g".getBytes()).build();
+    RegionInfo newRiT5 = RegionInfoBuilder.newBuilder(T5).setStartKey("a".getBytes())
+      .setEndKey("g".getBytes()).build();
     Put putForT5 = MetaTableAccessor.makePutFromRegionInfo(newRiT5, System.currentTimeMillis());
     MetaTableAccessor.putsToMetaTable(TEST_UTIL.getConnection(), Arrays.asList(putForT5));
 
@@ -230,11 +219,11 @@ public class TestCatalogJanitorCluster {
   /**
    * Take last byte and add one to it.
    */
-  private static byte [] incrementRow(byte [] row) {
+  private static byte[] incrementRow(byte[] row) {
     if (row.length == 0) {
-      return new byte []{'0'};
+      return new byte[] { '0' };
     }
-    row[row.length - 1] = (byte)(((int)row[row.length - 1]) + 1);
+    row[row.length - 1] = (byte) (((int) row[row.length - 1]) + 1);
     return row;
   }
 
@@ -242,23 +231,23 @@ public class TestCatalogJanitorCluster {
   public void testHoles() throws IOException {
     CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
 
-    CatalogJanitor.Report report = janitor.getLastReport();
+    Report report = janitor.getLastReport();
     // Assert no problems.
     assertTrue(report.isEmpty());
-    //Verify start and end region holes
+    // Verify start and end region holes
     verifyCornerHoles(janitor, T1);
-    //Verify start and end region holes
+    // Verify start and end region holes
     verifyCornerHoles(janitor, T2);
     verifyMiddleHole(janitor);
-    //Verify that MetaFixer is able to fix these holes
+    // Verify that MetaFixer is able to fix these holes
     fixHoles(janitor);
   }
 
   private void fixHoles(CatalogJanitor janitor) throws IOException {
     MetaFixer metaFixer = new MetaFixer(TEST_UTIL.getHBaseCluster().getMaster());
     janitor.scan();
-    CatalogJanitor.Report report = janitor.getLastReport();
-    //Verify total number of holes, 2 in t1 and t2 each and one in t3
+    Report report = janitor.getLastReport();
+    // Verify total number of holes, 2 in t1 and t2 each and one in t3
     assertEquals("Number of holes are not matching", 5, report.getHoles().size());
     metaFixer.fix();
     janitor.scan();
@@ -267,7 +256,7 @@ public class TestCatalogJanitorCluster {
   }
 
   private void verifyMiddleHole(CatalogJanitor janitor) throws IOException {
-    //Verify middle holes
+    // Verify middle holes
     RegionInfo firstRegion = getRegionInfo(T3, "".getBytes());
     RegionInfo secondRegion = getRegionInfo(T3, "bbb".getBytes());
     RegionInfo thirdRegion = getRegionInfo(T3, "ccc".getBytes());
@@ -293,8 +282,8 @@ public class TestCatalogJanitorCluster {
 
     assertEquals(1, holes.size());
     Pair<RegionInfo, RegionInfo> regionInfoRegionInfoPair = holes.get(0);
-    assertTrue(
-      regionInfoRegionInfoPair.getFirst().getTable().equals(RegionInfo.UNDEFINED.getTable()));
+    assertTrue(regionInfoRegionInfoPair.getFirst().getTable()
+      .equals(RegionInfoBuilder.UNDEFINED.getTable()));
     assertTrue(regionInfoRegionInfoPair.getSecond().getTable().equals(tableName));
     assertTrue(
       regionInfoRegionInfoPair.getSecond().getEncodedName().equals(secondRegion.getEncodedName()));
@@ -307,20 +296,20 @@ public class TestCatalogJanitorCluster {
     regionInfoRegionInfoPair = holes.get(1);
     assertTrue(regionInfoRegionInfoPair.getFirst().getEncodedName()
       .equals(secondLastRegion.getEncodedName()));
-    assertTrue(
-      regionInfoRegionInfoPair.getSecond().getTable().equals(RegionInfo.UNDEFINED.getTable()));
+    assertTrue(regionInfoRegionInfoPair.getSecond().getTable()
+      .equals(RegionInfoBuilder.UNDEFINED.getTable()));
   }
 
-  //Get Holes filter by table
+  // Get Holes filter by table
   private LinkedList<Pair<RegionInfo, RegionInfo>> getHoles(CatalogJanitor janitor,
     TableName tableName) throws IOException {
     janitor.scan();
-    CatalogJanitor.Report lastReport = janitor.getLastReport();
+    Report lastReport = janitor.getLastReport();
     assertFalse(lastReport.isEmpty());
     LinkedList<Pair<RegionInfo, RegionInfo>> holes = new LinkedList<>();
     for (Pair<RegionInfo, RegionInfo> hole : lastReport.getHoles()) {
-      if (hole.getFirst().getTable().equals(tableName) || hole.getSecond().getTable()
-        .equals(tableName)) {
+      if (hole.getFirst().getTable().equals(tableName) ||
+        hole.getSecond().getTable().equals(tableName)) {
         holes.add(hole);
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java
similarity index 75%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java
index b5b5e63..07b3ca7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.master.janitor;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CatalogFamilyFormat;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaMockingUtil;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -41,6 +41,8 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -53,37 +55,30 @@ import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category({MasterTests.class, MediumTests.class})
+@Category({ MasterTests.class, MediumTests.class })
 public class TestCatalogJanitorInMemoryStates {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestCatalogJanitorInMemoryStates.class);
+    HBaseClassTestRule.forClass(TestCatalogJanitorInMemoryStates.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestCatalogJanitorInMemoryStates.class);
-  @Rule public final TestName name = new TestName();
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static byte [] ROW = Bytes.toBytes("testRow");
-  private static byte [] FAMILY = Bytes.toBytes("testFamily");
-  private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
-  private static byte [] VALUE = Bytes.toBytes("testValue");
 
-  /**
-   * @throws java.lang.Exception
-   */
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static byte[] FAMILY = Bytes.toBytes("testFamily");
+
+  @Rule
+  public final TableNameTestRule name = new TableNameTestRule();
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
     TEST_UTIL.startMiniCluster(1);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
@@ -94,17 +89,17 @@ public class TestCatalogJanitorInMemoryStates {
    */
   @Test
   public void testInMemoryParentCleanup()
-      throws IOException, InterruptedException, ExecutionException {
-    final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
-    final ServerManager sm = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();
-    final CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
+    throws IOException, InterruptedException, ExecutionException {
+    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+    final AssignmentManager am = master.getAssignmentManager();
+    final ServerManager sm = master.getServerManager();
 
     Admin admin = TEST_UTIL.getAdmin();
     admin.catalogJanitorSwitch(false);
 
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     Table t = TEST_UTIL.createTable(tableName, FAMILY);
-    int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
+    TEST_UTIL.loadTable(t, FAMILY, false);
 
     RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
     List<HRegionLocation> allRegionLocations = locator.getAllRegionLocations();
@@ -117,18 +112,18 @@ public class TestCatalogJanitorInMemoryStates {
     assertNotNull("Should have found daughter regions for " + parent, daughters);
 
     assertTrue("Parent region should exist in RegionStates",
-        am.getRegionStates().isRegionInRegionStates(parent.getRegion()));
+      am.getRegionStates().isRegionInRegionStates(parent.getRegion()));
     assertTrue("Parent region should exist in ServerManager",
-        sm.isRegionInServerManagerStates(parent.getRegion()));
+      sm.isRegionInServerManagerStates(parent.getRegion()));
 
     // clean the parent
     Result r = MetaMockingUtil.getMetaTableRowResult(parent.getRegion(), null,
-        daughters.get(0).getRegion(), daughters.get(1).getRegion());
-    janitor.cleanParent(parent.getRegion(), r);
+      daughters.get(0).getRegion(), daughters.get(1).getRegion());
+    CatalogJanitor.cleanParent(master, parent.getRegion(), r);
     assertFalse("Parent region should have been removed from RegionStates",
-        am.getRegionStates().isRegionInRegionStates(parent.getRegion()));
+      am.getRegionStates().isRegionInRegionStates(parent.getRegion()));
     assertFalse("Parent region should have been removed from ServerManager",
-        sm.isRegionInServerManagerStates(parent.getRegion()));
+      sm.isRegionInServerManagerStates(parent.getRegion()));
 
   }
 
@@ -138,7 +133,7 @@ public class TestCatalogJanitorInMemoryStates {
    * @return List of region locations
    */
   private List<HRegionLocation> splitRegion(final RegionInfo r)
-      throws IOException, InterruptedException, ExecutionException {
+    throws IOException, InterruptedException, ExecutionException {
     List<HRegionLocation> locations = new ArrayList<>();
     // Split this table in two.
     Admin admin = TEST_UTIL.getAdmin();
@@ -157,17 +152,16 @@ public class TestCatalogJanitorInMemoryStates {
   }
 
   /*
-   * Wait on region split. May return because we waited long enough on the split
-   * and it didn't happen.  Caller should check.
+   * Wait on region split. May return because we waited long enough on the split and it didn't
+   * happen. Caller should check.
    * @param r
    * @return Daughter regions; caller needs to check table actually split.
    */
-  private PairOfSameType<RegionInfo> waitOnDaughters(final RegionInfo r)
-      throws IOException {
+  private PairOfSameType<RegionInfo> waitOnDaughters(final RegionInfo r) throws IOException {
     long start = System.currentTimeMillis();
     PairOfSameType<RegionInfo> pair = null;
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-         Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) {
+      Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) {
       Result result = null;
       RegionInfo region = null;
       while ((System.currentTimeMillis() - start) < 60000) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixer.java
similarity index 94%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixer.java
index cb91008..1cb5a96 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixer.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.master.janitor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -39,6 +39,9 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.HbckChore;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.GCMultipleMergedRegionsProcedure;
 import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
@@ -93,7 +96,7 @@ public class TestMetaFixer {
     MasterServices services = TEST_UTIL.getHBaseCluster().getMaster();
     int initialSize = services.getAssignmentManager().getRegionStates().getRegionStates().size();
     services.getCatalogJanitor().scan();
-    CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
+    Report report = services.getCatalogJanitor().getLastReport();
     assertTrue(report.isEmpty());
     int originalCount = ris.size();
     // Remove first, last and middle region. See if hole gets plugged. Table has 26 * replicaCount regions.
@@ -151,7 +154,7 @@ public class TestMetaFixer {
     services.getCatalogJanitor().scan();
     deleteRegion(services, ris.get(0));
     services.getCatalogJanitor().scan();
-    CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
+    Report report = services.getCatalogJanitor().getLastReport();
     ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
     assertTrue(ris.isEmpty());
     MetaFixer fixer = new MetaFixer(services);
@@ -184,7 +187,7 @@ public class TestMetaFixer {
     assertTrue(ris.size() > 5);
     HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
     services.getCatalogJanitor().scan();
-    CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
+    Report report = services.getCatalogJanitor().getLastReport();
     assertTrue(report.isEmpty());
     // Make a simple overlap spanning second and third region.
     makeOverlap(services, ris.get(1), ris.get(3));
@@ -201,7 +204,7 @@ public class TestMetaFixer {
 
     CatalogJanitor cj = services.getCatalogJanitor();
     cj.scan();
-    CatalogJanitor.Report report = cj.getLastReport();
+    Report report = cj.getLastReport();
     assertEquals(6, report.getOverlaps().size());
     assertEquals(1,
       MetaFixer.calculateMerges(10, report.getOverlaps()).size());
@@ -237,12 +240,12 @@ public class TestMetaFixer {
     });
 
     // No orphan regions on FS
-    hbckChore.chore();
+    hbckChore.choreForTesting();
     assertEquals(0, hbckChore.getOrphanRegionsOnFS().size());
 
     // No holes reported.
     cj.scan();
-    final CatalogJanitor.Report postReport = cj.getLastReport();
+    final Report postReport = cj.getLastReport();
     assertTrue(postReport.isEmpty());
   }
 
@@ -256,7 +259,7 @@ public class TestMetaFixer {
 
     HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
     services.getCatalogJanitor().scan();
-    CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
+    Report report = services.getCatalogJanitor().getLastReport();
     assertTrue(report.isEmpty());
 
     // Make a simple overlap for t1
@@ -289,7 +292,7 @@ public class TestMetaFixer {
       HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
       CatalogJanitor cj = services.getCatalogJanitor();
       cj.scan();
-      CatalogJanitor.Report report = cj.getLastReport();
+      Report report = cj.getLastReport();
       assertEquals(6, report.getOverlaps().size());
       assertEquals(2,
         MetaFixer.calculateMerges(5, report.getOverlaps()).size());
@@ -313,7 +316,7 @@ public class TestMetaFixer {
       HBaseTestingUtility.await(200, () -> {
         try {
           cj.scan();
-          final CatalogJanitor.Report postReport = cj.getLastReport();
+          final Report postReport = cj.getLastReport();
           RegionStates regionStates = am.getRegionStates();
           RegionStateStore regionStateStore = am.getRegionStateStore();
           // Make sure that two merged regions are opened and GCs are done.
@@ -352,7 +355,7 @@ public class TestMetaFixer {
 
       // No holes reported.
       cj.scan();
-      final CatalogJanitor.Report postReport = cj.getLastReport();
+      final Report postReport = cj.getLastReport();
       assertTrue(postReport.isEmpty());
 
     } finally {
@@ -371,13 +374,13 @@ public class TestMetaFixer {
   @Test
   public void testMergeWithMergedChildRegion() throws Exception {
     TableName tn = TableName.valueOf(this.name.getMethodName());
-    Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY);
+    TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY);
     List<RegionInfo> ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
     assertTrue(ris.size() > 5);
     HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
     CatalogJanitor cj = services.getCatalogJanitor();
     cj.scan();
-    CatalogJanitor.Report report = cj.getLastReport();
+    Report report = cj.getLastReport();
     assertTrue(report.isEmpty());
     RegionInfo overlapRegion = makeOverlap(services, ris.get(1), ris.get(2));
 
@@ -443,7 +446,7 @@ public class TestMetaFixer {
     assertTrue(ris.size() > 5);
     MasterServices services = TEST_UTIL.getHBaseCluster().getMaster();
     services.getCatalogJanitor().scan();
-    CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
+    Report report = services.getCatalogJanitor().getLastReport();
     assertTrue(report.isEmpty());
     // Make a simple overlap spanning second and third region.
     makeOverlap(services, ris.get(1), ris.get(5));
@@ -467,7 +470,7 @@ public class TestMetaFixer {
     HBaseTestingUtility.await(10, () -> {
       try {
         services.getCatalogJanitor().scan();
-        final CatalogJanitor.Report postReport = services.getCatalogJanitor().getLastReport();
+        final Report postReport = services.getCatalogJanitor().getLastReport();
         return postReport.isEmpty();
       } catch (Exception e) {
         throw new RuntimeException(e);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixerNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java
similarity index 73%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixerNoCluster.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java
index 5b80f89..614385e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixerNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.master.janitor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.SortedSet;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -33,48 +32,39 @@ 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.Pair;
-
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-
-
 /**
- * Test small utility methods inside {@link MetaFixer}.
- * For cluster tests see {@link TestMetaFixer}
+ * Test small utility methods inside {@link MetaFixer}. For cluster tests see {@link TestMetaFixer}
  */
-@Category({MasterTests.class, SmallTests.class})
+@Category({ MasterTests.class, SmallTests.class })
 public class TestMetaFixerNoCluster {
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMetaFixerNoCluster.class);
-  private static byte [] A = Bytes.toBytes("a");
-  private static byte [] B = Bytes.toBytes("b");
-  private static byte [] C = Bytes.toBytes("c");
-  private static byte [] D = Bytes.toBytes("d");
+    HBaseClassTestRule.forClass(TestMetaFixerNoCluster.class);
+  private static byte[] A = Bytes.toBytes("a");
+  private static byte[] B = Bytes.toBytes("b");
+  private static byte[] C = Bytes.toBytes("c");
+  private static byte[] D = Bytes.toBytes("d");
   private static RegionInfo ALL = RegionInfoBuilder.FIRST_META_REGIONINFO;
-  private static RegionInfo _ARI = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-      setEndKey(A).build();
-  private static RegionInfo _BRI = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-          setEndKey(B).build();
-  private static RegionInfo ABRI = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-          setStartKey(A).setEndKey(B).build();
-  private static RegionInfo ACRI =
-      org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-          setStartKey(A).setEndKey(C).build();
-  private static RegionInfo CDRI =
-      org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-          setStartKey(C).setEndKey(D).build();
-  private static RegionInfo ADRI =
-      org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-          setStartKey(A).setEndKey(D).build();
-  private static RegionInfo D_RI =
-      org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-          setStartKey(D).build();
-  private static RegionInfo C_RI =
-      org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-          setStartKey(C).build();
+  private static RegionInfo _ARI =
+    RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(A).build();
+  private static RegionInfo _BRI =
+    RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(B).build();
+  private static RegionInfo ABRI =
+    RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(A).setEndKey(B).build();
+  private static RegionInfo ACRI = org.apache.hadoop.hbase.client.RegionInfoBuilder
+    .newBuilder(TableName.META_TABLE_NAME).setStartKey(A).setEndKey(C).build();
+  private static RegionInfo CDRI = org.apache.hadoop.hbase.client.RegionInfoBuilder
+    .newBuilder(TableName.META_TABLE_NAME).setStartKey(C).setEndKey(D).build();
+  private static RegionInfo ADRI = org.apache.hadoop.hbase.client.RegionInfoBuilder
+    .newBuilder(TableName.META_TABLE_NAME).setStartKey(A).setEndKey(D).build();
+  private static RegionInfo D_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder
+    .newBuilder(TableName.META_TABLE_NAME).setStartKey(D).build();
+  private static RegionInfo C_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder
+    .newBuilder(TableName.META_TABLE_NAME).setStartKey(C).build();
 
   @Test
   public void testGetRegionInfoWithLargestEndKey() {