You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2017/01/17 23:40:10 UTC

[2/7] hbase git commit: HBASE-17470 Remove merge region code from region server (Stephen Yuan Jiang)

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java
deleted file mode 100644
index 5c86429..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java
+++ /dev/null
@@ -1,742 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitationsME
- * under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.MetaMutationAnnotation;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl.LoggingProgressable;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.annotations.VisibleForTesting;
-
-@InterfaceAudience.Private
-public class RegionMergeTransactionImpl implements RegionMergeTransaction {
-  private static final Log LOG = LogFactory.getLog(RegionMergeTransactionImpl.class);
-
-  // Merged region info
-  private HRegionInfo mergedRegionInfo;
-  // region_a sorts before region_b
-  private final HRegion region_a;
-  private final HRegion region_b;
-  // merges dir is under region_a
-  private final Path mergesdir;
-  // We only merge adjacent regions if forcible is false
-  private final boolean forcible;
-  private final long masterSystemTime;
-
-  /*
-   * Transaction state for listener, only valid during execute and
-   * rollback
-   */
-  private RegionMergeTransactionPhase currentPhase = RegionMergeTransactionPhase.STARTED;
-  private Server server;
-  private RegionServerServices rsServices;
-
-  public static class JournalEntryImpl implements JournalEntry {
-    private RegionMergeTransactionPhase type;
-    private long timestamp;
-
-    public JournalEntryImpl(RegionMergeTransactionPhase type) {
-      this(type, EnvironmentEdgeManager.currentTime());
-    }
-
-    public JournalEntryImpl(RegionMergeTransactionPhase type, long timestamp) {
-      this.type = type;
-      this.timestamp = timestamp;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder();
-      sb.append(type);
-      sb.append(" at ");
-      sb.append(timestamp);
-      return sb.toString();
-    }
-
-    @Override
-    public RegionMergeTransactionPhase getPhase() {
-      return type;
-    }
-
-    @Override
-    public long getTimeStamp() {
-      return timestamp;
-    }
-  }
-
-  /*
-   * Journal of how far the merge transaction has progressed.
-   */
-  private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
-
-  /**
-   * Listeners
-   */
-  private final ArrayList<TransactionListener> listeners = new ArrayList<TransactionListener>();
-
-  private static IOException closedByOtherException = new IOException(
-      "Failed to close region: already closed by another thread");
-
-  private RegionServerCoprocessorHost rsCoprocessorHost = null;
-
-  /**
-   * Constructor
-   * @param a region a to merge
-   * @param b region b to merge
-   * @param forcible if false, we will only merge adjacent regions
-   */
-  public RegionMergeTransactionImpl(final Region a, final Region b,
-      final boolean forcible) {
-    this(a, b, forcible, EnvironmentEdgeManager.currentTime());
-  }
-  /**
-   * Constructor
-   * @param a region a to merge
-   * @param b region b to merge
-   * @param forcible if false, we will only merge adjacent regions
-   * @param masterSystemTime the time at the master side
-   */
-  public RegionMergeTransactionImpl(final Region a, final Region b,
-      final boolean forcible, long masterSystemTime) {
-    if (a.getRegionInfo().compareTo(b.getRegionInfo()) <= 0) {
-      this.region_a = (HRegion)a;
-      this.region_b = (HRegion)b;
-    } else {
-      this.region_a = (HRegion)b;
-      this.region_b = (HRegion)a;
-    }
-    this.forcible = forcible;
-    this.masterSystemTime = masterSystemTime;
-    this.mergesdir = region_a.getRegionFileSystem().getMergesDir();
-  }
-
-  private void transition(RegionMergeTransactionPhase nextPhase) throws IOException {
-    transition(nextPhase, false);
-  }
-
-  private void transition(RegionMergeTransactionPhase nextPhase, boolean isRollback)
-      throws IOException {
-    if (!isRollback) {
-      // Add to the journal first, because if the listener throws an exception
-      // we need to roll back starting at 'nextPhase'
-      this.journal.add(new JournalEntryImpl(nextPhase));
-    }
-    for (int i = 0; i < listeners.size(); i++) {
-      TransactionListener listener = listeners.get(i);
-      if (!isRollback) {
-        listener.transition(this, currentPhase, nextPhase);
-      } else {
-        listener.rollback(this, currentPhase, nextPhase);
-      }
-    }
-    currentPhase = nextPhase;
-  }
-
-  @Override
-  public boolean prepare(final RegionServerServices services) throws IOException {
-    if (!region_a.getTableDesc().getTableName()
-        .equals(region_b.getTableDesc().getTableName())) {
-      LOG.info("Can't merge regions " + region_a + "," + region_b
-          + " because they do not belong to the same table");
-      return false;
-    }
-    if (region_a.getRegionInfo().equals(region_b.getRegionInfo())) {
-      LOG.info("Can't merge the same region " + region_a);
-      return false;
-    }
-    if (!forcible && !HRegionInfo.areAdjacent(region_a.getRegionInfo(),
-            region_b.getRegionInfo())) {
-      String msg = "Skip merging " + region_a.getRegionInfo().getRegionNameAsString()
-          + " and " + region_b.getRegionInfo().getRegionNameAsString()
-          + ", because they are not adjacent.";
-      LOG.info(msg);
-      return false;
-    }
-    if (!this.region_a.isMergeable() || !this.region_b.isMergeable()) {
-      return false;
-    }
-    try {
-      boolean regionAHasMergeQualifier = hasMergeQualifierInMeta(services,
-          region_a.getRegionInfo().getRegionName());
-      if (regionAHasMergeQualifier ||
-          hasMergeQualifierInMeta(services, region_b.getRegionInfo().getRegionName())) {
-        LOG.debug("Region " + (regionAHasMergeQualifier ? 
-            region_a.getRegionInfo().getRegionNameAsString()
-                : region_b.getRegionInfo().getRegionNameAsString())
-            + " is not mergeable because it has merge qualifier in META");
-        return false;
-      }
-    } catch (IOException e) {
-      LOG.warn("Failed judging whether merge transaction is available for "
-              + region_a.getRegionInfo().getRegionNameAsString() + " and "
-              + region_b.getRegionInfo().getRegionNameAsString(), e);
-      return false;
-    }
-
-    // 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
-
-    this.mergedRegionInfo = getMergedRegionInfo(region_a.getRegionInfo(),
-        region_b.getRegionInfo());
-
-    transition(RegionMergeTransactionPhase.PREPARED);
-    return true;
-  }
-
-  @Override
-  public Region execute(final Server server, final RegionServerServices services)
-      throws IOException {
-    if (User.isHBaseSecurityEnabled(region_a.getBaseConf())) {
-      LOG.warn("Should use execute(Server, RegionServerServices, User)");
-    }
-    return execute(server, services, null);
-  }
-
-  @Override
-  public Region execute(final Server server, final RegionServerServices services, User user)
-      throws IOException {
-    this.server = server;
-    this.rsServices = services;
-    if (rsCoprocessorHost == null) {
-      rsCoprocessorHost = server != null ?
-        ((HRegionServer) server).getRegionServerCoprocessorHost() : null;
-    }
-    final HRegion mergedRegion = createMergedRegion(server, services, user);
-    if (rsCoprocessorHost != null) {
-      rsCoprocessorHost.postMergeCommit(this.region_a, this.region_b, mergedRegion, user);
-    }
-    stepsAfterPONR(server, services, mergedRegion, user);
-
-    transition(RegionMergeTransactionPhase.COMPLETED);
-
-    return mergedRegion;
-  }
-
-  @VisibleForTesting
-  public void stepsAfterPONR(final Server server, final RegionServerServices services,
-      final HRegion mergedRegion, User user) throws IOException {
-    openMergedRegion(server, services, mergedRegion);
-    if (rsCoprocessorHost != null) {
-      rsCoprocessorHost.postMerge(region_a, region_b, mergedRegion, user);
-    }
-  }
-
-  /**
-   * Prepare the merged region and region files.
-   * @param server Hosting server instance. Can be null when testing
-   * @param services Used to online/offline regions.
-   * @return merged region
-   * @throws IOException If thrown, transaction failed. Call
-   *           {@link #rollback(Server, RegionServerServices)}
-   */
-  private HRegion createMergedRegion(final Server server, final RegionServerServices services,
-      User user) throws IOException {
-    LOG.info("Starting merge of " + region_a + " and "
-        + region_b.getRegionInfo().getRegionNameAsString() + ", forcible=" + forcible);
-    if ((server != null && server.isStopped())
-        || (services != null && services.isStopping())) {
-      throw new IOException("Server is stopped or stopping");
-    }
-
-    if (rsCoprocessorHost != null) {
-      boolean ret = rsCoprocessorHost.preMerge(region_a, region_b, user);
-      if (ret) {
-        throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
-            + this.region_b + " merge.");
-      }
-    }
-
-    // If true, no cluster to write meta edits to or to use coordination.
-    boolean testing = server == null ? true : server.getConfiguration()
-        .getBoolean("hbase.testing.nocluster", false);
-
-    HRegion mergedRegion = stepsBeforePONR(server, services, testing);
-
-    @MetaMutationAnnotation
-    final List<Mutation> metaEntries = new ArrayList<Mutation>();
-    if (rsCoprocessorHost != null) {
-      boolean ret = rsCoprocessorHost.preMergeCommit(region_a, region_b, metaEntries, user);
-
-      if (ret) {
-        throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
-            + this.region_b + " merge.");
-      }
-      try {
-        for (Mutation p : metaEntries) {
-          HRegionInfo.parseRegionName(p.getRow());
-        }
-      } catch (IOException e) {
-        LOG.error("Row key of mutation from coprocessor is not parsable as region name."
-            + "Mutations from coprocessor should only be for hbase:meta table.", e);
-        throw e;
-      }
-    }
-
-    // This is the point of no return. Similar with SplitTransaction.
-    // IF we reach the PONR then subsequent failures need to crash out this
-    // regionserver
-    transition(RegionMergeTransactionPhase.PONR);
-
-    // Add merged region and delete region_a and region_b
-    // as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
-    // will determine whether the region is merged or not in case of failures.
-    // If it is successful, master will roll-forward, if not, master will
-    // rollback
-    if (services != null && !services.reportRegionStateTransition(TransitionCode.MERGE_PONR,
-        mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
-      // Passed PONR, let SSH clean it up
-      throw new IOException("Failed to notify master that merge passed PONR: "
-        + region_a.getRegionInfo().getRegionNameAsString() + " and "
-        + region_b.getRegionInfo().getRegionNameAsString());
-    }
-    return mergedRegion;
-  }
-
-  @VisibleForTesting
-  public void prepareMutationsForMerge(HRegionInfo mergedRegion, HRegionInfo regionA,
-      HRegionInfo regionB, ServerName serverName, List<Mutation> mutations) throws IOException {
-    HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
-
-    // use the maximum of what master passed us vs local time.
-    long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
-
-    // Put for parent
-    Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(copyOfMerged, time);
-    putOfMerged.addColumn(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
-        regionA.toByteArray());
-    putOfMerged.addColumn(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
-        regionB.toByteArray());
-    mutations.add(putOfMerged);
-    // Deletes for merging regions
-    Delete deleteA = MetaTableAccessor.makeDeleteFromRegionInfo(regionA, time);
-    Delete deleteB = MetaTableAccessor.makeDeleteFromRegionInfo(regionB, time);
-    mutations.add(deleteA);
-    mutations.add(deleteB);
-    // The merged is a new region, openSeqNum = 1 is fine.
-    addLocation(putOfMerged, serverName, 1);
-  }
-
-  @VisibleForTesting
-  Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
-    p.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes
-            .toBytes(sn.getHostAndPort()));
-    p.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, Bytes.toBytes(sn
-            .getStartcode()));
-    p.addColumn(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER, Bytes.toBytes(openSeqNum));
-    return p;
-  }
-
-  @VisibleForTesting
-  public HRegion stepsBeforePONR(final Server server, final RegionServerServices services,
-      boolean testing) throws IOException {
-    if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_MERGE,
-        mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
-      throw new IOException("Failed to get ok from master to merge "
-        + region_a.getRegionInfo().getRegionNameAsString() + " and "
-        + region_b.getRegionInfo().getRegionNameAsString());
-    }
-
-    transition(RegionMergeTransactionPhase.SET_MERGING);
-
-    this.region_a.getRegionFileSystem().createMergesDir();
-
-    transition(RegionMergeTransactionPhase.CREATED_MERGE_DIR);
-
-    Map<byte[], List<StoreFile>> hstoreFilesOfRegionA = closeAndOfflineRegion(
-        services, this.region_a, true, testing);
-    Map<byte[], List<StoreFile>> hstoreFilesOfRegionB = closeAndOfflineRegion(
-        services, this.region_b, false, testing);
-
-    assert hstoreFilesOfRegionA != null && hstoreFilesOfRegionB != null;
-
-    // mergeStoreFiles creates merged region dirs under the region_a merges dir
-    // Nothing to unroll here if failure -- clean up of CREATE_MERGE_DIR will
-    // clean this up.
-    mergeStoreFiles(hstoreFilesOfRegionA, hstoreFilesOfRegionB);
-
-    // Log to the journal that we are creating merged region. We could fail
-    // halfway through. If we do, we could have left
-    // stuff in fs that needs cleanup -- a storefile or two. Thats why we
-    // add entry to journal BEFORE rather than AFTER the change.
-
-    transition(RegionMergeTransactionPhase.STARTED_MERGED_REGION_CREATION);
-
-    HRegion mergedRegion = createMergedRegionFromMerges(this.region_a,
-        this.region_b, this.mergedRegionInfo);
-    return mergedRegion;
-  }
-
-  /**
-   * Create a merged region from the merges directory under region a. In order
-   * to mock it for tests, place it with a new method.
-   * @param a hri of region a
-   * @param b hri of region b
-   * @param mergedRegion hri of merged region
-   * @return merged HRegion.
-   * @throws IOException
-   */
-  @VisibleForTesting
-  HRegion createMergedRegionFromMerges(final HRegion a, final HRegion b,
-      final HRegionInfo mergedRegion) throws IOException {
-    return a.createMergedRegionFromMerges(mergedRegion, b);
-  }
-
-  /**
-   * Close the merging region and offline it in regionserver
-   * @param services
-   * @param region
-   * @param isRegionA true if it is merging region a, false if it is region b
-   * @param testing true if it is testing
-   * @return a map of family name to list of store files
-   * @throws IOException
-   */
-  private Map<byte[], List<StoreFile>> closeAndOfflineRegion(
-      final RegionServerServices services, final HRegion region,
-      final boolean isRegionA, final boolean testing) throws IOException {
-    Map<byte[], List<StoreFile>> hstoreFilesToMerge = null;
-    Exception exceptionToThrow = null;
-    try {
-      hstoreFilesToMerge = region.close(false);
-    } catch (Exception e) {
-      exceptionToThrow = e;
-    }
-    if (exceptionToThrow == null && hstoreFilesToMerge == null) {
-      // The region was closed by a concurrent thread. We can't continue
-      // with the merge, instead we must just abandon the merge. If we
-      // reopen or merge this could cause problems because the region has
-      // probably already been moved to a different server, or is in the
-      // process of moving to a different server.
-      exceptionToThrow = closedByOtherException;
-    }
-    if (exceptionToThrow != closedByOtherException) {
-      transition(isRegionA ? RegionMergeTransactionPhase.CLOSED_REGION_A
-          : RegionMergeTransactionPhase.CLOSED_REGION_B);
-    }
-    if (exceptionToThrow != null) {
-      if (exceptionToThrow instanceof IOException)
-        throw (IOException) exceptionToThrow;
-      throw new IOException(exceptionToThrow);
-    }
-    if (!testing) {
-      services.removeFromOnlineRegions(region, null);
-    }
-
-    transition(isRegionA ? RegionMergeTransactionPhase.OFFLINED_REGION_A
-        : RegionMergeTransactionPhase.OFFLINED_REGION_B);
-
-    return hstoreFilesToMerge;
-  }
-
-  /**
-   * Get merged region info through the specified two regions
-   * @param a merging region A
-   * @param b merging region B
-   * @return the merged region info
-   */
-  @VisibleForTesting
-  static HRegionInfo getMergedRegionInfo(final HRegionInfo a, final HRegionInfo b) {
-    long rid = EnvironmentEdgeManager.currentTime();
-    // Regionid is timestamp. Merged region's id can't be less than that of
-    // merging regions else will insert at wrong location in hbase:meta
-    if (rid < a.getRegionId() || rid < b.getRegionId()) {
-      LOG.warn("Clock skew; merging regions id are " + a.getRegionId()
-          + " and " + b.getRegionId() + ", but current time here is " + rid);
-      rid = Math.max(a.getRegionId(), b.getRegionId()) + 1;
-    }
-
-    byte[] startKey = null;
-    byte[] endKey = null;
-    // Choose the smaller as start key
-    if (a.compareTo(b) <= 0) {
-      startKey = a.getStartKey();
-    } else {
-      startKey = b.getStartKey();
-    }
-    // Choose the bigger as end key
-    if (Bytes.equals(a.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
-        || (!Bytes.equals(b.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
-            && Bytes.compareTo(a.getEndKey(), b.getEndKey()) > 0)) {
-      endKey = a.getEndKey();
-    } else {
-      endKey = b.getEndKey();
-    }
-
-    // Merged region is sorted between two merging regions in META
-    HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTable(), startKey,
-        endKey, false, rid);
-    return mergedRegionInfo;
-  }
-
-  /**
-   * Perform time consuming opening of the merged region.
-   * @param server Hosting server instance. Can be null when testing
-   * @param services Used to online/offline regions.
-   * @param merged the merged region
-   * @throws IOException If thrown, transaction failed. Call
-   *           {@link #rollback(Server, RegionServerServices)}
-   */
-  @VisibleForTesting
-  void openMergedRegion(final Server server,  final RegionServerServices services,
-      HRegion merged) throws IOException {
-    boolean stopped = server != null && server.isStopped();
-    boolean stopping = services != null && services.isStopping();
-    if (stopped || stopping) {
-      LOG.info("Not opening merged region  " + merged.getRegionInfo().getRegionNameAsString()
-          + " because stopping=" + stopping + ", stopped=" + stopped);
-      return;
-    }
-    HRegionInfo hri = merged.getRegionInfo();
-    LoggingProgressable reporter = server == null ? null
-        : new LoggingProgressable(hri, server.getConfiguration().getLong(
-            "hbase.regionserver.regionmerge.open.log.interval", 10000));
-    merged.openHRegion(reporter);
-
-    if (services != null) {
-      if (!services.reportRegionStateTransition(TransitionCode.MERGED,
-          mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
-        throw new IOException("Failed to report merged region to master: "
-          + mergedRegionInfo.getShortNameToLog());
-      }
-      services.addToOnlineRegions(merged);
-    }
-  }
-
-  /**
-   * Create reference file(s) of merging regions under the region_a merges dir
-   * @param hstoreFilesOfRegionA
-   * @param hstoreFilesOfRegionB
-   * @throws IOException
-   */
-  private void mergeStoreFiles(
-      Map<byte[], List<StoreFile>> hstoreFilesOfRegionA,
-      Map<byte[], List<StoreFile>> hstoreFilesOfRegionB)
-      throws IOException {
-    // Create reference file(s) of region A in mergdir
-    HRegionFileSystem fs_a = this.region_a.getRegionFileSystem();
-    for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionA
-        .entrySet()) {
-      String familyName = Bytes.toString(entry.getKey());
-      for (StoreFile storeFile : entry.getValue()) {
-        fs_a.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
-            this.mergesdir);
-      }
-    }
-    // Create reference file(s) of region B in mergedir
-    HRegionFileSystem fs_b = this.region_b.getRegionFileSystem();
-    for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionB
-        .entrySet()) {
-      String familyName = Bytes.toString(entry.getKey());
-      for (StoreFile storeFile : entry.getValue()) {
-        fs_b.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
-            this.mergesdir);
-      }
-    }
-  }
-
-  @Override
-  public boolean rollback(final Server server,
-      final RegionServerServices services) throws IOException {
-    if (User.isHBaseSecurityEnabled(region_a.getBaseConf())) {
-      LOG.warn("Should use execute(Server, RegionServerServices, User)");
-    }
-    return rollback(server, services, null);
-  }
-
-  @Override
-  public boolean rollback(final Server server,
-      final RegionServerServices services, User user) throws IOException {
-    assert this.mergedRegionInfo != null;
-    this.server = server;
-    this.rsServices = services;
-    // Coprocessor callback
-    if (rsCoprocessorHost != null) {
-      rsCoprocessorHost.preRollBackMerge(region_a, region_b, user);
-    }
-
-    boolean result = true;
-    ListIterator<JournalEntry> iterator = this.journal
-        .listIterator(this.journal.size());
-    // Iterate in reverse.
-    while (iterator.hasPrevious()) {
-      JournalEntry je = iterator.previous();
-
-      transition(je.getPhase(), true);
-
-      switch (je.getPhase()) {
-
-        case SET_MERGING:
-          if (services != null
-              && !services.reportRegionStateTransition(TransitionCode.MERGE_REVERTED,
-                  mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
-            return false;
-        }
-          break;
-
-        case CREATED_MERGE_DIR:
-          this.region_a.writestate.writesEnabled = true;
-          this.region_b.writestate.writesEnabled = true;
-          this.region_a.getRegionFileSystem().cleanupMergesDir();
-          break;
-
-        case CLOSED_REGION_A:
-          try {
-            // So, this returns a seqid but if we just closed and then reopened,
-            // we should be ok. On close, we flushed using sequenceid obtained
-            // from hosting regionserver so no need to propagate the sequenceid
-            // returned out of initialize below up into regionserver as we
-            // normally do.
-            this.region_a.initialize();
-          } catch (IOException e) {
-            LOG.error("Failed rollbacking CLOSED_REGION_A of region "
-                + region_a.getRegionInfo().getRegionNameAsString(), e);
-            throw new RuntimeException(e);
-          }
-          break;
-
-        case OFFLINED_REGION_A:
-          if (services != null)
-            services.addToOnlineRegions(this.region_a);
-          break;
-
-        case CLOSED_REGION_B:
-          try {
-            this.region_b.initialize();
-          } catch (IOException e) {
-            LOG.error("Failed rollbacking CLOSED_REGION_A of region "
-                + region_b.getRegionInfo().getRegionNameAsString(), e);
-            throw new RuntimeException(e);
-          }
-          break;
-
-        case OFFLINED_REGION_B:
-          if (services != null)
-            services.addToOnlineRegions(this.region_b);
-          break;
-
-        case STARTED_MERGED_REGION_CREATION:
-          this.region_a.getRegionFileSystem().cleanupMergedRegion(
-              this.mergedRegionInfo);
-          break;
-
-        case PONR:
-          // We got to the point-of-no-return so we need to just abort. Return
-          // immediately. Do not clean up created merged regions.
-          return false;
-
-         // Informational states only
-        case STARTED:
-        case PREPARED:
-        case COMPLETED:
-          break;
-
-        default:
-          throw new RuntimeException("Unhandled journal entry: " + je);
-      }
-    }
-    // Coprocessor callback
-    if (rsCoprocessorHost != null) {
-      rsCoprocessorHost.postRollBackMerge(region_a, region_b, user);
-    }
-
-    return result;
-  }
-
-  @Override
-  public HRegionInfo getMergedRegionInfo() {
-    return this.mergedRegionInfo;
-  }
-
-  @VisibleForTesting
-  Path getMergesDir() {
-    return this.mergesdir;
-  }
-
-  /**
-   * Checks if the given region has merge qualifier in hbase:meta
-   * @param services
-   * @param regionName name of specified region
-   * @return true if the given region has merge qualifier in META.(It will be
-   *         cleaned by CatalogJanitor)
-   * @throws IOException
-   */
-  @VisibleForTesting
-  boolean hasMergeQualifierInMeta(final RegionServerServices services, final byte[] regionName)
-      throws IOException {
-    if (services == null) return false;
-    // Get merge regions if it is a merged region and already has merge
-    // qualifier
-    Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
-        .getRegionsFromMergeQualifier(services.getConnection(), regionName);
-    if (mergeRegions != null &&
-        (mergeRegions.getFirst() != null || mergeRegions.getSecond() != null)) {
-      // It has merge qualifier
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public List<JournalEntry> getJournal() {
-    return journal;
-  }
-
-  @Override
-  public RegionMergeTransaction registerTransactionListener(TransactionListener listener) {
-    listeners.add(listener);
-    return this;
-  }
-
-  @Override
-  public Server getServer() {
-    return server;
-  }
-
-  @Override
-  public RegionServerServices getRegionServerServices() {
-    return rsServices;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
deleted file mode 100644
index 307568c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
+++ /dev/null
@@ -1,348 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-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.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotDisabledException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.ipc.RemoteException;
-
-/**
- * A non-instantiable class that has a static method capable of compacting
- * a table by merging adjacent regions.
- */
-@InterfaceAudience.Private
-class HMerge {
-  // TODO: Where is this class used?  How does it relate to Merge in same package?
-  private static final Log LOG = LogFactory.getLog(HMerge.class);
-  static final Random rand = new Random();
-
-  /*
-   * Not instantiable
-   */
-  private HMerge() {
-    super();
-  }
-
-  /**
-   * Scans the table and merges two adjacent regions if they are small. This
-   * only happens when a lot of rows are deleted.
-   *
-   * When merging the hbase:meta region, the HBase instance must be offline.
-   * When merging a normal table, the HBase instance must be online, but the
-   * table must be disabled.
-   *
-   * @param conf        - configuration object for HBase
-   * @param fs          - FileSystem where regions reside
-   * @param tableName   - Table to be compacted
-   * @throws IOException
-   */
-  public static void merge(Configuration conf, FileSystem fs,
-    final TableName tableName)
-  throws IOException {
-    merge(conf, fs, tableName, true);
-  }
-
-  /**
-   * Scans the table and merges two adjacent regions if they are small. This
-   * only happens when a lot of rows are deleted.
-   *
-   * When merging the hbase:meta region, the HBase instance must be offline.
-   * When merging a normal table, the HBase instance must be online, but the
-   * table must be disabled.
-   *
-   * @param conf        - configuration object for HBase
-   * @param fs          - FileSystem where regions reside
-   * @param tableName   - Table to be compacted
-   * @param testMasterRunning True if we are to verify master is down before
-   * running merge
-   * @throws IOException
-   */
-  public static void merge(Configuration conf, FileSystem fs,
-    final TableName tableName, final boolean testMasterRunning)
-  throws IOException {
-    boolean masterIsRunning = false;
-    ClusterConnection hConnection = null;
-    if (testMasterRunning) {
-      try {
-        hConnection = (ClusterConnection) ConnectionFactory.createConnection(conf);
-        masterIsRunning = hConnection.isMasterRunning();
-      } finally {
-        if (hConnection != null) {
-          hConnection.close();
-        }
-      }
-    }
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      if (masterIsRunning) {
-        throw new IllegalStateException(
-            "Can not compact hbase:meta table if instance is on-line");
-      }
-      // TODO reenable new OfflineMerger(conf, fs).process();
-    } else {
-      if(!masterIsRunning) {
-        throw new IllegalStateException(
-            "HBase instance must be running to merge a normal table");
-      }
-      try (Connection conn = ConnectionFactory.createConnection(conf);
-          Admin admin = conn.getAdmin()) {
-        if (!admin.isTableDisabled(tableName)) {
-          throw new TableNotDisabledException(tableName);
-        }
-      }
-      new OnlineMerger(conf, fs, tableName).process();
-    }
-  }
-
-  private static abstract class Merger {
-    protected final Configuration conf;
-    protected final FileSystem fs;
-    protected final Path rootDir;
-    protected final HTableDescriptor htd;
-    protected final WALFactory walFactory;
-    private final long maxFilesize;
-
-
-    protected Merger(Configuration conf, FileSystem fs, final TableName tableName)
-    throws IOException {
-      this.conf = conf;
-      this.fs = fs;
-      this.maxFilesize = conf.getLong(HConstants.HREGION_MAX_FILESIZE,
-          HConstants.DEFAULT_MAX_FILE_SIZE);
-
-      this.rootDir = FSUtils.getRootDir(conf);
-      Path tabledir = FSUtils.getTableDir(this.rootDir, tableName);
-      this.htd = FSTableDescriptors.getTableDescriptorFromFs(this.fs, tabledir);
-      String logname = "merge_" + System.currentTimeMillis() + HConstants.HREGION_LOGDIR_NAME;
-
-      final Configuration walConf = new Configuration(conf);
-      FSUtils.setRootDir(walConf, tabledir);
-      this.walFactory = new WALFactory(walConf, null, logname);
-    }
-
-    void process() throws IOException {
-      try {
-        for (HRegionInfo[] regionsToMerge = next();
-            regionsToMerge != null;
-            regionsToMerge = next()) {
-          if (!merge(regionsToMerge)) {
-            return;
-          }
-        }
-      } finally {
-        try {
-          walFactory.close();
-        } catch(IOException e) {
-          LOG.error(e);
-        }
-      }
-    }
-
-    protected boolean merge(final HRegionInfo[] info) throws IOException {
-      if (info.length < 2) {
-        LOG.info("only one region - nothing to merge");
-        return false;
-      }
-
-      HRegion currentRegion = null;
-      long currentSize = 0;
-      HRegion nextRegion = null;
-      long nextSize = 0;
-      for (int i = 0; i < info.length - 1; i++) {
-        if (currentRegion == null) {
-          currentRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i], this.htd,
-              walFactory.getWAL(info[i].getEncodedNameAsBytes(),
-                info[i].getTable().getNamespace()));
-          currentSize = currentRegion.getLargestHStoreSize();
-        }
-        nextRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i + 1], this.htd,
-            walFactory.getWAL(info[i + 1].getEncodedNameAsBytes(),
-              info[i + 1].getTable().getNamespace()));
-        nextSize = nextRegion.getLargestHStoreSize();
-
-        if ((currentSize + nextSize) <= (maxFilesize / 2)) {
-          // We merge two adjacent regions if their total size is less than
-          // one half of the desired maximum size
-          LOG.info("Merging regions " + currentRegion.getRegionInfo().getRegionNameAsString() +
-            " and " + nextRegion.getRegionInfo().getRegionNameAsString());
-          HRegion mergedRegion =
-            HRegion.mergeAdjacent(currentRegion, nextRegion);
-          updateMeta(currentRegion.getRegionInfo().getRegionName(),
-            nextRegion.getRegionInfo().getRegionName(), mergedRegion);
-          break;
-        }
-        LOG.info("not merging regions " +
-          Bytes.toStringBinary(currentRegion.getRegionInfo().getRegionName()) +
-            " and " + Bytes.toStringBinary(nextRegion.getRegionInfo().getRegionName()));
-        currentRegion.close();
-        currentRegion = nextRegion;
-        currentSize = nextSize;
-      }
-      if(currentRegion != null) {
-        currentRegion.close();
-      }
-      return true;
-    }
-
-    protected abstract HRegionInfo[] next() throws IOException;
-
-    protected abstract void updateMeta(final byte [] oldRegion1,
-      final byte [] oldRegion2, HRegion newRegion)
-    throws IOException;
-
-  }
-
-  /** Instantiated to compact a normal user table */
-  private static class OnlineMerger extends Merger {
-    private final TableName tableName;
-    private final Table table;
-    private final ResultScanner metaScanner;
-    private HRegionInfo latestRegion;
-
-    OnlineMerger(Configuration conf, FileSystem fs,
-      final TableName tableName)
-    throws IOException {
-      super(conf, fs, tableName);
-      this.tableName = tableName;
-      Connection connection = ConnectionFactory.createConnection(conf);
-      this.table = connection.getTable(TableName.META_TABLE_NAME);
-      this.metaScanner = table.getScanner(HConstants.CATALOG_FAMILY,
-          HConstants.REGIONINFO_QUALIFIER);
-      this.latestRegion = null;
-    }
-
-    private HRegionInfo nextRegion() throws IOException {
-      try {
-        Result results = getMetaRow();
-        if (results == null) {
-          return null;
-        }
-        HRegionInfo region = MetaTableAccessor.getHRegionInfo(results);
-        if (region == null) {
-          throw new NoSuchElementException("meta region entry missing " +
-              Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
-              Bytes.toString(HConstants.REGIONINFO_QUALIFIER));
-        }
-        if (!region.getTable().equals(this.tableName)) {
-          return null;
-        }
-        return region;
-      } catch (IOException e) {
-        e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
-        LOG.error("meta scanner error", e);
-        metaScanner.close();
-        throw e;
-      }
-    }
-
-    /*
-     * Check current row has a HRegionInfo.  Skip to next row if HRI is empty.
-     * @return A Map of the row content else null if we are off the end.
-     * @throws IOException
-     */
-    private Result getMetaRow() throws IOException {
-      Result currentRow = metaScanner.next();
-      boolean foundResult = false;
-      while (currentRow != null) {
-        LOG.info("Row: <" + Bytes.toStringBinary(currentRow.getRow()) + ">");
-        byte[] regionInfoValue = currentRow.getValue(HConstants.CATALOG_FAMILY,
-            HConstants.REGIONINFO_QUALIFIER);
-        if (regionInfoValue == null || regionInfoValue.length == 0) {
-          currentRow = metaScanner.next();
-          continue;
-        }
-        HRegionInfo region = MetaTableAccessor.getHRegionInfo(currentRow);
-        if (!region.getTable().equals(this.tableName)) {
-          currentRow = metaScanner.next();
-          continue;
-        }
-        foundResult = true;
-        break;
-      }
-      return foundResult ? currentRow : null;
-    }
-
-    @Override
-    protected HRegionInfo[] next() throws IOException {
-      List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
-      if(latestRegion == null) {
-        latestRegion = nextRegion();
-      }
-      if(latestRegion != null) {
-        regions.add(latestRegion);
-      }
-      latestRegion = nextRegion();
-      if(latestRegion != null) {
-        regions.add(latestRegion);
-      }
-      return regions.toArray(new HRegionInfo[regions.size()]);
-    }
-
-    @Override
-    protected void updateMeta(final byte [] oldRegion1,
-        final byte [] oldRegion2,
-      HRegion newRegion)
-    throws IOException {
-      byte[][] regionsToDelete = {oldRegion1, oldRegion2};
-      for (int r = 0; r < regionsToDelete.length; r++) {
-        if(Bytes.equals(regionsToDelete[r], latestRegion.getRegionName())) {
-          latestRegion = null;
-        }
-        Delete delete = new Delete(regionsToDelete[r]);
-        table.delete(delete);
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("updated columns in row: " + Bytes.toStringBinary(regionsToDelete[r]));
-        }
-      }
-      newRegion.getRegionInfo().setOffline(true);
-
-      MetaTableAccessor.addRegionToMeta(table, newRegion.getRegionInfo());
-
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("updated columns in row: "
-            + Bytes.toStringBinary(newRegion.getRegionInfo().getRegionName()));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
deleted file mode 100644
index 7b96660..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.util;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Utility that can merge any two regions in the same table: adjacent,
- * overlapping or disjoint.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class Merge extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(Merge.class);
-  private Path rootdir;
-  private volatile MetaUtils utils;
-  private TableName tableName;               // Name of table
-  private volatile byte [] region1;        // Name of region 1
-  private volatile byte [] region2;        // Name of region 2
-  private volatile HRegionInfo mergeInfo = null;
-
-  @Override
-  public int run(String[] args) throws Exception {
-    if (parseArgs(args) != 0) {
-      return -1;
-    }
-
-    // Verify file system is up.
-    FileSystem fs = FileSystem.get(getConf());              // get DFS handle
-    LOG.info("Verifying that file system is available...");
-    try {
-      FSUtils.checkFileSystemAvailable(fs);
-    } catch (IOException e) {
-      LOG.fatal("File system is not available", e);
-      return -1;
-    }
-
-    // Verify HBase is down
-    LOG.info("Verifying that HBase is not running...");
-    try {
-      HBaseAdmin.available(getConf());
-      LOG.fatal("HBase cluster must be off-line, and is not. Aborting.");
-      return -1;
-    } catch (ZooKeeperConnectionException zkce) {
-      // If no zk, presume no master.
-    }
-
-    // Initialize MetaUtils and and get the root of the HBase installation
-
-    this.utils = new MetaUtils(getConf());
-    this.rootdir = FSUtils.getRootDir(getConf());
-    try {
-      mergeTwoRegions();
-      return 0;
-    } catch (IOException e) {
-      LOG.fatal("Merge failed", e);
-      return -1;
-
-    } finally {
-      if (this.utils != null) {
-        this.utils.shutdown();
-      }
-    }
-  }
-
-  /** @return HRegionInfo for merge result */
-  HRegionInfo getMergedHRegionInfo() {
-    return this.mergeInfo;
-  }
-
-  /*
-   * Merges two regions from a user table.
-   */
-  private void mergeTwoRegions() throws IOException {
-    LOG.info("Merging regions " + Bytes.toStringBinary(this.region1) + " and " +
-        Bytes.toStringBinary(this.region2) + " in table " + this.tableName);
-    HRegion meta = this.utils.getMetaRegion();
-    Get get = new Get(region1);
-    get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    Result result1 =  meta.get(get);
-    Preconditions.checkState(!result1.isEmpty(),
-        "First region cells can not be null");
-    HRegionInfo info1 = MetaTableAccessor.getHRegionInfo(result1);
-    if (info1 == null) {
-      throw new NullPointerException("info1 is null using key " +
-          Bytes.toStringBinary(region1) + " in " + meta);
-    }
-    get = new Get(region2);
-    get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    Result result2 =  meta.get(get);
-    Preconditions.checkState(!result2.isEmpty(),
-        "Second region cells can not be null");
-    HRegionInfo info2 = MetaTableAccessor.getHRegionInfo(result2);
-    if (info2 == null) {
-      throw new NullPointerException("info2 is null using key " + meta);
-    }
-    HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(FileSystem.get(getConf()),
-      this.rootdir, this.tableName);
-    HRegion merged = merge(htd, meta, info1, info2);
-
-    LOG.info("Adding " + merged.getRegionInfo() + " to " +
-        meta.getRegionInfo());
-
-    HRegion.addRegionToMETA(meta, merged);
-    merged.close();
-  }
-
-  /*
-   * Actually merge two regions and update their info in the meta region(s)
-   * Returns HRegion object for newly merged region
-   */
-  private HRegion merge(final HTableDescriptor htd, HRegion meta,
-                        HRegionInfo info1, HRegionInfo info2)
-  throws IOException {
-    if (info1 == null) {
-      throw new IOException("Could not find " + Bytes.toStringBinary(region1) + " in " +
-          Bytes.toStringBinary(meta.getRegionInfo().getRegionName()));
-    }
-    if (info2 == null) {
-      throw new IOException("Could not find " + Bytes.toStringBinary(region2) + " in " +
-          Bytes.toStringBinary(meta.getRegionInfo().getRegionName()));
-    }
-    HRegion merged = null;
-    HRegion r1 = HRegion.openHRegion(info1, htd, utils.getLog(info1), getConf());
-    try {
-      HRegion r2 = HRegion.openHRegion(info2, htd, utils.getLog(info2), getConf());
-      try {
-        merged = HRegion.merge(r1, r2);
-      } finally {
-        if (!r2.isClosed()) {
-          r2.close();
-        }
-      }
-    } finally {
-      if (!r1.isClosed()) {
-        r1.close();
-      }
-    }
-
-    // Remove the old regions from meta.
-    // HRegion.merge has already deleted their files
-
-    removeRegionFromMeta(meta, info1);
-    removeRegionFromMeta(meta, info2);
-
-    this.mergeInfo = merged.getRegionInfo();
-    return merged;
-  }
-
-  /*
-   * Removes a region's meta information from the passed <code>meta</code>
-   * region.
-   *
-   * @param meta hbase:meta HRegion to be updated
-   * @param regioninfo HRegionInfo of region to remove from <code>meta</code>
-   *
-   * @throws IOException
-   */
-  private void removeRegionFromMeta(HRegion meta, HRegionInfo regioninfo)
-  throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Removing region: " + regioninfo + " from " + meta);
-    }
-
-    Delete delete  = new Delete(regioninfo.getRegionName(),
-        System.currentTimeMillis());
-    meta.delete(delete);
-  }
-
-  /**
-   * Parse given arguments and assign table name and regions names.
-   * (generic args are handled by ToolRunner.)
-   *
-   * @param args the arguments to parse
-   *
-   * @throws IOException
-   */
-  private int parseArgs(String[] args) throws IOException {
-    if (args.length != 3) {
-      usage();
-      return -1;
-    }
-    tableName = TableName.valueOf(args[0]);
-
-    region1 = Bytes.toBytesBinary(args[1]);
-    region2 = Bytes.toBytesBinary(args[2]);
-    int status = 0;
-    if (notInTable(tableName, region1) || notInTable(tableName, region2)) {
-      status = -1;
-    } else if (Bytes.equals(region1, region2)) {
-      LOG.error("Can't merge a region with itself");
-      status = -1;
-    }
-    return status;
-  }
-
-  private boolean notInTable(final TableName tn, final byte [] rn) {
-    if (WritableComparator.compareBytes(tn.getName(), 0, tn.getName().length,
-        rn, 0, tn.getName().length) != 0) {
-      LOG.error("Region " + Bytes.toStringBinary(rn) + " does not belong to table " +
-        tn);
-      return true;
-    }
-    return false;
-  }
-
-  private void usage() {
-    System.err
-        .println("For hadoop 0.21+, Usage: hbase org.apache.hadoop.hbase.util.Merge "
-            + "[-Dfs.defaultFS=hdfs://nn:port] <table-name> <region-1> <region-2>\n");
-  }
-
-  public static void main(String[] args) {
-    int status;
-    try {
-      status = ToolRunner.run(HBaseConfiguration.create(), new Merge(), args);
-    } catch (Exception e) {
-      LOG.error("exiting due to error", e);
-      status = -1;
-    }
-    System.exit(status);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 65cedda..b138c7d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
@@ -1189,15 +1189,17 @@ public class TestAdmin1 {
     assertTrue(gotException);
     // Try going to the master directly (that will skip the check in admin)
     try {
-      DispatchMergingRegionsRequest request = RequestConverter
-          .buildDispatchMergingRegionsRequest(
-            regions.get(1).getFirst().getEncodedNameAsBytes(),
-            regions.get(2).getFirst().getEncodedNameAsBytes(),
+      byte[][] nameofRegionsToMerge = new byte[2][];
+      nameofRegionsToMerge[0] =  regions.get(1).getFirst().getEncodedNameAsBytes();
+      nameofRegionsToMerge[1] = regions.get(2).getFirst().getEncodedNameAsBytes();
+      MergeTableRegionsRequest request = RequestConverter
+          .buildMergeTableRegionsRequest(
+            nameofRegionsToMerge,
             true,
             HConstants.NO_NONCE,
-            HConstants.NO_NONCE);
+            HConstants.NO_NONCE);   
       ((ClusterConnection) TEST_UTIL.getAdmin().getConnection()).getMaster()
-        .dispatchMergingRegions(null, request);
+        .mergeTableRegions(null, request);
     } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException m) {
       Throwable t = m.getCause();
       do {
@@ -1209,40 +1211,6 @@ public class TestAdmin1 {
       } while (t != null);
     }
     assertTrue(gotException);
-    gotException = false;
-    // Try going to the regionservers directly
-    // first move the region to the same regionserver
-    if (!regions.get(2).getSecond().equals(regions.get(1).getSecond())) {
-      moveRegionAndWait(regions.get(2).getFirst(), regions.get(1).getSecond());
-    }
-    try {
-      AdminService.BlockingInterface admin = ((ClusterConnection) TEST_UTIL.getAdmin()
-        .getConnection()).getAdmin(regions.get(1).getSecond());
-      ProtobufUtil.mergeRegions(null, admin, regions.get(1).getFirst(), regions.get(2).getFirst(),
-        true, null);
-    } catch (MergeRegionException mm) {
-      gotException = true;
-    }
-    assertTrue(gotException);
-  }
-
-  private void moveRegionAndWait(HRegionInfo destRegion, ServerName destServer)
-      throws InterruptedException, MasterNotRunningException,
-      ZooKeeperConnectionException, IOException {
-    HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
-    TEST_UTIL.getAdmin().move(
-        destRegion.getEncodedNameAsBytes(),
-        Bytes.toBytes(destServer.getServerName()));
-    while (true) {
-      ServerName serverName = master.getAssignmentManager()
-          .getRegionStates().getRegionServerOfRegion(destRegion);
-      if (serverName != null && serverName.equals(destServer)) {
-        TEST_UTIL.assertRegionOnServer(
-            destRegion, serverName, 200);
-        break;
-      }
-      Thread.sleep(10);
-    }
   }
 
   @Test (expected=IllegalArgumentException.class, timeout=300000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
deleted file mode 100644
index 4a62bff..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.coprocessor;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionFactory;
-import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionImpl;
-import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
-import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.RegionServerObserver}
- * interface hooks at all appropriate times during normal HMaster operations.
- */
-@Category({CoprocessorTests.class, MediumTests.class})
-public class TestRegionServerObserver {
-  private static final Log LOG = LogFactory.getLog(TestRegionServerObserver.class);
-
-  /**
-   * Test verifies the hooks in regions merge.
-   * @throws Exception
-   */
-  @Ignore
-  @Test
-  public void testCoprocessorHooksInRegionsMerge() throws Exception {
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 1;
-    final String TABLENAME = "testRegionServerObserver";
-    final String TABLENAME2 = "testRegionServerObserver_2";
-    final byte[] FAM = Bytes.toBytes("fam");
-
-    // Create config to use for this cluster
-    Configuration conf = HBaseConfiguration.create();
-    conf.setClass("hbase.coprocessor.regionserver.classes", CPRegionServerObserver.class,
-      RegionServerObserver.class);
-
-    // Start the cluster
-    HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    Admin admin = TEST_UTIL.getHBaseAdmin();
-    try {
-      MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-      HRegionServer regionServer = cluster.getRegionServer(0);
-      RegionServerCoprocessorHost cpHost = regionServer.getRegionServerCoprocessorHost();
-      Coprocessor coprocessor = cpHost.findCoprocessor(CPRegionServerObserver.class.getName());
-      CPRegionServerObserver regionServerObserver = (CPRegionServerObserver) coprocessor;
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TABLENAME));
-      desc.addFamily(new HColumnDescriptor(FAM));
-      admin.createTable(desc, new byte[][] { Bytes.toBytes("row") });
-      desc = new HTableDescriptor(TableName.valueOf(TABLENAME2));
-      desc.addFamily(new HColumnDescriptor(FAM));
-      admin.createTable(desc, new byte[][] { Bytes.toBytes("row") });
-      assertFalse(regionServerObserver.wasRegionMergeCalled());
-      List<Region> regions = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME));
-      admin.mergeRegionsAsync(
-        regions.get(0).getRegionInfo().getEncodedNameAsBytes(),
-        regions.get(1).getRegionInfo().getEncodedNameAsBytes(),
-        true);
-      int regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size();
-      while (regionsCount != 1) {
-        regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size();
-        Thread.sleep(1000);
-      }
-      assertTrue(regionServerObserver.wasRegionMergeCalled());
-      assertTrue(regionServerObserver.wasPreMergeCommit());
-      assertTrue(regionServerObserver.wasPostMergeCommit());
-      assertEquals(regionsCount, 1);
-      assertEquals(regionServer.getOnlineRegions(TableName.valueOf(TABLENAME2)).size(), 1);
-    } finally {
-      if (admin != null) admin.close();
-      TEST_UTIL.shutdownMiniCluster();
-    }
-  }
-
-  public static class CPRegionServerObserver extends BaseRegionServerObserver {
-    private RegionMergeTransactionImpl rmt = null;
-    private HRegion mergedRegion = null;
-
-    private boolean preMergeCalled;
-    private boolean preMergeBeforePONRCalled;
-    private boolean preMergeAfterPONRCalled;
-    private boolean preRollBackMergeCalled;
-    private boolean postRollBackMergeCalled;
-    private boolean postMergeCalled;
-
-    public void resetStates() {
-      preMergeCalled = false;
-      preMergeBeforePONRCalled = false;
-      preMergeAfterPONRCalled = false;
-      preRollBackMergeCalled = false;
-      postRollBackMergeCalled = false;
-      postMergeCalled = false;
-    }
-
-    @Override
-    public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
-        Region regionB) throws IOException {
-      preMergeCalled = true;
-    }
-
-    @Override
-    public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-        Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException {
-      preMergeBeforePONRCalled = true;
-      RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
-      HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
-      List<Region> onlineRegions =
-          rs.getOnlineRegions(TableName.valueOf("testRegionServerObserver_2"));
-      rmt = (RegionMergeTransactionImpl) new RegionMergeTransactionFactory(rs.getConfiguration())
-        .create(onlineRegions.get(0), onlineRegions.get(1), true);
-      if (!rmt.prepare(rs)) {
-        LOG.error("Prepare for the region merge of table "
-            + onlineRegions.get(0).getTableDesc().getNameAsString()
-            + " failed. So returning null. ");
-        ctx.bypass();
-        return;
-      }
-      mergedRegion = rmt.stepsBeforePONR(rs, rs, false);
-      rmt.prepareMutationsForMerge(mergedRegion.getRegionInfo(), regionA.getRegionInfo(),
-        regionB.getRegionInfo(), rs.getServerName(), metaEntries);
-      MetaTableAccessor.mutateMetaTable(rs.getConnection(), metaEntries);
-    }
-
-    @Override
-    public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-        Region regionA, Region regionB, Region mr) throws IOException {
-      preMergeAfterPONRCalled = true;
-      RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
-      HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
-      rmt.stepsAfterPONR(rs, rs, this.mergedRegion, null);
-    }
-
-    @Override
-    public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-        Region regionA, Region regionB) throws IOException {
-      preRollBackMergeCalled = true;
-    }
-
-    @Override
-    public void postRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-        Region regionA, Region regionB) throws IOException {
-      postRollBackMergeCalled = true;
-    }
-
-    @Override
-    public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, Region regionA,
-        Region regionB, Region mergedRegion) throws IOException {
-      postMergeCalled = true;
-    }
-
-    public boolean wasPreMergeCalled() {
-      return this.preMergeCalled;
-    }
-
-    public boolean wasPostMergeCalled() {
-      return this.postMergeCalled;
-    }
-
-    public boolean wasPreMergeCommit() {
-      return this.preMergeBeforePONRCalled;
-    }
-
-    public boolean wasPostMergeCommit() {
-      return this.preMergeAfterPONRCalled;
-    }
-
-    public boolean wasPreRollBackMerge() {
-      return this.preRollBackMergeCalled;
-    }
-
-    public boolean wasPostRollBackMerge() {
-      return this.postRollBackMergeCalled;
-    }
-
-    public boolean wasRegionMergeCalled() {
-      return this.preMergeCalled && this.postMergeCalled;
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 28bf14a..48cf8a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -320,16 +320,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public long dispatchMergingRegions(
-      final HRegionInfo region_a,
-      final HRegionInfo region_b,
-      final boolean forcible,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    return -1;
-  }
-
-  @Override
   public boolean isActiveMaster() {
     return true;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 950ec92..c5f294a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -68,8 +68,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerIn
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
@@ -531,13 +529,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public MergeRegionsResponse mergeRegions(RpcController controller,
-      MergeRegionsRequest request) throws ServiceException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public CompactRegionResponse compactRegion(RpcController controller,
       CompactRegionRequest request) throws ServiceException {
     // TODO Auto-generated method stub

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java
deleted file mode 100644
index c011321..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.master.procedure;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
-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.JVMClusterUtil.RegionServerThread;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-@Category({MasterTests.class, MediumTests.class})
-public class TestDispatchMergingRegionsProcedure {
-  private static final Log LOG = LogFactory.getLog(TestDispatchMergingRegionsProcedure.class);
-
-  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
-  final static Configuration conf = UTIL.getConfiguration();
-  private static Admin admin;
-
-  private static void setupConf(Configuration conf) {
-    // Reduce the maximum attempts to speed up the test
-    conf.setInt("hbase.assignment.maximum.attempts", 3);
-    conf.setInt("hbase.master.maximum.ping.server.attempts", 3);
-    conf.setInt("hbase.master.ping.server.retry.sleep.interval", 1);
-
-    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
-  }
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    setupConf(conf);
-    UTIL.startMiniCluster(1);
-    admin = UTIL.getHBaseAdmin();
-  }
-
-  @AfterClass
-  public static void cleanupTest() throws Exception {
-    try {
-      UTIL.shutdownMiniCluster();
-    } catch (Exception e) {
-      LOG.warn("failure shutting down cluster", e);
-    }
-  }
-
-  @Before
-  public void setup() throws Exception {
-    resetProcExecutorTestingKillFlag();
-
-    // Turn off balancer so it doesn't cut in and mess up our placements.
-    UTIL.getHBaseAdmin().setBalancerRunning(false, true);
-    // Turn off the meta scanner so it don't remove parent on us.
-    UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
-    resetProcExecutorTestingKillFlag();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    resetProcExecutorTestingKillFlag();
-    for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
-      LOG.info("Tear down, remove table=" + htd.getTableName());
-      UTIL.deleteTable(htd.getTableName());
-    }
-  }
-
-  private void resetProcExecutorTestingKillFlag() {
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
-    assertTrue("expected executor to be running", procExec.isRunning());
-  }
-
-  /**
-   * This tests two region merges
-   */
-  @Test(timeout=60000)
-  public void testMergeTwoRegions() throws Exception {
-    final TableName tableName = TableName.valueOf("testMergeTwoRegions");
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    List<HRegionInfo> tableRegions = createTable(tableName, 3);
-
-    HRegionInfo[] regionsToMerge = new HRegionInfo[2];
-    regionsToMerge[0] = tableRegions.get(0);
-    regionsToMerge[1] = tableRegions.get(1);
-
-    final int initCompletedTaskCount = countOfCompletedMergeTaskCount();
-    long procId = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
-      procExec.getEnvironment(), tableName, regionsToMerge, true));
-    ProcedureTestingUtility.waitProcedure(procExec, procId);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    assertRegionCount(tableName, 2, 1, initCompletedTaskCount);
-  }
-
-  /**
-   * This tests two concurrent region merges
-   */
-  @Test(timeout=60000)
-  public void testMergeRegionsConcurrently() throws Exception {
-    final TableName tableName = TableName.valueOf("testMergeRegionsConcurrently");
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    List<HRegionInfo> tableRegions = createTable(tableName, 4);
-
-    HRegionInfo[] regionsToMerge1 = new HRegionInfo[2];
-    HRegionInfo[] regionsToMerge2 = new HRegionInfo[2];
-    regionsToMerge1[0] = tableRegions.get(0);
-    regionsToMerge1[1] = tableRegions.get(1);
-    regionsToMerge2[0] = tableRegions.get(2);
-    regionsToMerge2[1] = tableRegions.get(3);
-
-    final int initCompletedTaskCount = countOfCompletedMergeTaskCount();
-    long procId1 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
-      procExec.getEnvironment(), tableName, regionsToMerge1, true));
-    long procId2 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
-      procExec.getEnvironment(), tableName, regionsToMerge2, true));
-    ProcedureTestingUtility.waitProcedure(procExec, procId1);
-    ProcedureTestingUtility.waitProcedure(procExec, procId2);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
-    assertRegionCount(tableName, 2, 2, initCompletedTaskCount);
-  }
-
-  private void waitForCompletedMergeTask(int expectedTaskCount, int initCompletedTaskCount)
-      throws IOException, InterruptedException {
-    while (true) {
-      long currentCompletedTaskCount = countOfCompletedMergeTaskCount() - initCompletedTaskCount;
-      if (currentCompletedTaskCount == expectedTaskCount) {
-        return;
-      }
-      LOG.info("There are " + (expectedTaskCount - currentCompletedTaskCount) +
-        " merge requests are not completed, wait 100 ms");
-      TimeUnit.MILLISECONDS.sleep(100);
-    }
-  }
-
-  private static int countOfCompletedMergeTaskCount() {
-    int completedTaskCount = 0;
-    for (RegionServerThread server : UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
-      completedTaskCount += server.getRegionServer().getCompactSplitThread().getCompletedMergeTaskCount();
-    }
-    return completedTaskCount;
-  }
-
-  @Test(timeout=60000)
-  public void testRecoveryAndDoubleExecution() throws Exception {
-    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    List<HRegionInfo> tableRegions = createTable(tableName, 3);
-
-    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
-    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
-
-    HRegionInfo[] regionsToMerge = new HRegionInfo[2];
-    regionsToMerge[0] = tableRegions.get(0);
-    regionsToMerge[1] = tableRegions.get(1);
-
-    final int initCompletedTaskCount = countOfCompletedMergeTaskCount();
-    long procId = procExec.submitProcedure(
-      new DispatchMergingRegionsProcedure(
-        procExec.getEnvironment(), tableName, regionsToMerge, true));
-
-    // Restart the executor and execute the step twice
-    int numberOfSteps = DispatchMergingRegionsState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    assertRegionCount(tableName, 2, 1, initCompletedTaskCount);
-  }
-
-  @Test(timeout = 60000)
-  public void testRollbackAndDoubleExecution() throws Exception {
-    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    List<HRegionInfo> tableRegions = createTable(tableName, 3);
-
-    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
-    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
-
-    HRegionInfo[] regionsToMerge = new HRegionInfo[2];
-    regionsToMerge[0] = tableRegions.get(0);
-    regionsToMerge[1] = tableRegions.get(1);
-
-    long procId = procExec.submitProcedure(
-      new DispatchMergingRegionsProcedure(
-        procExec.getEnvironment(), tableName, regionsToMerge, true));
-
-    int numberOfSteps = DispatchMergingRegionsState.values().length - 3;
-    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
-  }
-
-  private List<HRegionInfo> createTable(final TableName tableName, final int nregions)
-      throws Exception {
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor(FAMILY));
-    byte[][] splitRows = new byte[nregions - 1][];
-    for (int i = 0; i < splitRows.length; ++i) {
-      splitRows[i] = Bytes.toBytes(String.format("%d", i));
-    }
-    admin.createTable(desc, splitRows);
-    return assertRegionCount(tableName, nregions);
-  }
-
-  public List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
-      throws Exception {
-    UTIL.waitUntilNoRegionsInTransition();
-    List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
-    assertEquals(nregions, tableRegions.size());
-    return tableRegions;
-  }
-
-  public List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions,
-      int expectedTaskCount, int initCompletedTaskCount) throws Exception {
-    waitForCompletedMergeTask(expectedTaskCount, initCompletedTaskCount);
-    return assertRegionCount(tableName, nregions);
-  }
-
-  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
-    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index a63fec6..c973471 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -2646,55 +2646,6 @@ public class TestHRegion {
     }
   }
 
