You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2018/04/05 23:57:02 UTC

[2/2] hive git commit: HIVE-19031 Mark duplicate configs in HiveConf as deprecated (Alan Gates, reviewed by Thejas Nair)

HIVE-19031 Mark duplicate configs in HiveConf as deprecated (Alan Gates, reviewed by Thejas Nair)


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

Branch: refs/heads/master
Commit: 105cc6543051fe5697de754a0f093539bdac59ff
Parents: 7845089
Author: Alan Gates <ga...@hortonworks.com>
Authored: Thu Apr 5 16:55:55 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Thu Apr 5 16:55:55 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hive/beeline/HiveSchemaTool.java |   2 +-
 .../apache/hive/beeline/TestHiveSchemaTool.java |   4 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   | 571 +++++++++++++++++--
 .../hive/ql/parse/TestReplicationScenarios.java |   4 +-
 .../hadoop/hive/ql/session/SessionState.java    |  11 +-
 .../hadoop/hive/metastore/MetaStoreInit.java    |   8 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   4 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   3 -
 .../hive/metastore/RetryingHMSHandler.java      |   6 +-
 .../hive/metastore/conf/MetastoreConf.java      | 282 ++-------
 .../hive/metastore/conf/RangeValidator.java     |  38 ++
 .../hive/metastore/conf/StringSetValidator.java |  51 ++
 .../hive/metastore/conf/TimeValidator.java      |  67 +++
 .../hadoop/hive/metastore/conf/Validator.java   |  87 +++
 .../datasource/DataSourceProvider.java          |   2 +-
 .../hive/metastore/tools/HiveSchemaHelper.java  |   2 +-
 .../metastore/tools/MetastoreSchemaTool.java    |   2 +-
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   2 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   6 +-
 .../DummyRawStoreForJdoConnection.java          |   6 +-
 .../hive/metastore/TestHiveMetaStore.java       |   3 +-
 .../TestMetaStoreConnectionUrlHook.java         |   4 +-
 .../metastore/TestObjectStoreInitRetry.java     |   8 +-
 .../TestRetriesInRetryingHMSHandler.java        |   6 +-
 .../hive/metastore/TestRetryingHMSHandler.java  |   6 +-
 .../tools/TestSchemaToolForMetastore.java       |   2 +-
 .../metastore/txn/TestTxnHandlerNegative.java   |   2 +-
 27 files changed, 891 insertions(+), 298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
index 040724a..6a819da 100644
--- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
+++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
@@ -983,7 +983,7 @@ public class HiveSchemaTool {
     private String[] argsWith(String password) throws IOException {
       return new String[]
         {
-          "-u", url == null ? HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECTURLKEY, hiveConf) : url,
+          "-u", url == null ? HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECT_URL_KEY, hiveConf) : url,
           "-d", driver == null ? HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECTION_DRIVER, hiveConf) : driver,
           "-n", userName,
           "-p", password,

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java b/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java
index 7ee7e9d..3d5f086 100644
--- a/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java
+++ b/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java
@@ -56,7 +56,7 @@ public class TestHiveSchemaTool {
   public void setup() throws IOException {
     mockStatic(HiveSchemaHelper.class);
     when(HiveSchemaHelper
-        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTURLKEY), same(hiveConf)))
+        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECT_URL_KEY), same(hiveConf)))
         .thenReturn("someURL");
     when(HiveSchemaHelper
         .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTION_DRIVER), same(hiveConf)))
@@ -72,7 +72,7 @@ public class TestHiveSchemaTool {
   @After
   public void globalAssert() throws IOException {
     verifyStatic();
-    HiveSchemaHelper.getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTURLKEY), same(hiveConf));
+    HiveSchemaHelper.getValidConfVar(eq(MetastoreConf.ConfVars.CONNECT_URL_KEY), same(hiveConf));
     HiveSchemaHelper
         .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTION_DRIVER), same(hiveConf));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 5f07312..f2af61d 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -631,82 +631,188 @@ public class HiveConf extends Configuration {
     // Metastore stuff. Be sure to update HiveConf.metaVars when you add something here!
     METASTOREDBTYPE("hive.metastore.db.type", "DERBY", new StringSet("DERBY", "ORACLE", "MYSQL", "MSSQL", "POSTGRES"),
         "Type of database used by the metastore. Information schema & JDBCStorageHandler depend on it."),
+    /**
+     * @deprecated Use MetastoreConf.WAREHOUSE
+     */
+    @Deprecated
     METASTOREWAREHOUSE("hive.metastore.warehouse.dir", "/user/hive/warehouse",
         "location of default database for the warehouse"),
+    /**
+     * @deprecated Use MetastoreConf.THRIFT_URIS
+     */
+    @Deprecated
     METASTOREURIS("hive.metastore.uris", "",
         "Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore."),
+
+    /**
+     * @deprecated Use MetastoreConf.THRIFT_URI_SELECTION
+     */
+    @Deprecated
     METASTORESELECTION("hive.metastore.uri.selection", "RANDOM",
         new StringSet("SEQUENTIAL", "RANDOM"),
         "Determines the selection mechanism used by metastore client to connect to remote " +
             "metastore.  SEQUENTIAL implies that the first valid metastore from the URIs specified " +
             "as part of hive.metastore.uris will be picked.  RANDOM implies that the metastore " +
             "will be picked randomly"),
+    /**
+     * @deprecated Use MetastoreConf.CAPABILITY_CHECK
+     */
+    @Deprecated
     METASTORE_CAPABILITY_CHECK("hive.metastore.client.capability.check", true,
         "Whether to check client capabilities for potentially breaking API usage."),
     METASTORE_FASTPATH("hive.metastore.fastpath", false,
         "Used to avoid all of the proxies and object copies in the metastore.  Note, if this is " +
             "set, you MUST use a local metastore (hive.metastore.uris must be empty) otherwise " +
             "undefined and most likely undesired behavior will result"),
+    /**
+     * @deprecated Use MetastoreConf.FS_HANDLER_THREADS_COUNT
+     */
+    @Deprecated
     METASTORE_FS_HANDLER_THREADS_COUNT("hive.metastore.fshandler.threads", 15,
         "Number of threads to be allocated for metastore handler for fs operations."),
+    /**
+     * @deprecated Use MetastoreConf.FILE_METADATA_THREADS
+     */
+    @Deprecated
     METASTORE_HBASE_FILE_METADATA_THREADS("hive.metastore.hbase.file.metadata.threads", 1,
         "Number of threads to use to read file metadata in background to cache it."),
+
+    /**
+     * @deprecated Use MetastoreConf.URI_RESOLVER
+     */
+    @Deprecated
     METASTORE_URI_RESOLVER("hive.metastore.uri.resolver", "",
-            "If set, fully qualified class name of resolver for hive metastore uri's"),
+        "If set, fully qualified class name of resolver for hive metastore uri's"),
+
+    /**
+     * @deprecated Use MetastoreConf.THRIFT_CONNECTION_RETRIES
+     */
+    @Deprecated
     METASTORETHRIFTCONNECTIONRETRIES("hive.metastore.connect.retries", 3,
         "Number of retries while opening a connection to metastore"),
+    /**
+     * @deprecated Use MetastoreConf.THRIFT_FAILURE_RETRIES
+     */
+    @Deprecated
     METASTORETHRIFTFAILURERETRIES("hive.metastore.failure.retries", 1,
         "Number of retries upon failure of Thrift metastore calls"),
+    /**
+     * @deprecated Use MetastoreConf.SERVER_PORT
+     */
+    @Deprecated
     METASTORE_SERVER_PORT("hive.metastore.port", 9083, "Hive metastore listener port"),
+    /**
+     * @deprecated Use MetastoreConf.CLIENT_CONNECT_RETRY_DELAY
+     */
+    @Deprecated
     METASTORE_CLIENT_CONNECT_RETRY_DELAY("hive.metastore.client.connect.retry.delay", "1s",
         new TimeValidator(TimeUnit.SECONDS),
         "Number of seconds for the client to wait between consecutive connection attempts"),
+    /**
+     * @deprecated Use MetastoreConf.CLIENT_SOCKET_TIMEOUT
+     */
+    @Deprecated
     METASTORE_CLIENT_SOCKET_TIMEOUT("hive.metastore.client.socket.timeout", "600s",
         new TimeValidator(TimeUnit.SECONDS),
         "MetaStore Client socket timeout in seconds"),
