You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vi...@apache.org on 2018/08/14 18:00:07 UTC

hive git commit: HIVE-20196 : Remove MetastoreConf dependency on server-specific classes (Alexander Kolbasov reviewed by Alan Gates and Vihang Karajgaonkar)

Repository: hive
Updated Branches:
  refs/heads/master 0772f186f -> faf29143d


HIVE-20196 : Remove MetastoreConf dependency on server-specific classes (Alexander Kolbasov reviewed by Alan Gates and Vihang Karajgaonkar)


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

Branch: refs/heads/master
Commit: faf29143dd94effc6a7192f8a04a79d38d6382f7
Parents: 0772f18
Author: Alexander Kolbasov <ak...@cloudera.com>
Authored: Tue Aug 14 10:48:36 2018 -0700
Committer: Vihang Karajgaonkar <vi...@apache.org>
Committed: Tue Aug 14 10:48:36 2018 -0700

----------------------------------------------------------------------
 .../hive/metastore/conf/MetastoreConf.java      | 67 +++++++++++++-------
 .../hive/metastore/conf/TestMetastoreConf.java  | 44 +++++++++++++
 2 files changed, 89 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/faf29143/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index cf5fbbe..30ea7f8 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -19,17 +19,6 @@ package org.apache.hadoop.hive.metastore.conf;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.DefaultStorageSchemaReader;
