You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/12/02 03:50:08 UTC

hbase git commit: HBASE-12573 Backport HBASE-10591 Sanity check table configuration in createTable (Enis Soztutar)

Repository: hbase
Updated Branches:
  refs/heads/0.98 679e164e3 -> b04d1e950


HBASE-12573 Backport HBASE-10591 Sanity check table configuration in createTable (Enis Soztutar)


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

Branch: refs/heads/0.98
Commit: b04d1e9507ea52676c1303739426956af192914e
Parents: 679e164
Author: Andrew Purtell <ap...@apache.org>
Authored: Mon Dec 1 18:35:35 2014 -0500
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Dec 1 18:35:35 2014 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  13 ++-
 .../org/apache/hadoop/hbase/master/HMaster.java | 102 ++++++++++++++++++-
 .../hbase/regionserver/RegionSplitPolicy.java   |   2 +-
 .../org/apache/hadoop/hbase/TestZooKeeper.java  |   4 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |  13 ++-
 .../hadoop/hbase/client/TestFromClientSide.java |  88 +++++++++++++++-
 .../TestFromClientSideWithCoprocessor.java      |   1 +
 hbase-server/src/test/resources/hbase-site.xml  |   6 ++
 8 files changed, 219 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index e01db67..f3df203 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.WritableComparable;
 