+    /**
+     * @deprecated Use MetastoreConf.CLIENT_SOCKET_LIFETIME
+     */
+    @Deprecated
     METASTORE_CLIENT_SOCKET_LIFETIME("hive.metastore.client.socket.lifetime", "0s",
         new TimeValidator(TimeUnit.SECONDS),
         "MetaStore Client socket lifetime in seconds. After this time is exceeded, client\n" +
         "reconnects on the next MetaStore operation. A value of 0s means the connection\n" +
         "has an infinite lifetime."),
+    /**
+     * @deprecated Use MetastoreConf.PWD
+     */
+    @Deprecated
     METASTOREPWD("javax.jdo.option.ConnectionPassword", "mine",
         "password to use against metastore database"),
+    /**
+     * @deprecated Use MetastoreConf.CONNECT_URL_HOOK
+     */
+    @Deprecated
     METASTORECONNECTURLHOOK("hive.metastore.ds.connection.url.hook", "",
         "Name of the hook to use for retrieving the JDO connection URL. If empty, the value in javax.jdo.option.ConnectionURL is used"),
+    /**
+     * @deprecated Use MetastoreConf.MULTITHREADED
+     */
+    @Deprecated
     METASTOREMULTITHREADED("javax.jdo.option.Multithreaded", true,
         "Set this to true if multiple threads access metastore through JDO concurrently."),
+    /**
+     * @deprecated Use MetastoreConf.CONNECT_URL_KEY
+     */
+    @Deprecated
     METASTORECONNECTURLKEY("javax.jdo.option.ConnectionURL",
         "jdbc:derby:;databaseName=metastore_db;create=true",
         "JDBC connect string for a JDBC metastore.\n" +
         "To use SSL to encrypt/authenticate the connection, provide database-specific SSL flag in the connection URL.\n" +
         "For example, jdbc:postgresql://myhost/db?ssl=true for postgres database."),
+    /**
+     * @deprecated Use MetastoreConf.DBACCESS_SSL_PROPS
+     */
+    @Deprecated
     METASTORE_DBACCESS_SSL_PROPS("hive.metastore.dbaccess.ssl.properties", "",
            "Comma-separated SSL properties for metastore to access database when JDO connection URL\n" +
            "enables SSL access. e.g. javax.net.ssl.trustStore=/tmp/truststore,javax.net.ssl.trustStorePassword=pwd."),
+    /**
+     * @deprecated Use MetastoreConf.HMS_HANDLER_ATTEMPTS
+     */
+    @Deprecated
     HMSHANDLERATTEMPTS("hive.hmshandler.retry.attempts", 10,
         "The number of times to retry a HMSHandler call if there were a connection error."),
+    /**
+     * @deprecated Use MetastoreConf.HMS_HANDLER_INTERVAL
+     */
+    @Deprecated
     HMSHANDLERINTERVAL("hive.hmshandler.retry.interval", "2000ms",
         new TimeValidator(TimeUnit.MILLISECONDS), "The time between HMSHandler retry attempts on failure."),
+    /**
+     * @deprecated Use MetastoreConf.HMS_HANDLER_FORCE_RELOAD_CONF
+     */
+    @Deprecated
     HMSHANDLERFORCERELOADCONF("hive.hmshandler.force.reload.conf", false,
         "Whether to force reloading of the HMSHandler configuration (including\n" +
         "the connection URL, before the next metastore query that accesses the\n" +
         "datastore. Once reloaded, this value is reset to false. Used for\n" +
         "testing only."),
+    /**
+     * @deprecated Use MetastoreConf.SERVER_MAX_MESSAGE_SIZE
+     */
+    @Deprecated
     METASTORESERVERMAXMESSAGESIZE("hive.metastore.server.max.message.size", 100*1024*1024L,
         "Maximum message size in bytes a HMS will accept."),
+    /**
+     * @deprecated Use MetastoreConf.SERVER_MIN_THREADS
+     */
+    @Deprecated
     METASTORESERVERMINTHREADS("hive.metastore.server.min.threads", 200,
         "Minimum number of worker threads in the Thrift server's pool."),
+    /**
+     * @deprecated Use MetastoreConf.SERVER_MAX_THREADS
+     */
+    @Deprecated
     METASTORESERVERMAXTHREADS("hive.metastore.server.max.threads", 1000,
         "Maximum number of worker threads in the Thrift server's pool."),
+    /**
+     * @deprecated Use MetastoreConf.TCP_KEEP_ALIVE
+     */
+    @Deprecated
     METASTORE_TCP_KEEP_ALIVE("hive.metastore.server.tcp.keepalive", true,
         "Whether to enable TCP keepalive for the metastore server. Keepalive will prevent accumulation of half-open connections."),
 
+    /**
+     * @deprecated Use MetastoreConf.WM_DEFAULT_POOL_SIZE
+     */
+    @Deprecated
     METASTORE_WM_DEFAULT_POOL_SIZE("hive.metastore.wm.default.pool.size", 4,
         "The size of a default pool to create when creating an empty resource plan;\n" +
         "If not positive, no default pool will be created."),
 