-import org.apache.hadoop.hive.metastore.HiveAlterHandler;
-import org.apache.hadoop.hive.metastore.MaterializationsRebuildLockCleanerTask;
-import org.apache.hadoop.hive.metastore.MetastoreTaskThread;
-import org.apache.hadoop.hive.metastore.RuntimeStatsCleanerTask;
-import org.apache.hadoop.hive.metastore.events.EventCleanerTask;
-import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager;
-import org.apache.hadoop.hive.metastore.txn.AcidCompactionHistoryService;
-import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
-import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService;
-import org.apache.hadoop.hive.metastore.txn.AcidWriteSetService;
 import org.apache.hadoop.hive.metastore.utils.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,6 +59,40 @@ public class MetastoreConf {
   private static final Logger LOG = LoggerFactory.getLogger(MetastoreConf.class);
   private static final Pattern TIME_UNIT_SUFFIX = Pattern.compile("([0-9]+)([a-zA-Z]+)");
 
+  @VisibleForTesting
+  static final String DEFAULT_STORAGE_SCHEMA_READER_CLASS =
+      "org.apache.hadoop.hive.metastore.DefaultStorageSchemaReader";
+  @VisibleForTesting
+  static final String HIVE_ALTER_HANDLE_CLASS =
+      "org.apache.hadoop.hive.metastore.HiveAlterHandler";
+  @VisibleForTesting
+  static final String MATERIALZIATIONS_REBUILD_LOCK_CLEANER_TASK_CLASS =
+      "org.apache.hadoop.hive.metastore.MaterializationsRebuildLockCleanerTask";
+  @VisibleForTesting
+  static final String METASTORE_TASK_THREAD_CLASS =
+      "org.apache.hadoop.hive.metastore.MetastoreTaskThread";
+  @VisibleForTesting
+  static final String RUNTIME_STATS_CLEANER_TASK_CLASS =
+      "org.apache.hadoop.hive.metastore.RuntimeStatsCleanerTask";
+  @VisibleForTesting
+  static final String EVENT_CLEANER_TASK_CLASS =
+      "org.apache.hadoop.hive.metastore.events.EventCleanerTask";
+  @VisibleForTesting
+  static final String METASTORE_DELEGATION_MANAGER_CLASS =
+      "org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager";
+  @VisibleForTesting
+  static final String ACID_COMPACTION_HISTORY_SERVICE_CLASS =
+      "org.apache.hadoop.hive.metastore.txn.AcidCompactionHistoryService";
+  @VisibleForTesting
+  static final String ACID_HOUSE_KEEPER_SERVICE_CLASS =
+      "org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService";
+  @VisibleForTesting
+  static final String ACID_OPEN_TXNS_COUNTER_SERVICE_CLASS =
+      "org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService";
+  @VisibleForTesting
+  static final String ACID_WRITE_SET_SERVICE_CLASS =
+      "org.apache.hadoop.hive.metastore.txn.AcidWriteSetService";
+
   private static final Map<String, ConfVars> metaConfs = new HashMap<>();
   private static URL hiveDefaultURL = null;
   private static URL hiveSiteURL = null;
@@ -260,7 +283,7 @@ public class MetastoreConf {
         "hive.metastore.aggregate.stats.cache.ttl", 600, TimeUnit.SECONDS,
         "Number of seconds for a cached node to be active in the cache before they become stale."),
     ALTER_HANDLER("metastore.alter.handler", "hive.metastore.alter.impl",
-        HiveAlterHandler.class.getName(),
+        HIVE_ALTER_HANDLE_CLASS,
         "Alter handler.  For now defaults to the Hive one.  Really need a better default option"),
     ASYNC_LOG_ENABLED("metastore.async.log.enabled", "hive.async.log.enabled", true,
         "Whether to enable Log4j2's asynchronous logging. Asynchronous logging can give\n" +
@@ -436,7 +459,7 @@ public class MetastoreConf {
     DELEGATION_TOKEN_RENEW_INTERVAL("metastore.cluster.delegation.token.renew-interval",
       "hive.cluster.delegation.token.renew-interval", 1, TimeUnit.DAYS, ""),
     DELEGATION_TOKEN_STORE_CLS("metastore.cluster.delegation.token.store.class",
-        "hive.cluster.delegation.token.store.class", MetastoreDelegationTokenManager.class.getName(),
+        "hive.cluster.delegation.token.store.class", METASTORE_DELEGATION_MANAGER_CLASS,
         "Class to store delegation tokens"),
     DETACH_ALL_ON_COMMIT("javax.jdo.option.DetachAllOnCommit",
         "javax.jdo.option.DetachAllOnCommit", true,
@@ -747,7 +770,7 @@ public class MetastoreConf {
         "hive.metastore.stats.auto.analyze.worker.count", 1,
         "Number of parallel analyze commands to run for background stats update."),
     STORAGE_SCHEMA_READER_IMPL("metastore.storage.schema.reader.impl", "metastore.storage.schema.reader.impl",
-        DefaultStorageSchemaReader.class.getName(),
+        DEFAULT_STORAGE_SCHEMA_READER_CLASS,
         "The class to use to read schemas from storage.  It must implement " +
         "org.apache.hadoop.hive.metastore.StorageSchemaReader"),
     STORE_MANAGER_TYPE("datanucleus.storeManagerType", "datanucleus.storeManagerType", "rdbms", "metadata store type"),
@@ -761,21 +784,21 @@ public class MetastoreConf {
             + "The only supported special character right now is '/'. This flag applies only to quoted table names.\n"
             + "The default value is true."),
     TASK_THREADS_ALWAYS("metastore.task.threads.always", "metastore.task.threads.always",
-        EventCleanerTask.class.getName() + "," + RuntimeStatsCleanerTask.class.getName() + "," +
+        EVENT_CLEANER_TASK_CLASS + "," + RUNTIME_STATS_CLEANER_TASK_CLASS + "," +
         "org.apache.hadoop.hive.metastore.repl.DumpDirCleanerTask" + "," +
             "org.apache.hadoop.hive.metastore.HiveProtoEventsCleanerTask",
         "Comma separated list of tasks that will be started in separate threads.  These will " +
             "always be started, regardless of whether the metastore is running in embedded mode " +
-            "or in server mode.  They must implement " + MetastoreTaskThread.class.getName()),
+            "or in server mode.  They must implement " + METASTORE_TASK_THREAD_CLASS),
     TASK_THREADS_REMOTE_ONLY("metastore.task.threads.remote", "metastore.task.threads.remote",
-        AcidHouseKeeperService.class.getName() + "," +
-            AcidOpenTxnsCounterService.class.getName() + "," +
-            AcidCompactionHistoryService.class.getName() + "," +
-            AcidWriteSetService.class.getName() + "," +
-            MaterializationsRebuildLockCleanerTask.class.getName(),
+        ACID_HOUSE_KEEPER_SERVICE_CLASS + "," +
+            ACID_OPEN_TXNS_COUNTER_SERVICE_CLASS + "," +
+            ACID_COMPACTION_HISTORY_SERVICE_CLASS + "," +
+            ACID_WRITE_SET_SERVICE_CLASS + "," +
+            MATERIALZIATIONS_REBUILD_LOCK_CLEANER_TASK_CLASS,
         "Command separated list of tasks that will be started in separate threads.  These will be" +
             " started only when the metastore is running as a separate service.  They must " +
-            "implement " + MetastoreTaskThread.class.getName()),
+            "implement " + METASTORE_TASK_THREAD_CLASS),
     TCP_KEEP_ALIVE("metastore.server.tcp.keepalive",
         "hive.metastore.server.tcp.keepalive", true,
         "Whether to enable TCP keepalive for the metastore server. Keepalive will prevent accumulation of half-open connections."),

http://git-wip-us.apache.org/repos/asf/hive/blob/faf29143/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java
index 1422701..9905a14 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java
@@ -40,6 +40,18 @@ import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.hive.metastore.DefaultStorageSchemaReader;
+import org.apache.hadoop.hive.metastore.HiveAlterHandler;
+import org.apache.hadoop.hive.metastore.MaterializationsRebuildLockCleanerTask;
+import org.apache.hadoop.hive.metastore.MetastoreTaskThread;
+import org.apache.hadoop.hive.metastore.RuntimeStatsCleanerTask;
+import org.apache.hadoop.hive.metastore.events.EventCleanerTask;
+import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager;
+import org.apache.hadoop.hive.metastore.txn.AcidCompactionHistoryService;
+import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
+import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService;
+import org.apache.hadoop.hive.metastore.txn.AcidWriteSetService;
+
 @Category(MetastoreUnitTest.class)
 public class TestMetastoreConf {
 
@@ -430,4 +442,36 @@ public class TestMetastoreConf {
     // Make sure the hidden keys didn't get published
     Assert.assertThat(dump, CoreMatchers.not(new StringContains(ConfVars.PWD.getVarname())));
   }
+
+  /**
+   * Test class names hardcoded in MetastoreConf.
+   * MetastoreConf uses several hard-coded class names. If one of these classes is renamed or
+   * moved to a different package we want to be able to catch this. So we compare expected
+   * class name with the actual one.
+   */
+  @Test
+  public void testClassNames() {
+    Assert.assertEquals(MetastoreConf.DEFAULT_STORAGE_SCHEMA_READER_CLASS,
+        DefaultStorageSchemaReader.class.getName());
+    Assert.assertEquals(MetastoreConf.HIVE_ALTER_HANDLE_CLASS,
+        HiveAlterHandler.class.getName());
+    Assert.assertEquals(MetastoreConf.MATERIALZIATIONS_REBUILD_LOCK_CLEANER_TASK_CLASS,
+        MaterializationsRebuildLockCleanerTask.class.getName());
+    Assert.assertEquals(MetastoreConf.METASTORE_TASK_THREAD_CLASS,
+        MetastoreTaskThread.class.getName());
+    Assert.assertEquals(MetastoreConf.RUNTIME_STATS_CLEANER_TASK_CLASS,
+        RuntimeStatsCleanerTask.class.getName());
+    Assert.assertEquals(MetastoreConf.EVENT_CLEANER_TASK_CLASS,
+        EventCleanerTask.class.getName());
+    Assert.assertEquals(MetastoreConf.METASTORE_DELEGATION_MANAGER_CLASS,
+        MetastoreDelegationTokenManager.class.getName());
+    Assert.assertEquals(MetastoreConf.ACID_COMPACTION_HISTORY_SERVICE_CLASS,
+        AcidCompactionHistoryService.class.getName());
+    Assert.assertEquals(MetastoreConf.ACID_HOUSE_KEEPER_SERVICE_CLASS,
+        AcidHouseKeeperService.class.getName());
+    Assert.assertEquals(MetastoreConf.ACID_OPEN_TXNS_COUNTER_SERVICE_CLASS,
+        AcidOpenTxnsCounterService.class.getName());
+    Assert.assertEquals(MetastoreConf.ACID_WRITE_SET_SERVICE_CLASS,
+        AcidWriteSetService.class.getName());
+  }
 }