You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2019/08/08 00:01:40 UTC

[hbase] branch branch-2 updated: HBASE-22777 Add a multi-region merge (for fixing overlaps)

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

stack pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 225c6cd  HBASE-22777 Add a multi-region merge (for fixing overlaps)
225c6cd is described below

commit 225c6cd5bc3bcb3a1146c38a2053c45c6bfef877
Author: stack <st...@apache.org>
AuthorDate: Thu Aug 1 13:53:53 2019 -0700

    HBASE-22777 Add a multi-region merge (for fixing overlaps)
    
    Makes MergeTableRegionsProcedure do more than just two regions at a
    time. Compatible as MTRP was done considering one day it'd do more than
    two at a time.
    
    Changes hardcoded assumption that merge parent regions are named
    mergeA and mergeB in a column on the resultant region. Instead
    can have N columns on the merged region, one for each parent
    merged. Column qualifiers all being with 'merge'.
    
    Most of code below is undoing the assumption that there are two
    parents on a merge only.
---
 .../org/apache/hadoop/hbase/MetaTableAccessor.java | 462 ++++++++++-----------
 .../org/apache/hadoop/hbase/client/RegionInfo.java |  84 ++--
 .../java/org/apache/hadoop/hbase/HConstants.java   |  28 +-
 .../org/apache/hadoop/hbase/PrivateCellUtil.java   |  19 +
 .../src/main/protobuf/MasterProcedure.proto        |  11 +-
 .../apache/hadoop/hbase/rest/RegionsResource.java  |  14 +-
 .../hbase/favored/FavoredNodeLoadBalancer.java     |   6 +-
 .../hadoop/hbase/favored/FavoredNodesPromoter.java |   6 +-
 .../apache/hadoop/hbase/master/CatalogJanitor.java |  85 ++--
 .../org/apache/hadoop/hbase/master/HMaster.java    |  36 +-
 .../hbase/master/assignment/AssignmentManager.java |  26 +-
 .../assignment/GCMergedRegionsProcedure.java       |   2 +
 ....java => GCMultipleMergedRegionsProcedure.java} | 111 ++---
 .../assignment/MergeTableRegionsProcedure.java     | 427 ++++++++-----------
 .../hbase/master/assignment/RegionStateStore.java  |  20 +-
 .../master/balancer/FavoredStochasticBalancer.java |   8 +-
 .../hbase/regionserver/HRegionFileSystem.java      |  18 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |   3 +-
 .../apache/hadoop/hbase/regionserver/HStore.java   |  13 +-
 .../hadoop/hbase/regionserver/StoreFileInfo.java   |   2 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |  15 +-
 .../apache/hadoop/hbase/TestMetaTableAccessor.java | 186 +++++----
 .../master/TestMergeTableRegionsWhileRSCrash.java  |   2 +-
 .../TestMasterAbortWhileMergingTable.java          |   2 +-
 .../assignment/TestMergeTableRegionsProcedure.java |  88 +++-
 .../assignment/TestModifyTableWhileMerging.java    |   2 +-
 .../TestRegionMergeTransactionOnCluster.java       |  26 +-
 27 files changed, 854 insertions(+), 848 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index c2ca02d..a6fad98 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -22,9 +22,9 @@ import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -32,8 +32,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Set;
-import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.SortedMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -59,6 +59,8 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.RegexStringComparator;
 import org.apache.hadoop.hbase.filter.RowFilter;
 import org.apache.hadoop.hbase.filter.SubstringComparator;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
@@ -84,11 +86,12 @@ import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 
 /**
  * <p>
- * Read/write operations on region and assignment information store in <code>hbase:meta</code>.
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
  * </p>
  * <p>
  * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
- * because when used on client-side (like from HBaseAdmin), we want to use short-living connection
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
  * (opened before each operation, closed right after), while when used on HM or HRS (like in
  * AssignmentManager) we want permanent connection.
  * </p>
@@ -98,16 +101,17 @@ import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
  * called default replica.
  * </p>
  * <p>
- * Meta layout (as of 0.98 + HBASE-10070) is like:
+ * <h2>Meta layout</h2>
  *
  * <pre>
- * For each table there is single row in column family 'table' formatted:
- * &lt;tableName&gt; including namespace and columns are:
- * table: state             => contains table state
+ * For each table there is single row named for the table with a 'table' column family.
+ * The column family currently has one column in it, the 'state' column:
  *
- * For each table range, there is a single row, formatted like:
+ * table:state             => contains table state
+ *
+ * Then for each table range ('Region'), there is a single row, formatted as:
  * &lt;tableName&gt;,&lt;startKey&gt;,&lt;regionId&gt;,&lt;encodedRegionName&gt;.
- * This row corresponds to the regionName of the default region replica.
+ * This row is the serialized regionName of the default region replica.
  * Columns are:
  * info:regioninfo         => contains serialized HRI for the default region replica
  * info:server             => contains hostname:port (in string form) for the server hosting
@@ -128,10 +132,14 @@ import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
  *                             region is split
  * info:splitB             => contains a serialized HRI for the second daughter region if the
  *                             region is split
- * info:mergeA             => contains a serialized HRI for the first parent region if the
- *                             region is the result of a merge
- * info:mergeB             => contains a serialized HRI for the second parent region if the
- *                             region is the result of a merge
+ * info:merge*             => contains a serialized HRI for a merge parent region. There will be two
+ *                             or more of these columns in a row. A row that has these columns is
+ *                             undergoing a merge and is the result of the merge. Columns listed
+ *                             in marge* columns are the parents of this merged region. Example
+ *                             columns: info:merge0001, info:merge0002. You make also see 'mergeA',
+ *                             and 'mergeB'. This is old form replaced by the new format that allows
+ *                             for more than two parents to be merged at a time.
+ * TODO: Add rep_barrier for serial replication explaination.
  * </pre>
  * </p>
  * <p>
@@ -152,36 +160,6 @@ public class MetaTableAccessor {
 
   private static final byte SEPARATED_BYTE = 0x00;
 
-  /**
-   * Lists all of the table regions currently in META.
-   * Deprecated, keep there until some test use this.
-   * @param connection what we will use
-   * @param tableName table to list
-   * @return Map of all user-space regions to servers
-   * @deprecated use {@link #getTableRegionsAndLocations}, region can have multiple locations
-   */
-  @Deprecated
-  public static NavigableMap<RegionInfo, ServerName> allTableRegions(
-      Connection connection, final TableName tableName) throws IOException {
-    final NavigableMap<RegionInfo, ServerName> regions = new TreeMap<>();
-    Visitor visitor = new TableVisitorBase(tableName) {
-      @Override
-      public boolean visitInternal(Result result) throws IOException {
-        RegionLocations locations = getRegionLocations(result);
-        if (locations == null) return true;
-        for (HRegionLocation loc : locations.getRegionLocations()) {
-          if (loc != null) {
-            RegionInfo regionInfo = loc.getRegionInfo();
-            regions.put(regionInfo, loc.getServerName());
-          }
-        }
-        return true;
-      }
-    };
-    scanMetaForTableRegions(connection, visitor, tableName);
-    return regions;
-  }
-
   @InterfaceAudience.Private
   public enum QueryType {
     ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY),
@@ -201,7 +179,7 @@ public class MetaTableAccessor {
   }
 
   /** The delimiter for meta columns for replicaIds &gt; 0 */
-  protected static final char META_REPLICA_ID_DELIMITER = '_';
+  static final char META_REPLICA_ID_DELIMITER = '_';
 
   /** A regex for parsing server columns from meta. See above javadoc for meta layout */
   private static final Pattern SERVER_COLUMN_PATTERN
