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/08/27 19:08:39 UTC

hbase git commit: HBASE-20941 Created and implemented HbckService in master

Repository: hbase
Updated Branches:
  refs/heads/branch-2.0 346aa6c99 -> 8fd5e039e


HBASE-20941 Created and implemented HbckService in master

Added API setTableStateInMeta() to update table state only in Meta. This will be used by hbck2 tool.


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

Branch: refs/heads/branch-2.0
Commit: 8fd5e039e61f2996be9dbf2f750d45cac6e898e4
Parents: 346aa6c
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Tue Aug 7 11:46:42 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Aug 27 12:08:25 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/ClusterConnection.java  |  32 ++++++
 .../hbase/client/ConnectionImplementation.java  |  22 ++++
 .../apache/hadoop/hbase/client/HBaseHbck.java   |  95 +++++++++++++++++
 .../org/apache/hadoop/hbase/client/Hbck.java    |  50 +++++++++
 .../hbase/shaded/protobuf/RequestConverter.java |  12 +++
 .../hadoop/hbase/HBaseInterfaceAudience.java    |   5 +
 .../src/main/protobuf/Master.proto              |  11 ++
 .../hadoop/hbase/master/MasterRpcServices.java  |  31 +++++-
 .../hadoop/hbase/HBaseTestingUtility.java       |   9 +-
 .../apache/hadoop/hbase/client/TestHbck.java    | 104 +++++++++++++++++++
 .../hadoop/hbase/master/TestMasterMetrics.java  |   1 +
 11 files changed, 370 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
index 3e055b0..45f2404 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
@@ -326,4 +326,36 @@ public interface ClusterConnection extends Connection {
    * @throws IOException if a remote or network exception occurs
    */
   int getCurrentNrHRS() throws IOException;
+
+  /**
+   * Retrieve an Hbck implementation to fix an HBase cluster.
+   * The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
+   * each thread. This is a lightweight operation. Pooling or caching of the returned Hbck instance
+   * is not recommended.
+   * <br>
+   * The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
+   *<br>
+   * This will be used mostly by hbck tool.
+   *
+   * @return an Hbck instance for active master. Active master is fetched from the zookeeper.
+   */
+  Hbck getHbck() throws IOException;
+
+  /**
+   * Retrieve an Hbck implementation to fix an HBase cluster.
+   * The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
+   * each thread. This is a lightweight operation. Pooling or caching of the returned Hbck instance
+   * is not recommended.
+   * <br>
+   * The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
+   *<br>
+   * This will be used mostly by hbck tool. This may only be used to by pass getting
+   * registered master from ZK. In situations where ZK is not available or active master is not
+   * registered with ZK and user can get master address by other means, master can be explicitly
+   * specified.
+   *
+   * @param masterServer explicit {@link ServerName} for master server
+   * @return an Hbck instance for a specified master server
+   */
+  Hbck getHbck(ServerName masterServer) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 53e4b7f..2b0f4c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -408,6 +408,28 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   @Override
+  public Hbck getHbck() throws IOException {
+    return getHbck(get(registry.getMasterAddress()));
+  }
+
+  @Override
+  public Hbck getHbck(ServerName masterServer) throws IOException {
+    checkClosed();
+    if (isDeadServer(masterServer)) {
+      throw new RegionServerStoppedException(masterServer + " is dead.");
+    }
+    String key = getStubKey(MasterProtos.HbckService.BlockingInterface.class.getName(),
+        masterServer, this.hostnamesCanChange);
+
+    return new HBaseHbck(this,
+        (MasterProtos.HbckService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
+          BlockingRpcChannel channel =
+              this.rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout);
+          return MasterProtos.HbckService.newBlockingStub(channel);
+        }));
+  }
+
+  @Override
   public MetricsConnection getConnectionMetrics() {
     return this.metrics;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/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
new file mode 100644
index 0000000..03a6f69
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
@@ -0,0 +1,95 @@
+/*
+ * 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.client;
+
+import java.io.IOException;
+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.GetTableStateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService.BlockingInterface;
+
+
+/**
+ * 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.
+ *
+ * <p>Connection should be an <i>unmanaged</i> connection obtained via
+ * {@link ConnectionFactory#createConnection(Configuration)}.</p>
+ *
+ * <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>
+ *
+ * @see ConnectionFactory
+ * @see ClusterConnection
+ * @see Hbck
+ */
+@InterfaceAudience.Private
+public class HBaseHbck implements Hbck {
+  private static final Logger LOG = LoggerFactory.getLogger(HBaseHbck.class);
+
+  private boolean aborted;
+  private final BlockingInterface hbck;
+
+  private RpcControllerFactory rpcControllerFactory;
+
+  HBaseHbck(ClusterConnection connection, BlockingInterface hbck) throws IOException {
+    this.hbck = hbck;
+    this.rpcControllerFactory = connection.getRpcControllerFactory();
+  }
+
+  @Override
+  public void close() throws IOException {
+    // currently does nothing
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    this.aborted = true;
+    // Currently does nothing but throw the passed message and exception
+    throw new RuntimeException(why, e);
+  }
+
+  @Override
+  public boolean isAborted() {
+    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 {
+      GetTableStateResponse response = hbck.setTableStateInMeta(
+          rpcControllerFactory.newController(),
+          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());
+      throw new IOException(se);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/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
new file mode 100644
index 0000000..a216cdb
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
@@ -0,0 +1,50 @@
+/*
+ * 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.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+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>
+ *
+ * @see ConnectionFactory
+ * @see ClusterConnection
+ * @since 2.2.0
+ */
+@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.
+   *
+   * 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;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/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 8ce2f1b..19b9fc8 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
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -133,6 +134,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalance
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
@@ -1443,6 +1445,16 @@ public final class RequestConverter {
   }
 
   /**
+   * Creates a protocol buffer SetTableStateInMetaRequest
+   * @param state table state to update in Meta
+   * @return a SetTableStateInMetaRequest
+   */
+  public static SetTableStateInMetaRequest buildSetTableStateInMetaRequest(final TableState state) {
+    return SetTableStateInMetaRequest.newBuilder().setTableState(state.convert())
+        .setTableName(ProtobufUtil.toProtoTableName(state.getTableName())).build();
+  }
+
+  /**
    * Creates a protocol buffer GetTableDescriptorsRequest for a single table
    *
    * @param tableName the table name

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
index 4e03457..d5d4643 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
@@ -48,4 +48,9 @@ public final class HBaseInterfaceAudience {
    * for class name, and arguments.
    */
   public static final String TOOLS = "Tools";
+
+  /**
+   * Denotes classes used by hbck tool for fixing inconsistent state of HBase.
+   */
+  public static final String HBCK = "HBCK";
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/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 3a236c0..ca8d915 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -485,6 +485,11 @@ 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;
 }
@@ -987,3 +992,9 @@ service MasterService {
     returns(ClearDeadServersResponse);
 
 }
+
+service HbckService {
+  /** Update state of the table in meta only*/
+  rpc SetTableStateInMeta(SetTableStateInMetaRequest)
+    returns(GetTableStateResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/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 780ee81..a0b6d8d 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -175,6 +176,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
@@ -239,6 +241,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRe
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
@@ -296,7 +299,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.Snapshot
 @SuppressWarnings("deprecation")
 public class MasterRpcServices extends RSRpcServices
       implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
-        LockService.BlockingInterface {
+        LockService.BlockingInterface, HbckService.BlockingInterface {
   private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
 
   private final HMaster master;
@@ -424,6 +427,8 @@ public class MasterRpcServices extends RSRpcServices
       RegionServerStatusService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
         LockService.BlockingInterface.class));
+    bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
+        HbckService.BlockingInterface.class));
     bssi.addAll(super.getServices());
     return bssi;
   }
@@ -2237,4 +2242,28 @@ public class MasterRpcServices extends RSRpcServices
     }
     return response.build();
   }
+
+  // HBCK Services
+
+  /**
+   * Update state of the table in meta only. This is required by hbck in some situations to cleanup
+   * stuck assign/ unassign regions procedures for the table.
+   *
+   * @return previous state of the table
+   */
+  @Override
+  public GetTableStateResponse setTableStateInMeta(RpcController controller,
+      SetTableStateInMetaRequest request) throws ServiceException {
+    Connection conn = master.getConnection();
+    TableName tn = ProtobufUtil.toTableName(request.getTableName());
+
+    try {
+      HBaseProtos.TableState prevState = MetaTableAccessor.getTableState(conn, tn).convert();
+      MetaTableAccessor.updateTableState(conn, tn,
+          TableState.convert(tn, request.getTableState()).getState());
+      return GetTableStateResponse.newBuilder().setTableState(prevState).build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index d21442d..56a2ca1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
@@ -73,6 +74,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Hbck;
 import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
 import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
 import org.apache.hadoop.hbase.client.Put;
@@ -2804,7 +2806,12 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   private HBaseAdmin hbaseAdmin = null;
 
-
+  /**
+   * Returns an {@link Hbck} instance. Needs be closed when done.
+   */
+  public Hbck getHbck() throws IOException {
+    return ((ClusterConnection) getConnection()).getHbck();
+  }
 
   /**
    * Unassign the named region.

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/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
new file mode 100644
index 0000000..86652d8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
@@ -0,0 +1,104 @@
+/*
+ * 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.client;
+
+import static junit.framework.TestCase.assertTrue;
+
+import java.io.IOException;
+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.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to test HBaseHbck.
+ * Spins up the minicluster once at test start and then takes it down afterward.
+ * Add any testing of HBaseHbck functionality here.
+ */
+@Category({LargeTests.class, ClientTests.class})
+public class TestHbck {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestHbck.class);
+
+  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());
+
+  @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");
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    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 {
+    // set table state to DISABLED
+    hbck.setTableStateInMeta(new TableState(tableName, 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));
+    assertTrue("Incorrect previous state! expeced=DISABLED, found=" + prevState.getState(),
+        prevState.isDisabled());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8fd5e039/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
index 91955f8..e211b9c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
@@ -65,6 +65,7 @@ public class TestMasterMetrics {
         long reportStartTime, long reportEndTime) {
       // do nothing
     }
+
   }
 
   @BeforeClass