+import com.google.protobuf.HBaseZeroCopyByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
@@ -686,7 +687,17 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
   }
 
   /**
-   * This get the class associated with the region split policy which
+   * This sets the class associated with the region split policy which
+   * determines when a region split should occur.  The class used by
+   * default is defined in {@link org.apache.hadoop.hbase.regionserver.RegionSplitPolicy}
+   * @param clazz the class name
+   */
+  public void setRegionSplitPolicyClassName(String clazz) {
+    setValue(SPLIT_POLICY, clazz);
+  }
+
+  /**
+   * This gets the class associated with the region split policy which
    * determines when a region split should occur.  The class used by
    * default is defined in {@link org.apache.hadoop.hbase.regionserver.RegionSplitPolicy}
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/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 5448592..a02b8fc 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
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -115,6 +116,7 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.protobuf.generated.*;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@@ -145,6 +147,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJani
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
@@ -161,6 +165,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorE
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
@@ -213,6 +219,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Repor
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
+import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -1776,7 +1783,7 @@ MasterServices, Server {
 
     HRegionInfo[] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
-    checkCompression(hTableDescriptor);
+    sanityCheckTableDescriptor(hTableDescriptor);
     if (cpHost != null) {
       cpHost.preCreateTable(hTableDescriptor, newRegions);
     }
@@ -1790,6 +1797,97 @@ MasterServices, Server {
 
   }
 
+  /**
+   * Checks whether the table conforms to some sane limits, and configured
+   * values (compression, etc) work. Throws an exception if something is wrong.
+   * @throws IOException
+   */
+  private void sanityCheckTableDescriptor(final HTableDescriptor htd) throws IOException {
+    final String CONF_KEY = "hbase.table.sanity.checks";
+    if (!conf.getBoolean(CONF_KEY, true)) {
+      return;
+    }
+    String tableVal = htd.getConfigurationValue(CONF_KEY);
+    if (tableVal != null && !Boolean.valueOf(tableVal)) {
+      return;
+    }
+
+    // check max file size
+    long maxFileSizeLowerLimit = 2 * 1024 * 1024L; // 2M is the default lower limit
+    long maxFileSize = htd.getMaxFileSize();
+    if (maxFileSize < 0) {
+      maxFileSize = conf.getLong(HConstants.HREGION_MAX_FILESIZE, maxFileSizeLowerLimit);
+    }
+    if (maxFileSize < conf.getLong("hbase.hregion.max.filesize.limit", maxFileSizeLowerLimit)) {
+      throw new DoNotRetryIOException("MAX_FILESIZE for table descriptor or "
+        + "\"hbase.hregion.max.filesize\" (" + maxFileSize
+        + ") is too small, which might cause over splitting into unmanageable "
+        + "number of regions. Set " + CONF_KEY + " to false at conf or table descriptor "
+          + "if you want to bypass sanity checks");
+    }
+
+    // check flush size
+    long flushSizeLowerLimit = 1024 * 1024L; // 1M is the default lower limit
+    long flushSize = htd.getMemStoreFlushSize();
+    if (flushSize < 0) {
+      flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeLowerLimit);
+    }
+    if (flushSize < conf.getLong("hbase.hregion.memstore.flush.size.limit", flushSizeLowerLimit)) {
+      throw new DoNotRetryIOException("MEMSTORE_FLUSHSIZE for table descriptor or "
+          + "\"hbase.hregion.memstore.flush.size\" ("+flushSize+") is too small, which might cause"
+          + " very frequent flushing. Set " + CONF_KEY + " to false at conf or table descriptor "
+          + "if you want to bypass sanity checks");
+    }
+
+    // check split policy class can be loaded
+    try {
+      RegionSplitPolicy.getSplitPolicyClass(htd, conf);
+    } catch (Exception ex) {
+      throw new DoNotRetryIOException(ex);
+    }
+
+    // check compression can be loaded
+    checkCompression(htd);
+
+    // check that we have at least 1 CF
+    if (htd.getColumnFamilies().length == 0) {
+      throw new DoNotRetryIOException("Table should have at least one column family "
+          + "Set "+CONF_KEY+" at conf or table descriptor if you want to bypass sanity checks");
+    }
+
+    for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+      if (hcd.getTimeToLive() <= 0) {
+        throw new DoNotRetryIOException("TTL for column family " + hcd.getNameAsString()
+          + "  must be positive. Set " + CONF_KEY + " to false at conf or table descriptor "
+          + "if you want to bypass sanity checks");
+      }
+
+      // check blockSize
+      if (hcd.getBlocksize() < 1024 || hcd.getBlocksize() > 16 * 1024 * 1024) {
+        throw new DoNotRetryIOException("Block size for column family " + hcd.getNameAsString()
+          + "  must be between 1K and 16MB Set "+CONF_KEY+" to false at conf or table descriptor "
+          + "if you want to bypass sanity checks");
+      }
+
+      // check versions
+      if (hcd.getMinVersions() < 0) {
+        throw new DoNotRetryIOException("Min versions for column family " + hcd.getNameAsString()
+          + "  must be positive. Set " + CONF_KEY + " to false at conf or table descriptor "
+          + "if you want to bypass sanity checks");
+      }
+      // max versions already being checked
+
+      // check replication scope
+      if (hcd.getScope() < 0) {
+        throw new DoNotRetryIOException("Replication scope for column family "
+          + hcd.getNameAsString() + "  must be positive. Set " + CONF_KEY + " to false at conf "
+          + "or table descriptor if you want to bypass sanity checks");
+      }
+
+      // TODO: should we check coprocessors and encryption ?
+    }
+  }
+
   private void checkCompression(final HTableDescriptor htd)
   throws IOException {
     if (!this.masterCheckCompression) return;
@@ -2071,7 +2169,7 @@ MasterServices, Server {
   public void modifyTable(final TableName tableName, final HTableDescriptor descriptor)
       throws IOException {
     checkInitialized();
-    checkCompression(descriptor);
+    sanityCheckTableDescriptor(descriptor);
     if (cpHost != null) {
       cpHost.preModifyTable(tableName, descriptor);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
index 037755f..53979af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
@@ -106,7 +106,7 @@ public abstract class RegionSplitPolicy extends Configured {
     return policy;
   }
 
-  static Class<? extends RegionSplitPolicy> getSplitPolicyClass(
+  public static Class<? extends RegionSplitPolicy> getSplitPolicyClass(
       HTableDescriptor htd, Configuration conf) throws IOException {
     String className = htd.getRegionSplitPolicyClassName();
     if (className == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index 3dcf10b..e2b4f0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -495,7 +495,9 @@ public class TestZooKeeper {
         Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
         Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i"), Bytes.toBytes("j") };
       String tableName = "testRegionAssignmentAfterMasterRecoveryDueToZKExpiry";
-      admin.createTable(new HTableDescriptor(TableName.valueOf(tableName)), SPLIT_KEYS);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
+      htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+      admin.createTable(htd, SPLIT_KEYS);
       ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
       ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
       m.getZooKeeperWatcher().close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 7c5b90b..d3a5494 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -171,8 +171,12 @@ public class TestAdmin2 {
   @Test (timeout=300000)
   public void testTableNameClash() throws Exception {
     String name = "testTableNameClash";
-    admin.createTable(new HTableDescriptor(TableName.valueOf(name + "SOMEUPPERCASE")));
-    admin.createTable(new HTableDescriptor(TableName.valueOf(name)));
+    HTableDescriptor htd1 = new HTableDescriptor(TableName.valueOf(name + "SOMEUPPERCASE"));
+    HTableDescriptor htd2 = new HTableDescriptor(TableName.valueOf(name));
+    htd1.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+    htd2.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+    admin.createTable(htd1);
+    admin.createTable(htd2);
     // Before fix, below would fail throwing a NoServerForRegionException.
     new HTable(TEST_UTIL.getConfiguration(), name).close();
   }
@@ -196,8 +200,9 @@ public class TestAdmin2 {
       byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
       byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
       HBaseAdmin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
-      hbaseadmin.createTable(new HTableDescriptor(TableName.valueOf(name)), startKey, endKey,
-        expectedRegions);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
+      htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+      hbaseadmin.createTable(htd, startKey, endKey, expectedRegions);
       hbaseadmin.close();
     } finally {
       TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 84932c6..7ab1d0f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -137,6 +137,7 @@ public class TestFromClientSide {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
         MultiRowMutationEndpoint.class.getName());
+    conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
     // We need more than one region server in this test
     TEST_UTIL.startMiniCluster(SLAVES);
   }
@@ -4358,7 +4359,7 @@ public class TestFromClientSide {
 
     HTable table =
         TEST_UTIL.createTable(tableAname,
-          new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 64);
+          new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024);
     // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
     // in Store.rowAtOrBeforeFromStoreFile
     table.setAutoFlush(true);
@@ -5362,6 +5363,91 @@ public class TestFromClientSide {
   }
 
   @Test
+  public void testIllegalTableDescriptor() throws Exception {
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testIllegalTableDescriptor"));
+    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
+
+    // create table with 0 families
+    checkTableIsIllegal(htd);
+    htd.addFamily(hcd);
+    checkTableIsLegal(htd);
+
+    htd.setMaxFileSize(1024); // 1K
+    checkTableIsIllegal(htd);
+    htd.setMaxFileSize(0);
+    checkTableIsIllegal(htd);
+    htd.setMaxFileSize(1024 * 1024 * 1024); // 1G
+    checkTableIsLegal(htd);
+
+    htd.setMemStoreFlushSize(1024);
+    checkTableIsIllegal(htd);
+    htd.setMemStoreFlushSize(0);
+    checkTableIsIllegal(htd);
+    htd.setMemStoreFlushSize(128 * 1024 * 1024); // 128M
+    checkTableIsLegal(htd);
+
+    htd.setRegionSplitPolicyClassName("nonexisting.foo.class");
+    checkTableIsIllegal(htd);
+    htd.setRegionSplitPolicyClassName(null);
+    checkTableIsLegal(htd);
+
+    hcd.setBlocksize(0);
+    checkTableIsIllegal(htd);
+    hcd.setBlocksize(1024 * 1024 * 128); // 128M
+    checkTableIsIllegal(htd);
+    hcd.setBlocksize(1024);
+    checkTableIsLegal(htd);
+
+    hcd.setTimeToLive(0);
+    checkTableIsIllegal(htd);
+    hcd.setTimeToLive(-1);
+    checkTableIsIllegal(htd);
+    hcd.setTimeToLive(1);
+    checkTableIsLegal(htd);
+
+    hcd.setMinVersions(-1);
+    checkTableIsIllegal(htd);
+    hcd.setMinVersions(3);
+    try {
+      hcd.setMaxVersions(2);
+      fail();
+    } catch (IllegalArgumentException ex) {
+      // expected
+      hcd.setMaxVersions(10);
+    }
+    checkTableIsLegal(htd);
+
+    hcd.setScope(-1);
+    checkTableIsIllegal(htd);
+    hcd.setScope(0);
+    checkTableIsLegal(htd);
+
+    // check the conf settings to disable sanity checks
+    htd.setMemStoreFlushSize(0);
+    htd.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString());
+    checkTableIsLegal(htd);
+  }
+
+  private void checkTableIsLegal(HTableDescriptor htd) throws IOException {
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    admin.createTable(htd);
+    assertTrue(admin.tableExists(htd.getTableName()));
+    admin.disableTable(htd.getTableName());
+    admin.deleteTable(htd.getTableName());
+  }
+
+  private void checkTableIsIllegal(HTableDescriptor htd) throws IOException {
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    try {
+      admin.createTable(htd);
+      fail();
+    } catch(Exception ex) {
+      // should throw ex
+    }
+    assertFalse(admin.tableExists(htd.getTableName()));
+  }
+
+  @Test
   public void testRawScanRespectsVersions() throws Exception {
     byte[] TABLE = Bytes.toBytes("testRawScan");
     HTable table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY });

http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
index a49e94c..27099a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
@@ -36,6 +36,7 @@ public class TestFromClientSideWithCoprocessor extends TestFromClientSide {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
         MultiRowMutationEndpoint.class.getName(), NoOpScanPolicyObserver.class.getName());
+    conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
     // We need more than one region server in this test
     TEST_UTIL.startMiniCluster(SLAVES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b04d1e95/hbase-server/src/test/resources/hbase-site.xml
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/hbase-site.xml b/hbase-server/src/test/resources/hbase-site.xml
index 69c5206..8c8312c 100644
--- a/hbase-server/src/test/resources/hbase-site.xml
+++ b/hbase-server/src/test/resources/hbase-site.xml
@@ -141,4 +141,10 @@
     version is X.X.X-SNAPSHOT"
     </description>
   </property>
+  <property>
+    <name>hbase.table.sanity.checks</name>
+    <value>false</value>
+    <description>Skip sanity checks in tests
+    </description>
+  </property>
 </configuration>