diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java index 328e5d563421..e4d961a56d80 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.util.MultiThreadedUpdater; import org.apache.hadoop.hbase.util.MultiThreadedWriter; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; import org.apache.hadoop.util.StringUtils; @@ -94,7 +95,7 @@ public void setConf(Configuration conf) { String.format("%s.%s", TEST_NAME, LoadTestTool.OPT_COLUMN_FAMILIES), StringUtils.join(",", DEFAULT_COLUMN_FAMILIES)); - conf.setBoolean("hbase.table.sanity.checks", true); + conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, true); // enable async wal replication to region replicas for unit tests conf.setBoolean(ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_CONF_KEY, true); 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 89c159162104..4adf648f9185 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 @@ -64,7 +64,6 @@ import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.ClusterMetricsBuilder; -import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; @@ -81,7 +80,6 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.MasterSwitchType; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -175,14 +173,8 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory; import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy; -import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.RSRpcServices; -import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; -import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy; -import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy; -import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationLoadSource; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; @@ -198,10 +190,7 @@ import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.util.Addressing; -import org.apache.hadoop.hbase.util.BloomFilterUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.CompressionTest; -import org.apache.hadoop.hbase.util.EncryptionTest; import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.HBaseFsck; import org.apache.hadoop.hbase.util.HFileArchiveUtil; @@ -211,6 +200,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.RetryCounterFactory; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; @@ -238,7 +228,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; /** * HMaster is the "master server" for HBase. An HBase cluster has one active @@ -417,12 +406,6 @@ public void run() { // Time stamp for when HMaster finishes becoming Active Master private long masterFinishedInitializationTime; - //should we check the compression codec type at master side, default true, HBASE-6370 - private final boolean masterCheckCompression; - - //should we check encryption settings at master side, default true - private final boolean masterCheckEncryption; - Map coprocessorServiceHandlers = Maps.newHashMap(); // monitor for snapshot of hbase tables @@ -541,12 +524,6 @@ public HMaster(final Configuration conf) this.conf.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName.toString()); } - // should we check the compression codec type at master side, default true, HBASE-6370 - this.masterCheckCompression = conf.getBoolean("hbase.master.check.compression", true); - - // should we check encryption settings at master side, default true - this.masterCheckEncryption = conf.getBoolean("hbase.master.check.encryption", true); - this.metricsMaster = new MetricsMaster(new MetricsMasterWrapperImpl(this)); // preload table descriptor at startup @@ -2080,7 +2057,7 @@ public long createTable(final TableDescriptor tableDescriptor, final byte[][] sp this.clusterSchemaService.getNamespace(namespace); RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(desc, splitKeys); - sanityCheckTableDescriptor(desc); + TableDescriptorChecker.sanityCheck(conf, desc); return MasterProcedureUtil .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { @@ -2134,224 +2111,6 @@ public long createSystemTable(final TableDescriptor tableDescriptor) throws IOEx return procId; } - /** - * 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 TableDescriptor htd) throws IOException { - final String CONF_KEY = "hbase.table.sanity.checks"; - boolean logWarn = false; - if (!conf.getBoolean(CONF_KEY, true)) { - logWarn = true; - } - String tableVal = htd.getValue(CONF_KEY); - if (tableVal != null && !Boolean.valueOf(tableVal)) { - logWarn = true; - } - - // 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)) { - String message = "MAX_FILESIZE for table descriptor or " - + "\"hbase.hregion.max.filesize\" (" + maxFileSize - + ") is too small, which might cause over splitting into unmanageable " - + "number of regions."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - // 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)) { - String message = "MEMSTORE_FLUSHSIZE for table descriptor or " - + "\"hbase.hregion.memstore.flush.size\" ("+flushSize+") is too small, which might cause" - + " very frequent flushing."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - // check that coprocessors and other specified plugin classes can be loaded - try { - checkClassLoading(conf, htd); - } catch (Exception ex) { - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, ex.getMessage(), null); - } - - // check compression can be loaded - try { - checkCompression(htd); - } catch (IOException e) { - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, e.getMessage(), e); - } - - // check encryption can be loaded - try { - checkEncryption(conf, htd); - } catch (IOException e) { - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, e.getMessage(), e); - } - // Verify compaction policy - try{ - checkCompactionPolicy(conf, htd); - } catch(IOException e){ - warnOrThrowExceptionForFailure(false, CONF_KEY, e.getMessage(), e); - } - // check that we have at least 1 CF - if (htd.getColumnFamilyCount() == 0) { - String message = "Table should have at least one column family."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - // check that we have minimum 1 region replicas - int regionReplicas = htd.getRegionReplication(); - if (regionReplicas < 1) { - String message = "Table region replication should be at least one."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - if (hcd.getTimeToLive() <= 0) { - String message = "TTL for column family " + hcd.getNameAsString() + " must be positive."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - // check blockSize - if (hcd.getBlocksize() < 1024 || hcd.getBlocksize() > 16 * 1024 * 1024) { - String message = "Block size for column family " + hcd.getNameAsString() - + " must be between 1K and 16MB."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - // check versions - if (hcd.getMinVersions() < 0) { - String message = "Min versions for column family " + hcd.getNameAsString() - + " must be positive."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - // max versions already being checked - - // HBASE-13776 Setting illegal versions for ColumnFamilyDescriptor - // does not throw IllegalArgumentException - // check minVersions <= maxVerions - if (hcd.getMinVersions() > hcd.getMaxVersions()) { - String message = "Min versions for column family " + hcd.getNameAsString() - + " must be less than the Max versions."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - // check replication scope - checkReplicationScope(hcd); - // check bloom filter type - checkBloomFilterType(hcd); - - // check data replication factor, it can be 0(default value) when user has not explicitly - // set the value, in this case we use default replication factor set in the file system. - if (hcd.getDFSReplication() < 0) { - String message = "HFile Replication for column family " + hcd.getNameAsString() - + " must be greater than zero."; - warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null); - } - - // TODO: should we check coprocessors and encryption ? - } - } - - private void checkReplicationScope(ColumnFamilyDescriptor hcd) throws IOException{ - // check replication scope - WALProtos.ScopeType scop = WALProtos.ScopeType.valueOf(hcd.getScope()); - if (scop == null) { - String message = "Replication scope for column family " - + hcd.getNameAsString() + " is " + hcd.getScope() + " which is invalid."; - - LOG.error(message); - throw new DoNotRetryIOException(message); - } - } - - private void checkCompactionPolicy(Configuration conf, TableDescriptor htd) - throws IOException { - // FIFO compaction has some requirements - // Actually FCP ignores periodic major compactions - String className = htd.getValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY); - if (className == null) { - className = - conf.get(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, - ExploringCompactionPolicy.class.getName()); - } - - int blockingFileCount = HStore.DEFAULT_BLOCKING_STOREFILE_COUNT; - String sv = htd.getValue(HStore.BLOCKING_STOREFILES_KEY); - if (sv != null) { - blockingFileCount = Integer.parseInt(sv); - } else { - blockingFileCount = conf.getInt(HStore.BLOCKING_STOREFILES_KEY, blockingFileCount); - } - - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - String compactionPolicy = - hcd.getConfigurationValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY); - if (compactionPolicy == null) { - compactionPolicy = className; - } - if (!compactionPolicy.equals(FIFOCompactionPolicy.class.getName())) { - continue; - } - // FIFOCompaction - String message = null; - - // 1. Check TTL - if (hcd.getTimeToLive() == ColumnFamilyDescriptorBuilder.DEFAULT_TTL) { - message = "Default TTL is not supported for FIFO compaction"; - throw new IOException(message); - } - - // 2. Check min versions - if (hcd.getMinVersions() > 0) { - message = "MIN_VERSION > 0 is not supported for FIFO compaction"; - throw new IOException(message); - } - - // 3. blocking file count - sv = hcd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY); - if (sv != null) { - blockingFileCount = Integer.parseInt(sv); - } - if (blockingFileCount < 1000) { - message = - "Blocking file count '" + HStore.BLOCKING_STOREFILES_KEY + "' " + blockingFileCount - + " is below recommended minimum of 1000 for column family "+ hcd.getNameAsString(); - throw new IOException(message); - } - } - } - - private static void checkBloomFilterType(ColumnFamilyDescriptor cfd) - throws IOException { - Configuration conf = new CompoundConfiguration().addStringMap(cfd.getConfiguration()); - try { - BloomFilterUtil.getBloomFilterParam(cfd.getBloomFilterType(), conf); - } catch (IllegalArgumentException e) { - throw new DoNotRetryIOException("Failed to get bloom filter param", e); - } - } - - // HBASE-13350 - Helper method to log warning on sanity check failures if checks disabled. - private static void warnOrThrowExceptionForFailure(boolean logWarn, String confKey, - String message, Exception cause) throws IOException { - if (!logWarn) { - throw new DoNotRetryIOException(message + " Set " + confKey + - " to false at conf or table descriptor if you want to bypass sanity checks", cause); - } - LOG.warn(message); - } - private void startActiveMasterManager(int infoPort) throws KeeperException { String backupZNode = ZNodePaths.joinZNode( zooKeeper.getZNodePaths().backupMasterAddressesZNode, serverName.toString()); @@ -2405,41 +2164,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException { } } - private void checkCompression(final TableDescriptor htd) - throws IOException { - if (!this.masterCheckCompression) return; - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - checkCompression(hcd); - } - } - - private void checkCompression(final ColumnFamilyDescriptor hcd) - throws IOException { - if (!this.masterCheckCompression) return; - CompressionTest.testCompression(hcd.getCompressionType()); - CompressionTest.testCompression(hcd.getCompactionCompressionType()); - } - - private void checkEncryption(final Configuration conf, final TableDescriptor htd) - throws IOException { - if (!this.masterCheckEncryption) return; - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - checkEncryption(conf, hcd); - } - } - - private void checkEncryption(final Configuration conf, final ColumnFamilyDescriptor hcd) - throws IOException { - if (!this.masterCheckEncryption) return; - EncryptionTest.testEncryption(conf, hcd.getEncryptionType(), hcd.getEncryptionKey()); - } - - private void checkClassLoading(final Configuration conf, final TableDescriptor htd) - throws IOException { - RegionSplitPolicy.getSplitPolicyClass(htd, conf); - RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd); - } - private static boolean isCatalogTable(final TableName tableName) { return tableName.equals(TableName.META_TABLE_NAME); } @@ -2682,7 +2406,7 @@ protected void run() throws IOException { TableDescriptor oldDescriptor = getMaster().getTableDescriptors().get(tableName); TableDescriptor newDescriptor = getMaster().getMasterCoprocessorHost() .preModifyTable(tableName, oldDescriptor, newDescriptorGetter.get()); - sanityCheckTableDescriptor(newDescriptor); + TableDescriptorChecker.sanityCheck(conf, newDescriptor); LOG.info("{} modify table {} from {} to {}", getClientIdAuditPrefix(), tableName, oldDescriptor, newDescriptor); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 936730b9b8d1..94d7785c9083 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.NonceKey; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.zookeeper.KeeperException; @@ -826,6 +827,9 @@ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final TableDescriptor snapshotTableDesc = manifest.getTableDescriptor(); TableName tableName = TableName.valueOf(reqSnapshot.getTable()); + // sanity check the new table descriptor + TableDescriptorChecker.sanityCheck(master.getConfiguration(), snapshotTableDesc); + // stop tracking "abandoned" handlers cleanupSentinels(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/TableDescriptorChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/TableDescriptorChecker.java new file mode 100644 index 000000000000..a2ee1ff25593 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/TableDescriptorChecker.java @@ -0,0 +1,301 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.util; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CompoundConfiguration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; +import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; + +/** + * Only used for master to sanity check {@link org.apache.hadoop.hbase.client.TableDescriptor}. + */ +@InterfaceAudience.Private +public final class TableDescriptorChecker { + private static Logger LOG = LoggerFactory.getLogger(TableDescriptorChecker.class); + + public static final String TABLE_SANITY_CHECKS = "hbase.table.sanity.checks"; + public static final boolean DEFAULT_TABLE_SANITY_CHECKS = true; + + //should we check the compression codec type at master side, default true, HBASE-6370 + public static final String MASTER_CHECK_COMPRESSION = "hbase.master.check.compression"; + public static final boolean DEFAULT_MASTER_CHECK_COMPRESSION = true; + + //should we check encryption settings at master side, default true + public static final String MASTER_CHECK_ENCRYPTION = "hbase.master.check.encryption"; + public static final boolean DEFAULT_MASTER_CHECK_ENCRYPTION = true; + + private TableDescriptorChecker() { + } + + /** + * Checks whether the table conforms to some sane limits, and configured + * values (compression, etc) work. Throws an exception if something is wrong. + */ + public static void sanityCheck(final Configuration conf, final TableDescriptor td) + throws IOException { + // Setting this to true logs the warning instead of throwing exception + boolean logWarn = false; + if (!conf.getBoolean(TABLE_SANITY_CHECKS, DEFAULT_TABLE_SANITY_CHECKS)) { + logWarn = true; + } + String tableVal = td.getValue(TABLE_SANITY_CHECKS); + if (tableVal != null && !Boolean.valueOf(tableVal)) { + logWarn = true; + } + + // check max file size + long maxFileSizeLowerLimit = 2 * 1024 * 1024L; // 2M is the default lower limit + long maxFileSize = td.getMaxFileSize(); + if (maxFileSize < 0) { + maxFileSize = conf.getLong(HConstants.HREGION_MAX_FILESIZE, maxFileSizeLowerLimit); + } + if (maxFileSize < conf.getLong("hbase.hregion.max.filesize.limit", maxFileSizeLowerLimit)) { + String message = + "MAX_FILESIZE for table descriptor or " + "\"hbase.hregion.max.filesize\" (" + + maxFileSize + ") is too small, which might cause over splitting into unmanageable " + + "number of regions."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + + // check flush size + long flushSizeLowerLimit = 1024 * 1024L; // 1M is the default lower limit + long flushSize = td.getMemStoreFlushSize(); + if (flushSize < 0) { + flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeLowerLimit); + } + if (flushSize < conf.getLong("hbase.hregion.memstore.flush.size.limit", flushSizeLowerLimit)) { + String message = "MEMSTORE_FLUSHSIZE for table descriptor or " + + "\"hbase.hregion.memstore.flush.size\" (" + flushSize + + ") is too small, which might cause" + " very frequent flushing."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + + // check that coprocessors and other specified plugin classes can be loaded + try { + checkClassLoading(conf, td); + } catch (Exception ex) { + warnOrThrowExceptionForFailure(logWarn, ex.getMessage(), null); + } + + if (conf.getBoolean(MASTER_CHECK_COMPRESSION, DEFAULT_MASTER_CHECK_COMPRESSION)) { + // check compression can be loaded + try { + checkCompression(td); + } catch (IOException e) { + warnOrThrowExceptionForFailure(logWarn, e.getMessage(), e); + } + } + + if (conf.getBoolean(MASTER_CHECK_ENCRYPTION, DEFAULT_MASTER_CHECK_ENCRYPTION)) { + // check encryption can be loaded + try { + checkEncryption(conf, td); + } catch (IOException e) { + warnOrThrowExceptionForFailure(logWarn, e.getMessage(), e); + } + } + + // Verify compaction policy + try { + checkCompactionPolicy(conf, td); + } catch (IOException e) { + warnOrThrowExceptionForFailure(false, e.getMessage(), e); + } + // check that we have at least 1 CF + if (td.getColumnFamilyCount() == 0) { + String message = "Table should have at least one column family."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + + // check that we have minimum 1 region replicas + int regionReplicas = td.getRegionReplication(); + if (regionReplicas < 1) { + String message = "Table region replication should be at least one."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + + for (ColumnFamilyDescriptor hcd : td.getColumnFamilies()) { + if (hcd.getTimeToLive() <= 0) { + String message = "TTL for column family " + hcd.getNameAsString() + " must be positive."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + + // check blockSize + if (hcd.getBlocksize() < 1024 || hcd.getBlocksize() > 16 * 1024 * 1024) { + String message = "Block size for column family " + hcd.getNameAsString() + + " must be between 1K and 16MB."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + + // check versions + if (hcd.getMinVersions() < 0) { + String message = + "Min versions for column family " + hcd.getNameAsString() + " must be positive."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + // max versions already being checked + + // HBASE-13776 Setting illegal versions for ColumnFamilyDescriptor + // does not throw IllegalArgumentException + // check minVersions <= maxVerions + if (hcd.getMinVersions() > hcd.getMaxVersions()) { + String message = "Min versions for column family " + hcd.getNameAsString() + + " must be less than the Max versions."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + + // check replication scope + checkReplicationScope(hcd); + // check bloom filter type + checkBloomFilterType(hcd); + + // check data replication factor, it can be 0(default value) when user has not explicitly + // set the value, in this case we use default replication factor set in the file system. + if (hcd.getDFSReplication() < 0) { + String message = "HFile Replication for column family " + hcd.getNameAsString() + + " must be greater than zero."; + warnOrThrowExceptionForFailure(logWarn, message, null); + } + } + } + + private static void checkReplicationScope(final ColumnFamilyDescriptor cfd) throws IOException { + // check replication scope + WALProtos.ScopeType scop = WALProtos.ScopeType.valueOf(cfd.getScope()); + if (scop == null) { + String message = + "Replication scope for column family " + cfd.getNameAsString() + " is " + cfd.getScope() + + " which is invalid."; + + LOG.error(message); + throw new DoNotRetryIOException(message); + } + } + + private static void checkCompactionPolicy(Configuration conf, TableDescriptor td) + throws IOException { + // FIFO compaction has some requirements + // Actually FCP ignores periodic major compactions + String className = td.getValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY); + if (className == null) { + className = conf.get(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, + ExploringCompactionPolicy.class.getName()); + } + + int blockingFileCount = HStore.DEFAULT_BLOCKING_STOREFILE_COUNT; + String sv = td.getValue(HStore.BLOCKING_STOREFILES_KEY); + if (sv != null) { + blockingFileCount = Integer.parseInt(sv); + } else { + blockingFileCount = conf.getInt(HStore.BLOCKING_STOREFILES_KEY, blockingFileCount); + } + + for (ColumnFamilyDescriptor hcd : td.getColumnFamilies()) { + String compactionPolicy = + hcd.getConfigurationValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY); + if (compactionPolicy == null) { + compactionPolicy = className; + } + if (!compactionPolicy.equals(FIFOCompactionPolicy.class.getName())) { + continue; + } + // FIFOCompaction + String message = null; + + // 1. Check TTL + if (hcd.getTimeToLive() == ColumnFamilyDescriptorBuilder.DEFAULT_TTL) { + message = "Default TTL is not supported for FIFO compaction"; + throw new IOException(message); + } + + // 2. Check min versions + if (hcd.getMinVersions() > 0) { + message = "MIN_VERSION > 0 is not supported for FIFO compaction"; + throw new IOException(message); + } + + // 3. blocking file count + sv = hcd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY); + if (sv != null) { + blockingFileCount = Integer.parseInt(sv); + } + if (blockingFileCount < 1000) { + message = + "Blocking file count '" + HStore.BLOCKING_STOREFILES_KEY + "' " + blockingFileCount + + " is below recommended minimum of 1000 for column family " + hcd.getNameAsString(); + throw new IOException(message); + } + } + } + + private static void checkBloomFilterType(ColumnFamilyDescriptor cfd) throws IOException { + Configuration conf = new CompoundConfiguration().addStringMap(cfd.getConfiguration()); + try { + BloomFilterUtil.getBloomFilterParam(cfd.getBloomFilterType(), conf); + } catch (IllegalArgumentException e) { + throw new DoNotRetryIOException("Failed to get bloom filter param", e); + } + } + + private static void checkCompression(final TableDescriptor td) throws IOException { + for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) { + CompressionTest.testCompression(cfd.getCompressionType()); + CompressionTest.testCompression(cfd.getCompactionCompressionType()); + } + } + + private static void checkEncryption(final Configuration conf, final TableDescriptor td) + throws IOException { + for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) { + EncryptionTest.testEncryption(conf, cfd.getEncryptionType(), cfd.getEncryptionKey()); + } + } + + private static void checkClassLoading(final Configuration conf, final TableDescriptor td) + throws IOException { + RegionSplitPolicy.getSplitPolicyClass(td, conf); + RegionCoprocessorHost.testTableCoprocessorAttrs(conf, td); + } + + // HBASE-13350 - Helper method to log warning on sanity check failures if checks disabled. + private static void warnOrThrowExceptionForFailure(boolean logWarn, String message, + Exception cause) throws IOException { + if (!logWarn) { + throw new DoNotRetryIOException(message + " Set " + TABLE_SANITY_CHECKS + + " to false at conf or table descriptor if you want to bypass sanity checks", cause); + } + LOG.warn(message); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java index ec3d4685c210..34b45f023143 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java @@ -103,8 +103,6 @@ public static void setUpBeforeClass() throws Exception { 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 conf.setInt("hbase.regionserver.handler.count", 20); conf.setInt("hbase.bucketcache.size", 400); conf.setStrings(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java index cbded23b2dd9..d5babfc4864d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java @@ -113,8 +113,6 @@ public static void setUpBeforeClass() throws Exception { 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 conf.setInt("hbase.regionserver.handler.count", 20); conf.setInt("hbase.bucketcache.size", 400); conf.setStrings(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap"); 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 9c6c8ff2a4e0..ec44bd09bf1d 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 @@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -150,7 +151,7 @@ protected static final void initialize(Class... cps) throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, Arrays.stream(cps).map(Class::getName).toArray(String[]::new)); - conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests + conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, true); // enable for below tests // We need more than one region server in this test TEST_UTIL.startMiniCluster(SLAVES); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java index 0bb2b3f93e6d..e7bc8b3d4b2c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java @@ -26,16 +26,17 @@ import java.io.IOException; import java.lang.reflect.Field; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -53,7 +54,7 @@ public class TestIllegalTableDescriptor { HBaseClassTestRule.forClass(TestIllegalTableDescriptor.class); // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide. - private static final Logger masterLogger; + private static final Logger LOGGER; protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -63,19 +64,17 @@ public class TestIllegalTableDescriptor { public TestName name = new TestName(); static { - masterLogger = mock(Logger.class); + LOGGER = mock(Logger.class); } @BeforeClass public static void setUpBeforeClass() throws Exception { // replacing HMaster.LOG with our mock logger for verifying logging - Field field = HMaster.class.getDeclaredField("LOG"); + Field field = TableDescriptorChecker.class.getDeclaredField("LOG"); field.setAccessible(true); - field.set(null, masterLogger); - + field.set(null, LOGGER); Configuration conf = TEST_UTIL.getConfiguration(); - conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests - // We need more than one region server in this test + conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, true); // enable for below tests TEST_UTIL.startMiniCluster(1); } @@ -172,10 +171,10 @@ public void testIllegalTableDescriptor() throws Exception { htd.setMemStoreFlushSize(0); // Check that logs warn on invalid table but allow it. - htd.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString()); + htd.setConfiguration(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString()); checkTableIsLegal(htd); - verify(masterLogger).warn(contains("MEMSTORE_FLUSHSIZE for table " + verify(LOGGER).warn(contains("MEMSTORE_FLUSHSIZE for table " + "descriptor or \"hbase.hregion.memstore.flush.size\" (0) is too small, which might " + "cause very frequent flushing.")); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java index bd4cc91c217d..368750e2856b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java @@ -46,6 +46,5 @@ public void before() throws Exception { 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 } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java index 8a41470992d5..4b2886ed6e6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java @@ -83,7 +83,6 @@ public static void beforeClass() throws Exception { 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); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java index cadce8ac0caa..9d073743cb64 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java @@ -63,7 +63,6 @@ public static void setUpBeforeClass() throws Exception { //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); Configuration conf = TEST_UTIL.getConfiguration(); - conf.setBoolean("hbase.table.sanity.checks", true); // ignore sanity checks in the server TEST_UTIL.startMiniCluster(SLAVES); // Write a bunch of data diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java index 7c338cdf55e7..6103de075af2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -74,7 +75,7 @@ public static void startCluster() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); // Disable sanity check for coprocessor - conf.setBoolean("hbase.table.sanity.checks", false); + conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, false); // set RIT stuck warning threshold to a small value conf.setInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 20); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java index 26686c39ef27..6609e6291f40 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.ModifyRegionUtils; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -95,7 +96,7 @@ public void testCreateWithoutColumnFamily() throws Exception { TableDescriptorBuilder.newBuilder(MasterProcedureTestingUtility.createHTD(tableName)); // disable sanity check - builder.setValue("hbase.table.sanity.checks", Boolean.FALSE.toString()); + builder.setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString()); TableDescriptor htd = builder.build(); final RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java index 8bd3958f94e6..f7952c4695a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.NonceKey; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Rule; @@ -174,7 +175,7 @@ public void testModifyTableDeleteCF() throws Exception { new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName)); htd2.removeFamily(Bytes.toBytes(cf3)); // Disable Sanity check - htd2.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString()); + htd2.setConfiguration(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString()); long procId2 = ProcedureTestingUtility.submitAndWait(procExec, diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java index fe6655e7a978..592d5f1e3900 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.transport.THttpClient; @@ -79,7 +80,7 @@ public class TestThriftHttpServer { @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.thrift.http", true); - TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false); + TEST_UTIL.getConfiguration().setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, false); TEST_UTIL.startMiniCluster(); //ensure that server time increments every time we do an operation, otherwise //successive puts having the same timestamp will override each other diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java index 09d0605bce0c..f2b1e765b102 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java @@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.thrift.generated.TRowResult; import org.apache.hadoop.hbase.thrift.generated.TScan; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -114,7 +115,7 @@ private static ByteBuffer asByteBuffer(long l) { @BeforeClass public static void beforeClass() throws Exception { UTIL.getConfiguration().setBoolean(COALESCE_INC_KEY, true); - UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false); + UTIL.getConfiguration().setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, false); UTIL.getConfiguration().setInt("hbase.client.retries.number", 3); UTIL.startMiniCluster(); } diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java index 08dae61d5cc8..74007963e77b 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.protocol.TProtocol; @@ -126,7 +127,7 @@ private String getParametersString() { @BeforeClass public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false); + TEST_UTIL.getConfiguration().setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, false); TEST_UTIL.startMiniCluster(); //ensure that server time increments every time we do an operation, otherwise //successive puts having the same timestamp will override each other diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java index 45abe79a995e..acee81419151 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.thrift.generated.Hbase; +import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.apache.hadoop.security.authentication.util.KerberosName; import org.apache.http.HttpHeaders; import org.apache.http.auth.AuthSchemeProvider; @@ -166,7 +167,7 @@ public static void setUpBeforeClass() throws Exception { setupUser(kdc, spnegoServerKeytab, spnegoServerPrincipal); TEST_UTIL.getConfiguration().setBoolean(Constants.USE_HTTP_CONF_KEY, true); - TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false); + TEST_UTIL.getConfiguration().setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, false); addSecurityConfigurations(TEST_UTIL.getConfiguration()); TestThriftHttpServer.setUpBeforeClass();