@@ -216,8 +194,7 @@ public class MetaTableAccessor {
    * @param connection connection we're using
    * @param visitor Visitor invoked against each row in regions family.
    */
-  public static void fullScanRegions(Connection connection,
-      final Visitor visitor)
+  public static void fullScanRegions(Connection connection, final Visitor visitor)
       throws IOException {
     scanMeta(connection, null, null, QueryType.REGION, visitor);
   }
@@ -226,8 +203,7 @@ public class MetaTableAccessor {
    * Performs a full scan of <code>hbase:meta</code> for regions.
    * @param connection connection we're using
    */
-  public static List<Result> fullScanRegions(Connection connection)
-      throws IOException {
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
     return fullScan(connection, QueryType.REGION);
   }
 
@@ -236,8 +212,7 @@ public class MetaTableAccessor {
    * @param connection connection we're using
    * @param visitor Visitor invoked against each row in tables family.
    */
-  public static void fullScanTables(Connection connection,
-      final Visitor visitor)
+  public static void fullScanTables(Connection connection, final Visitor visitor)
       throws IOException {
     scanMeta(connection, null, null, QueryType.TABLE, visitor);
   }
@@ -248,8 +223,7 @@ public class MetaTableAccessor {
    * @param type scanned part of meta
    * @return List of {@link Result}
    */
-  public static List<Result> fullScan(Connection connection, QueryType type)
-    throws IOException {
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
     CollectAllVisitor v = new CollectAllVisitor();
     scanMeta(connection, null, null, type, v);
     return v.getResults();
@@ -394,20 +368,89 @@ public class MetaTableAccessor {
   }
 
   /**
-   * Get regions from the merge qualifier of the specified merged region
-   * @return null if it doesn't contain merge qualifier, else two merge regions
+   * @return Return all regioninfos listed in the 'info:merge*' columns of
+   *   the <code>regionName</code> row.
    */
   @Nullable
-  public static Pair<RegionInfo, RegionInfo> getRegionsFromMergeQualifier(
-      Connection connection, byte[] regionName) throws IOException {
-    Result result = getRegionResult(connection, regionName);
-    RegionInfo mergeA = getRegionInfo(result, HConstants.MERGEA_QUALIFIER);
-    RegionInfo mergeB = getRegionInfo(result, HConstants.MERGEB_QUALIFIER);
-    if (mergeA == null && mergeB == null) {
+  public static List<RegionInfo> getMergeRegions(Connection connection, byte[] regionName)
+      throws IOException {
+    return getMergeRegions(getMergeRegionsRaw(connection, regionName));
+  }
+
+  /**
+   * @return Deserialized regioninfo values taken from column values that match
+   *   the regex 'info:merge.*' in array of <code>cells</code>.
+   */
+  @Nullable
+  public static List<RegionInfo> getMergeRegions(Cell [] cells) {
+    if (cells == null) {
       return null;
     }
-    return new Pair<>(mergeA, mergeB);
- }
+    List<RegionInfo> regionsToMerge = null;
+    for (Cell cell: cells) {
+      if (!isMergeQualifierPrefix(cell)) {
+        continue;
+      }
+      // Ok. This cell is that of a info:merge* column.
+      RegionInfo ri = RegionInfo.parseFromOrNull(cell.getValueArray(), cell.getValueOffset(),
+         cell.getValueLength());
+      if (ri != null) {
+        if (regionsToMerge == null) {
+          regionsToMerge = new ArrayList<>();
+        }
+        regionsToMerge.add(ri);
+      }
+    }
+    return regionsToMerge;
+  }
+
+  /**
+   * @return True if any merge regions present in <code>cells</code>; i.e.
+   *   the column in <code>cell</code> matches the regex 'info:merge.*'.
+   */
+  public static boolean hasMergeRegions(Cell [] cells) {
+    for (Cell cell: cells) {
+      if (!isMergeQualifierPrefix(cell)) {
+        continue;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * @return True if the column in <code>cell</code> matches the regex 'info:merge.*'.
+   */
+  private static boolean isMergeQualifierPrefix(Cell cell) {
+    // Check to see if has family and that qualifier starts with the merge qualifier 'merge'
+    return CellUtil.matchingFamily(cell, HConstants.CATALOG_FAMILY) &&
+      PrivateCellUtil.qualifierStartsWith(cell, HConstants.MERGE_QUALIFIER_PREFIX);
+  }
+
+  /**
+   * @return Array of Cells made from all columns on the <code>regionName</code> row
+   *   that match the regex 'info:merge.*'.
+   */
+  @Nullable
+  private static Cell [] getMergeRegionsRaw(Connection connection, byte [] regionName)
+      throws IOException {
+    Scan scan = new Scan().withStartRow(regionName).
+        setOneRowLimit().
+        readVersions(1).
+        addFamily(HConstants.CATALOG_FAMILY).
+        setFilter(new QualifierFilter(CompareOperator.EQUAL,
+          new RegexStringComparator(HConstants.MERGE_QUALIFIER_PREFIX_STR+ ".*")));
+    try (Table m = getMetaHTable(connection); ResultScanner scanner = m.getScanner(scan)) {
+      // Should be only one result in this scanner if any.
+      Result result = scanner.next();
+      if (result == null) {
+        return null;
+      }
+      // Should be safe to just return all Cells found since we had filter in place.
+      // All values should be RegionInfos or something wrong.
+      return result.rawCells();
+    }
+  }
 
   /**
    * Checks if the specified table exists.  Looks at the hbase:meta table hosted on
@@ -420,8 +463,8 @@ public class MetaTableAccessor {
       final TableName tableName)
   throws IOException {
     // Catalog tables always exist.
-    return tableName.equals(TableName.META_TABLE_NAME)
-        || getTableState(connection, tableName) != null;
+    return tableName.equals(TableName.META_TABLE_NAME) ||
+        getTableState(connection, tableName) != null;
   }
 
   /**
@@ -578,7 +621,6 @@ public class MetaTableAccessor {
    * @param connection connection we're using
    * @param tableName table we're looking for
    * @return Return list of regioninfos and server.
-   * @throws IOException
    */
   public static List<Pair<RegionInfo, ServerName>>
     getTableRegionsAndLocations(Connection connection, TableName tableName)
@@ -592,7 +634,6 @@ public class MetaTableAccessor {
    * @param tableName table to work with, can be null for getting all regions
    * @param excludeOfflinedSplitParents don't return split parents
    * @return Return list of regioninfos and server addresses.
-   * @throws IOException
    */
   public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
       Connection connection, @Nullable final TableName tableName,
@@ -672,25 +713,26 @@ public class MetaTableAccessor {
 
   public static void fullScanMetaAndPrint(Connection connection)
     throws IOException {
-    Visitor v = new Visitor() {
-      @Override
-      public boolean visit(Result r) throws IOException {
-        if (r ==  null || r.isEmpty()) return true;
-        LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
-        TableState state = getTableState(r);
-        if (state != null) {
-          LOG.info("Table State: " + state);
-        } else {
-          RegionLocations locations = getRegionLocations(r);
-          if (locations == null) return true;
-          for (HRegionLocation loc : locations.getRegionLocations()) {
-            if (loc != null) {
-              LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegion());
-            }
+    Visitor v = r -> {
+      if (r ==  null || r.isEmpty()) {
+        return true;
+      }
+      LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
+      TableState state = getTableState(r);
+      if (state != null) {
+        LOG.info("fullScanMetaAndPrint.Table State={}" + state);
+      } else {
+        RegionLocations locations = getRegionLocations(r);
+        if (locations == null) {
+          return true;
+        }
+        for (HRegionLocation loc : locations.getRegionLocations()) {
+          if (loc != null) {
+            LOG.info("fullScanMetaAndPrint.HRI Print={}", loc.getRegion());
           }
         }
-        return true;
       }
+      return true;
     };
     scanMeta(connection, null, null, QueryType.ALL, v);
   }
@@ -700,21 +742,19 @@ public class MetaTableAccessor {
     scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);
   }
 
-  public static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows,
+  private static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows,
       final Visitor visitor) throws IOException {
     scanMeta(connection, getTableStartRowForMeta(table, type), getTableStopRowForMeta(table, type),
       type, maxRows, visitor);
   }
 
-  public static void scanMeta(Connection connection, @Nullable final byte[] startRow,
+  private static void scanMeta(Connection connection, @Nullable final byte[] startRow,
       @Nullable final byte[] stopRow, QueryType type, final Visitor visitor) throws IOException {
     scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor);
   }
 
   /**
-   * Performs a scan of META table for given table starting from
-   * given row.
-   *
+   * Performs a scan of META table for given table starting from given row.
    * @param connection connection we're using
    * @param visitor    visitor to call
    * @param tableName  table withing we scan
@@ -737,7 +777,6 @@ public class MetaTableAccessor {
     scanMeta(connection, startRow, stopRow, QueryType.REGION, rowLimit, visitor);
   }
 
-
   /**
    * Performs a scan of META table.
    * @param connection connection we're using
@@ -749,8 +788,8 @@ public class MetaTableAccessor {
    * @param maxRows maximum rows to return
    * @param visitor Visitor invoked against each row.
    */
-  public static void scanMeta(Connection connection, @Nullable final byte[] startRow,
-      @Nullable final byte[] stopRow, QueryType type, int maxRows, final Visitor visitor)
+  static void scanMeta(Connection connection, @Nullable final byte[] startRow,
+        @Nullable final byte[] stopRow, QueryType type, int maxRows, final Visitor visitor)
       throws IOException {
     scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor);
   }
@@ -792,7 +831,7 @@ public class MetaTableAccessor {
         }
       }
     }
-    if (visitor != null && visitor instanceof Closeable) {
+    if (visitor instanceof Closeable) {
       try {
         ((Closeable) visitor).close();
       } catch (Throwable t) {
@@ -873,7 +912,7 @@ public class MetaTableAccessor {
    * @return a byte[] for state qualifier
    */
   @VisibleForTesting
-  public static byte[] getRegionStateColumn(int replicaId) {
+  static byte[] getRegionStateColumn(int replicaId) {
     return replicaId == 0 ? HConstants.STATE_QUALIFIER
         : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
             + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
@@ -885,7 +924,7 @@ public class MetaTableAccessor {
    * @return a byte[] for sn column qualifier
    */
   @VisibleForTesting
-  public static byte[] getServerNameColumn(int replicaId) {
+  static byte[] getServerNameColumn(int replicaId) {
     return replicaId == 0 ? HConstants.SERVERNAME_QUALIFIER
         : Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
             + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
@@ -990,6 +1029,18 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Returns the daughter regions by reading the corresponding columns of the catalog table
+   * Result.
+   * @param data a Result object from the catalog table scan
+   * @return pair of RegionInfo or PairOfSameType(null, null) if region is not a split parent
+   */
+  public static PairOfSameType<RegionInfo> getDaughterRegions(Result data) {
+    RegionInfo splitA = getRegionInfo(data, HConstants.SPLITA_QUALIFIER);
+    RegionInfo splitB = getRegionInfo(data, HConstants.SPLITB_QUALIFIER);
+    return new PairOfSameType<>(splitA, splitB);
+  }
+
+  /**
    * Returns an HRegionLocationList extracted from the result.
    * @return an HRegionLocationList containing all locations for the region range or null if
    *  we can't deserialize the result.
@@ -1011,7 +1062,7 @@ public class MetaTableAccessor {
     // iterate until all serverName columns are seen
     int replicaId = 0;
     byte[] serverColumn = getServerColumn(replicaId);
-    SortedMap<byte[], byte[]> serverMap = null;
+    SortedMap<byte[], byte[]> serverMap;
     serverMap = infoMap.tailMap(serverColumn, false);
 
     if (serverMap.isEmpty()) return new RegionLocations(locations);
@@ -1024,7 +1075,7 @@ public class MetaTableAccessor {
       HRegionLocation location = getRegionLocation(r, regionInfo, replicaId);
       // In case the region replica is newly created, it's location might be null. We usually do not
       // have HRL's in RegionLocations object with null ServerName. They are handled as null HRLs.
-      if (location == null || location.getServerName() == null) {
+      if (location.getServerName() == null) {
         locations.add(null);
       } else {
         locations.add(location);
@@ -1078,39 +1129,9 @@ public class MetaTableAccessor {
   }
 
   /**
-   * Returns the daughter regions by reading the corresponding columns of the catalog table
-   * Result.
-   * @param data a Result object from the catalog table scan
-   * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
-   * parent
-   */
-  public static PairOfSameType<RegionInfo> getDaughterRegions(Result data) {
-    RegionInfo splitA = getRegionInfo(data, HConstants.SPLITA_QUALIFIER);
-    RegionInfo splitB = getRegionInfo(data, HConstants.SPLITB_QUALIFIER);
-
-    return new PairOfSameType<>(splitA, splitB);
-  }
-
-  /**
-   * Returns the merge regions by reading the corresponding columns of the catalog table
-   * Result.
-   * @param data a Result object from the catalog table scan
-   * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
-   * parent
-   */
-  public static PairOfSameType<RegionInfo> getMergeRegions(Result data) {
-    RegionInfo mergeA = getRegionInfo(data, HConstants.MERGEA_QUALIFIER);
-    RegionInfo mergeB = getRegionInfo(data, HConstants.MERGEB_QUALIFIER);
-
-    return new PairOfSameType<>(mergeA, mergeB);
-  }
-
-  /**
    * Fetch table state for given table from META table
    * @param conn connection to use
    * @param tableName table to fetch state for
-   * @return state
-   * @throws IOException
    */
   @Nullable
   public static TableState getTableState(Connection conn, TableName tableName)
@@ -1128,19 +1149,16 @@ public class MetaTableAccessor {
    * Fetch table states from META table
    * @param conn connection to use
    * @return map {tableName -&gt; state}
-   * @throws IOException
    */
   public static Map<TableName, TableState> getTableStates(Connection conn)
       throws IOException {
     final Map<TableName, TableState> states = new LinkedHashMap<>();
-    Visitor collector = new Visitor() {
-      @Override
-      public boolean visit(Result r) throws IOException {
-        TableState state = getTableState(r);
-        if (state != null)
-          states.put(state.getTableName(), state);
-        return true;
+    Visitor collector = r -> {
+      TableState state = getTableState(r);
+      if (state != null) {
+        states.put(state.getTableName(), state);
       }
+      return true;
     };
     fullScanTables(conn, collector);
     return states;
@@ -1150,7 +1168,6 @@ public class MetaTableAccessor {
    * Updates state in META
    * @param conn connection to use
    * @param tableName table to look for
-   * @throws IOException
    */
   public static void updateTableState(Connection conn, TableName tableName,
       TableState.State actual) throws IOException {
@@ -1160,7 +1177,6 @@ public class MetaTableAccessor {
   /**
    * Decode table state from META Result.
    * Should contain cell from HConstants.TABLE_FAMILY
-   * @param r result
    * @return null if not found
    */
   @Nullable
@@ -1236,7 +1252,7 @@ public class MetaTableAccessor {
    */
   public static abstract class DefaultVisitorBase implements Visitor {
 
-    public DefaultVisitorBase() {
+    DefaultVisitorBase() {
       super();
     }
 
@@ -1324,8 +1340,7 @@ public class MetaTableAccessor {
   }
 
   /**
-   * Generates and returns a Delete containing the region info for the catalog
-   * table
+   * Generates and returns a Delete containing the region info for the catalog table
    */
   private static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
     if (regionInfo == null) {
@@ -1339,7 +1354,7 @@ public class MetaTableAccessor {
   /**
    * Adds split daughters to the Put
    */
-  public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB)
+  private static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB)
       throws IOException {
     if (splitA != null) {
       put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
@@ -1459,28 +1474,6 @@ public class MetaTableAccessor {
     }
   }
 
-  /**
-   * Execute the passed <code>mutations</code> against <code>hbase:meta</code> table.
-   * @param connection connection we're using
-   * @param mutations Puts and Deletes to execute on hbase:meta
-   * @throws IOException
-   */
-  public static void mutateMetaTable(final Connection connection,
-                                     final List<Mutation> mutations)
-    throws IOException {
-    Table t = getMetaHTable(connection);
-    try {
-      debugLogMutations(mutations);
-      t.batch(mutations, null);
-    } catch (InterruptedException e) {
-      InterruptedIOException ie = new InterruptedIOException(e.getMessage());
-      ie.initCause(e);
-      throw ie;
-    } finally {
-      t.close();
-    }
-  }
-
   private static void addRegionStateToPut(Put put, RegionState.State state) throws IOException {
     put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
         .setRow(put.getRow())
@@ -1506,15 +1499,12 @@ public class MetaTableAccessor {
    */
   public static void addSplitsToParent(Connection connection, RegionInfo regionInfo,
       RegionInfo splitA, RegionInfo splitB) throws IOException {
-    Table meta = getMetaHTable(connection);
-    try {
+    try (Table meta = getMetaHTable(connection)) {
       Put put = makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
       addDaughtersToPut(put, splitA, splitB);
       meta.put(put);
       debugLogMutation(put);
       LOG.debug("Added region {}", regionInfo.getRegionNameAsString());
-    } finally {
-      meta.close();
     }
   }
 
@@ -1552,7 +1542,6 @@ public class MetaTableAccessor {
    * is CLOSED.
    * @param connection connection we're using
    * @param regionInfos region information list
-   * @param regionReplication
    * @param ts desired timestamp
    * @throws IOException if problem connecting or updating meta
    */
@@ -1576,58 +1565,59 @@ public class MetaTableAccessor {
     LOG.info("Added {} regions to meta.", puts.size());
   }
 
+  static Put addMergeRegions(Put put, Collection<RegionInfo> mergeRegions) throws IOException {
+    int limit = 10000; // Arbitrary limit. No room in our formatted 'task0000' below for more.
+    int max = mergeRegions.size();
+    if (max > limit) {
+      // Should never happen!!!!! But just in case.
+      throw new RuntimeException("Can't merge " + max + " regions in one go; " + limit +
+          " is upper-limit.");
+    }
+    int counter = 0;
+    for (RegionInfo ri: mergeRegions) {
+      String qualifier = String.format(HConstants.MERGE_QUALIFIER_PREFIX_STR + "%04d", counter++);
+      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).
+          setRow(put.getRow()).
+          setFamily(HConstants.CATALOG_FAMILY).
+          setQualifier(Bytes.toBytes(qualifier)).
+          setTimestamp(put.getTimestamp()).
+          setType(Type.Put).
+          setValue(RegionInfo.toByteArray(ri)).
+          build());
+    }
+    return put;
+  }
+
   /**
-   * Merge the two regions into one in an atomic operation. Deletes the two merging regions in
-   * hbase:meta and adds the merged region with the information of two merging regions.
+   * Merge regions into one in an atomic operation. Deletes the merging regions in
+   * hbase:meta and adds the merged region.
    * @param connection connection we're using
    * @param mergedRegion the merged region
-   * @param regionA merge parent region A
-   * @param regionAOpenSeqNum the next open sequence id for region A, used by serial replication. -1
-   *          if not necessary.
-   * @param regionB merge parent region B
-   * @param regionBOpenSeqNum the next open sequence id for region B, used by serial replication. -1
-   *          if not necessary.
+   * @param parentSeqNum Parent regions to merge and their next open sequence id used
+   *   by serial replication. Set to -1 if not needed by this table.
    * @param sn the location of the region
    */
   public static void mergeRegions(Connection connection, RegionInfo mergedRegion,
-      RegionInfo regionA, long regionAOpenSeqNum, RegionInfo regionB, long regionBOpenSeqNum,
-      ServerName sn, int regionReplication) throws IOException {
+        Map<RegionInfo, Long> parentSeqNum, ServerName sn, int regionReplication)
+      throws IOException {
     try (Table meta = getMetaHTable(connection)) {
-      long time = EnvironmentEdgeManager.currentTime();
+      long time = HConstants.LATEST_TIMESTAMP;
       List<Mutation> mutations = new ArrayList<>();
-
-      List<RegionInfo> replicationParents = new ArrayList<>(2);
-      // Deletes for merging regions
-      mutations.add(makeDeleteFromRegionInfo(regionA, time));
-      if (regionAOpenSeqNum > 0) {
-        mutations.add(makePutForReplicationBarrier(regionA, regionAOpenSeqNum, time));
-        replicationParents.add(regionA);
-      }
-      mutations.add(makeDeleteFromRegionInfo(regionB, time));
-      if (regionBOpenSeqNum > 0) {
-        mutations.add(makePutForReplicationBarrier(regionB, regionBOpenSeqNum, time));
-        replicationParents.add(regionB);
+      List<RegionInfo> replicationParents = new ArrayList<>();
+      for (Map.Entry<RegionInfo, Long> e: parentSeqNum.entrySet()) {
+        RegionInfo ri = e.getKey();
+        long seqNum = e.getValue();
+        // Deletes for merging regions
+        mutations.add(makeDeleteFromRegionInfo(ri, time));
+        if (seqNum > 0) {
+          mutations.add(makePutForReplicationBarrier(ri, seqNum, time));
+          replicationParents.add(ri);
+        }
       }
-
       // Put for parent
       Put putOfMerged = makePutFromRegionInfo(mergedRegion, time);
-      putOfMerged.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
-              .setRow(putOfMerged.getRow())
-              .setFamily(HConstants.CATALOG_FAMILY)
-              .setQualifier(HConstants.MERGEA_QUALIFIER)
-              .setTimestamp(putOfMerged.getTimestamp())
-              .setType(Type.Put)
-              .setValue(RegionInfo.toByteArray(regionA))
-              .build())
-          .add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
-              .setRow(putOfMerged.getRow())
-              .setFamily(HConstants.CATALOG_FAMILY)
-              .setQualifier(HConstants.MERGEB_QUALIFIER)
-              .setTimestamp(putOfMerged.getTimestamp())
-              .setType(Type.Put)
-              .setValue(RegionInfo.toByteArray(regionB))
-              .build());
-      // Set initial state to CLOSED
+      putOfMerged = addMergeRegions(putOfMerged, parentSeqNum.keySet());
+      // Set initial state to CLOSED.
       // NOTE: If initial state is not set to CLOSED then merged region gets added with the
       // default OFFLINE state. If Master gets restarted after this step, start up sequence of
       // master tries to assign this offline region. This is followed by re-assignments of the
@@ -1642,8 +1632,8 @@ public class MetaTableAccessor {
         addLocation(putOfMerged, sn, 1, mergedRegion.getReplicaId());
       }
 
-      // Add empty locations for region replicas of the merged region so that number of replicas can
-      // be cached whenever the primary region is looked up from meta
+      // Add empty locations for region replicas of the merged region so that number of replicas
+      // can be cached whenever the primary region is looked up from meta
       for (int i = 1; i < regionReplication; i++) {
         addEmptyLocation(putOfMerged, i);
       }
@@ -1652,7 +1642,7 @@ public class MetaTableAccessor {
         addReplicationParent(putOfMerged, replicationParents);
       }
       byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString() + HConstants.DELIMITER);
-      multiMutate(connection, meta, tableRow, mutations);
+      multiMutate(meta, tableRow, mutations);
     }
   }
 
@@ -1706,7 +1696,7 @@ public class MetaTableAccessor {
       }
 
       byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
-      multiMutate(connection, meta, tableRow, putParent, putA, putB);
+      multiMutate(meta, tableRow, putParent, putA, putB);
     }
   }
 
@@ -1725,7 +1715,7 @@ public class MetaTableAccessor {
    * Construct PUT for given state
    * @param state new state
    */
-  public static Put makePutFromTableState(TableState state, long ts) {
+  private static Put makePutFromTableState(TableState state, long ts) {
     Put put = new Put(state.getTableName().getName(), ts);
     put.addColumn(getTableFamily(), getTableStateColumn(), state.convert().toByteArray());
     return put;
@@ -1745,16 +1735,16 @@ public class MetaTableAccessor {
     LOG.info("Deleted table " + table + " state from META");
   }
 
-  private static void multiMutate(Connection connection, Table table, byte[] row,
+  private static void multiMutate(Table table, byte[] row,
       Mutation... mutations) throws IOException {
-    multiMutate(connection, table, row, Arrays.asList(mutations));
+    multiMutate(table, row, Arrays.asList(mutations));
   }
 
   /**
    * Performs an atomic multi-mutate operation against the given table.
    */
-  private static void multiMutate(Connection connection, final Table table, byte[] row,
-      final List<Mutation> mutations) throws IOException {
+  private static void multiMutate(final Table table, byte[] row, final List<Mutation> mutations)
+      throws IOException {
     debugLogMutations(mutations);
     Batch.Call<MultiRowMutationService, MutateRowsResponse> callable =
       new Batch.Call<MultiRowMutationService, MutateRowsResponse>() {
@@ -1838,7 +1828,6 @@ public class MetaTableAccessor {
    * Deletes the specified region from META.
    * @param connection connection we're using
    * @param regionInfo region to be deleted from META
-   * @throws IOException
    */
   public static void deleteRegionInfo(Connection connection, RegionInfo regionInfo)
       throws IOException {
@@ -1863,7 +1852,8 @@ public class MetaTableAccessor {
    * @param connection connection we're using
    * @param regionsInfo list of regions to be deleted from META
    */
-  public static void deleteRegionInfos(Connection connection, List<RegionInfo> regionsInfo, long ts)
+  private static void deleteRegionInfos(Connection connection, List<RegionInfo> regionsInfo,
+        long ts)
       throws IOException {
     List<Delete> deletes = new ArrayList<>(regionsInfo.size());
     for (RegionInfo hri : regionsInfo) {
@@ -1899,21 +1889,29 @@ public class MetaTableAccessor {
   }
 
   /**
-   * Deletes merge qualifiers for the specified merged region.
+   * Deletes merge qualifiers for the specified merge region.
    * @param connection connection we're using
-   * @param mergedRegion the merged region
+   * @param mergeRegion the merged region
    */
-  public static void deleteMergeQualifiers(Connection connection, final RegionInfo mergedRegion)
+  public static void deleteMergeQualifiers(Connection connection, final RegionInfo mergeRegion)
       throws IOException {
-    long time = EnvironmentEdgeManager.currentTime();
-    Delete delete = new Delete(mergedRegion.getRegionName());
-    delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time);
-    delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER, time);
+    Delete delete = new Delete(mergeRegion.getRegionName());
+    // NOTE: We are doing a new hbase:meta read here.
+    Cell [] cells = getMergeRegionsRaw(connection, mergeRegion.getRegionName());
+    if (cells == null || cells.length == 0) {
+      return;
+    }
+    List<byte[]> qualifiers = new ArrayList<>(cells.length);
+    for (Cell cell : cells) {
+      byte[] qualifier = CellUtil.cloneQualifier(cell);
+      qualifiers.add(qualifier);
+      delete.addColumns(getCatalogFamily(), qualifier, HConstants.LATEST_TIMESTAMP);
+    }
     deleteFromMetaTable(connection, delete);
-    LOG.info("Deleted references in merged region "
-      + mergedRegion.getRegionNameAsString() + ", qualifier="
-      + Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier="
-      + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
+    LOG.info("Deleted merge references in " + mergeRegion.getRegionNameAsString() +
+        ", deleted qualifiers " +
+        qualifiers.stream().map(Bytes::toStringBinary).
+            collect(Collectors.joining(", ")));
   }
 
   public static Put addRegionInfo(final Put p, final RegionInfo hri)
@@ -2056,7 +2054,7 @@ public class MetaTableAccessor {
     private final RegionState.State state;
     private final List<byte[]> parentRegionNames;
 
-    public ReplicationBarrierResult(long[] barriers, State state, List<byte[]> parentRegionNames) {
+    ReplicationBarrierResult(long[] barriers, State state, List<byte[]> parentRegionNames) {
       this.barriers = barriers;
       this.state = state;
       this.parentRegionNames = parentRegionNames;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index e8afead..f2829e8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -70,8 +70,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
  */
 @InterfaceAudience.Public
 public interface RegionInfo {
-  public static final RegionInfo UNDEFINED =
-      RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
+  RegionInfo UNDEFINED = RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
   /**
    * Separator used to demarcate the encodedName in a region name
    * in the new format. See description on new format above.
@@ -141,11 +140,16 @@ public interface RegionInfo {
       }
 
       int replicaDiff = lhs.getReplicaId() - rhs.getReplicaId();
-      if (replicaDiff != 0) return replicaDiff;
+      if (replicaDiff != 0) {
+        return replicaDiff;
+      }
 
-      if (lhs.isOffline() == rhs.isOffline())
+      if (lhs.isOffline() == rhs.isOffline()) {
         return 0;
-      if (lhs.isOffline() == true) return -1;
+      }
+      if (lhs.isOffline()) {
+        return -1;
+      }
 
       return 1;
   };
@@ -224,8 +228,6 @@ public interface RegionInfo {
   boolean isMetaRegion();
 
   /**
-   * @param rangeStartKey
-   * @param rangeEndKey
    * @return true if the given inclusive range of rows is fully contained
    * by this region. For example, if the region is foo,a,g and this is
    * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
@@ -235,7 +237,6 @@ public interface RegionInfo {
   boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey);
 
   /**
-   * @param row
    * @return true if the given row falls in this region.
    */
   boolean containsRow(byte[] row);
@@ -339,9 +340,7 @@ public interface RegionInfo {
 
   /**
    * Gets the start key from the specified region name.
-   * @param regionName
    * @return Start key.
-   * @throws java.io.IOException
    */
   static byte[] getStartKey(final byte[] regionName) throws IOException {
     return parseRegionName(regionName)[1];
@@ -362,7 +361,6 @@ public interface RegionInfo {
   }
 
   /**
-   * @param bytes
    * @return A deserialized {@link RegionInfo}
    * or null if we failed deserialize or passed bytes null
    */
@@ -373,9 +371,6 @@ public interface RegionInfo {
   }
 
   /**
-   * @param bytes
-   * @param offset
-   * @param len
    * @return A deserialized {@link RegionInfo} or null
    *  if we failed deserialize or passed bytes null
    */
@@ -392,7 +387,6 @@ public interface RegionInfo {
   /**
    * @param bytes A pb RegionInfo serialized with a pb magic prefix.
    * @return A deserialized {@link RegionInfo}
-   * @throws DeserializationException
    */
   @InterfaceAudience.Private
   static RegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
@@ -405,7 +399,6 @@ public interface RegionInfo {
    * @param offset starting point in the byte array
    * @param len length to read on the byte array
    * @return A deserialized {@link RegionInfo}
-   * @throws DeserializationException
    */
   @InterfaceAudience.Private
   static RegionInfo parseFrom(final byte [] bytes, int offset, int len)
@@ -426,9 +419,8 @@ public interface RegionInfo {
   }
 
   /**
-   * Check whether two regions are adjacent
-   * @param regionA
-   * @param regionB
+   * Check whether two regions are adjacent; i.e. lies just before or just
+   * after in a table.
    * @return true if two regions are adjacent
    */
   static boolean areAdjacent(RegionInfo regionA, RegionInfo regionB) {
@@ -436,20 +428,19 @@ public interface RegionInfo {
       throw new IllegalArgumentException(
       "Can't check whether adjacent for null region");
     }
+    if (!regionA.getTable().equals(regionB.getTable())) {
+      return false;
+    }
     RegionInfo a = regionA;
     RegionInfo b = regionB;
     if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) {
       a = regionB;
       b = regionA;
     }
-    if (Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0) {
-      return true;
-    }
-    return false;
+    return Bytes.equals(a.getEndKey(), b.getStartKey());
   }
 
   /**
-   * @param ri
    * @return This instance serialized as protobuf w/ a magic pb prefix.
    * @see #parseFrom(byte[])
    */
@@ -473,7 +464,6 @@ public interface RegionInfo {
 
   /**
    * Make a region name of passed parameters.
-   * @param tableName
    * @param startKey Can be null
    * @param regionid Region id (Usually timestamp from when region was created).
    * @param newFormat should we create the region name in the new format
@@ -487,7 +477,6 @@ public interface RegionInfo {
 
   /**
    * Make a region name of passed parameters.
-   * @param tableName
    * @param startKey Can be null
    * @param id Region id (Usually timestamp from when region was created).
    * @param newFormat should we create the region name in the new format
@@ -501,10 +490,8 @@ public interface RegionInfo {
 
   /**
    * Make a region name of passed parameters.
-   * @param tableName
    * @param startKey Can be null
    * @param regionid Region id (Usually timestamp from when region was created).
-   * @param replicaId
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey, id and replicaId
@@ -517,7 +504,6 @@ public interface RegionInfo {
 
   /**
    * Make a region name of passed parameters.
-   * @param tableName
    * @param startKey Can be null
    * @param id Region id (Usually timestamp from when region was created).
    * @param newFormat should we create the region name in the new format
@@ -531,10 +517,8 @@ public interface RegionInfo {
 
   /**
    * Make a region name of passed parameters.
-   * @param tableName
    * @param startKey Can be null
    * @param id Region id (Usually timestamp from when region was created).
-   * @param replicaId
    * @param newFormat should we create the region name in the new format
    * @return Region name made of passed tableName, startKey, id and replicaId
    */
@@ -593,7 +577,7 @@ public interface RegionInfo {
       b[offset++] = ENC_SEPARATOR;
       System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
       offset += MD5_HEX_LENGTH;
-      b[offset++] = ENC_SEPARATOR;
+      b[offset] = ENC_SEPARATOR;
     }
 
     return b;
@@ -612,9 +596,7 @@ public interface RegionInfo {
 
   /**
    * Separate elements of a regionName.
-   * @param regionName
    * @return Array of byte[] containing tableName, startKey and id
-   * @throws IOException
    */
   static byte [][] parseRegionName(final byte[] regionName)
   throws IOException {
@@ -693,7 +675,6 @@ public interface RegionInfo {
    * be used to read back the instances.
    * @param infos RegionInfo objects to serialize
    * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
-   * @throws IOException
    */
   static byte[] toDelimitedByteArray(RegionInfo... infos) throws IOException {
     byte[][] bytes = new byte[infos.length][];
@@ -715,9 +696,7 @@ public interface RegionInfo {
   /**
    * Use this instead of {@link RegionInfo#toByteArray(RegionInfo)} when writing to a stream and you want to use
    * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want).
-   * @param ri
    * @return This instance serialized as a delimied protobuf w/ a magic pb prefix.
-   * @throws IOException
    */
   static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException {
     return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toRegionInfo(ri));
@@ -727,9 +706,7 @@ public interface RegionInfo {
    * Parses an RegionInfo instance from the passed in stream.
    * Presumes the RegionInfo was serialized to the stream with
    * {@link #toDelimitedByteArray(RegionInfo)}.
-   * @param in
    * @return An instance of RegionInfo.
-   * @throws IOException
    */
   static RegionInfo parseFrom(final DataInputStream in) throws IOException {
     // I need to be able to move back in the stream if this is not a pb
@@ -757,28 +734,23 @@ public interface RegionInfo {
    * @param offset the start offset into the byte[] buffer
    * @param length how far we should read into the byte[] buffer
    * @return All the RegionInfos that are in the byte array. Keeps reading till we hit the end.
-   * @throws IOException
    */
   static List<RegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
                                              final int length) throws IOException {
     if (bytes == null) {
       throw new IllegalArgumentException("Can't build an object with empty bytes array");
     }
-    DataInputBuffer in = new DataInputBuffer();
     List<RegionInfo> ris = new ArrayList<>();
-    try {
+    try (DataInputBuffer in = new DataInputBuffer()) {
       in.reset(bytes, offset, length);
       while (in.available() > 0) {
         RegionInfo ri = parseFrom(in);
         ris.add(ri);
       }
-    } finally {
-      in.close();
     }
     return ris;
   }
 
-
   /**
    * @return True if this is first Region in Table
    */
@@ -794,10 +766,20 @@ public interface RegionInfo {
   }
 
   /**
-   * @return True if regions are adjacent, if 'after' next. Does not do tablename compare.
+   * @return True if region is next, adjacent but 'after' this one.
+   * @see #isAdjacent(RegionInfo)
+   * @see #areAdjacent(RegionInfo, RegionInfo)
    */
   default boolean isNext(RegionInfo after) {
-    return Bytes.equals(getEndKey(), after.getStartKey());
+    return getTable().equals(after.getTable()) && Bytes.equals(getEndKey(), after.getStartKey());
+  }
+
+  /**
+   * @return True if region is adjacent, either just before or just after this one.
+   * @see #isNext(RegionInfo)
+   */
+  default boolean isAdjacent(RegionInfo other) {
+    return getTable().equals(other.getTable()) && areAdjacent(this, other);
   }
 
   /**
@@ -808,11 +790,13 @@ public interface RegionInfo {
   }
 
   /**
-   * @return True if an overlap in region range. Does not do tablename compare.
-   *   Does not check if <code>other</code> has degenerate range.
+   * @return True if an overlap in region range.
    * @see #isDegenerate()
    */
   default boolean isOverlap(RegionInfo other) {
+    if (!getTable().equals(other.getTable())) {
+      return false;
+    }
     int startKeyCompare = Bytes.compareTo(getStartKey(), other.getStartKey());
     if (startKeyCompare == 0) {
       return true;
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index e1977d4..9cee166 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -527,11 +527,31 @@ public final class HConstants {
   /** The upper-half split region column qualifier */
   public static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
 
-  /** The lower-half merge region column qualifier */
-  public static final byte[] MERGEA_QUALIFIER = Bytes.toBytes("mergeA");
+  /**
+   * Merge qualifier prefix.
+   * We used to only allow two regions merge; mergeA and mergeB.
+   * Now we allow many to merge. Each region to merge will be referenced
+   * in a column whose qualifier starts with this define.
+   */
+  public static final String MERGE_QUALIFIER_PREFIX_STR = "merge";
+  public static final byte [] MERGE_QUALIFIER_PREFIX =
+      Bytes.toBytes(MERGE_QUALIFIER_PREFIX_STR);
 
-  /** The upper-half merge region column qualifier */
-  public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB");
+  /**
+   * The lower-half merge region column qualifier
+   * @deprecated Since 2.3.0 and 2.2.1. Not used anymore. Instead we look for
+   *   the {@link #MERGE_QUALIFIER_PREFIX_STR} prefix.
+   */
+  @Deprecated
+  public static final byte[] MERGEA_QUALIFIER = Bytes.toBytes(MERGE_QUALIFIER_PREFIX_STR + "A");
+
+  /**
+   * The upper-half merge region column qualifier
+   * @deprecated Since 2.3.0 and 2.2.1. Not used anymore. Instead we look for
+   *   the {@link #MERGE_QUALIFIER_PREFIX_STR} prefix.
+   */
+  @Deprecated
+  public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes(MERGE_QUALIFIER_PREFIX_STR + "B");
 
   /** The catalog family as a string*/
   public static final String TABLE_FAMILY_STR = "table";
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
index df150ad..768ab00 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
@@ -757,6 +757,25 @@ public final class PrivateCellUtil {
       left.getQualifierLength(), buf, offset, length);
   }
 
+  /**
+   * Finds if the start of the qualifier part of the Cell matches <code>buf</code>
+   * @param left the cell with which we need to match the qualifier
+   * @param startsWith the serialized keyvalue format byte[]
+   * @return true if the qualifier have same staring characters, false otherwise
+   */
+  public static boolean qualifierStartsWith(final Cell left, final byte[] startsWith) {
+    if (startsWith == null || startsWith.length == 0) {
+      throw new IllegalArgumentException("Cannot pass an empty startsWith");
+    }
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(), startsWith.length,
+          startsWith, 0, startsWith.length);
+    }
+    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
+        startsWith.length, startsWith, 0, startsWith.length);
+  }
+
   public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
       final int flength, final byte[] qual, final int qoffset, final int qlength) {
     if (!matchingFamily(left, fam, foffset, flength)) {
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 44fa199..de33345 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -375,6 +375,8 @@ message GCRegionStateData {
   required RegionInfo region_info = 1;
 }
 
+// NOTE: This message is used by GCMergedRegionStateProcedure
+// AND GCMultipleMergedRegionStateProcedure.
 enum GCMergedRegionsState {
   GC_MERGED_REGIONS_PREPARE = 1;
   GC_MERGED_REGIONS_PURGE = 2;
@@ -382,11 +384,18 @@ enum GCMergedRegionsState {
 }
 
 message GCMergedRegionsStateData {
+  // Use GCMultipleMergedRegionsStateData instead.
+  option deprecated = true;
   required RegionInfo parent_a = 1;
   required RegionInfo parent_b = 2;
   required RegionInfo merged_child = 3;
 }
 
+message GCMultipleMergedRegionsStateData {
+  repeated RegionInfo parents = 1;
+  required RegionInfo merged_child = 2;
+}
+
 enum PeerModificationState {
   PRE_PEER_MODIFICATION = 1;
   UPDATE_PEER_STORAGE = 2;
@@ -544,4 +553,4 @@ enum SplitWALState {
   ACQUIRE_SPLIT_WAL_WORKER = 1;
   DISPATCH_WAL_TO_WORKER = 2;
   RELEASE_SPLIT_WORKER = 3;
-}
\ No newline at end of file
+}
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
index 1e0f7be..8e93acf 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
@@ -27,7 +27,7 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.UriInfo;
 import java.io.IOException;
-import java.util.Map;
+import java.util.List;
 
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.rest.model.TableInfoModel;
 import org.apache.hadoop.hbase.rest.model.TableRegionModel;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -78,13 +79,12 @@ public class RegionsResource extends ResourceBase {
       TableInfoModel model = new TableInfoModel(tableName.getNameAsString());
 
       Connection connection = ConnectionFactory.createConnection(servlet.getConfiguration());
-      @SuppressWarnings("deprecation")
-      Map<RegionInfo, ServerName> regions = MetaTableAccessor
-          .allTableRegions(connection, tableName);
+      List<Pair<RegionInfo, ServerName>> regions = MetaTableAccessor
+          .getTableRegionsAndLocations(connection, tableName);
       connection.close();
-      for (Map.Entry<RegionInfo,ServerName> e: regions.entrySet()) {
-        RegionInfo hri = e.getKey();
-        ServerName addr = e.getValue();
+      for (Pair<RegionInfo,ServerName> e: regions) {
+        RegionInfo hri = e.getFirst();
+        ServerName addr = e.getSecond();
         model.add(
           new TableRegionModel(tableName.getNameAsString(), hri.getRegionId(),
             hri.getStartKey(), hri.getEndKey(), addr.getHostAndPort()));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java
index 6869390..b8d4b09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java
@@ -428,10 +428,10 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
    * keep it simple.
    */
   @Override
-  public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA,
-      RegionInfo regionB) throws IOException {
+  public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo [] mergeParents)
+      throws IOException {
     Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
-    regionFNMap.put(merged, getFavoredNodes(regionA));
+    regionFNMap.put(merged, getFavoredNodes(mergeParents[0]));
     fnm.updateFavoredNodes(regionFNMap);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java
index 95160a6..322eb1df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -33,6 +33,6 @@ public interface FavoredNodesPromoter {
   void generateFavoredNodesForDaughter(List<ServerName> servers,
       RegionInfo parent, RegionInfo hriA, RegionInfo hriB) throws IOException;
 
-  void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo hriA,
-      RegionInfo hriB) throws IOException;
+  void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo [] mergeParents)
+      throws IOException;
 }
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
index e5537fb..25b7191 100644
--- 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
@@ -28,6 +28,7 @@ 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;
@@ -51,7 +52,7 @@ 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.GCMergedRegionsProcedure;
+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;
@@ -185,26 +186,15 @@ public class CatalogJanitor extends ScheduledChore {
           break;
         }
 
-        PairOfSameType<RegionInfo> p = MetaTableAccessor.getMergeRegions(e.getValue());
-        RegionInfo regionA = p.getFirst();
-        RegionInfo regionB = p.getSecond();
-        if (regionA == null || regionB == null) {
-          LOG.warn("Unexpected references regionA="
-              + (regionA == null ? "null" : regionA.getShortNameToLog())
-              + ",regionB="
-              + (regionB == null ? "null" : regionB.getShortNameToLog())
-              + " in merged region " + e.getKey().getShortNameToLog());
-        } else {
-          if (cleanMergeRegion(e.getKey(), regionA, regionB)) {
-            gcs++;
-          }
+        List<RegionInfo> parents = MetaTableAccessor.getMergeRegions(e.getValue().rawCells());
+        if (parents != null && cleanMergeRegion(e.getKey(), parents)) {
+          gcs++;
         }
       }
       // Clean split parents
       Map<RegionInfo, Result> splitParents = report.splitParents;
 
       // Now work on our list of found parents. See if any we can clean up.
-      // regions whose parents are still around
       HashSet<String> parentNotCleaned = new HashSet<>();
       for (Map.Entry<RegionInfo, Result> e : splitParents.entrySet()) {
         if (this.services.isInMaintenanceMode()) {
@@ -253,10 +243,10 @@ public class CatalogJanitor extends ScheduledChore {
    * 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
+   *   the files on the file system
    */
-  private boolean cleanMergeRegion(final RegionInfo mergedRegion,
-     final RegionInfo regionA, final RegionInfo regionB) throws IOException {
+  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 = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
@@ -269,17 +259,19 @@ public class CatalogJanitor extends ScheduledChore {
       LOG.warn("Merged region does not exist: " + mergedRegion.getEncodedName());
     }
     if (regionFs == null || !regionFs.hasReferences(htd)) {
-      LOG.debug("Deleting region " + regionA.getShortNameToLog() + " and "
-          + regionB.getShortNameToLog()
-          + " from fs because merged region no longer holds references");
+      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 GCMergedRegionsProcedure(pe.getEnvironment(),
-          mergedRegion, regionA, regionB));
-      // Remove from in-memory states
-      this.services.getAssignmentManager().getRegionStates().deleteRegion(regionA);
-      this.services.getAssignmentManager().getRegionStates().deleteRegion(regionB);
-      this.services.getServerManager().removeRegion(regionA);
-      this.services.getServerManager().removeRegion(regionB);
+      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;
@@ -327,11 +319,9 @@ public class CatalogJanitor extends ScheduledChore {
    */
   boolean cleanParent(final RegionInfo parent, Result rowContent)
   throws IOException {
-    // Check whether it is a merged region and not clean reference
-    // No necessary to check MERGEB_QUALIFIER because these two qualifiers will
-    // be inserted/deleted together
-    if (rowContent.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) {
-      // wait cleaning merge region first
+    // Check whether it is a merged region and if it is clean of references.
+    if (MetaTableAccessor.hasMergeRegions(rowContent.rawCells())) {
+      // Wait until clean of merge parent regions first
       return false;
     }
     // Run checks on each daughter split.
@@ -424,28 +414,19 @@ public class CatalogJanitor extends ScheduledChore {
   }
 
   /**
-   * Checks if the specified region has merge qualifiers, if so, try to clean
-   * them
-   * @return true if the specified region doesn't have merge qualifier now
+   * Checks if the specified region has merge qualifiers, if so, try to clean them.
+   * @return true if no info:merge* columns; i.e. the specified region doesn't have
+   *   any merge qualifiers.
    */
   public boolean cleanMergeQualifier(final RegionInfo region) throws IOException {
-    // Get merge regions if it is a merged region and already has merge
-    // qualifier
-    Pair<RegionInfo, RegionInfo> mergeRegions = MetaTableAccessor
-        .getRegionsFromMergeQualifier(this.services.getConnection(),
-          region.getRegionName());
-    if (mergeRegions == null
-        || (mergeRegions.getFirst() == null && mergeRegions.getSecond() == null)) {
+    // Get merge regions if it is a merged region and already has merge qualifier
+    List<RegionInfo> parents = MetaTableAccessor.getMergeRegions(this.services.getConnection(),
+        region.getRegionName());
+    if (parents == null || parents.isEmpty()) {
       // It doesn't have merge qualifier, no need to clean
       return true;
     }
-    // It shouldn't happen, we must insert/delete these two qualifiers together
-    if (mergeRegions.getFirst() == null || mergeRegions.getSecond() == null) {
-      LOG.error("Merged region " + region.getRegionNameAsString()
-          + " has only one merge qualifier in META.");
-      return false;
-    }
-    return cleanMergeRegion(region, mergeRegions.getFirst(), mergeRegions.getSecond());
+    return cleanMergeRegion(region, parents);
   }
 
   /**
@@ -580,7 +561,7 @@ public class CatalogJanitor extends ScheduledChore {
         if (regionInfo.isSplitParent()) { // splitParent means split and offline.
           this.report.splitParents.put(regionInfo, r);
         }
-        if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) {
+        if (MetaTableAccessor.hasMergeRegions(r.rawCells())) {
           this.report.mergedRegions.put(regionInfo, r);
         }
       }
@@ -755,7 +736,7 @@ public class CatalogJanitor extends ScheduledChore {
       try (Table t = connection.getTable(TableName.META_TABLE_NAME)) {
         Result r = t.get(g);
         byte [] row = g.getRow();
-        row[row.length - 2] <<= ((byte)row[row.length - 2]);
+        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));
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 b397bf6..0d37943 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
@@ -88,7 +88,6 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.exceptions.MergeRegionException;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
@@ -131,6 +130,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@@ -1873,40 +1873,20 @@ public class HMaster extends HRegionServer implements MasterServices {
   public long mergeRegions(
       final RegionInfo[] regionsToMerge,
       final boolean forcible,
-      final long nonceGroup,
+      final long ng,
       final long nonce) throws IOException {
     checkInitialized();
 
-    assert(regionsToMerge.length == 2);
-
-    TableName tableName = regionsToMerge[0].getTable();
-    if (tableName == null || regionsToMerge[1].getTable() == null) {
-      throw new UnknownRegionException ("Can't merge regions without table associated");
-    }
-
-    if (!tableName.equals(regionsToMerge[1].getTable())) {
-      throw new IOException (
-        "Cannot merge regions from two different tables " + regionsToMerge[0].getTable()
-        + " and " + regionsToMerge[1].getTable());
-    }
-
-    if (RegionInfo.COMPARATOR.compare(regionsToMerge[0], regionsToMerge[1]) == 0) {
-      throw new MergeRegionException(
-        "Cannot merge a region to itself " + regionsToMerge[0] + ", " + regionsToMerge[1]);
-    }
-
-    return MasterProcedureUtil.submitProcedure(
-        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
+    final String mergeRegionsStr = Arrays.stream(regionsToMerge).
+      map(r -> RegionInfo.getShortNameToLog(r)).collect(Collectors.joining(", "));
+    return MasterProcedureUtil.submitProcedure(new NonceProcedureRunnable(this, ng, nonce) {
       @Override
       protected void run() throws IOException {
         getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge);
-
-        LOG.info(getClientIdAuditPrefix() + " Merge regions " +
-          regionsToMerge[0].getEncodedName() + " and " + regionsToMerge[1].getEncodedName());
-
+        String aid = getClientIdAuditPrefix();
+        LOG.info("{} merge regions {}", aid, mergeRegionsStr);
         submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(),
-          regionsToMerge, forcible));
-
+            regionsToMerge, forcible));
         getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge);
       }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 89d6e4d..6889352 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -836,9 +836,8 @@ public class AssignmentManager {
     return new SplitTableRegionProcedure(getProcedureEnvironment(), regionToSplit, splitKey);
   }
 
-  public MergeTableRegionsProcedure createMergeProcedure(final RegionInfo regionToMergeA,
-      final RegionInfo regionToMergeB) throws IOException {
-    return new MergeTableRegionsProcedure(getProcedureEnvironment(), regionToMergeA,regionToMergeB);
+  public MergeTableRegionsProcedure createMergeProcedure(RegionInfo ... ris) throws IOException {
+    return new MergeTableRegionsProcedure(getProcedureEnvironment(), ris, false);
   }
 
   /**
@@ -1774,23 +1773,28 @@ public class AssignmentManager {
   }
 
   /**
-   * When called here, the merge has happened. The two merged regions have been
+   * When called here, the merge has happened. The merged regions have been
    * unassigned and the above markRegionClosed has been called on each so they have been
    * disassociated from a hosting Server. The merged region will be open after this call. The
-   * merged regions are removed from hbase:meta below> Later they are deleted from the filesystem
+   * merged regions are removed from hbase:meta below. Later they are deleted from the filesystem
    * by the catalog janitor running against hbase:meta. It notices when the merged region no
-   * longer holds references to the old regions.
+   * longer holds references to the old regions (References are deleted after a compaction
+   * rewrites what the Reference points at but not until the archiver chore runs, are the
+   * References removed).
    */
   public void markRegionAsMerged(final RegionInfo child, final ServerName serverName,
-      final RegionInfo mother, final RegionInfo father) throws IOException {
+        RegionInfo [] mergeParents)
+      throws IOException {
     final RegionStateNode node = regionStates.getOrCreateRegionStateNode(child);
     node.setState(State.MERGED);
-    regionStates.deleteRegion(mother);
-    regionStates.deleteRegion(father);
-    regionStateStore.mergeRegions(child, mother, father, serverName);
+    for (RegionInfo ri: mergeParents) {
+      regionStates.deleteRegion(ri);
+
+    }
+    regionStateStore.mergeRegions(child, mergeParents, serverName);
     if (shouldAssignFavoredNodes(child)) {
       ((FavoredNodesPromoter)getBalancer()).
-        generateFavoredNodesForMergedRegion(child, mother, father);
+        generateFavoredNodesForMergedRegion(child, mergeParents);
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
index 610003d..a8097e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
@@ -40,8 +40,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.G
  * <p>This is a Table Procedure. We take a read lock on the Table.
  * We do NOT keep a lock for the life of this procedure. The subprocedures
  * take locks on the Regions they are purging.
+ * @deprecated 2.3.0 Use {@link GCMultipleMergedRegionsProcedure}.
  */
 @InterfaceAudience.Private
+@Deprecated
 public class GCMergedRegionsProcedure
 extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
   private static final Logger LOG = LoggerFactory.getLogger(GCMergedRegionsProcedure.class);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMultipleMergedRegionsProcedure.java
similarity index 54%
copy from hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
copy to hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMultipleMergedRegionsProcedure.java
index 610003d..285891e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMultipleMergedRegionsProcedure.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
@@ -27,39 +29,36 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMultipleMergedRegionsStateData;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * GC regions that have been Merged.
- * Caller determines if it is GC time. This Procedure does not check.
- * <p>This is a Table Procedure. We take a read lock on the Table.
- * We do NOT keep a lock for the life of this procedure. The subprocedures
- * take locks on the Regions they are purging.
+ * GC regions that have been Merged. Caller determines if it is GC time. This Procedure does not
+ * check. This is a Table Procedure. We take a read lock on the Table. We do NOT keep a lock for
+ * the life of this procedure. The sub-procedures take locks on the Regions they are purging.
+ * Replaces a Procedure that did two regions only at a time instead doing multiple merges in the
+ * one go; only difference from the old {@link GCMergedRegionsState} is the serialization; this
+ * class has a different serialization profile writing out more than just two regions.
  */
-@InterfaceAudience.Private
-public class GCMergedRegionsProcedure
-extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
-  private static final Logger LOG = LoggerFactory.getLogger(GCMergedRegionsProcedure.class);
-  private RegionInfo father;
-  private RegionInfo mother;
+@org.apache.yetus.audience.InterfaceAudience.Private
+public class GCMultipleMergedRegionsProcedure extends
+    AbstractStateMachineTableProcedure<GCMergedRegionsState> {
+  private static final Logger LOG = LoggerFactory.getLogger(GCMultipleMergedRegionsProcedure.class);
+  private List<RegionInfo> parents;
   private RegionInfo mergedChild;
 
-  public GCMergedRegionsProcedure(final MasterProcedureEnv env,
-      final RegionInfo mergedChild,
-      final RegionInfo father,
-      final RegionInfo mother) {
+  public GCMultipleMergedRegionsProcedure(final MasterProcedureEnv env,
+      final RegionInfo mergedChild, final List<RegionInfo> parents) {
     super(env);
-    this.father = father;
-    this.mother = mother;
+    this.parents = parents;
     this.mergedChild = mergedChild;
   }
 
-  public GCMergedRegionsProcedure() {
+  public GCMultipleMergedRegionsProcedure() {
     // Required by the Procedure framework to create the procedure on replay
     super();
   }
@@ -71,39 +70,42 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
 
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, GCMergedRegionsState state)
-  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     if (LOG.isTraceEnabled()) {
       LOG.trace(this + " execute state=" + state);
     }
     try {
       switch (state) {
-      case GC_MERGED_REGIONS_PREPARE:
-        // Nothing to do to prepare.
-        setNextState(GCMergedRegionsState.GC_MERGED_REGIONS_PURGE);
-        break;
-      case GC_MERGED_REGIONS_PURGE:
-        addChildProcedure(createGCRegionProcedures(env));
-        setNextState(GCMergedRegionsState.GC_REGION_EDIT_METADATA);
-        break;
-      case GC_REGION_EDIT_METADATA:
-        MetaTableAccessor.deleteMergeQualifiers(env.getMasterServices().getConnection(), mergedChild);
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+        case GC_MERGED_REGIONS_PREPARE:
+          // Nothing to do to prepare.
+          setNextState(GCMergedRegionsState.GC_MERGED_REGIONS_PURGE);
+          break;
+        case GC_MERGED_REGIONS_PURGE:
+          addChildProcedure(createGCRegionProcedures(env));
+          setNextState(GCMergedRegionsState.GC_REGION_EDIT_METADATA);
+          break;
+        case GC_REGION_EDIT_METADATA:
+          MetaTableAccessor.deleteMergeQualifiers(env.getMasterServices().getConnection(),
+              mergedChild);
+          return Flow.NO_MORE_STATE;
+        default:
+          throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException ioe) {
       // TODO: This is going to spew log?
-      LOG.warn("Error trying to GC merged regions " + this.father.getShortNameToLog() +
-          " & " + this.mother.getShortNameToLog() + "; retrying...", ioe);
+      LOG.warn("Error trying to GC merged regions {}; retrying...",
+          this.parents.stream().map(r -> RegionInfo.getShortNameToLog(r)).
+              collect(Collectors.joining(", ")),
+          ioe);
     }
     return Flow.HAS_MORE_STATE;
   }
 
   private GCRegionProcedure[] createGCRegionProcedures(final MasterProcedureEnv env) {
-    GCRegionProcedure [] procs = new GCRegionProcedure[2];
+    GCRegionProcedure [] procs = new GCRegionProcedure[this.parents.size()];
     int index = 0;
-    for (RegionInfo hri: new RegionInfo [] {this.father, this.mother}) {
-      GCRegionProcedure proc = new GCRegionProcedure(env, hri);
+    for (RegionInfo ri: this.parents) {
+      GCRegionProcedure proc = new GCRegionProcedure(env, ri);
       proc.setOwner(env.getRequestUser().getShortName());
       procs[index++] = proc;
     }
@@ -112,7 +114,7 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
 
   @Override
   protected void rollbackState(MasterProcedureEnv env, GCMergedRegionsState state)
-  throws IOException, InterruptedException {
+      throws IOException, InterruptedException {
     // no-op
   }
 
@@ -135,11 +137,11 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
   protected void serializeStateData(ProcedureStateSerializer serializer)
       throws IOException {
     super.serializeStateData(serializer);
-    final MasterProcedureProtos.GCMergedRegionsStateData.Builder msg =
-        MasterProcedureProtos.GCMergedRegionsStateData.newBuilder().
-        setParentA(ProtobufUtil.toRegionInfo(this.father)).
-        setParentB(ProtobufUtil.toRegionInfo(this.mother)).
-        setMergedChild(ProtobufUtil.toRegionInfo(this.mergedChild));
+    final GCMultipleMergedRegionsStateData.Builder msg =
+        GCMultipleMergedRegionsStateData.newBuilder().
+            addAllParents(this.parents.stream().map(ProtobufUtil::toRegionInfo).
+                collect(Collectors.toList())).
+            setMergedChild(ProtobufUtil.toRegionInfo(this.mergedChild));
     serializer.serialize(msg.build());
   }
 
@@ -147,10 +149,10 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
   protected void deserializeStateData(ProcedureStateSerializer serializer)
       throws IOException {
     super.deserializeStateData(serializer);
-    final MasterProcedureProtos.GCMergedRegionsStateData msg =
-        serializer.deserialize(MasterProcedureProtos.GCMergedRegionsStateData.class);
-    this.father = ProtobufUtil.toRegionInfo(msg.getParentA());
-    this.mother = ProtobufUtil.toRegionInfo(msg.getParentB());
+    final GCMultipleMergedRegionsStateData msg =
+        serializer.deserialize(GCMultipleMergedRegionsStateData.class);
+    this.parents = msg.getParentsList().stream().map(ProtobufUtil::toRegionInfo).
+        collect(Collectors.toList());
     this.mergedChild = ProtobufUtil.toRegionInfo(msg.getMergedChild());
   }
 
@@ -159,10 +161,9 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
     sb.append(getClass().getSimpleName());
     sb.append(" child=");
     sb.append(this.mergedChild.getShortNameToLog());
-    sb.append(", father=");
-    sb.append(this.father.getShortNameToLog());
-    sb.append(", mother=");
-    sb.append(this.mother.getShortNameToLog());
+    sb.append(", parents:");
+    sb.append(this.parents.stream().map(r -> RegionInfo.getShortNameToLog(r)).
+        collect(Collectors.joining(", ")));
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 404d89c..8375dd0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -27,7 +27,6 @@ import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -57,7 +56,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WALSplitUtil;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -72,12 +70,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
 
 /**
- * The procedure to Merge a region in a table.
- * <p/>
- * This procedure takes an exclusive table lock since it is working over multiple regions.
- * <p/>
- * It holds the lock for the life of the procedure.
- * <p/>
+ * The procedure to Merge regions in a table. This procedure takes an exclusive table
+ * lock since it is working over multiple regions. It holds the lock for the life of the procedure.
  * Throws exception on construction if determines context hostile to merge (cluster going down or
  * master is shutting down or table is disabled).
  */
@@ -85,153 +79,113 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.M
 public class MergeTableRegionsProcedure
     extends AbstractStateMachineTableProcedure<MergeTableRegionsState> {
   private static final Logger LOG = LoggerFactory.getLogger(MergeTableRegionsProcedure.class);
-  private Boolean traceEnabled;
   private ServerName regionLocation;
+
+  /**
+   * Two or more regions to merge, the 'merge parents'.
+   */
   private RegionInfo[] regionsToMerge;
+
+  /**
+   * The resulting merged region.
+   */
   private RegionInfo mergedRegion;
-  private boolean forcible;
+
+  private boolean force;
 
   public MergeTableRegionsProcedure() {
     // Required by the Procedure framework to create the procedure on replay
   }
 
   public MergeTableRegionsProcedure(final MasterProcedureEnv env,
-      final RegionInfo regionToMergeA, final RegionInfo regionToMergeB) throws IOException {
-    this(env, regionToMergeA, regionToMergeB, false);
-  }
-
-  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
-      final RegionInfo regionToMergeA, final RegionInfo regionToMergeB,
-      final boolean forcible) throws IOException {
-    this(env, new RegionInfo[] {regionToMergeA, regionToMergeB}, forcible);
-  }
-
-  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
-      final RegionInfo[] regionsToMerge, final boolean forcible)
+      final RegionInfo[] regionsToMerge, final boolean force)
       throws IOException {
     super(env);
-
-    // Check daughter regions and make sure that we have valid daughter regions
-    // before doing the real work. This check calls the super method #checkOnline also.
-    checkRegionsToMerge(env, regionsToMerge, forcible);
-
-    // WARN: make sure there is no parent region of the two merging regions in
-    // hbase:meta If exists, fixing up daughters would cause daughter regions(we
-    // have merged one) online again when we restart master, so we should clear
-    // the parent region to prevent the above case
-    // Since HBASE-7721, we don't need fix up daughters any more. so here do nothing
+    // Check parent regions. Make sure valid before starting work.
+    // This check calls the super method #checkOnline also.
+    checkRegionsToMerge(env, regionsToMerge, force);
+    // Sort the regions going into the merge.
+    Arrays.sort(regionsToMerge);
     this.regionsToMerge = regionsToMerge;
     this.mergedRegion = createMergedRegionInfo(regionsToMerge);
+    // Preflight depends on mergedRegion being set (at least).
     preflightChecks(env, true);
-    this.forcible = forcible;
-  }
-
-  private static void checkRegionsToMerge(MasterProcedureEnv env, final RegionInfo[] regionsToMerge,
-      final boolean forcible) throws MergeRegionException {
-    // For now, we only merge 2 regions.
-    // It could be extended to more than 2 regions in the future.
-    if (regionsToMerge == null || regionsToMerge.length != 2) {
-      throw new MergeRegionException("Expected to merge 2 regions, got: " +
-        Arrays.toString(regionsToMerge));
-    }
-
-    checkRegionsToMerge(env, regionsToMerge[0], regionsToMerge[1], forcible);
+    this.force = force;
   }
 
   /**
-   * One time checks.
+   * @throws MergeRegionException If unable to merge regions for whatever reasons.
    */
-  private static void checkRegionsToMerge(MasterProcedureEnv env, final RegionInfo regionToMergeA,
-      final RegionInfo regionToMergeB, final boolean forcible) throws MergeRegionException {
-    if (!regionToMergeA.getTable().equals(regionToMergeB.getTable())) {
-      throw new MergeRegionException("Can't merge regions from two different tables: " +
-        regionToMergeA + ", " + regionToMergeB);
-    }
-
-    if (regionToMergeA.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID ||
-        regionToMergeB.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
-      throw new MergeRegionException("Can't merge non-default replicas");
-    }
-
-    try {
-      checkOnline(env, regionToMergeA);
-      checkOnline(env, regionToMergeB);
-    } catch (DoNotRetryRegionException dnrre) {
-      throw new MergeRegionException(dnrre);
-    }
-
-    if (!RegionInfo.areAdjacent(regionToMergeA, regionToMergeB)) {
-      String msg = "Unable to merge non-adjacent regions " + regionToMergeA.getShortNameToLog() +
-          ", " + regionToMergeB.getShortNameToLog() + " where forcible = " + forcible;
-      LOG.warn(msg);
-      if (!forcible) {
-        throw new MergeRegionException(msg);
+  private static void checkRegionsToMerge(MasterProcedureEnv env, final RegionInfo[] regions,
+      final boolean force) throws MergeRegionException {
+    long count = Arrays.stream(regions).distinct().count();
+    if (regions.length != count) {
+      throw new MergeRegionException("Duplicate regions specified; cannot merge a region to " +
+          "itself. Passed in " + regions.length + " but only " + count + " unique.");
+    }
+    if (count < 2) {
+      throw new MergeRegionException("Need two Regions at least to run a Merge");
+    }
+    RegionInfo previous = null;
+    for (RegionInfo ri: regions) {
+      if (previous != null) {
+        if (!previous.getTable().equals(ri.getTable())) {
+          String msg = "Can't merge regions from different tables: " + previous + ", " + ri;
+          LOG.warn(msg);
+          throw new MergeRegionException(msg);
+        }
+        if (!force && !ri.isAdjacent(previous) && !ri.isOverlap(previous)) {
+          String msg = "Unable to merge non-adjacent or non-overlapping regions " +
+              previous.getShortNameToLog() + ", " + ri.getShortNameToLog() + " when force=false";
+          LOG.warn(msg);
+          throw new MergeRegionException(msg);
+        }
       }
-    }
-  }
 
+      if (ri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+        throw new MergeRegionException("Can't merge non-default replicas; " + ri);
+      }
+      try {
+        checkOnline(env, ri);
+      } catch (DoNotRetryRegionException dnrre) {
+        throw new MergeRegionException(dnrre);
+      }
 
-  private static RegionInfo createMergedRegionInfo(final RegionInfo[] regionsToMerge) {
-    return createMergedRegionInfo(regionsToMerge[0], regionsToMerge[1]);
+      previous = ri;
+    }
   }
 
   /**
-   * Create merged region info through the specified two regions
+   * Create merged region info by looking at passed in <code>regionsToMerge</code>
+   * to figure what extremes for start and end keys to use; merged region needs
+   * to have an extent sufficient to cover all regions-to-merge.
    */
-  private static RegionInfo createMergedRegionInfo(final RegionInfo regionToMergeA,
-      final RegionInfo regionToMergeB) {
-    // Choose the smaller as start key
-    final byte[] startKey;
-    if (RegionInfo.COMPARATOR.compare(regionToMergeA, regionToMergeB) <= 0) {
-      startKey = regionToMergeA.getStartKey();
-    } else {
-      startKey = regionToMergeB.getStartKey();
-    }
-
-    // Choose the bigger as end key
-    final byte[] endKey;
-    if (Bytes.equals(regionToMergeA.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
-        || (!Bytes.equals(regionToMergeB.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
-            && Bytes.compareTo(regionToMergeA.getEndKey(), regionToMergeB.getEndKey()) > 0)) {
-      endKey = regionToMergeA.getEndKey();
-    } else {
-      endKey = regionToMergeB.getEndKey();
-    }
-
-    // Merged region is sorted between two merging regions in META
-    return RegionInfoBuilder.newBuilder(regionToMergeA.getTable())
-        .setStartKey(startKey)
-        .setEndKey(endKey)
-        .setSplit(false)
-        .setRegionId(getMergedRegionIdTimestamp(regionToMergeA, regionToMergeB))
-        .build();
-  }
-
-  private static long getMergedRegionIdTimestamp(final RegionInfo regionToMergeA,
-      final RegionInfo regionToMergeB) {
-    long rid = EnvironmentEdgeManager.currentTime();
+  private static RegionInfo createMergedRegionInfo(final RegionInfo[] regionsToMerge) {
+    byte [] lowestStartKey = null;
+    byte [] highestEndKey = null;
     // Region Id is a timestamp. Merged region's id can't be less than that of
     // merging regions else will insert at wrong location in hbase:meta (See HBASE-710).
-    if (rid < regionToMergeA.getRegionId() || rid < regionToMergeB.getRegionId()) {
-      LOG.warn("Clock skew; merging regions id are " + regionToMergeA.getRegionId()
-          + " and " + regionToMergeB.getRegionId() + ", but current time here is " + rid);
-      rid = Math.max(regionToMergeA.getRegionId(), regionToMergeB.getRegionId()) + 1;
-    }
-    return rid;
-  }
-
-
-  private void removeNonDefaultReplicas(MasterProcedureEnv env) throws IOException {
-    AssignmentManagerUtil.removeNonDefaultReplicas(env, Stream.of(regionsToMerge),
-      getRegionReplication(env));
-  }
-
-  private void checkClosedRegions(MasterProcedureEnv env) throws IOException {
-    // theoretically this should not happen any more after we use TRSP, but anyway let's add a check
-    // here
-    for (RegionInfo region : regionsToMerge) {
-      AssignmentManagerUtil.checkClosedRegion(env, region);
+    long highestRegionId = -1;
+    for (RegionInfo ri: regionsToMerge) {
+      if (lowestStartKey == null) {
+        lowestStartKey = ri.getStartKey();
+      } else if (Bytes.compareTo(ri.getStartKey(), lowestStartKey) < 0) {
+        lowestStartKey = ri.getStartKey();
+      }
+      if (highestEndKey == null) {
+        highestEndKey = ri.getEndKey();
+      } else if (ri.isLast() || Bytes.compareTo(ri.getEndKey(), highestEndKey) > 0) {
+        highestEndKey = ri.getEndKey();
+      }
+      highestRegionId = ri.getRegionId() > highestRegionId? ri.getRegionId(): highestRegionId;
     }
+    // Merged region is sorted between two merging regions in META
+    return RegionInfoBuilder.newBuilder(regionsToMerge[0].getTable()).
+        setStartKey(lowestStartKey).
+        setEndKey(highestEndKey).
+        setSplit(false).
+        setRegionId(highestRegionId + 1/*Add one so new merged region is highest*/).
+        build();
   }
 
   @Override
@@ -291,9 +245,8 @@ public class MergeTableRegionsProcedure
           throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException e) {
-      String msg = "Error trying to merge regions " +
-        RegionInfo.getShortNameToLog(regionsToMerge) + " in the table " + getTableName() +
-           " (in state=" + state + ")";
+      String msg = "Error trying to merge " + RegionInfo.getShortNameToLog(regionsToMerge) +
+          " in " + getTableName() + " (in state=" + state + ")";
       if (!isRollbackSupported(state)) {
         // We reach a state that cannot be rolled back. We just need to keep retrying.
         LOG.warn(msg, e);
@@ -314,9 +267,7 @@ public class MergeTableRegionsProcedure
   @Override
   protected void rollbackState(final MasterProcedureEnv env, final MergeTableRegionsState state)
       throws IOException {
-    if (isTraceEnabled()) {
-      LOG.trace(this + " rollback state=" + state);
-    }
+    LOG.trace("{} rollback state={}", this, state);
 
     try {
       switch (state) {
@@ -376,6 +327,19 @@ public class MergeTableRegionsProcedure
     return true;
   }
 
+  private void removeNonDefaultReplicas(MasterProcedureEnv env) throws IOException {
+    AssignmentManagerUtil.removeNonDefaultReplicas(env, Stream.of(regionsToMerge),
+        getRegionReplication(env));
+  }
+
+  private void checkClosedRegions(MasterProcedureEnv env) throws IOException {
+    // Theoretically this should not happen any more after we use TRSP, but anyway
+    // let's add a check here
+    for (RegionInfo region : regionsToMerge) {
+      AssignmentManagerUtil.checkClosedRegion(env, region);
+    }
+  }
+
   @Override
   protected MergeTableRegionsState getState(final int stateId) {
     return MergeTableRegionsState.forNumber(stateId);
@@ -400,9 +364,9 @@ public class MergeTableRegionsProcedure
         MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
         .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
         .setMergedRegionInfo(ProtobufUtil.toRegionInfo(mergedRegion))
-        .setForcible(forcible);
-    for (int i = 0; i < regionsToMerge.length; ++i) {
-      mergeTableRegionsMsg.addRegionInfo(ProtobufUtil.toRegionInfo(regionsToMerge[i]));
+        .setForcible(force);
+    for (RegionInfo ri: regionsToMerge) {
+      mergeTableRegionsMsg.addRegionInfo(ProtobufUtil.toRegionInfo(ri));
     }
     serializer.serialize(mergeTableRegionsMsg.build());
   }
@@ -432,8 +396,8 @@ public class MergeTableRegionsProcedure
     sb.append(getTableName());
     sb.append(", regions=");
     sb.append(RegionInfo.getShortNameToLog(regionsToMerge));
-    sb.append(", forcibly=");
-    sb.append(forcible);
+    sb.append(", force=");
+    sb.append(force);
   }
 
   @Override
@@ -481,43 +445,14 @@ public class MergeTableRegionsProcedure
    */
   private boolean prepareMergeRegion(final MasterProcedureEnv env) throws IOException {
     // Fail if we are taking snapshot for the given table
-    if (env.getMasterServices().getSnapshotManager()
-      .isTakingSnapshot(regionsToMerge[0].getTable())) {
-      throw new MergeRegionException(
-        "Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
-          ", because we are taking snapshot for the table " + regionsToMerge[0].getTable());
+    TableName tn = regionsToMerge[0].getTable();
+    if (env.getMasterServices().getSnapshotManager().isTakingSnapshot(tn)) {
+      throw new MergeRegionException("Skip merging regions " +
+          RegionInfo.getShortNameToLog(regionsToMerge) + ", because we are snapshotting " + tn);
     }
-    // Note: the following logic assumes that we only have 2 regions to merge.  In the future,
-    // if we want to extend to more than 2 regions, the code needs to be modified a little bit.
-    CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();
-    boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
-    if (regionAHasMergeQualifier
-        || !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
-      String msg = "Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
-        ", because region "
-        + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
-              .getEncodedName()) + " has merge qualifier";
-      LOG.warn(msg);
-      throw new MergeRegionException(msg);
-    }
-
-    RegionStates regionStates = env.getAssignmentManager().getRegionStates();
-    RegionState regionStateA = regionStates.getRegionState(regionsToMerge[0].getEncodedName());
-    RegionState regionStateB = regionStates.getRegionState(regionsToMerge[1].getEncodedName());
-    if (regionStateA == null || regionStateB == null) {
-      throw new UnknownRegionException(
-        regionStateA == null ?
-            regionsToMerge[0].getEncodedName() : regionsToMerge[1].getEncodedName());
-    }
-
-    if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
-      throw new MergeRegionException(
-        "Unable to merge regions that are not online " + regionStateA + ", " + regionStateB);
-    }
-
     if (!env.getMasterServices().isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
-      String regionsStr = Arrays.deepToString(regionsToMerge);
-      LOG.warn("merge switch is off! skip merge of " + regionsStr);
+      String regionsStr = Arrays.deepToString(this.regionsToMerge);
+      LOG.warn("Merge switch is off! skip merge of " + regionsStr);
       super.setFailure(getClass().getSimpleName(),
           new IOException("Merge of " + regionsStr + " failed because merge switch is off"));
       return false;
@@ -531,30 +466,34 @@ public class MergeTableRegionsProcedure
       return false;
     }
 
-    // Ask the remote regionserver if regions are mergeable. If we get an IOE, report it
-    // along with the failure, so we can see why regions are not mergeable at this time.
-    IOException mergeableCheckIOE = null;
-    boolean mergeable = false;
-    RegionState current = regionStateA;
-    try {
-      mergeable = isMergeable(env, current);
-    } catch (IOException e) {
-      mergeableCheckIOE = e;
-    }
-    if (mergeable && mergeableCheckIOE == null) {
-      current = regionStateB;
+    CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();
+    RegionStates regionStates = env.getAssignmentManager().getRegionStates();
+    for (RegionInfo ri: this.regionsToMerge) {
+      if (!catalogJanitor.cleanMergeQualifier(ri)) {
+        String msg = "Skip merging " + RegionInfo.getShortNameToLog(regionsToMerge) +
+            ", because parent " + RegionInfo.getShortNameToLog(ri) + " has a merge qualifier";
+        LOG.warn(msg);
+        throw new MergeRegionException(msg);
+      }
+      RegionState state = regionStates.getRegionState(ri.getEncodedName());
+      if (state == null) {
+        throw new UnknownRegionException("No state for " + RegionInfo.getShortNameToLog(ri));
+      }
+      if (!state.isOpened()) {
+        throw new MergeRegionException("Unable to merge regions that are not online: " + ri);
+      }
+      // Ask the remote regionserver if regions are mergeable. If we get an IOE, report it
+      // along with the failure, so we can see why regions are not mergeable at this time.
       try {
-        mergeable = isMergeable(env, current);
+        if (!isMergeable(env, state)) {
+          return false;
+        }
       } catch (IOException e) {
-        mergeableCheckIOE = e;
+        IOException ioe = new IOException(RegionInfo.getShortNameToLog(ri) + " NOT mergeable", e);
+        super.setFailure(getClass().getSimpleName(), ioe);
+        return false;
       }
     }
-    if (!mergeable) {
-      IOException e = new IOException(current.getRegion().getShortNameToLog() + " NOT mergeable");
-      if (mergeableCheckIOE != null) e.initCause(mergeableCheckIOE);
-      super.setFailure(getClass().getSimpleName(), e);
-      return false;
-    }
 
     // Update region states to Merging
     setRegionStateToMerging(env);
@@ -589,8 +528,6 @@ public class MergeTableRegionsProcedure
 
   /**
    * Action after rollback a merge table regions action.
-   * @param env MasterProcedureEnv
-   * @throws IOException
    */
   private void postRollBackMergeRegions(final MasterProcedureEnv env) throws IOException {
     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
@@ -605,71 +542,88 @@ public class MergeTableRegionsProcedure
   private void setRegionStateToMerging(final MasterProcedureEnv env) {
     // Set State.MERGING to regions to be merged
     RegionStates regionStates = env.getAssignmentManager().getRegionStates();
-    regionStates.getRegionStateNode(regionsToMerge[0]).setState(State.MERGING);
-    regionStates.getRegionStateNode(regionsToMerge[1]).setState(State.MERGING);
+    for (RegionInfo ri: this.regionsToMerge) {
+      regionStates.getRegionStateNode(ri).setState(State.MERGING);
+    }
   }
 
   /**
-   * Create a merged region
-   * @param env MasterProcedureEnv
+   * Create merged region.
+   * The way the merge works is that we make a 'merges' temporary
+   * directory in the FIRST parent region to merge (Do not change this without
+   * also changing the rollback where we look in this FIRST region for the
+   * merge dir). We then collect here references to all the store files in all
+   * the parent regions including those of the FIRST parent region into a
+   * subdirectory, named for the resultant merged region. We then call
+   * commitMergeRegion. It finds this subdirectory of storefile references
+   * and moves them under the new merge region (creating the region layout
+   * as side effect). After assign of the new merge region, we will run a
+   * compaction. This will undo the references but the reference files remain
+   * in place until the archiver runs (which it does on a period as a chore
+   * in the RegionServer that hosts the merge region -- see
+   * CompactedHFilesDischarger). Once the archiver has moved aside the
+   * no-longer used references, the merge region no longer has references.
+   * The catalog janitor will notice when it runs next and it will remove
+   * the old parent regions.
    */
   private void createMergedRegion(final MasterProcedureEnv env) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
     final FileSystem fs = mfs.getFileSystem();
-    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
-    regionFs.createMergesDir();
-
-    mergeStoreFiles(env, regionFs, regionFs.getMergesDir());
-    HRegionFileSystem regionFs2 = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[1], false);
-    mergeStoreFiles(env, regionFs2, regionFs.getMergesDir());
-
-    regionFs.commitMergedRegion(mergedRegion);
+    HRegionFileSystem mergeRegionFs = null;
+    for (RegionInfo ri: this.regionsToMerge) {
+      HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+          env.getMasterConfiguration(), fs, tabledir, ri, false);
+      if (mergeRegionFs == null) {
+        mergeRegionFs = regionFs;
+        mergeRegionFs.createMergesDir();
+      }
+      mergeStoreFiles(env, regionFs, mergeRegionFs.getMergesDir());
+    }
+    assert mergeRegionFs != null;
+    mergeRegionFs.commitMergedRegion(mergedRegion);
 
-    //Prepare to create merged regions
+    // Prepare to create merged regions
     env.getAssignmentManager().getRegionStates().
         getOrCreateRegionStateNode(mergedRegion).setState(State.MERGING_NEW);
   }
 
   /**
-   * Create reference file(s) of merging regions under the merged directory
-   * @param env MasterProcedureEnv
-   * @param regionFs region file system
-   * @param mergedDir the temp directory of merged region
+   * Create reference file(s) to parent region hfiles in the <code>mergeDir</code>
+   * @param regionFs merge parent region file system
+   * @param mergeDir the temp directory in which we are accumulating references.
    */
   private void mergeStoreFiles(final MasterProcedureEnv env, final HRegionFileSystem regionFs,
-      final Path mergedDir) throws IOException {
+      final Path mergeDir) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     final Configuration conf = env.getMasterConfiguration();
     final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
-
     for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       String family = hcd.getNameAsString();
       final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
-
       if (storeFiles != null && storeFiles.size() > 0) {
         for (StoreFileInfo storeFileInfo : storeFiles) {
-          // Create reference file(s) of the region in mergedDir.
+          // Create reference file(s) to parent region file here in mergedDir.
           // As this procedure is running on master, use CacheConfig.DISABLED means
           // don't cache any block.
           regionFs.mergeStoreFile(mergedRegion, family, new HStoreFile(mfs.getFileSystem(),
               storeFileInfo, conf, CacheConfig.DISABLED, hcd.getBloomFilterType(), true),
-            mergedDir);
+            mergeDir);
         }
       }
     }
   }
 
   /**
-   * Clean up a merged region
-   * @param env MasterProcedureEnv
+   * Clean up a merged region on rollback after failure.
    */
   private void cleanupMergedRegion(final MasterProcedureEnv env) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
+    TableName tn = this.regionsToMerge[0].getTable();
+    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), tn);
     final FileSystem fs = mfs.getFileSystem();
+    // See createMergedRegion above where we specify the merge dir as being in the
+    // FIRST merge parent region.
     HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
       env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
     regionFs.cleanupMergedRegion(mergedRegion);
@@ -696,8 +650,8 @@ public class MergeTableRegionsProcedure
   }
 
   private int getRegionReplication(final MasterProcedureEnv env) throws IOException {
-    final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
-    return htd.getRegionReplication();
+    return env.getMasterServices().getTableDescriptors().get(getTableName()).
+        getRegionReplication();
   }
 
   /**
@@ -726,9 +680,8 @@ public class MergeTableRegionsProcedure
    * Add merged region to META and delete original regions.
    */
   private void updateMetaForMergedRegions(final MasterProcedureEnv env) throws IOException {
-    final ServerName serverName = getServerName(env);
-    env.getAssignmentManager().markRegionAsMerged(mergedRegion, serverName,
-      regionsToMerge[0], regionsToMerge[1]);
+    env.getAssignmentManager().markRegionAsMerged(mergedRegion, getServerName(env),
+        this.regionsToMerge);
   }
 
   /**
@@ -786,22 +739,10 @@ public class MergeTableRegionsProcedure
   }
 
   /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @return traceEnabled
-   */
-  private Boolean isTraceEnabled() {
-    if (traceEnabled == null) {
-      traceEnabled = LOG.isTraceEnabled();
-    }
-    return traceEnabled;
-  }
-
-  /**
    * @return The merged region. Maybe be null if called to early or we failed.
    */
   @VisibleForTesting
-  public RegionInfo getMergedRegion() {
+  RegionInfo getMergedRegion() {
     return this.mergedRegion;
   }
 
@@ -810,6 +751,6 @@ public class MergeTableRegionsProcedure
     // Abort means rollback. We can't rollback all steps. HBASE-18018 added abort to all
     // Procedures. Here is a Procedure that has a PONR and cannot be aborted once it enters this
     // range of steps; what do we do for these should an operator want to cancel them? HBASE-20022.
-    return isRollbackSupported(getCurrentState())? super.abort(env): false;
+    return isRollbackSupported(getCurrentState()) && super.abort(env);
   }
-}
+}
\ No newline at end of file
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index 169543a..5a8e7dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellBuilderFactory;
 import org.apache.hadoop.hbase.CellBuilderType;
@@ -256,17 +259,16 @@ public class RegionStateStore {
   // ============================================================================================
   //  Update Region Merging State helpers
   // ============================================================================================
-  public void mergeRegions(RegionInfo child, RegionInfo hriA, RegionInfo hriB,
-      ServerName serverName) throws IOException {
+  public void mergeRegions(RegionInfo child, RegionInfo [] parents, ServerName serverName)
+      throws IOException {
     TableDescriptor htd = getTableDescriptor(child.getTable());
-    long regionAOpenSeqNum = -1L;
-    long regionBOpenSeqNum = -1L;
-    if (htd.hasGlobalReplicationScope()) {
-      regionAOpenSeqNum = getOpenSeqNumForParentRegion(hriA);
-      regionBOpenSeqNum = getOpenSeqNumForParentRegion(hriB);
+    boolean globalScope = htd.hasGlobalReplicationScope();
+    Map<RegionInfo, Long> parentSeqNums = new HashMap<>(parents.length);
+    for (RegionInfo ri: parents) {
+      parentSeqNums.put(ri, globalScope? getOpenSeqNumForParentRegion(ri): -1);
     }
-    MetaTableAccessor.mergeRegions(master.getConnection(), child, hriA, regionAOpenSeqNum, hriB,
-      regionBOpenSeqNum, serverName, getRegionReplication(htd));
+    MetaTableAccessor.mergeRegions(master.getConnection(), child, parentSeqNums,
+        serverName, getRegionReplication(htd));
   }
 
   // ============================================================================================
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
index 5a6659e..1daa9e7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -543,9 +543,9 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
    * keep it simple.
    */
   @Override
-  public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA,
-      RegionInfo regionB) throws IOException {
-    updateFavoredNodesForRegion(merged, fnm.getFavoredNodes(regionA));
+  public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo [] mergeParents)
+      throws IOException {
+    updateFavoredNodesForRegion(merged, fnm.getFavoredNodes(mergeParents[0]));
   }
 
   /*
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 45a2627..d2888e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -326,8 +326,8 @@ public class HRegionFileSystem {
         if(stat.isDirectory()) {
           continue;
         }
-        if(StoreFileInfo.isReference(stat.getPath())) {
-          if (LOG.isTraceEnabled()) LOG.trace("Reference " + stat.getPath());
+        if (StoreFileInfo.isReference(stat.getPath())) {
+          LOG.trace("Reference {}", stat.getPath());
           return true;
         }
       }
@@ -758,22 +758,22 @@ public class HRegionFileSystem {
   }
 
   /**
-   * Create the region merges directory.
+   * Create the region merges directory, a temporary directory to accumulate
+   * merges in.
    * @throws IOException If merges dir already exists or we fail to create it.
    * @see HRegionFileSystem#cleanupMergesDir()
    */
   public void createMergesDir() throws IOException {
     Path mergesdir = getMergesDir();
     if (fs.exists(mergesdir)) {
-      LOG.info("The " + mergesdir
-          + " directory exists.  Hence deleting it to recreate it");
+      LOG.info("{} directory exists. Deleting it to recreate it anew", mergesdir);
       if (!fs.delete(mergesdir, true)) {
-        throw new IOException("Failed deletion of " + mergesdir
-            + " before creating them again.");
+        throw new IOException("Failed deletion of " + mergesdir + " before recreate.");
       }
     }
-    if (!mkdirs(fs, conf, mergesdir))
+    if (!mkdirs(fs, conf, mergesdir)) {
       throw new IOException("Failed create of " + mergesdir);
+    }
   }
 
   /**
@@ -813,7 +813,7 @@ public class HRegionFileSystem {
   public void commitMergedRegion(final RegionInfo mergedRegionInfo) throws IOException {
     Path regionDir = new Path(this.tableDir, mergedRegionInfo.getEncodedName());
     Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
-    // Move the tmp dir in the expected location
+    // Move the tmp dir to the expected location
     if (mergedRegionTmpDir != null && fs.exists(mergedRegionTmpDir)) {
 
       // Write HRI to a file in case we need to recover hbase:meta
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index ef8913c..c4fd11c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1922,8 +1922,7 @@ public class HRegionServer extends HasThread implements
     this.procedureResultReporter = new RemoteProcedureResultReporter(this);
 
     // Create the CompactedFileDischarger chore executorService. This chore helps to
-    // remove the compacted files
-    // that will no longer be used in reads.
+    // remove the compacted files that will no longer be used in reads.
     // Default is 2 mins. The default value for TTLCleaner is 5 mins so we set this to
     // 2 mins so that compacted files can be archived before the TTLCleaner runs
     int cleanerInterval =
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 32c9ef2..fd8f19d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1894,21 +1894,12 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
   }
 
   /**
-   * <p>It works by processing a compaction that's been written to disk.
-   *
-   * <p>It is usually invoked at the end of a compaction, but might also be
-   * invoked at HStore startup, if the prior execution died midway through.
-   *
-   * <p>Moving the compacted TreeMap into place means:
-   * <pre>
-   * 1) Unload all replaced StoreFile, close and collect list to delete.
-   * 2) Compute new store size
-   * </pre>
-   *
+   * Update counts.
    * @param compactedFiles list of files that were compacted
    */
   @VisibleForTesting
   protected void completeCompaction(Collection<HStoreFile> compactedFiles)
+  // Rename this method! TODO.
     throws IOException {
     this.storeSize.set(0L);
     this.totalUncompressedBytes.set(0L);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
index 779ca44..eb12765 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
@@ -389,7 +389,7 @@ public class StoreFileInfo {
   @Override
   public String toString() {
     return this.getPath() +
-      (isReference() ? "-" + getReferredToFile(this.getPath()) + "-" + reference : "");
+      (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : "");
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 731bc10..266b7dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -2795,13 +2795,14 @@ public class HBaseFsck extends Configured implements Closeable {
               throw new IOException("Two entries in hbase:meta are same " + previous);
             }
           }
-          PairOfSameType<RegionInfo> mergeRegions = MetaTableAccessor.getMergeRegions(result);
-          for (RegionInfo mergeRegion : new RegionInfo[] {
-              mergeRegions.getFirst(), mergeRegions.getSecond() }) {
-            if (mergeRegion != null) {
-              // This region is already been merged
-              HbckRegionInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName());
-              hbInfo.setMerged(true);
+          List<RegionInfo> mergeParents = MetaTableAccessor.getMergeRegions(result.rawCells());
+          if (mergeParents != null) {
+            for (RegionInfo mergeRegion : mergeParents) {
+              if (mergeRegion != null) {
+                // This region is already being merged
+                HbckRegionInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName());
+                hbInfo.setMerged(true);
+              }
             }
           }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index 8ef2ab1..f303d3d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.anyObject;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
@@ -30,13 +25,17 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Admin;
 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.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -78,7 +77,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 @Category({MiscTests.class, MediumTests.class})
 @SuppressWarnings("deprecation")
 public class TestMetaTableAccessor {
-
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMetaTableAccessor.class);
@@ -108,9 +106,34 @@ public class TestMetaTableAccessor {
   }
 
   @Test
+  public void testGettingMergeRegions() throws IOException {
+    TableName tn = TableName.valueOf(this.name.getMethodName());
+    Put put = new Put(Bytes.toBytes(this.name.getMethodName()));
+    List<RegionInfo> ris = new ArrayList<>();
+    int limit = 10;
+    byte [] previous = HConstants.EMPTY_START_ROW;
+    for (int i = 0; i < limit; i++) {
+      RegionInfo ri = RegionInfoBuilder.newBuilder(tn).
+          setStartKey(previous).setEndKey(Bytes.toBytes(i)).build();
+      ris.add(ri);
+    }
+    put = MetaTableAccessor.addMergeRegions(put, ris);
+    List<Cell> cells = put.getFamilyCellMap().get(HConstants.CATALOG_FAMILY);
+    String previousQualifier = null;
+    Assert.assertEquals(limit, cells.size());
+    for (Cell cell: cells) {
+      LOG.info(cell.toString());
+      String qualifier = Bytes.toString(cell.getQualifierArray());
+      Assert.assertTrue(qualifier.startsWith(HConstants.MERGE_QUALIFIER_PREFIX_STR));
+      Assert.assertNotEquals(qualifier, previousQualifier);
+      previousQualifier = qualifier;
+    }
+  }
+
+  @Test
   public void testIsMetaWhenAllHealthy() throws InterruptedException {
     HMaster m = UTIL.getMiniHBaseCluster().getMaster();
-    assertTrue(m.waitForMetaOnline());
+    Assert.assertTrue(m.waitForMetaOnline());
   }
 
   @Test
@@ -119,7 +142,7 @@ public class TestMetaTableAccessor {
     int index = UTIL.getMiniHBaseCluster().getServerWithMeta();
     HRegionServer rsWithMeta = UTIL.getMiniHBaseCluster().getRegionServer(index);
     rsWithMeta.abort("TESTING");
-    assertTrue(m.waitForMetaOnline());
+    Assert.assertTrue(m.waitForMetaOnline());
   }
 
   /**
@@ -164,8 +187,8 @@ public class TestMetaTableAccessor {
 
     try {
       // Make sure reader and writer are working.
-      assertTrue(reader.isProgressing());
-      assertTrue(writer.isProgressing());
+      Assert.assertTrue(reader.isProgressing());
+      Assert.assertTrue(writer.isProgressing());
 
       // Kill server hosting meta -- twice  . See if our reader/writer ride over the
       // meta moves.  They'll need to retry.
@@ -184,8 +207,8 @@ public class TestMetaTableAccessor {
         }
       }
 
-      assertTrue("reader: " + reader.toString(), reader.isProgressing());
-      assertTrue("writer: " + writer.toString(), writer.isProgressing());
+      Assert.assertTrue("reader: " + reader.toString(), reader.isProgressing());
+      Assert.assertTrue("writer: " + writer.toString(), writer.isProgressing());
     } catch (IOException e) {
       throw e;
     } finally {
@@ -196,7 +219,7 @@ public class TestMetaTableAccessor {
       t.close();
     }
     long exeTime = System.currentTimeMillis() - startTime;
-    assertTrue("Timeout: test took " + exeTime / 1000 + " sec", exeTime < timeOut);
+    Assert.assertTrue("Timeout: test took " + exeTime / 1000 + " sec", exeTime < timeOut);
   }
 
   /**
@@ -251,26 +274,27 @@ public class TestMetaTableAccessor {
   @Test
   public void testGetRegionsFromMetaTable() throws IOException, InterruptedException {
     List<RegionInfo> regions = MetaTableLocator.getMetaRegions(UTIL.getZooKeeperWatcher());
-    assertTrue(regions.size() >= 1);
-    assertTrue(MetaTableLocator.getMetaRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1);
+    Assert.assertTrue(regions.size() >= 1);
+    Assert.assertTrue(
+        MetaTableLocator.getMetaRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1);
   }
 
   @Test public void testTableExists() throws IOException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    assertFalse(MetaTableAccessor.tableExists(connection, tableName));
+    Assert.assertFalse(MetaTableAccessor.tableExists(connection, tableName));
     UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
-    assertTrue(MetaTableAccessor.tableExists(connection, tableName));
+    Assert.assertTrue(MetaTableAccessor.tableExists(connection, tableName));
     Admin admin = UTIL.getAdmin();
     admin.disableTable(tableName);
     admin.deleteTable(tableName);
-    assertFalse(MetaTableAccessor.tableExists(connection, tableName));
-    assertTrue(MetaTableAccessor.tableExists(connection,
+    Assert.assertFalse(MetaTableAccessor.tableExists(connection, tableName));
+    Assert.assertTrue(MetaTableAccessor.tableExists(connection,
         TableName.META_TABLE_NAME));
     UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
-    assertTrue(MetaTableAccessor.tableExists(connection, tableName));
+    Assert.assertTrue(MetaTableAccessor.tableExists(connection, tableName));
     admin.disableTable(tableName);
     admin.deleteTable(tableName);
-    assertFalse(MetaTableAccessor.tableExists(connection, tableName));
+    Assert.assertFalse(MetaTableAccessor.tableExists(connection, tableName));
   }
 
   @Test public void testGetRegion() throws IOException, InterruptedException {
@@ -279,7 +303,7 @@ public class TestMetaTableAccessor {
     // Test get on non-existent region.
     Pair<RegionInfo, ServerName> pair =
       MetaTableAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region"));
-    assertNull(pair);
+    Assert.assertNull(pair);
     LOG.info("Finished " + name);
   }
 
@@ -302,18 +326,18 @@ public class TestMetaTableAccessor {
 
     // Now make sure we only get the regions from 1 of the tables at a time
 
-    assertEquals(1, MetaTableAccessor.getTableRegions(connection, tableName).size());
-    assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size());
+    Assert.assertEquals(1, MetaTableAccessor.getTableRegions(connection, tableName).size());
+    Assert.assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size());
   }
 
   private static List<RegionInfo> testGettingTableRegions(final Connection connection,
       final TableName name, final int regionCount)
   throws IOException, InterruptedException {
     List<RegionInfo> regions = MetaTableAccessor.getTableRegions(connection, name);
-    assertEquals(regionCount, regions.size());
+    Assert.assertEquals(regionCount, regions.size());
     Pair<RegionInfo, ServerName> pair =
       MetaTableAccessor.getRegion(connection, regions.get(0).getRegionName());
-    assertEquals(regions.get(0).getEncodedName(),
+    Assert.assertEquals(regions.get(0).getEncodedName(),
       pair.getFirst().getEncodedName());
     return regions;
   }
@@ -323,24 +347,30 @@ public class TestMetaTableAccessor {
   throws IOException, InterruptedException {
     Pair<RegionInfo, ServerName> pair =
       MetaTableAccessor.getRegion(connection, region.getRegionName());
-    assertEquals(region.getEncodedName(),
+    Assert.assertEquals(region.getEncodedName(),
       pair.getFirst().getEncodedName());
   }
 
   @Test
   public void testParseReplicaIdFromServerColumn() {
     String column1 = HConstants.SERVER_QUALIFIER_STR;
-    assertEquals(0, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column1)));
+    Assert.assertEquals(0,
+        MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column1)));
     String column2 = column1 + MetaTableAccessor.META_REPLICA_ID_DELIMITER;
-    assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column2)));
+    Assert.assertEquals(-1,
+        MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column2)));
     String column3 = column2 + "00";
-    assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column3)));
+    Assert.assertEquals(-1,
+        MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column3)));
     String column4 = column3 + "2A";
-    assertEquals(42, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column4)));
+    Assert.assertEquals(42,
+        MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column4)));
     String column5 = column4 + "2A";
-    assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column5)));
+    Assert.assertEquals(-1,
+        MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column5)));
     String column6 = HConstants.STARTCODE_QUALIFIER_STR;
-    assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column6)));
+    Assert.assertEquals(-1,
+        MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column6)));
   }
 
   @Test
@@ -427,14 +457,14 @@ public class TestMetaTableAccessor {
       long seqNum, int replicaId, boolean checkSeqNum) throws IOException {
     Get get = new Get(row);
     Result result = meta.get(get);
-    assertTrue(Bytes.equals(
+    Assert.assertTrue(Bytes.equals(
       result.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(replicaId)),
       Bytes.toBytes(serverName.getHostAndPort())));
-    assertTrue(Bytes.equals(
+    Assert.assertTrue(Bytes.equals(
       result.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(replicaId)),
       Bytes.toBytes(serverName.getStartcode())));
     if (checkSeqNum) {
-      assertTrue(Bytes.equals(
+      Assert.assertTrue(Bytes.equals(
         result.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(replicaId)),
         Bytes.toBytes(seqNum)));
     }
@@ -448,10 +478,10 @@ public class TestMetaTableAccessor {
         MetaTableAccessor.getServerColumn(replicaId));
     Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
       MetaTableAccessor.getStartCodeColumn(replicaId));
-    assertNotNull(serverCell);
-    assertNotNull(startCodeCell);
-    assertEquals(0, serverCell.getValueLength());
-    assertEquals(0, startCodeCell.getValueLength());
+    Assert.assertNotNull(serverCell);
+    Assert.assertNotNull(startCodeCell);
+    Assert.assertEquals(0, serverCell.getValueLength());
+    Assert.assertEquals(0, startCodeCell.getValueLength());
   }
 
   @Test
@@ -479,12 +509,12 @@ public class TestMetaTableAccessor {
         Cell snCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
           MetaTableAccessor.getServerNameColumn(replicaId));
         if (replicaId == 0) {
-          assertNotNull(stateCell);
+          Assert.assertNotNull(stateCell);
         } else {
-          assertNull(serverCell);
-          assertNull(startCodeCell);
-          assertNull(stateCell);
-          assertNull(snCell);
+          Assert.assertNull(serverCell);
+          Assert.assertNull(startCodeCell);
+          Assert.assertNull(stateCell);
+          Assert.assertNull(snCell);
         }
       }
     } finally {
@@ -586,15 +616,21 @@ public class TestMetaTableAccessor {
     try (Table meta = MetaTableAccessor.getMetaHTable(connection)) {
       List<RegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
-
-      MetaTableAccessor.mergeRegions(connection, merged, parentA, -1L, parentB, -1L, serverName0,
-        3);
-
+      MetaTableAccessor.mergeRegions(connection, merged, getMapOfRegionsToSeqNum(parentA, parentB),
+          serverName0, 3);
       assertEmptyMetaLocation(meta, merged.getRegionName(), 1);
       assertEmptyMetaLocation(meta, merged.getRegionName(), 2);
     }
   }
 
+  private Map<RegionInfo, Long> getMapOfRegionsToSeqNum(RegionInfo ... regions) {
+    Map<RegionInfo, Long> mids = new HashMap<>(regions.length);
+    for (RegionInfo region: regions) {
+      mids.put(region, -1L);
+    }
+    return mids;
+  }
+
   @Test
   public void testMetaScanner() throws Exception {
     LOG.info("Starting " + name.getMethodName());
@@ -669,15 +705,15 @@ public class TestMetaTableAccessor {
         MetaTableAccessor.getStartCodeColumn(0));
       Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
         MetaTableAccessor.getSeqNumColumn(0));
-      assertNotNull(serverCell);
-      assertNotNull(startCodeCell);
-      assertNotNull(seqNumCell);
-      assertTrue(serverCell.getValueLength() > 0);
-      assertTrue(startCodeCell.getValueLength() > 0);
-      assertTrue(seqNumCell.getValueLength() > 0);
-      assertEquals(masterSystemTime, serverCell.getTimestamp());
-      assertEquals(masterSystemTime, startCodeCell.getTimestamp());
-      assertEquals(masterSystemTime, seqNumCell.getTimestamp());
+      Assert.assertNotNull(serverCell);
+      Assert.assertNotNull(startCodeCell);
+      Assert.assertNotNull(seqNumCell);
+      Assert.assertTrue(serverCell.getValueLength() > 0);
+      Assert.assertTrue(startCodeCell.getValueLength() > 0);
+      Assert.assertTrue(seqNumCell.getValueLength() > 0);
+      Assert.assertEquals(masterSystemTime, serverCell.getTimestamp());
+      Assert.assertEquals(masterSystemTime, startCodeCell.getTimestamp());
+      Assert.assertEquals(masterSystemTime, seqNumCell.getTimestamp());
     }
   }
 
@@ -727,16 +763,16 @@ public class TestMetaTableAccessor {
       Result result = meta.get(get);
       Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
           MetaTableAccessor.getServerColumn(0));
-      assertNotNull(serverCell);
-      assertEquals(serverNameTime, serverCell.getTimestamp());
+      Assert.assertNotNull(serverCell);
+      Assert.assertEquals(serverNameTime, serverCell.getTimestamp());
 
       ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
       edge.setValue(masterSystemTime);
       EnvironmentEdgeManager.injectEdge(edge);
       try {
         // now merge the regions, effectively deleting the rows for region a and b.
-        MetaTableAccessor.mergeRegions(connection, mergedRegionInfo, regionInfoA, -1L, regionInfoB,
-          -1L, sn, 1);
+        MetaTableAccessor.mergeRegions(connection, mergedRegionInfo,
+            getMapOfRegionsToSeqNum(regionInfoA, regionInfoB), sn, 1);
       } finally {
         EnvironmentEdgeManager.reset();
       }
@@ -749,9 +785,9 @@ public class TestMetaTableAccessor {
         MetaTableAccessor.getStartCodeColumn(0));
       Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
         MetaTableAccessor.getSeqNumColumn(0));
-      assertNull(serverCell);
-      assertNull(startCodeCell);
-      assertNull(seqNumCell);
+      Assert.assertNull(serverCell);
+      Assert.assertNull(startCodeCell);
+      Assert.assertNull(seqNumCell);
     }
   }
 
@@ -832,7 +868,7 @@ public class TestMetaTableAccessor {
       MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, loc.getServerName(),
         1);
 
-      assertTrue(prevCalls < scheduler.numPriorityCalls);
+      Assert.assertTrue(prevCalls < scheduler.numPriorityCalls);
     }
   }
 
@@ -875,8 +911,8 @@ public class TestMetaTableAccessor {
         MetaTableAccessor.getServerColumn(splitA.getReplicaId()));
       Cell startCodeCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
         MetaTableAccessor.getStartCodeColumn(splitA.getReplicaId()));
-      assertNull(serverCellA);
-      assertNull(startCodeCellA);
+      Assert.assertNull(serverCellA);
+      Assert.assertNull(startCodeCellA);
 
       Get get2 = new Get(splitA.getRegionName());
       Result resultB = meta.get(get2);
@@ -884,8 +920,8 @@ public class TestMetaTableAccessor {
         MetaTableAccessor.getServerColumn(splitB.getReplicaId()));
       Cell startCodeCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY,
         MetaTableAccessor.getStartCodeColumn(splitB.getReplicaId()));
-      assertNull(serverCellB);
-      assertNull(startCodeCellB);
+      Assert.assertNull(serverCellB);
+      Assert.assertNull(startCodeCellB);
     } finally {
       if (meta != null) {
         meta.close();
@@ -901,10 +937,10 @@ public class TestMetaTableAccessor {
     final String encodedName = regions.get(0).getRegionInfo().getEncodedName();
     final Result result = MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(),
       encodedName);
-    assertNotNull(result);
-    assertTrue(result.advance());
+    Assert.assertNotNull(result);
+    Assert.assertTrue(result.advance());
     final String resultingRowKey = CellUtil.getCellKeyAsString(result.current());
-    assertTrue(resultingRowKey.contains(encodedName));
+    Assert.assertTrue(resultingRowKey.contains(encodedName));
     UTIL.deleteTable(tableName);
   }
 
@@ -913,7 +949,7 @@ public class TestMetaTableAccessor {
     final String encodedName = "nonexistingregion";
     final Result result = MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(),
       encodedName);
-    assertNull(result);
+    Assert.assertNull(result);
   }
 }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
index 7cf794a..5151343 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
@@ -100,7 +100,7 @@ public class TestMergeTableRegionsWhileRSCrash {
         .getMaster().getMasterProcedureExecutor();
     List<RegionInfo> regionInfos = admin.getRegions(TABLE_NAME);
     MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure(
-        env, regionInfos.get(0), regionInfos.get(1));
+        env, new RegionInfo [] {regionInfos.get(0), regionInfos.get(1)}, false);
     executor.submitProcedure(mergeTableRegionsProcedure);
     UTIL.waitFor(30000,
       () -> executor.getProcedures().stream().filter(p -> p instanceof TransitRegionStateProcedure)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
index 1af9bd0..ac1726c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
@@ -89,7 +89,7 @@ public class TestMasterAbortWhileMergingTable {
     List<RegionInfo> regionInfos = admin.getRegions(TABLE_NAME);
     MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure(
         UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
-            .getEnvironment(), regionInfos.get(0), regionInfos.get(1));
+            .getEnvironment(), new RegionInfo [] {regionInfos.get(0), regionInfos.get(1)}, false);
     long procID = UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
         .submitProcedure(mergeTableRegionsProcedure);
     mergeCommitArrive.await();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
index 6a8c4b3..d17a4aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,15 +20,20 @@ package org.apache.hadoop.hbase.master.assignment;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
@@ -41,7 +46,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -69,10 +74,8 @@ public class TestMergeTableRegionsProcedure {
 
   private static final int initialRegionCount = 4;
   private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
-  private final static Configuration conf = UTIL.getConfiguration();
   private static Admin admin;
 
-  private AssignmentManager am;
   private ProcedureMetrics mergeProcMetrics;
   private ProcedureMetrics assignProcMetrics;
   private ProcedureMetrics unassignProcMetrics;
@@ -93,7 +96,7 @@ public class TestMergeTableRegionsProcedure {
 
   @BeforeClass
   public static void setupCluster() throws Exception {
-    setupConf(conf);
+    setupConf(UTIL.getConfiguration());
     UTIL.startMiniCluster(1);
     admin = UTIL.getAdmin();
   }
@@ -113,7 +116,7 @@ public class TestMergeTableRegionsProcedure {
     // Turn off the meta scanner so it don't remove parent on us.
     UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
     resetProcExecutorTestingKillFlag();
-    am = UTIL.getHBaseCluster().getMaster().getAssignmentManager();
+    AssignmentManager am = UTIL.getHBaseCluster().getMaster().getAssignmentManager();
     mergeProcMetrics = am.getAssignmentManagerMetrics().getMergeProcMetrics();
     assignProcMetrics = am.getAssignmentManagerMetrics().getAssignProcMetrics();
     unassignProcMetrics = am.getAssignmentManagerMetrics().getUnassignProcMetrics();
@@ -134,41 +137,68 @@ public class TestMergeTableRegionsProcedure {
     assertTrue("expected executor to be running", procExec.isRunning());
   }
 
+  private int loadARowPerRegion(final Table t, List<RegionInfo> ris)
+      throws IOException {
+    List<Put> puts = new ArrayList<>();
+    for (RegionInfo ri: ris) {
+      Put put = new Put(ri.getStartKey() == null || ri.getStartKey().length == 0?
+          new byte [] {'a'}: ri.getStartKey());
+      put.addColumn(HConstants.CATALOG_FAMILY, HConstants.CATALOG_FAMILY,
+          HConstants.CATALOG_FAMILY);
+      puts.add(put);
+    }
+    t.put(puts);
+    return puts.size();
+  }
+
+
   /**
    * This tests two region merges
    */
   @Test
   public void testMergeTwoRegions() throws Exception {
     final TableName tableName = TableName.valueOf(this.name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    List<RegionInfo> tableRegions = createTable(tableName);
+    UTIL.createTable(tableName, new byte[][]{HConstants.CATALOG_FAMILY},
+        new byte[][]{new byte[]{'b'}, new byte[]{'c'}, new byte[]{'d'}, new byte[]{'e'}});
+    testMerge(tableName, 2);
+  }
 
-    RegionInfo[] regionsToMerge = new RegionInfo[2];
-    regionsToMerge[0] = tableRegions.get(0);
-    regionsToMerge[1] = tableRegions.get(1);
+  private void testMerge(TableName tableName, int mergeCount) throws IOException {
+    List<RegionInfo> ris = MetaTableAccessor.getTableRegions(UTIL.getConnection(), tableName);
+    int originalRegionCount = ris.size();
+    assertTrue(originalRegionCount > mergeCount);
+    RegionInfo[] regionsToMerge = ris.subList(0, mergeCount).toArray(new RegionInfo [] {});
+    int countOfRowsLoaded = 0;
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      countOfRowsLoaded = loadARowPerRegion(table, ris);
+    }
+    assertEquals(countOfRowsLoaded, UTIL.countRows(tableName));
 
     // collect AM metrics before test
     collectAssignmentManagerMetrics();
-
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     MergeTableRegionsProcedure proc =
         new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true);
     long procId = procExec.submitProcedure(proc);
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    assertRegionCount(tableName, initialRegionCount - 1);
+    MetaTableAccessor.fullScanMetaAndPrint(UTIL.getConnection());
+    assertEquals(originalRegionCount - mergeCount + 1,
+        MetaTableAccessor.getTableRegions(UTIL.getConnection(), tableName).size());
 
     assertEquals(mergeSubmittedCount + 1, mergeProcMetrics.getSubmittedCounter().getCount());
     assertEquals(mergeFailedCount, mergeProcMetrics.getFailedCounter().getCount());
     assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount());
     assertEquals(assignFailedCount, assignProcMetrics.getFailedCounter().getCount());
-    assertEquals(unassignSubmittedCount + 2, unassignProcMetrics.getSubmittedCounter().getCount());
+    assertEquals(unassignSubmittedCount + mergeCount,
+        unassignProcMetrics.getSubmittedCounter().getCount());
     assertEquals(unassignFailedCount, unassignProcMetrics.getFailedCounter().getCount());
 
-    Pair<RegionInfo, RegionInfo> pair =
-      MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
-        proc.getMergedRegion().getRegionName());
-    assertTrue(pair.getFirst() != null && pair.getSecond() != null);
+    // Need to get the references cleaned out. Close of region will move them
+    // to archive so disable and reopen just to get rid of references to later
+    // when the catalogjanitor runs, it can do merged region cleanup.
+    admin.disableTable(tableName);
+    admin.enableTable(tableName);
 
     // Can I purge the merged regions from hbase:meta? Check that all went
     // well by looking at the merged row up in hbase:meta. It should have no
@@ -176,10 +206,24 @@ public class TestMergeTableRegionsProcedure {
     // the merged regions cleanup.
     UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
     UTIL.getHBaseCluster().getMaster().getCatalogJanitor().triggerNow();
-    while (pair != null && pair.getFirst() != null && pair.getSecond() != null) {
-      pair = MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
-          proc.getMergedRegion().getRegionName());
+    byte [] mergedRegion = proc.getMergedRegion().getRegionName();
+    while (ris != null && ris.get(0) != null && ris.get(1) != null) {
+      ris = MetaTableAccessor.getMergeRegions(UTIL.getConnection(), mergedRegion);
+      LOG.info("{} {}", Bytes.toStringBinary(mergedRegion), ris);
+      Threads.sleep(1000);
     }
+    assertEquals(countOfRowsLoaded, UTIL.countRows(tableName));
+  }
+
+  /**
+   * This tests ten region merges in one go.
+   */
+  @Test
+  public void testMergeTenRegions() throws Exception {
+    final TableName tableName = TableName.valueOf(this.name.getMethodName());
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
+    testMerge(tableName, 10);
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
index 16ad373..b1969d1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
@@ -92,7 +92,7 @@ public class TestModifyTableWhileMerging {
     List<RegionInfo> regionInfos = admin.getRegions(TABLE_NAME);
     MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure(
       UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
-        .getEnvironment(), regionInfos.get(0), regionInfos.get(1));
+        .getEnvironment(), new RegionInfo [] {regionInfos.get(0), regionInfos.get(1)}, false);
     ModifyTableProcedure modifyTableProcedure = new ModifyTableProcedure(env, tableDescriptor);
     long procModify = executor.submitProcedure(modifyTableProcedure);
     UTIL.waitFor(30000, () -> executor.getProcedures().stream()
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index 9d9226e..8dc2ada 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -34,7 +34,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
@@ -63,6 +62,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
@@ -216,15 +216,12 @@ public class TestRegionMergeTransactionOnCluster {
         MASTER.getConnection(), mergedRegionInfo.getRegionName());
 
       // contains merge reference in META
-      assertTrue(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY,
-          HConstants.MERGEA_QUALIFIER) != null);
-      assertTrue(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY,
-          HConstants.MERGEB_QUALIFIER) != null);
+      assertTrue(MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells()));
 
       // merging regions' directory are in the file system all the same
-      PairOfSameType<RegionInfo> p = MetaTableAccessor.getMergeRegions(mergedRegionResult);
-      RegionInfo regionA = p.getFirst();
-      RegionInfo regionB = p.getSecond();
+      List<RegionInfo> p = MetaTableAccessor.getMergeRegions(mergedRegionResult.rawCells());
+      RegionInfo regionA = p.get(0);
+      RegionInfo regionB = p.get(1);
       FileSystem fs = MASTER.getMasterFileSystem().getFileSystem();
       Path rootDir = MASTER.getMasterFileSystem().getRootDir();
 
@@ -291,11 +288,7 @@ public class TestRegionMergeTransactionOnCluster {
 
       mergedRegionResult = MetaTableAccessor.getRegionResult(
         TEST_UTIL.getConnection(), mergedRegionInfo.getRegionName());
-      assertFalse(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY,
-          HConstants.MERGEA_QUALIFIER) != null);
-      assertFalse(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY,
-          HConstants.MERGEB_QUALIFIER) != null);
-
+      assertFalse(MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells()));
     } finally {
       ADMIN.enableCatalogJanitor(true);
     }
@@ -336,8 +329,9 @@ public class TestRegionMergeTransactionOnCluster {
 
       try {
         // Merge the same region: b and b.
-        admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true);
-        fail("A region should not be able to merge with itself, even forcifully");
+        FutureUtils
+          .get(admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true));
+        fail("A region should not be able to merge with itself, even forcfully");
       } catch (IOException ie) {
         assertTrue("Exception should mention regions not online",
           StringUtils.stringifyException(ie).contains("region to itself")