You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2014/02/26 23:24:20 UTC

svn commit: r1572301 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/master/ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ hbase-server/src/test/java/org/apach...

Author: enis
Date: Wed Feb 26 22:24:20 2014
New Revision: 1572301

URL: http://svn.apache.org/r1572301
Log:
HBASE-10591 Sanity check table configuration in createTable

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
    hbase/trunk/hbase-server/src/test/resources/hbase-site.xml

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java Wed Feb 26 22:24:20 2014
@@ -34,7 +34,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.regex.Matcher;
 
-import com.google.protobuf.HBaseZeroCopyByteString;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.WritableComparable;
 
+import com.google.protobuf.HBaseZeroCopyByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
@@ -655,7 +655,17 @@ public class HTableDescriptor implements
   }
 
   /**
-   * 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}
    *

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Wed Feb 26 22:24:20 2014
@@ -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;
@@ -116,10 +117,10 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 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;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
@@ -147,6 +148,8 @@ import org.apache.hadoop.hbase.protobuf.
 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;
@@ -163,6 +166,8 @@ import org.apache.hadoop.hbase.protobuf.
 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;
@@ -197,10 +202,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
-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.IsProcedureDoneRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
@@ -210,6 +211,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
+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;
@@ -1745,7 +1747,7 @@ MasterServices, Server {
 
     HRegionInfo[] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
-    checkCompression(hTableDescriptor);
+    sanityCheckTableDescriptor(hTableDescriptor);
     if (cpHost != null) {
       cpHost.preCreateTable(hTableDescriptor, newRegions);
     }
@@ -1759,6 +1761,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;
@@ -2039,7 +2132,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);
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java Wed Feb 26 22:24:20 2014
@@ -105,7 +105,7 @@ public abstract class RegionSplitPolicy 
     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) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Wed Feb 26 22:24:20 2014
@@ -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();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Wed Feb 26 22:24:20 2014
@@ -197,6 +197,7 @@ public class TestAdmin {
     exception = null;
     try {
       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(nonexistent));
+      htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
       this.admin.modifyTable(htd.getTableName(), htd);
     } catch (IOException e) {
       exception = e;
@@ -1156,8 +1157,12 @@ public class TestAdmin {
   @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();
   }
@@ -1181,8 +1186,9 @@ public class TestAdmin {
       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);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Wed Feb 26 22:24:20 2014
@@ -85,8 +85,8 @@ import org.apache.hadoop.hbase.ipc.RpcSe
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
-import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
+import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
@@ -130,6 +130,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);
   }
@@ -4252,7 +4253,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);
@@ -5396,6 +5397,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 });

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java Wed Feb 26 22:24:20 2014
@@ -36,6 +36,7 @@ public class TestFromClientSideWithCopro
     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);
   }

Modified: hbase/trunk/hbase-server/src/test/resources/hbase-site.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/resources/hbase-site.xml?rev=1572301&r1=1572300&r2=1572301&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/resources/hbase-site.xml (original)
+++ hbase/trunk/hbase-server/src/test/resources/hbase-site.xml Wed Feb 26 22:24:20 2014
@@ -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>