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 2018/09/19 16:03:41 UTC

hbase git commit: HBASE-21156 [hbck2] Queue an assign of hbase:meta and bulk assign/unassign

Repository: hbase
Updated Branches:
  refs/heads/branch-2.1 27b772ddc -> 37cc07a77


HBASE-21156 [hbck2] Queue an assign of hbase:meta and bulk assign/unassign

Adds 'raw' assigns and unassigns methods to Hbck Service.

Fixes HbckService so it works when cluster is Kerberized.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/37cc07a7
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/37cc07a7
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/37cc07a7

Branch: refs/heads/branch-2.1
Commit: 37cc07a7723d8e87c42240faa87cb43eab29d6dc
Parents: 27b772d
Author: Michael Stack <st...@apache.org>
Authored: Wed Sep 12 21:28:52 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Sep 19 09:02:43 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  1 -
 .../apache/hadoop/hbase/client/HBaseHbck.java   | 60 ++++++++++---
 .../org/apache/hadoop/hbase/client/Hbck.java    | 38 ++++++--
 .../hadoop/hbase/security/SecurityInfo.java     | 15 +++-
 .../hbase/shaded/protobuf/RequestConverter.java | 22 +++++
 .../src/main/protobuf/Master.proto              | 61 ++++++++++++-
 .../org/apache/hadoop/hbase/master/HMaster.java |  4 +-
 .../hadoop/hbase/master/MasterRpcServices.java  | 92 ++++++++++++++++++++
 .../master/assignment/AssignmentManager.java    |  5 ++
 .../hbase/security/HBasePolicyProvider.java     | 14 +--
 .../apache/hadoop/hbase/client/TestHbck.java    | 87 ++++++++++++------
 11 files changed, 338 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 7f3abea..a160204 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -226,7 +226,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
  * @see Admin
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public class HBaseAdmin implements Admin {
   private static final Logger LOG = LoggerFactory.getLogger(HBaseAdmin.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
index 03a6f69..af9e7b7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
@@ -18,25 +18,36 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService.BlockingInterface;
 
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 
 /**
  * Use {@link ClusterConnection#getHbck()} to obtain an instance of {@link Hbck} instead of
- * constructing
- * an HBaseHbck directly. This will be mostly used by hbck tool.
+ * constructing an HBaseHbck directly.
  *
  * <p>Connection should be an <i>unmanaged</i> connection obtained via
  * {@link ConnectionFactory#createConnection(Configuration)}.</p>
  *
+ * <p>NOTE: The methods in here can do damage to a cluster if applied in the wrong sequence or at
+ * the wrong time. Use with caution. For experts only. These methods are only for the
+ * extreme case where the cluster has been damaged or has achieved an inconsistent state because
+ * of some unforeseen circumstance or bug and requires manual intervention.
+ *
  * <p>An instance of this class is lightweight and not-thread safe. A new instance should be created
  * by each thread. Pooling or caching of the instance is not recommended.</p>
  *
@@ -75,10 +86,6 @@ public class HBaseHbck implements Hbck {
     return this.aborted;
   }
 
-  /**
-   * NOTE: This is a dangerous action, as existing running procedures for the table or regions
-   * which belong to the table may get confused.
-   */
   @Override
   public TableState setTableStateInMeta(TableState state) throws IOException {
     try {
@@ -87,9 +94,38 @@ public class HBaseHbck implements Hbck {
           RequestConverter.buildSetTableStateInMetaRequest(state));
       return TableState.convert(state.getTableName(), response.getTableState());
     } catch (ServiceException se) {
-      LOG.debug("ServiceException while updating table state in meta. table={}, state={}",
-          state.getTableName(), state.getState());
+      LOG.debug("table={}, state={}", state.getTableName(), state.getState(), se);
       throw new IOException(se);
     }
   }
+
+  @Override
+  public List<Long> assigns(List<String> encodedRegionNames) throws IOException {
+    try {
+      MasterProtos.AssignsResponse response =
+          this.hbck.assigns(rpcControllerFactory.newController(),
+              RequestConverter.toAssignRegionsRequest(encodedRegionNames));
+      return response.getPidList();
+    } catch (ServiceException se) {
+      LOG.debug(toCommaDelimitedString(encodedRegionNames), se);
+      throw new IOException(se);
+    }
+  }
+
+  @Override
+  public List<Long> unassigns(List<String> encodedRegionNames) throws IOException {
+    try {
+      MasterProtos.UnassignsResponse response =
+          this.hbck.unassigns(rpcControllerFactory.newController(),
+              RequestConverter.toUnassignRegionsRequest(encodedRegionNames));
+      return response.getPidList();
+    } catch (ServiceException se) {
+      LOG.debug(toCommaDelimitedString(encodedRegionNames), se);
+      throw new IOException(se);
+    }
+  }
+
+  private static String toCommaDelimitedString(List<String> list) {
+    return list.stream().collect(Collectors.joining(", "));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
index a216cdb..c6a3caa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.List;
+
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -27,7 +29,12 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Hbck APIs for HBase. Obtain an instance from {@link ClusterConnection#getHbck()} and call
  * {@link #close()} when done.
  * <p>Hbck client APIs will be mostly used by hbck tool which in turn can be used by operators to
- * fix HBase and bringging it to consistent state.</p>
+ * fix HBase and bringing it to consistent state.</p>
+ *
+ * <p>NOTE: The methods in here can do damage to a cluster if applied in the wrong sequence or at
+ * the wrong time. Use with caution. For experts only. These methods are only for the
+ * extreme case where the cluster has been damaged or has achieved an inconsistent state because
+ * of some unforeseen circumstance or bug and requires manual intervention.
  *
  * @see ConnectionFactory
  * @see ClusterConnection
@@ -36,15 +43,36 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.HBCK)
 public interface Hbck extends Abortable, Closeable {
   /**
-   * Update table state in Meta only. No procedures are submitted to open/ assign or close/
-   * unassign regions of the table. This is useful only when some procedures/ actions are stuck
-   * beause of inconsistency between region and table states.
+   * Update table state in Meta only. No procedures are submitted to open/assign or
+   * close/unassign regions of the table.
    *
-   * NOTE: This is a dangerous action, as existing running procedures for the table or regions
+   * <p>>NOTE: This is a dangerous action, as existing running procedures for the table or regions
    * which belong to the table may get confused.
    *
    * @param state table state
    * @return previous state of the table in Meta
    */
   TableState setTableStateInMeta(TableState state) throws IOException;
+
+  /**
+   * Like {@link Admin#assign(byte[])} but 'raw' in that it can do more than one Region at a time
+   * -- good if many Regions to online -- and it will schedule the assigns even in the case where
+   * Master is initializing (as long as the ProcedureExecutor is up). Does NOT call Coprocessor
+   * hooks.
+   * @param encodedRegionNames Region encoded names; e.g. 1588230740 is the hard-coded encoding
+   *                           for hbase:meta region and de00010733901a05f5a2a3a382e27dd4 is an
+   *                           example of what a random user-space encoded Region name looks like.
+   */
+  List<Long> assigns(List<String> encodedRegionNames) throws IOException;
+
+  /**
+   * Like {@link Admin#unassign(byte[], boolean)} but 'raw' in that it can do more than one Region
+   * at a time -- good if many Regions to offline -- and it will schedule the assigns even in the
+   * case where Master is initializing (as long as the ProcedureExecutor is up). Does NOT call
+   * Coprocessor hooks.
+   * @param encodedRegionNames Region encoded names; e.g. 1588230740 is the hard-coded encoding
+   *                           for hbase:meta region and de00010733901a05f5a2a3a382e27dd4 is an
+   *                           example of what a random user-space encoded Region name looks like.
+   */
+  List<Long> unassigns(List<String> encodedRegionNames) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
index 8fca997..eb9d209 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
@@ -20,12 +20,13 @@ package org.apache.hadoop.hbase.security;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Maps RPC protocol interfaces to required configuration
@@ -37,13 +38,19 @@ public class SecurityInfo {
   // populate info for known services
   static {
     infos.put(AdminProtos.AdminService.getDescriptor().getName(),
-        new SecurityInfo("hbase.regionserver.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
+        new SecurityInfo("hbase.regionserver.kerberos.principal",
+            Kind.HBASE_AUTH_TOKEN));
     infos.put(ClientProtos.ClientService.getDescriptor().getName(),
-        new SecurityInfo("hbase.regionserver.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
+        new SecurityInfo("hbase.regionserver.kerberos.principal",
+            Kind.HBASE_AUTH_TOKEN));
     infos.put(MasterService.getDescriptor().getName(),
         new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
     infos.put(RegionServerStatusProtos.RegionServerStatusService.getDescriptor().getName(),
         new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
+    infos.put(MasterProtos.HbckService.getDescriptor().getName(),
+        new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
+    // NOTE: IF ADDING A NEW SERVICE, BE SURE TO UPDATE HBasePolicyProvider ALSO ELSE
+    // new Service will not be found when all is Kerberized!!!!
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 64da3a3..6ab81e2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
@@ -1879,4 +1880,25 @@ public final class RequestConverter {
     }
     return pbServers;
   }
+
+  // HBCK2
+  public static MasterProtos.AssignsRequest toAssignRegionsRequest(
+      List<String> encodedRegionNames) {
+    MasterProtos.AssignsRequest.Builder b = MasterProtos.AssignsRequest.newBuilder();
+    return b.addAllRegion(toEncodedRegionNameRegionSpecifiers(encodedRegionNames)).build();
+  }
+
+  public static MasterProtos.UnassignsRequest toUnassignRegionsRequest(
+      List<String> encodedRegionNames) {
+    MasterProtos.UnassignsRequest.Builder b =
+        MasterProtos.UnassignsRequest.newBuilder();
+    return b.addAllRegion(toEncodedRegionNameRegionSpecifiers(encodedRegionNames)).build();
+  }
+
+  private static List<RegionSpecifier> toEncodedRegionNameRegionSpecifiers(
+      List<String> encodedRegionNames) {
+    return encodedRegionNames.stream().
+        map(r -> buildRegionSpecifier(RegionSpecifierType.ENCODED_REGION_NAME, Bytes.toBytes(r))).
+        collect(Collectors.toList());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index ca8d915..8cef398 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -485,10 +485,6 @@ message GetTableStateResponse {
   required TableState table_state = 1;
 }
 
-message SetTableStateInMetaRequest {
-  required TableName table_name = 1;
-  required TableState table_state = 2;
-}
 
 message GetClusterStatusRequest {
   repeated Option options = 1;
@@ -993,8 +989,65 @@ service MasterService {
 
 }
 
+// HBCK Service definitions.
+
+message SetTableStateInMetaRequest {
+  required TableName table_name = 1;
+  required TableState table_state = 2;
+}
+
+/** Like Admin's AssignRegionRequest except it can
+ * take one or more Regions at a time.
+ */
+// NOTE: In hbck.proto, there is a define for
+// AssignRegionRequest -- singular 'Region'. This
+// is plural to convey it can carry more than one
+// Region at a time.
+message AssignsRequest {
+  repeated RegionSpecifier region = 1;
+}
+
+/** Like Admin's AssignRegionResponse except it can
+ * return one or more pids as result -- one per assign.
+ */
+message AssignsResponse {
+  repeated uint64 pid = 1;
+}
+
+/** Like Admin's UnassignRegionRequest except it can
+ * take one or more Regions at a time.
+ */
+message UnassignsRequest {
+  repeated RegionSpecifier region = 1;
+}
+
+/** Like Admin's UnassignRegionResponse except it can
+ * return one or more pids as result -- one per unassign.
+ */
+message UnassignsResponse {
+  repeated uint64 pid = 1;
+}
+
 service HbckService {
   /** Update state of the table in meta only*/
   rpc SetTableStateInMeta(SetTableStateInMetaRequest)
     returns(GetTableStateResponse);
+
+  /**
+   * Assign regions.
+   * Like Admin's assign but works even if the
+   * Master is initializing. Also allows bulk'ing up
+   * assigns rather than one region at a time.
+   */
+  rpc Assigns(AssignsRequest)
+    returns(AssignsResponse);
+
+  /**
+   * Unassign regions
+   * Like Admin's unssign but works even if the
+   * Master is initializing. Also allows bulk'ing up
+   * assigns rather than one region at a time.
+   */
+  rpc Unassigns(UnassignsRequest)
+    returns(UnassignsResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
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 69c1caa..d7e57e8 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
@@ -1118,8 +1118,8 @@ public class HMaster extends HRegionServer implements MasterServices {
       // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
       // then how to assign including how to break region lock if one held.
       LOG.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
-          "progress, in holding-pattern until region onlined; operator intervention required. " +
-          "Schedule an assign.", ri.getRegionNameAsString(), rs, optProc.isPresent());
+          "progress, in holding-pattern until region onlined.",
+          ri.getRegionNameAsString(), rs, optProc.isPresent());
       // Check once-a-minute.
       if (rc == null) {
         rc = new RetryCounterFactory(1000).create();

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index a315027..b4c437b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
@@ -2306,4 +2307,95 @@ public class MasterRpcServices extends RSRpcServices
       throw new ServiceException(e);
     }
   }
+
+  /**
+   * Get RegionInfo from Master using content of RegionSpecifier as key.
+   * @return RegionInfo found by decoding <code>rs</code> or null if none found
+   */
+  private RegionInfo getRegionInfo(HBaseProtos.RegionSpecifier rs) throws UnknownRegionException {
+    RegionInfo ri = null;
+    switch(rs.getType()) {
+      case REGION_NAME:
+        final byte[] regionName = rs.getValue().toByteArray();
+        ri = this.master.getAssignmentManager().getRegionInfo(regionName);
+        break;
+      case ENCODED_REGION_NAME:
+        String encodedRegionName = Bytes.toString(rs.getValue().toByteArray());
+        RegionState regionState = this.master.getAssignmentManager().getRegionStates().
+            getRegionState(encodedRegionName);
+        ri = regionState == null? null: regionState.getRegion();
+        break;
+      default:
+        break;
+    }
+    return ri;
+  }
+
+  /**
+   * Submit the Procedure that gets created by <code>f</code>
+   * @return pid of the submitted Procedure.
+   */
+  private long submitProcedure(HBaseProtos.RegionSpecifier rs, Function<RegionInfo, Procedure> f)
+    throws UnknownRegionException {
+    RegionInfo ri = getRegionInfo(rs);
+    long pid = Procedure.NO_PROC_ID;
+    if (ri == null) {
+      LOG.warn("No RegionInfo found to match {}", rs);
+    } else {
+      pid = this.master.getMasterProcedureExecutor().submitProcedure(f.apply(ri));
+    }
+    return pid;
+  }
+
+  /**
+   * A 'raw' version of assign that does bulk and skirts Master state checks (assigns can be made
+   * during Master startup). For use by Hbck2.
+   */
+  @Override
+  public MasterProtos.AssignsResponse assigns(RpcController controller,
+      MasterProtos.AssignsRequest request)
+    throws ServiceException {
+    LOG.info(master.getClientIdAuditPrefix() + " assigns");
+    if (this.master.getMasterProcedureExecutor() == null) {
+      throw new ServiceException("Master's ProcedureExecutor not initialized; retry later");
+    }
+    MasterProtos.AssignsResponse.Builder responseBuilder =
+        MasterProtos.AssignsResponse.newBuilder();
+    try {
+      for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) {
+        long pid = submitProcedure(rs,
+          r -> this.master.getAssignmentManager().createAssignProcedure(r));
+        responseBuilder.addPid(pid);
+      }
+      return responseBuilder.build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
+  /**
+   * A 'raw' version of unassign that does bulk and skirts Master state checks (unassigns can be
+   * made during Master startup). For use by Hbck2.
+   */
+  @Override
+  public MasterProtos.UnassignsResponse unassigns(RpcController controller,
+      MasterProtos.UnassignsRequest request)
+      throws ServiceException {
+    LOG.info(master.getClientIdAuditPrefix() + " unassigns");
+    if (this.master.getMasterProcedureExecutor() == null) {
+      throw new ServiceException("Master's ProcedureExecutor not initialized; retry later");
+    }
+    MasterProtos.UnassignsResponse.Builder responseBuilder =
+        MasterProtos.UnassignsResponse.newBuilder();
+    try {
+      for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) {
+        long pid = submitProcedure(rs,
+          ri -> this.master.getAssignmentManager().createUnassignProcedure(ri));
+        responseBuilder.addPid(pid);
+      }
+      return responseBuilder.build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
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 e33ffdf..123abc2 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
@@ -694,6 +694,11 @@ public class AssignmentManager implements ServerListener {
     return proc;
   }
 
+  public UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo) {
+    return createUnassignProcedure(regionInfo, null, false);
+
+  }
+
   UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo,
       final ServerName destinationServer, final boolean force) {
     return createUnassignProcedure(regionInfo, destinationServer, force, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
index 76d185f..72e36a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.security;
 
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
@@ -27,6 +27,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.Service;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Implementation of secure Hadoop policy provider for mapping
@@ -35,10 +36,13 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 @InterfaceAudience.Private
 public class HBasePolicyProvider extends PolicyProvider {
   protected final static Service[] services = {
-      new Service("security.client.protocol.acl", ClientService.BlockingInterface.class),
-      new Service("security.client.protocol.acl", AdminService.BlockingInterface.class),
-      new Service("security.admin.protocol.acl", MasterService.BlockingInterface.class),
-      new Service("security.masterregion.protocol.acl", RegionServerStatusService.BlockingInterface.class)
+    new Service("security.client.protocol.acl", ClientService.BlockingInterface.class),
+    new Service("security.client.protocol.acl", AdminService.BlockingInterface.class),
+    new Service("security.client.protocol.acl",
+      MasterProtos.HbckService.BlockingInterface.class),
+    new Service("security.admin.protocol.acl", MasterService.BlockingInterface.class),
+    new Service("security.masterregion.protocol.acl",
+      RegionServerStatusService.BlockingInterface.class)
   };
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/37cc07a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
index 86652d8..f80e2d1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
@@ -19,17 +19,22 @@
 package org.apache.hadoop.hbase.client;
 
 import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.After;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -52,23 +57,16 @@ public class TestHbck {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestHbck.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private Admin admin;
-  private Hbck hbck;
 
   @Rule
   public TestName name = new TestName();
 
-  private static final TableName tableName = TableName.valueOf(TestHbck.class.getSimpleName());
+  private static final TableName TABLE_NAME = TableName.valueOf(TestHbck.class.getSimpleName());
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
-    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
-    TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
-    TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
     TEST_UTIL.startMiniCluster(3);
-
-    TEST_UTIL.createTable(tableName, "family1");
+    TEST_UTIL.createMultiRegionTable(TABLE_NAME, Bytes.toBytes("family1"), 5);
   }
 
   @AfterClass
@@ -76,29 +74,62 @@ public class TestHbck {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Before
-  public void setUp() throws Exception {
-    this.admin = TEST_UTIL.getAdmin();
-    this.hbck = TEST_UTIL.getHbck();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    for (HTableDescriptor htd : this.admin.listTables()) {
-      TEST_UTIL.deleteTable(htd.getTableName());
-    }
-    this.hbck.close();
-  }
-
   @Test
   public void testSetTableStateInMeta() throws IOException {
+    Hbck hbck = TEST_UTIL.getHbck();
     // set table state to DISABLED
-    hbck.setTableStateInMeta(new TableState(tableName, TableState.State.DISABLED));
+    hbck.setTableStateInMeta(new TableState(TABLE_NAME, TableState.State.DISABLED));
     // Method {@link Hbck#setTableStateInMeta()} returns previous state, which in this case
     // will be DISABLED
     TableState prevState =
-        hbck.setTableStateInMeta(new TableState(tableName, TableState.State.ENABLED));
+        hbck.setTableStateInMeta(new TableState(TABLE_NAME, TableState.State.ENABLED));
     assertTrue("Incorrect previous state! expeced=DISABLED, found=" + prevState.getState(),
         prevState.isDisabled());
   }
+
+  @Test
+  public void testAssigns() throws IOException {
+    Hbck hbck = TEST_UTIL.getHbck();
+    try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
+      List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
+      for (RegionInfo ri: regions) {
+        RegionState rs = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
+            getRegionStates().getRegionState(ri.getEncodedName());
+        LOG.info("RS: {}", rs.toString());
+      }
+      List<Long> pids = hbck.unassigns(regions.stream().map(r -> r.getEncodedName()).
+          collect(Collectors.toList()));
+      waitOnPids(pids);
+      for (RegionInfo ri: regions) {
+        RegionState rs = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
+            getRegionStates().getRegionState(ri.getEncodedName());
+        LOG.info("RS: {}", rs.toString());
+        assertTrue(rs.toString(), rs.isClosed());
+      }
+      pids = hbck.assigns(regions.stream().map(r -> r.getEncodedName()).
+          collect(Collectors.toList()));
+      waitOnPids(pids);
+      for (RegionInfo ri: regions) {
+        RegionState rs = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
+            getRegionStates().getRegionState(ri.getEncodedName());
+        LOG.info("RS: {}", rs.toString());
+        assertTrue(rs.toString(), rs.isOpened());
+      }
+      // What happens if crappy region list passed?
+      pids = hbck.assigns(Arrays.stream(new String [] {"a", "some rubbish name"}).
+          collect(Collectors.toList()));
+      for (long pid: pids) {
+        assertEquals(org.apache.hadoop.hbase.procedure2.Procedure.NO_PROC_ID, pid);
+      }
+    }
+  }
+
+  private void waitOnPids(List<Long> pids) {
+    for (Long pid: pids) {
+      while (!TEST_UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor().
+          isFinished(pid)) {
+        Threads.sleep(100);
+      }
+    }
+  }
 }