-  // ////////////////////////////////////////////////////////////////////////////
-  // Merge test
-  // ////////////////////////////////////////////////////////////////////////////
-  @Test
-  public void testMerge() throws IOException {
-    byte[][] families = { fam1, fam2, fam3 };
-    Configuration hc = initSplit();
-    // Setting up region
-    this.region = initHRegion(tableName, method, hc, families);
-    try {
-      LOG.info("" + HBaseTestCase.addContent(region, fam3));
-      region.flush(true);
-      region.compactStores();
-      byte[] splitRow = region.checkSplit();
-      assertNotNull(splitRow);
-      LOG.info("SplitRow: " + Bytes.toString(splitRow));
-      HRegion[] subregions = splitRegion(region, splitRow);
-      try {
-        // Need to open the regions.
-        for (int i = 0; i < subregions.length; i++) {
-          HRegion.openHRegion(subregions[i], null);
-          subregions[i].compactStores();
-        }
-        Path oldRegionPath = region.getRegionFileSystem().getRegionDir();
-        Path oldRegion1 = subregions[0].getRegionFileSystem().getRegionDir();
-        Path oldRegion2 = subregions[1].getRegionFileSystem().getRegionDir();
-        long startTime = System.currentTimeMillis();
-        region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
-        LOG.info("Merge regions elapsed time: "
-            + ((System.currentTimeMillis() - startTime) / 1000.0));
-        FILESYSTEM.delete(oldRegion1, true);
-        FILESYSTEM.delete(oldRegion2, true);
-        FILESYSTEM.delete(oldRegionPath, true);
-        LOG.info("splitAndMerge completed.");
-      } finally {
-        for (int i = 0; i < subregions.length; i++) {
-          try {
-            HBaseTestingUtility.closeRegionAndWAL(subregions[i]);
-          } catch (IOException e) {
-            // Ignore.
-          }
-        }
-      }
-    } finally {
-      HBaseTestingUtility.closeRegionAndWAL(this.region);
-      this.region = null;
-    }
-  }
-
   /**
    * @param parent
    *          Region to split.