-
-   METASTORE_INT_ORIGINAL("hive.metastore.archive.intermediate.original",
+    METASTORE_INT_ORIGINAL("hive.metastore.archive.intermediate.original",
         "_INTERMEDIATE_ORIGINAL",
         "Intermediate dir suffixes used for archiving. Not important what they\n" +
         "are, as long as collisions are avoided"),
@@ -714,24 +820,56 @@ public class HiveConf extends Configuration {
         "_INTERMEDIATE_ARCHIVED", ""),
     METASTORE_INT_EXTRACTED("hive.metastore.archive.intermediate.extracted",
         "_INTERMEDIATE_EXTRACTED", ""),
+    /**
+     * @deprecated Use MetastoreConf.KERBEROS_KEYTAB_FILE
+     */
+    @Deprecated
     METASTORE_KERBEROS_KEYTAB_FILE("hive.metastore.kerberos.keytab.file", "",
         "The path to the Kerberos Keytab file containing the metastore Thrift server's service principal."),
+    /**
+     * @deprecated Use MetastoreConf.KERBEROS_PRINCIPAL
+     */
+    @Deprecated
     METASTORE_KERBEROS_PRINCIPAL("hive.metastore.kerberos.principal",
         "hive-metastore/_HOST@EXAMPLE.COM",
         "The service principal for the metastore Thrift server. \n" +
         "The special string _HOST will be replaced automatically with the correct host name."),
+    /**
+     * @deprecated Use MetastoreConf.CLIENT_KERBEROS_PRINCIPAL
+     */
+    @Deprecated
     METASTORE_CLIENT_KERBEROS_PRINCIPAL("hive.metastore.client.kerberos.principal",
         "", // E.g. "hive-metastore/_HOST@EXAMPLE.COM".
         "The Kerberos principal associated with the HA cluster of hcat_servers."),
+    /**
+     * @deprecated Use MetastoreConf.USE_THRIFT_SASL
+     */
+    @Deprecated
     METASTORE_USE_THRIFT_SASL("hive.metastore.sasl.enabled", false,
         "If true, the metastore Thrift interface will be secured with SASL. Clients must authenticate with Kerberos."),
+    /**
+     * @deprecated Use MetastoreConf.USE_THRIFT_FRAMED_TRANSPORT
+     */
+    @Deprecated
     METASTORE_USE_THRIFT_FRAMED_TRANSPORT("hive.metastore.thrift.framed.transport.enabled", false,
         "If true, the metastore Thrift interface will use TFramedTransport. When false (default) a standard TTransport is used."),
+    /**
+     * @deprecated Use MetastoreConf.USE_THRIFT_COMPACT_PROTOCOL
+     */
+    @Deprecated
     METASTORE_USE_THRIFT_COMPACT_PROTOCOL("hive.metastore.thrift.compact.protocol.enabled", false,
         "If true, the metastore Thrift interface will use TCompactProtocol. When false (default) TBinaryProtocol will be used.\n" +
         "Setting it to true will break compatibility with older clients running TBinaryProtocol."),
+    /**
+     * @deprecated Use MetastoreConf.TOKEN_SIGNATURE
+     */
+    @Deprecated
     METASTORE_TOKEN_SIGNATURE("hive.metastore.token.signature", "",
         "The delegation token service name to match when selecting a token from the current user's tokens."),
+    /**
+     * @deprecated Use MetastoreConf.DELEGATION_TOKEN_STORE_CLS
+     */
+    @Deprecated
     METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS("hive.cluster.delegation.token.store.class",
         "org.apache.hadoop.hive.thrift.MemoryTokenStore",
         "The delegation token store implementation. Set to org.apache.hadoop.hive.thrift.ZooKeeperTokenStore for load-balanced cluster."),
@@ -750,11 +888,23 @@ public class HiveConf extends Configuration {
         "ACL for token store entries. Comma separated list of ACL entries. For example:\n" +
         "sasl:hive/host1@MY.DOMAIN:cdrwa,sasl:hive/host2@MY.DOMAIN:cdrwa\n" +
         "Defaults to all permissions for the hiveserver2/metastore process user."),
+    /**
+     * @deprecated Use MetastoreConf.CACHE_PINOBJTYPES
+     */
+    @Deprecated
     METASTORE_CACHE_PINOBJTYPES("hive.metastore.cache.pinobjtypes", "Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order",
         "List of comma separated metastore object types that should be pinned in the cache"),
+    /**
+     * @deprecated Use MetastoreConf.CONNECTION_POOLING_TYPE
+     */
+    @Deprecated
     METASTORE_CONNECTION_POOLING_TYPE("datanucleus.connectionPoolingType", "HikariCP", new StringSet("BONECP", "DBCP",
       "HikariCP", "NONE"),
         "Specify connection pool library for datanucleus"),
+    /**
+     * @deprecated Use MetastoreConf.CONNECTION_POOLING_MAX_CONNECTIONS
+     */
+    @Deprecated
     METASTORE_CONNECTION_POOLING_MAX_CONNECTIONS("datanucleus.connectionPool.maxPoolSize", 10,
       "Specify the maximum number of connections in the connection pool. Note: The configured size will be used by\n" +
         "2 connection pools (TxnHandler and ObjectStore). When configuring the max connection pool size, it is\n" +
@@ -764,77 +914,179 @@ public class HiveConf extends Configuration {
         "(2 * physical_core_count + hard_disk_count)."),
     // Workaround for DN bug on Postgres:
     // http://www.datanucleus.org/servlet/forum/viewthread_thread,7985_offset
+    /**
+     * @deprecated Use MetastoreConf.DATANUCLEUS_INIT_COL_INFO
+     */
+    @Deprecated
     METASTORE_DATANUCLEUS_INIT_COL_INFO("datanucleus.rdbms.initializeColumnInfo", "NONE",
         "initializeColumnInfo setting for DataNucleus; set to NONE at least on Postgres."),
+    /**
+     * @deprecated Use MetastoreConf.VALIDATE_TABLES
+     */
+    @Deprecated
     METASTORE_VALIDATE_TABLES("datanucleus.schema.validateTables", false,
         "validates existing schema against code. turn this on if you want to verify existing schema"),
+    /**
+     * @deprecated Use MetastoreConf.VALIDATE_COLUMNS
+     */
+    @Deprecated
     METASTORE_VALIDATE_COLUMNS("datanucleus.schema.validateColumns", false,
         "validates existing schema against code. turn this on if you want to verify existing schema"),
+    /**
+     * @deprecated Use MetastoreConf.VALIDATE_CONSTRAINTS
+     */
+    @Deprecated
     METASTORE_VALIDATE_CONSTRAINTS("datanucleus.schema.validateConstraints", false,
         "validates existing schema against code. turn this on if you want to verify existing schema"),
+    /**
+     * @deprecated Use MetastoreConf.STORE_MANAGER_TYPE
+     */
+    @Deprecated
     METASTORE_STORE_MANAGER_TYPE("datanucleus.storeManagerType", "rdbms", "metadata store type"),
+    /**
+     * @deprecated Use MetastoreConf.AUTO_CREATE_ALL
+     */
+    @Deprecated
     METASTORE_AUTO_CREATE_ALL("datanucleus.schema.autoCreateAll", false,
         "Auto creates necessary schema on a startup if one doesn't exist. Set this to false, after creating it once."
         + "To enable auto create also set hive.metastore.schema.verification=false. Auto creation is not "
         + "recommended for production use cases, run schematool command instead." ),
+    /**
+     * @deprecated Use MetastoreConf.SCHEMA_VERIFICATION
+     */
+    @Deprecated
     METASTORE_SCHEMA_VERIFICATION("hive.metastore.schema.verification", true,
         "Enforce metastore schema version consistency.\n" +
         "True: Verify that version information stored in is compatible with one from Hive jars.  Also disable automatic\n" +
         "      schema migration attempt. Users are required to manually migrate schema after Hive upgrade which ensures\n" +
         "      proper metastore schema migration. (Default)\n" +
         "False: Warn if the version information stored in metastore doesn't match with one from in Hive jars."),
+    /**
+     * @deprecated Use MetastoreConf.SCHEMA_VERIFICATION_RECORD_VERSION
+     */
+    @Deprecated
     METASTORE_SCHEMA_VERIFICATION_RECORD_VERSION("hive.metastore.schema.verification.record.version", false,
       "When true the current MS version is recorded in the VERSION table. If this is disabled and verification is\n" +
       " enabled the MS will be unusable."),
+    /**
+     * @deprecated Use MetastoreConf.SCHEMA_INFO_CLASS
+     */
+    @Deprecated
     METASTORE_SCHEMA_INFO_CLASS("hive.metastore.schema.info.class",
         "org.apache.hadoop.hive.metastore.MetaStoreSchemaInfo",
         "Fully qualified class name for the metastore schema information class \n"
         + "which is used by schematool to fetch the schema information.\n"
         + " This class should implement the IMetaStoreSchemaInfo interface"),
+    /**
+     * @deprecated Use MetastoreConf.DATANUCLEUS_TRANSACTION_ISOLATION
+     */
+    @Deprecated
     METASTORE_TRANSACTION_ISOLATION("datanucleus.transactionIsolation", "read-committed",
         "Default transaction isolation level for identity generation."),
+    /**
+     * @deprecated Use MetastoreConf.DATANUCLEUS_CACHE_LEVEL2
+     */
+    @Deprecated
     METASTORE_CACHE_LEVEL2("datanucleus.cache.level2", false,
         "Use a level 2 cache. Turn this off if metadata is changed independently of Hive metastore server"),
     METASTORE_CACHE_LEVEL2_TYPE("datanucleus.cache.level2.type", "none", ""),
+    /**
+     * @deprecated Use MetastoreConf.IDENTIFIER_FACTORY
+     */
+    @Deprecated
     METASTORE_IDENTIFIER_FACTORY("datanucleus.identifierFactory", "datanucleus1",
         "Name of the identifier factory to use when generating table/column names etc. \n" +
         "'datanucleus1' is used for backward compatibility with DataNucleus v1"),
+    /**
+     * @deprecated Use MetastoreConf.DATANUCLEUS_USE_LEGACY_VALUE_STRATEGY
+     */
+    @Deprecated
     METASTORE_USE_LEGACY_VALUE_STRATEGY("datanucleus.rdbms.useLegacyNativeValueStrategy", true, ""),
+    /**
+     * @deprecated Use MetastoreConf.DATANUCLEUS_PLUGIN_REGISTRY_BUNDLE_CHECK
+     */
+    @Deprecated
     METASTORE_PLUGIN_REGISTRY_BUNDLE_CHECK("datanucleus.plugin.pluginRegistryBundleCheck", "LOG",
         "Defines what happens when plugin bundles are found and are duplicated [EXCEPTION|LOG|NONE]"),
+    /**
+     * @deprecated Use MetastoreConf.BATCH_RETRIEVE_MAX
+     */
+    @Deprecated
     METASTORE_BATCH_RETRIEVE_MAX("hive.metastore.batch.retrieve.max", 300,
         "Maximum number of objects (tables/partitions) can be retrieved from metastore in one batch. \n" +
         "The higher the number, the less the number of round trips is needed to the Hive metastore server, \n" +
         "but it may also cause higher memory requirement at the client side."),
+    /**
+     * @deprecated Use MetastoreConf.BATCH_RETRIEVE_OBJECTS_MAX
+     */
+    @Deprecated
     METASTORE_BATCH_RETRIEVE_OBJECTS_MAX(
         "hive.metastore.batch.retrieve.table.partition.max", 1000,
         "Maximum number of objects that metastore internally retrieves in one batch."),
 
+    /**
+     * @deprecated Use MetastoreConf.INIT_HOOKS
+     */
+    @Deprecated
     METASTORE_INIT_HOOKS("hive.metastore.init.hooks", "",
         "A comma separated list of hooks to be invoked at the beginning of HMSHandler initialization. \n" +
         "An init hook is specified as the name of Java class which extends org.apache.hadoop.hive.metastore.MetaStoreInitListener."),
+    /**
+     * @deprecated Use MetastoreConf.PRE_EVENT_LISTENERS
+     */
+    @Deprecated
     METASTORE_PRE_EVENT_LISTENERS("hive.metastore.pre.event.listeners", "",
         "List of comma separated listeners for metastore events."),
+    /**
+     * @deprecated Use MetastoreConf.EVENT_LISTENERS
+     */
+    @Deprecated
     METASTORE_EVENT_LISTENERS("hive.metastore.event.listeners", "",
         "A comma separated list of Java classes that implement the org.apache.hadoop.hive.metastore.MetaStoreEventListener" +
             " interface. The metastore event and corresponding listener method will be invoked in separate JDO transactions. " +
             "Alternatively, configure hive.metastore.transactional.event.listeners to ensure both are invoked in same JDO transaction."),
+    /**
+     * @deprecated Use MetastoreConf.TRANSACTIONAL_EVENT_LISTENERS
+     */
+    @Deprecated
     METASTORE_TRANSACTIONAL_EVENT_LISTENERS("hive.metastore.transactional.event.listeners", "",
         "A comma separated list of Java classes that implement the org.apache.hadoop.hive.metastore.MetaStoreEventListener" +
             " interface. Both the metastore event and corresponding listener method will be invoked in the same JDO transaction."),
+    /**
+     * @deprecated Use MetastoreConf.NOTIFICATION_SEQUENCE_LOCK_MAX_RETRIES
+     */
+    @Deprecated
     NOTIFICATION_SEQUENCE_LOCK_MAX_RETRIES("hive.notification.sequence.lock.max.retries", 5,
         "Number of retries required to acquire a lock when getting the next notification sequential ID for entries "
             + "in the NOTIFICATION_LOG table."),
+    /**
+     * @deprecated Use MetastoreConf.NOTIFICATION_SEQUENCE_LOCK_RETRY_SLEEP_INTERVAL
+     */
+    @Deprecated
     NOTIFICATION_SEQUENCE_LOCK_RETRY_SLEEP_INTERVAL("hive.notification.sequence.lock.retry.sleep.interval", 500L,
         new TimeValidator(TimeUnit.MILLISECONDS),
         "Sleep interval between retries to acquire a notification lock as described part of property "
             + NOTIFICATION_SEQUENCE_LOCK_MAX_RETRIES.name()),
+    /**
+     * @deprecated Use MetastoreConf.EVENT_DB_LISTENER_TTL
+     */
+    @Deprecated
     METASTORE_EVENT_DB_LISTENER_TTL("hive.metastore.event.db.listener.timetolive", "86400s",
         new TimeValidator(TimeUnit.SECONDS),
         "time after which events will be removed from the database listener queue"),
+
+    /**
+     * @deprecated Use MetastoreConf.EVENT_DB_NOTIFICATION_API_AUTH
+     */
+    @Deprecated
     METASTORE_EVENT_DB_NOTIFICATION_API_AUTH("hive.metastore.event.db.notification.api.auth", true,
         "Should metastore do authorization against database notification related APIs such as get_next_notification.\n" +
         "If set to true, then only the superusers in proxy settings have the permission"),
+
+    /**
+     * @deprecated Use MetastoreConf.AUTHORIZATION_STORAGE_AUTH_CHECKS
+     */
+    @Deprecated
     METASTORE_AUTHORIZATION_STORAGE_AUTH_CHECKS("hive.metastore.authorization.storage.checks", false,
         "Should the metastore do authorization checks against the underlying storage (usually hdfs) \n" +
         "for operations like drop-partition (disallow the drop-partition if the user in\n" +
@@ -845,29 +1097,56 @@ public class HiveConf extends Configuration {
         "StorageBasedAuthorization already does this check for managed table. For external table however,\n" +
         "anyone who has read permission of the directory could drop external table, which is surprising.\n" +
         "The flag is set to false by default to maintain backward compatibility."),
+    /**
+     * @deprecated Use MetastoreConf.EVENT_CLEAN_FREQ
+     */
+    @Deprecated
     METASTORE_EVENT_CLEAN_FREQ("hive.metastore.event.clean.freq", "0s",
         new TimeValidator(TimeUnit.SECONDS),
         "Frequency at which timer task runs to purge expired events in metastore."),
+    /**
+     * @deprecated Use MetastoreConf.EVENT_EXPIRY_DURATION
+     */
+    @Deprecated
     METASTORE_EVENT_EXPIRY_DURATION("hive.metastore.event.expiry.duration", "0s",
         new TimeValidator(TimeUnit.SECONDS),
         "Duration after which events expire from events table"),
+    /**
+     * @deprecated Use MetastoreConf.EVENT_MESSAGE_FACTORY
+     */
+    @Deprecated
     METASTORE_EVENT_MESSAGE_FACTORY("hive.metastore.event.message.factory",
         "org.apache.hadoop.hive.metastore.messaging.json.JSONMessageFactory",
         "Factory class for making encoding and decoding messages in the events generated."),
+    /**
+     * @deprecated Use MetastoreConf.EXECUTE_SET_UGI
+     */
+    @Deprecated
     METASTORE_EXECUTE_SET_UGI("hive.metastore.execute.setugi", true,
         "In unsecure mode, setting this property to true will cause the metastore to execute DFS operations using \n" +
         "the client's reported user and group permissions. Note that this property must be set on \n" +
         "both the client and server sides. Further note that its best effort. \n" +
         "If client sets its to true and server sets it to false, client setting will be ignored."),
+    /**
+     * @deprecated Use MetastoreConf.PARTITION_NAME_WHITELIST_PATTERN
+     */
+    @Deprecated
     METASTORE_PARTITION_NAME_WHITELIST_PATTERN("hive.metastore.partition.name.whitelist.pattern", "",
         "Partition names will be checked against this regex pattern and rejected if not matched."),
-
+    /**
+     * @deprecated Use MetastoreConf.INTEGER_JDO_PUSHDOWN
+     */
+    @Deprecated
     METASTORE_INTEGER_JDO_PUSHDOWN("hive.metastore.integral.jdo.pushdown", false,
         "Allow JDO query pushdown for integral partition columns in metastore. Off by default. This\n" +
         "improves metastore perf for integral columns, especially if there's a large number of partitions.\n" +
         "However, it doesn't work correctly with integral values that are not normalized (e.g. have\n" +
         "leading zeroes, like 0012). If metastore direct SQL is enabled and works, this optimization\n" +
         "is also irrelevant."),
+    /**
+     * @deprecated Use MetastoreConf.TRY_DIRECT_SQL
+     */
+    @Deprecated
     METASTORE_TRY_DIRECT_SQL("hive.metastore.try.direct.sql", true,
         "Whether the Hive metastore should try to use direct SQL queries instead of the\n" +
         "DataNucleus for certain read paths. This can improve metastore performance when\n" +
@@ -877,29 +1156,57 @@ public class HiveConf extends Configuration {
         "work for all queries on your datastore. If all SQL queries fail (for example, your\n" +
         "metastore is backed by MongoDB), you might want to disable this to save the\n" +
         "try-and-fall-back cost."),
+    /**
+     * @deprecated Use MetastoreConf.DIRECT_SQL_PARTITION_BATCH_SIZE
+     */
+    @Deprecated
     METASTORE_DIRECT_SQL_PARTITION_BATCH_SIZE("hive.metastore.direct.sql.batch.size", 0,
         "Batch size for partition and other object retrieval from the underlying DB in direct\n" +
         "SQL. For some DBs like Oracle and MSSQL, there are hardcoded or perf-based limitations\n" +
         "that necessitate this. For DBs that can handle the queries, this isn't necessary and\n" +
         "may impede performance. -1 means no batching, 0 means automatic batching."),
+    /**
+     * @deprecated Use MetastoreConf.TRY_DIRECT_SQL_DDL
+     */
+    @Deprecated
     METASTORE_TRY_DIRECT_SQL_DDL("hive.metastore.try.direct.sql.ddl", true,
         "Same as hive.metastore.try.direct.sql, for read statements within a transaction that\n" +
         "modifies metastore data. Due to non-standard behavior in Postgres, if a direct SQL\n" +
         "select query has incorrect syntax or something similar inside a transaction, the\n" +
         "entire transaction will fail and fall-back to DataNucleus will not be possible. You\n" +
         "should disable the usage of direct SQL inside transactions if that happens in your case."),
+    /**
+     * @deprecated Use MetastoreConf.DIRECT_SQL_MAX_QUERY_LENGTH
+     */
+    @Deprecated
     METASTORE_DIRECT_SQL_MAX_QUERY_LENGTH("hive.direct.sql.max.query.length", 100, "The maximum\n" +
         " size of a query string (in KB)."),
+    /**
+     * @deprecated Use MetastoreConf.DIRECT_SQL_MAX_ELEMENTS_IN_CLAUSE
+     */
+    @Deprecated
     METASTORE_DIRECT_SQL_MAX_ELEMENTS_IN_CLAUSE("hive.direct.sql.max.elements.in.clause", 1000,
         "The maximum number of values in a IN clause. Once exceeded, it will be broken into\n" +
         " multiple OR separated IN clauses."),
+    /**
+     * @deprecated Use MetastoreConf.DIRECT_SQL_MAX_ELEMENTS_VALUES_CLAUSE
+     */
+    @Deprecated
     METASTORE_DIRECT_SQL_MAX_ELEMENTS_VALUES_CLAUSE("hive.direct.sql.max.elements.values.clause",
         1000, "The maximum number of values in a VALUES clause for INSERT statement."),
+    /**
+     * @deprecated Use MetastoreConf.ORM_RETRIEVE_MAPNULLS_AS_EMPTY_STRINGS
+     */
+    @Deprecated
     METASTORE_ORM_RETRIEVE_MAPNULLS_AS_EMPTY_STRINGS("hive.metastore.orm.retrieveMapNullsAsEmptyStrings",false,
         "Thrift does not support nulls in maps, so any nulls present in maps retrieved from ORM must " +
         "either be pruned or converted to empty strings. Some backing dbs such as Oracle persist empty strings " +
         "as nulls, so we should set this parameter if we wish to reverse that behaviour. For others, " +
         "pruning is the correct behaviour"),
+    /**
+     * @deprecated Use MetastoreConf.DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES
+     */
+    @Deprecated
     METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES(
         "hive.metastore.disallow.incompatible.col.type.changes", true,
         "If true (default is false), ALTER TABLE operations which change the type of a\n" +
@@ -914,6 +1221,10 @@ public class HiveConf extends Configuration {
         "not blocked.\n" +
         "\n" +
         "See HIVE-4409 for more details."),
+    /**
+     * @deprecated Use MetastoreConf.LIMIT_PARTITION_REQUEST
+     */
+    @Deprecated
     METASTORE_LIMIT_PARTITION_REQUEST("hive.metastore.limit.partition.request", -1,
         "This limits the number of partitions that can be requested from the metastore for a given table.\n" +
             "The default value \"-1\" means no limit."),
@@ -922,48 +1233,75 @@ public class HiveConf extends Configuration {
         "Default property values for newly created tables"),
     DDL_CTL_PARAMETERS_WHITELIST("hive.ddl.createtablelike.properties.whitelist", "",
         "Table Properties to copy over when executing a Create Table Like."),
+    /**
+     * @deprecated Use MetastoreConf.RAW_STORE_IMPL
+     */
+    @Deprecated
     METASTORE_RAW_STORE_IMPL("hive.metastore.rawstore.impl", "org.apache.hadoop.hive.metastore.ObjectStore",
         "Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. \n" +
         "This class is used to store and retrieval of raw metadata objects such as table, database"),
-    METASTORE_CACHED_RAW_STORE_IMPL("hive.metastore.cached.rawstore.impl", "org.apache.hadoop.hive.metastore.ObjectStore",
-        "Name of the wrapped RawStore class"),
-    METASTORE_CACHED_RAW_STORE_CACHE_UPDATE_FREQUENCY(
-        "hive.metastore.cached.rawstore.cache.update.frequency", "60", new TimeValidator(
-            TimeUnit.SECONDS),
-        "The time after which metastore cache is updated from metastore DB."),
-    METASTORE_CACHED_RAW_STORE_CACHED_OBJECTS_WHITELIST(
-        "hive.metastore.cached.rawstore.cached.object.whitelist", ".*", "Comma separated list of regular expressions \n " +
-        "to select the tables (and its partitions, stats etc) that will be cached by CachedStore. \n" +
-        "This can be used in conjunction with hive.metastore.cached.rawstore.cached.object.blacklist. \n" +
-        "Example: .*, db1.*, db2\\.tbl.*. The last item can potentially override patterns specified before."),
-    METASTORE_CACHED_RAW_STORE_CACHED_OBJECTS_BLACKLIST(
-         "hive.metastore.cached.rawstore.cached.object.blacklist", "", "Comma separated list of regular expressions \n " +
-         "to filter out the tables (and its partitions, stats etc) that will be cached by CachedStore. \n" +
-         "This can be used in conjunction with hive.metastore.cached.rawstore.cached.object.whitelist. \n" +
-         "Example: db2.*, db3\\.tbl1, db3\\..*. The last item can potentially override patterns specified before. \n" +
-         "The blacklist also overrides the whitelist."),
+    /**
+     * @deprecated Use MetastoreConf.TXN_STORE_IMPL
+     */
+    @Deprecated
     METASTORE_TXN_STORE_IMPL("hive.metastore.txn.store.impl",
         "org.apache.hadoop.hive.metastore.txn.CompactionTxnHandler",
         "Name of class that implements org.apache.hadoop.hive.metastore.txn.TxnStore.  This " +
         "class is used to store and retrieve transactions and locks"),
+    /**
+     * @deprecated Use MetastoreConf.CONNECTION_DRIVER
+     */
+    @Deprecated
     METASTORE_CONNECTION_DRIVER("javax.jdo.option.ConnectionDriverName", "org.apache.derby.jdbc.EmbeddedDriver",
         "Driver class name for a JDBC metastore"),
+    /**
+     * @deprecated Use MetastoreConf.MANAGER_FACTORY_CLASS
+     */
+    @Deprecated
     METASTORE_MANAGER_FACTORY_CLASS("javax.jdo.PersistenceManagerFactoryClass",
         "org.datanucleus.api.jdo.JDOPersistenceManagerFactory",
         "class implementing the jdo persistence"),
+    /**
+     * @deprecated Use MetastoreConf.EXPRESSION_PROXY_CLASS
+     */
+    @Deprecated
     METASTORE_EXPRESSION_PROXY_CLASS("hive.metastore.expression.proxy",
         "org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore", ""),
+    /**
+     * @deprecated Use MetastoreConf.DETACH_ALL_ON_COMMIT
+     */
+    @Deprecated
     METASTORE_DETACH_ALL_ON_COMMIT("javax.jdo.option.DetachAllOnCommit", true,
         "Detaches all objects from session so that they can be used after transaction is committed"),
+    /**
+     * @deprecated Use MetastoreConf.NON_TRANSACTIONAL_READ
+     */
+    @Deprecated
     METASTORE_NON_TRANSACTIONAL_READ("javax.jdo.option.NonTransactionalRead", true,
         "Reads outside of transactions"),
+    /**
+     * @deprecated Use MetastoreConf.CONNECTION_USER_NAME
+     */
+    @Deprecated
     METASTORE_CONNECTION_USER_NAME("javax.jdo.option.ConnectionUserName", "APP",
         "Username to use against metastore database"),
+    /**
+     * @deprecated Use MetastoreConf.END_FUNCTION_LISTENERS
+     */
+    @Deprecated
     METASTORE_END_FUNCTION_LISTENERS("hive.metastore.end.function.listeners", "",
         "List of comma separated listeners for the end of metastore functions."),
+    /**
+     * @deprecated Use MetastoreConf.PART_INHERIT_TBL_PROPS
+     */
+    @Deprecated
     METASTORE_PART_INHERIT_TBL_PROPS("hive.metastore.partition.inherit.table.properties", "",
         "List of comma separated keys occurring in table properties which will get inherited to newly created partitions. \n" +
         "* implies all the keys will get inherited."),
+    /**
+     * @deprecated Use MetastoreConf.FILTER_HOOK
+     */
+    @Deprecated
     METASTORE_FILTER_HOOK("hive.metastore.filter.hook", "org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl",
         "Metastore hook class for filtering the metadata read results. If hive.security.authorization.manager"
         + "is set to instance of HiveAuthorizerFactory, then this value is ignored."),
@@ -973,50 +1311,126 @@ public class HiveConf extends Configuration {
         "Choose whether dropping partitions with HCatClient pushes the partition-predicate to the metastore, " +
             "or drops partitions iteratively"),
 
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_ENABLED
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_ENABLED("hive.metastore.aggregate.stats.cache.enabled", true,
         "Whether aggregate stats caching is enabled or not."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_SIZE
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_SIZE("hive.metastore.aggregate.stats.cache.size", 10000,
         "Maximum number of aggregate stats nodes that we will place in the metastore aggregate stats cache."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_MAX_PARTITIONS
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_MAX_PARTITIONS("hive.metastore.aggregate.stats.cache.max.partitions", 10000,
         "Maximum number of partitions that are aggregated per cache node."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_FPP
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_FPP("hive.metastore.aggregate.stats.cache.fpp", (float) 0.01,
         "Maximum false positive probability for the Bloom Filter used in each aggregate stats cache node (default 1%)."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_MAX_VARIANCE
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_MAX_VARIANCE("hive.metastore.aggregate.stats.cache.max.variance", (float) 0.01,
         "Maximum tolerable variance in number of partitions between a cached node and our request (default 1%)."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_TTL
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_TTL("hive.metastore.aggregate.stats.cache.ttl", "600s", new TimeValidator(TimeUnit.SECONDS),
         "Number of seconds for a cached node to be active in the cache before they become stale."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT("hive.metastore.aggregate.stats.cache.max.writer.wait", "5000ms",
         new TimeValidator(TimeUnit.MILLISECONDS),
         "Number of milliseconds a writer will wait to acquire the writelock before giving up."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_MAX_READER_WAIT
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT("hive.metastore.aggregate.stats.cache.max.reader.wait", "1000ms",
         new TimeValidator(TimeUnit.MILLISECONDS),
         "Number of milliseconds a reader will wait to acquire the readlock before giving up."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_MAX_FULL
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_MAX_FULL("hive.metastore.aggregate.stats.cache.max.full", (float) 0.9,
         "Maximum cache full % after which the cache cleaner thread kicks in."),
+    /**
+     * @deprecated Use MetastoreConf.AGGREGATE_STATS_CACHE_CLEAN_UNTIL
+     */
+    @Deprecated
     METASTORE_AGGREGATE_STATS_CACHE_CLEAN_UNTIL("hive.metastore.aggregate.stats.cache.clean.until", (float) 0.8,
         "The cleaner thread cleans until cache reaches this % full size."),
+    /**
+     * @deprecated Use MetastoreConf.METRICS_ENABLED
+     */
+    @Deprecated
     METASTORE_METRICS("hive.metastore.metrics.enabled", false, "Enable metrics on the metastore."),
+    /**
+     * @deprecated Use MetastoreConf.INIT_METADATA_COUNT_ENABLED
+     */
+    @Deprecated
     METASTORE_INIT_METADATA_COUNT_ENABLED("hive.metastore.initial.metadata.count.enabled", true,
       "Enable a metadata count at metastore startup for metrics."),
 
     // Metastore SSL settings
+    /**
+     * @deprecated Use MetastoreConf.USE_SSL
+     */
+    @Deprecated
     HIVE_METASTORE_USE_SSL("hive.metastore.use.SSL", false,
         "Set this to true for using SSL encryption in HMS server."),
+    /**
+     * @deprecated Use MetastoreConf.SSL_KEYSTORE_PATH
+     */
+    @Deprecated
     HIVE_METASTORE_SSL_KEYSTORE_PATH("hive.metastore.keystore.path", "",
         "Metastore SSL certificate keystore location."),
+    /**
+     * @deprecated Use MetastoreConf.SSL_KEYSTORE_PASSWORD
+     */
+    @Deprecated
     HIVE_METASTORE_SSL_KEYSTORE_PASSWORD("hive.metastore.keystore.password", "",
         "Metastore SSL certificate keystore password."),
+    /**
+     * @deprecated Use MetastoreConf.SSL_TRUSTSTORE_PATH
+     */
+    @Deprecated
     HIVE_METASTORE_SSL_TRUSTSTORE_PATH("hive.metastore.truststore.path", "",
         "Metastore SSL certificate truststore location."),
+    /**
+     * @deprecated Use MetastoreConf.SSL_TRUSTSTORE_PASSWORD
+     */
+    @Deprecated
     HIVE_METASTORE_SSL_TRUSTSTORE_PASSWORD("hive.metastore.truststore.password", "",
         "Metastore SSL certificate truststore password."),
 
     // Parameters for exporting metadata on table drop (requires the use of the)
     // org.apache.hadoop.hive.ql.parse.MetaDataExportListener preevent listener
+    /**
+     * @deprecated Use MetastoreConf.METADATA_EXPORT_LOCATION
+     */
+    @Deprecated
     METADATA_EXPORT_LOCATION("hive.metadata.export.location", "",
         "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" +
         "it is the location to which the metadata will be exported. The default is an empty string, which results in the \n" +
         "metadata being exported to the current user's home directory on HDFS."),
+    /**
+     * @deprecated Use MetastoreConf.MOVE_EXPORTED_METADATA_TO_TRASH
+     */
+    @Deprecated
     MOVE_EXPORTED_METADATA_TO_TRASH("hive.metadata.move.exported.metadata.to.trash", true,
         "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" +
         "this setting determines if the metadata that is exported will subsequently be moved to the user's trash directory \n" +
@@ -1033,6 +1447,10 @@ public class HiveConf extends Configuration {
         "The number of columns to use when formatting output generated by the DESCRIBE PRETTY table_name command.\n" +
         "If the value of this property is -1, then Hive will use the auto-detected terminal width."),
 
+    /**
+     * @deprecated Use MetastoreConf.FS_HANDLER_CLS
+     */
+    @Deprecated
     HIVE_METASTORE_FS_HANDLER_CLS("hive.metastore.fs.handler.class", "org.apache.hadoop.hive.metastore.HiveMetaStoreFsImpl", ""),
 
     // Things we log in the jobconf
@@ -1294,6 +1712,10 @@ public class HiveConf extends Configuration {
         "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
         "The default SerDe Hive will use for storage formats that do not specify a SerDe."),
 
+    /**
+     * @deprecated Use MetastoreConf.SERDES_USING_METASTORE_FOR_SCHEMA
+     */
+    @Deprecated
     SERDESUSINGMETASTOREFORSCHEMA("hive.serdes.using.metastore.for.schema",
         "org.apache.hadoop.hive.ql.io.orc.OrcSerde," +
         "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe," +
@@ -1790,8 +2212,16 @@ public class HiveConf extends Configuration {
         "The storage that stores temporary Hive statistics. In filesystem based statistics collection ('fs'), \n" +
         "each task writes statistics it has collected in a file on the filesystem, which will be aggregated \n" +
         "after the job has finished. Supported values are fs (filesystem) and custom as defined in StatsSetupConst.java."), // StatsSetupConst.StatDB
+    /**
+     * @deprecated Use MetastoreConf.STATS_DEFAULT_PUBLISHER
+     */
+    @Deprecated
     HIVE_STATS_DEFAULT_PUBLISHER("hive.stats.default.publisher", "",
         "The Java class (implementing the StatsPublisher interface) that is used by default if hive.stats.dbclass is custom type."),
+    /**
+     * @deprecated Use MetastoreConf.STATS_DEFAULT_AGGRETATOR
+     */
+    @Deprecated
     HIVE_STATS_DEFAULT_AGGREGATOR("hive.stats.default.aggregator", "",
         "The Java class (implementing the StatsAggregator interface) that is used by default if hive.stats.dbclass is custom type."),
     CLIENT_STATS_COUNTERS("hive.client.stats.counters", "",
@@ -1818,6 +2248,10 @@ public class HiveConf extends Configuration {
         "This is useful to identify how tables are accessed and to determine if there are wasted columns that can be trimmed."),
     HIVE_STATS_NDV_ALGO("hive.stats.ndv.algo", "hll", new PatternSet("hll", "fm"),
         "hll and fm stand for HyperLogLog and FM-sketch, respectively for computing ndv."),
+    /**
+     * @deprecated Use MetastoreConf.STATS_FETCH_BITVECTOR
+     */
+    @Deprecated
     HIVE_STATS_FETCH_BITVECTOR("hive.stats.fetch.bitvector", false,
         "Whether we fetch bitvector when we compute ndv. Users can turn it off if they want to use old schema"),
     // standard error allowed for ndv estimates for FM-sketch. A lower value indicates higher accuracy and a
@@ -1825,10 +2259,18 @@ public class HiveConf extends Configuration {
     HIVE_STATS_NDV_ERROR("hive.stats.ndv.error", (float)20.0,
         "Standard error expressed in percentage. Provides a tradeoff between accuracy and compute cost. \n" +
         "A lower value for error indicates higher accuracy and a higher compute cost."),
+    /**
+     * @deprecated Use MetastoreConf.STATS_NDV_TUNER
+     */
+    @Deprecated
     HIVE_METASTORE_STATS_NDV_TUNER("hive.metastore.stats.ndv.tuner", (float)0.0,
          "Provides a tunable parameter between the lower bound and the higher bound of ndv for aggregate ndv across all the partitions. \n" +
          "The lower bound is equal to the maximum of ndv of all the partitions. The higher bound is equal to the sum of ndv of all the partitions.\n" +
          "Its value should be between 0.0 (i.e., choose lower bound) and 1.0 (i.e., choose higher bound)"),
+    /**
+     * @deprecated Use MetastoreConf.STATS_NDV_DENSITY_FUNCTION
+     */
+    @Deprecated
     HIVE_METASTORE_STATS_NDV_DENSITY_FUNCTION("hive.metastore.stats.ndv.densityfunction", false,
         "Whether to use density function to estimate the NDV for the whole table based on the NDV of partitions"),
     HIVE_STATS_KEY_PREFIX("hive.stats.key.prefix", "", "", true), // internal usage only
@@ -1953,8 +2395,16 @@ public class HiveConf extends Configuration {
         "In nonstrict mode, for non-ACID resources, INSERT will only acquire shared lock, which\n" +
         "allows two concurrent writes to the same partition but still lets lock manager prevent\n" +
         "DROP TABLE etc. when the table is being written to"),
+    /**
+     * @deprecated Use MetastoreConf.TXN_TIMEOUT
+     */
+    @Deprecated
     HIVE_TXN_TIMEOUT("hive.txn.timeout", "300s", new TimeValidator(TimeUnit.SECONDS),
         "time after which transactions are declared aborted if the client has not sent a heartbeat."),
+    /**
+     * @deprecated Use MetastoreConf.TXN_HEARTBEAT_THREADPOOL_SIZE
+     */
+    @Deprecated
     HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE("hive.txn.heartbeat.threadpool.size", 5, "The number of " +
         "threads to use for heartbeating. For Hive CLI, 1 is enough. For HiveServer2, we need a few"),
     TXN_MGR_DUMP_LOCK_STATE_ON_ACQUIRE_TIMEOUT("hive.txn.manager.dump.lock.state.on.acquire.timeout", false,
@@ -1967,13 +2417,23 @@ public class HiveConf extends Configuration {
       "4: Make the table 'quarter-acid' as it only supports insert. But it doesn't require ORC or bucketing.\n" +
       "This is intended to be used as an internal property for future versions of ACID. (See\n" +
         "HIVE-14035 for details.  User sets it tblproperites via transactional_properties.)", true),
-
+    /**
+     * @deprecated Use MetastoreConf.MAX_OPEN_TXNS
+     */
+    @Deprecated
     HIVE_MAX_OPEN_TXNS("hive.max.open.txns", 100000, "Maximum number of open transactions. If \n" +
         "current open transactions reach this limit, future open transaction requests will be \n" +
         "rejected, until this number goes below the limit."),
+    /**
+     * @deprecated Use MetastoreConf.COUNT_OPEN_TXNS_INTERVAL
+     */
+    @Deprecated
     HIVE_COUNT_OPEN_TXNS_INTERVAL("hive.count.open.txns.interval", "1s",
         new TimeValidator(TimeUnit.SECONDS), "Time in seconds between checks to count open transactions."),
-
+    /**
+     * @deprecated Use MetastoreConf.TXN_MAX_OPEN_BATCH
+     */
+    @Deprecated
     HIVE_TXN_MAX_OPEN_BATCH("hive.txn.max.open.batch", 1000,
         "Maximum number of transactions that can be fetched in one call to open_txns().\n" +
         "This controls how many transactions streaming agents such as Flume or Storm open\n" +
@@ -1982,20 +2442,29 @@ public class HiveConf extends Configuration {
         "of delta files created by streaming agents. But it also increases the number of open\n" +
         "transactions that Hive has to track at any given time, which may negatively affect\n" +
         "read performance."),
-
+    /**
+     * @deprecated Use MetastoreConf.TXN_RETRYABLE_SQLEX_REGEX
+     */
+    @Deprecated
     HIVE_TXN_RETRYABLE_SQLEX_REGEX("hive.txn.retryable.sqlex.regex", "", "Comma separated list\n" +
         "of regular expression patterns for SQL state, error code, and error message of\n" +
         "retryable SQLExceptions, that's suitable for the metastore DB.\n" +
         "For example: Can't serialize.*,40001$,^Deadlock,.*ORA-08176.*\n" +
         "The string that the regex will be matched against is of the following form, where ex is a SQLException:\n" +
         "ex.getMessage() + \" (SQLState=\" + ex.getSQLState() + \", ErrorCode=\" + ex.getErrorCode() + \")\""),
-
+    /**
+     * @deprecated Use MetastoreConf.COMPACTOR_INITIATOR_ON
+     */
+    @Deprecated
     HIVE_COMPACTOR_INITIATOR_ON("hive.compactor.initiator.on", false,
         "Whether to run the initiator and cleaner threads on this metastore instance or not.\n" +
         "Set this to true on one instance of the Thrift metastore service as part of turning\n" +
         "on Hive transactions. For a complete list of parameters required for turning on\n" +
         "transactions, see hive.txn.manager."),
-
+    /**
+     * @deprecated Use MetastoreConf.COMPACTOR_WORKER_THREADS
+     */
+    @Deprecated
     HIVE_COMPACTOR_WORKER_THREADS("hive.compactor.worker.threads", 0,
         "How many compactor worker threads to run on this metastore instance. Set this to a\n" +
         "positive number on one or more instances of the Thrift metastore service as part of\n" +
@@ -2036,7 +2505,10 @@ public class HiveConf extends Configuration {
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
         "a major compaction."),
-
+    /**
+     * @deprecated Use MetastoreConf.COMPACTOR_INITIATOR_FAILED_THRESHOLD
+     */
+    @Deprecated
     COMPACTOR_INITIATOR_FAILED_THRESHOLD("hive.compactor.initiator.failed.compacts.threshold", 2,
       new RangeValidator(1, 20), "Number of consecutive compaction failures (per table/partition) " +
       "after which automatic compactions will not be scheduled any more.  Note that this must be less " +
@@ -2046,26 +2518,49 @@ public class HiveConf extends Configuration {
         new TimeValidator(TimeUnit.MILLISECONDS), "Time between runs of the cleaner thread"),
     COMPACTOR_JOB_QUEUE("hive.compactor.job.queue", "", "Used to specify name of Hadoop queue to which\n" +
       "Compaction jobs will be submitted.  Set to empty string to let Hadoop choose the queue."),
-
+    /**
+     * @deprecated Use MetastoreConf.COMPACTOR_HISTORY_RETENTION_SUCCEEDED
+     */
+    @Deprecated
     COMPACTOR_HISTORY_RETENTION_SUCCEEDED("hive.compactor.history.retention.succeeded", 3,
       new RangeValidator(0, 100), "Determines how many successful compaction records will be " +
       "retained in compaction history for a given table/partition."),
-
+    /**
+     * @deprecated Use MetastoreConf.COMPACTOR_HISTORY_RETENTION_FAILED
+     */
+    @Deprecated
     COMPACTOR_HISTORY_RETENTION_FAILED("hive.compactor.history.retention.failed", 3,
       new RangeValidator(0, 100), "Determines how many failed compaction records will be " +
       "retained in compaction history for a given table/partition."),
-
+    /**
+     * @deprecated Use MetastoreConf.COMPACTOR_HISTORY_RETENTION_ATTEMPTED
+     */
+    @Deprecated
     COMPACTOR_HISTORY_RETENTION_ATTEMPTED("hive.compactor.history.retention.attempted", 2,
       new RangeValidator(0, 100), "Determines how many attempted compaction records will be " +
       "retained in compaction history for a given table/partition."),
-
+    /**
+     * @deprecated Use MetastoreConf.COMPACTOR_HISTORY_REAPER_INTERVAL
+     */
+    @Deprecated
     COMPACTOR_HISTORY_REAPER_INTERVAL("hive.compactor.history.reaper.interval", "2m",
       new TimeValidator(TimeUnit.MILLISECONDS), "Determines how often compaction history reaper runs"),
-
+    /**
+     * @deprecated Use MetastoreConf.TIMEDOUT_TXN_REAPER_START
+     */
+    @Deprecated
     HIVE_TIMEDOUT_TXN_REAPER_START("hive.timedout.txn.reaper.start", "100s",
       new TimeValidator(TimeUnit.MILLISECONDS), "Time delay of 1st reaper run after metastore start"),
+    /**
+     * @deprecated Use MetastoreConf.TIMEDOUT_TXN_REAPER_INTERVAL
+     */
+    @Deprecated
     HIVE_TIMEDOUT_TXN_REAPER_INTERVAL("hive.timedout.txn.reaper.interval", "180s",
       new TimeValidator(TimeUnit.MILLISECONDS), "Time interval describing how often the reaper runs"),
+    /**
+     * @deprecated Use MetastoreConf.WRITE_SET_REAPER_INTERVAL
+     */
+    @Deprecated
     WRITE_SET_REAPER_INTERVAL("hive.writeset.reaper.interval", "60s",
       new TimeValidator(TimeUnit.MILLISECONDS), "Frequency of WriteSet reaper runs"),
 
@@ -3054,6 +3549,10 @@ public class HiveConf extends Configuration {
         "  none: default(past) behavior. Implies only alphaNumeric and underscore are valid characters in identifiers.\n" +
         "  column: implies column names can contain any character."
     ),
+    /**
+     * @deprecated Use MetastoreConf.SUPPORT_SPECIAL_CHARACTERS_IN_TABLE_NAMES
+     */
+    @Deprecated
     HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES("hive.support.special.characters.tablename", true,
         "This flag should be set to true to enable support for special characters in table names.\n"
         + "When it is set to false, only [a-zA-Z_0-9]+ are supported.\n"
@@ -4517,6 +5016,14 @@ public class HiveConf extends Configuration {
     // if embedded metastore is to be used as per config so far
     // then this is considered like the metastore server case
     String msUri = this.getVar(HiveConf.ConfVars.METASTOREURIS);
+    // This is hackery, but having hive-common depend on standalone-metastore is really bad
+    // because it will pull all of the metastore code into every module.  We need to check that
+    // we aren't using the standalone metastore.  If we are, we should treat it the same as a
+    // remote metastore situation.
+    if (msUri == null || msUri.isEmpty()) {
+      msUri = this.get("metastore.thrift.uris");
+    }
+    LOG.debug("Found metastore URI of " + msUri);
     if(HiveConfUtil.isEmbeddedMetaStore(msUri)){
       setLoadMetastoreConfig(true);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 7f1b662..22d5876 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -3492,13 +3492,13 @@ public class TestReplicationScenarios {
     }
     assertNotNull(ex);
     // Disable auth so the call should succeed
-    hconf.setBoolVar(HiveConf.ConfVars.METASTORE_EVENT_DB_NOTIFICATION_API_AUTH, false);
+    MetastoreConf.setBoolVar(hconf, MetastoreConf.ConfVars.EVENT_DB_NOTIFICATION_API_AUTH, false);
     try {
       rsp = metaStoreClient.getNextNotification(firstEventId, 0, null);
       assertEquals(1, rsp.getEventsSize());
     } finally {
       // Restore the settings
-      hconf.setBoolVar(HiveConf.ConfVars.METASTORE_EVENT_DB_NOTIFICATION_API_AUTH, true);
+      MetastoreConf.setBoolVar(hconf, MetastoreConf.ConfVars.EVENT_DB_NOTIFICATION_API_AUTH, true);
       hconf.set(proxySettingName, "*");
       ProxyUsers.refreshSuperUserGroupsConfiguration(hconf);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 27e42b1..6003ced 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hive.conf.HiveConfUtil;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.cache.CachedStore;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.MapRedStats;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
 import org.apache.hadoop.hive.ql.exec.Registry;
@@ -1746,14 +1747,16 @@ public class SessionState {
 
   private void unCacheDataNucleusClassLoaders() {
     try {
-      boolean isLocalMetastore =
-          HiveConfUtil.isEmbeddedMetaStore(sessionConf.getVar(HiveConf.ConfVars.METASTOREURIS));
+      boolean isLocalMetastore = HiveConfUtil.isEmbeddedMetaStore(
+          MetastoreConf.getVar(sessionConf, MetastoreConf.ConfVars.THRIFT_URIS));
       if (isLocalMetastore) {
 
-        String rawStoreImpl = sessionConf.getVar(ConfVars.METASTORE_RAW_STORE_IMPL);
+        String rawStoreImpl =
+            MetastoreConf.getVar(sessionConf, MetastoreConf.ConfVars.RAW_STORE_IMPL);
         String realStoreImpl;
         if (rawStoreImpl.equals(CachedStore.class.getName())) {
-          realStoreImpl = sessionConf.getVar(ConfVars.METASTORE_CACHED_RAW_STORE_IMPL);
+          realStoreImpl =
+              MetastoreConf.getVar(sessionConf, MetastoreConf.ConfVars.CACHED_RAW_STORE_IMPL);
         } else {
           realStoreImpl = rawStoreImpl;
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInit.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInit.java
index c880a9a..9417cf5 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInit.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInit.java
@@ -75,23 +75,23 @@ public class MetaStoreInit {
     if (connectUrl != null && !connectUrl.equals(currentUrl)) {
       LOG.error(
           String.format("Overriding %s with %s",
-              MetastoreConf.ConfVars.CONNECTURLKEY.toString(),
+              MetastoreConf.ConfVars.CONNECT_URL_KEY.toString(),
               connectUrl));
-      MetastoreConf.setVar(activeConf, ConfVars.CONNECTURLKEY, connectUrl);
+      MetastoreConf.setVar(activeConf, ConfVars.CONNECT_URL_KEY, connectUrl);
       return true;
     }
     return false;
   }
 
   static String getConnectionURL(Configuration conf) {
-    return MetastoreConf.getVar(conf, ConfVars.CONNECTURLKEY, "");
+    return MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_KEY, "");
   }
 
   // Multiple threads could try to initialize at the same time.
   synchronized private static void initConnectionUrlHook(Configuration conf,
     MetaStoreInitData updateData) throws ClassNotFoundException {
 
-    String className = MetastoreConf.getVar(conf, ConfVars.CONNECTURLHOOK, "").trim();
+    String className = MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_HOOK, "").trim();
     if (className.equals("")) {
       updateData.urlHookClassName = "";
       updateData.urlHook = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index dec2163..2056930 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -405,9 +405,9 @@ public class ObjectStore implements RawStore, Configurable {
 
   @SuppressWarnings("nls")
   private void initialize(Properties dsProps) {
-    int retryLimit = MetastoreConf.getIntVar(conf, ConfVars.HMSHANDLERATTEMPTS);
+    int retryLimit = MetastoreConf.getIntVar(conf, ConfVars.HMS_HANDLER_ATTEMPTS);
     long retryInterval = MetastoreConf.getTimeVar(conf,
-        ConfVars.HMSHANDLERINTERVAL, TimeUnit.MILLISECONDS);
+        ConfVars.HMS_HANDLER_INTERVAL, TimeUnit.MILLISECONDS);
     int numTries = retryLimit;
 
     while (numTries > 0){

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index af545d1..f6c46ee 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -1156,8 +1156,6 @@ public interface RawStore extends Configurable {
     List<String> partNames, List<String> colNames) throws MetaException, NoSuchObjectException;
 
   /**
-<<<<<<< HEAD
-=======
    * Get column stats for all partitions of all tables in the database
    * @param catName catalog name
    * @param dbName database name
@@ -1169,7 +1167,6 @@ public interface RawStore extends Configurable {
       throws MetaException, NoSuchObjectException;
 
   /**
->>>>>>> e6d9605492... HIVE-18755 Modifications to the metastore for catalogs
    * Get the next notification event.
    * @param rqst Request containing information on the last processed notification.
    * @return list of notifications, sorted by eventId

http://git-wip-us.apache.org/repos/asf/hive/blob/105cc654/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
index 559ff34..a830eb9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
@@ -120,10 +120,10 @@ public class RetryingHMSHandler implements InvocationHandler {
   public Result invokeInternal(final Object proxy, final Method method, final Object[] args) throws Throwable {
 
     boolean gotNewConnectUrl = false;
-    boolean reloadConf = MetastoreConf.getBoolVar(origConf, ConfVars.HMSHANDLERFORCERELOADCONF);
+    boolean reloadConf = MetastoreConf.getBoolVar(origConf, ConfVars.HMS_HANDLER_FORCE_RELOAD_CONF);
     long retryInterval = MetastoreConf.getTimeVar(origConf,
-        ConfVars.HMSHANDLERINTERVAL, TimeUnit.MILLISECONDS);
-    int retryLimit = MetastoreConf.getIntVar(origConf, ConfVars.HMSHANDLERATTEMPTS);
+        ConfVars.HMS_HANDLER_INTERVAL, TimeUnit.MILLISECONDS);
+    int retryLimit = MetastoreConf.getIntVar(origConf, ConfVars.HMS_HANDLER_ATTEMPTS);
     long timeout = MetastoreConf.getTimeVar(origConf,
         ConfVars.CLIENT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS);