You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by yc...@apache.org on 2019/01/31 15:04:22 UTC

[hive] branch branch-3 updated: HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen)

This is an automated email from the ASF dual-hosted git repository.

ychena pushed a commit to branch branch-3
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/branch-3 by this push:
     new c1c33cc  HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen)
c1c33cc is described below

commit c1c33cce5d185067df191aa57f4e1d31f1a5d582
Author: Yongzhi Chen <yc...@apache.org>
AuthorDate: Thu Jan 31 10:02:55 2019 -0500

    HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen)
---
 .../hive/metastore/PersistenceManagerProvider.java |  48 +++++----
 .../datasource/BoneCPDataSourceProvider.java       | 109 +++++++++++++++++----
 .../metastore/datasource/DataSourceProvider.java   |  18 ++--
 .../datasource/DataSourceProviderFactory.java      |  32 +++---
 .../datasource/HikariCPDataSourceProvider.java     |  35 ++++---
 .../hive/metastore/metrics/MetricsConstants.java   |   2 +
 .../hadoop/hive/metastore/metrics/PerfLogger.java  |  17 +++-
 .../datasource/TestDataSourceProviderFactory.java  |  22 ++---
 8 files changed, 188 insertions(+), 95 deletions(-)

diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java
index 20f0738..876437b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java
@@ -19,6 +19,28 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import javax.jdo.JDOCanRetryException;
+import javax.jdo.JDOHelper;
+import javax.jdo.PersistenceManager;
+import javax.jdo.PersistenceManagerFactory;
+import javax.jdo.datastore.DataStoreCache;
+import javax.sql.DataSource;
+
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
@@ -33,7 +55,6 @@ import org.apache.hadoop.hive.metastore.model.MSerDeInfo;
 import org.apache.hadoop.hive.metastore.model.MStorageDescriptor;
 import org.apache.hadoop.hive.metastore.model.MTable;
 import org.apache.hadoop.hive.metastore.model.MType;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.datanucleus.AbstractNucleusContext;
 import org.datanucleus.ClassLoaderResolver;
 import org.datanucleus.ClassLoaderResolverImpl;
@@ -46,29 +67,6 @@ import org.datanucleus.util.WeakValueMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.jdo.JDOCanRetryException;
-import javax.jdo.JDOHelper;
-import javax.jdo.PersistenceManager;
-import javax.jdo.PersistenceManagerFactory;
-import javax.jdo.datastore.DataStoreCache;
-import javax.sql.DataSource;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Supplier;
-
 /**
  * This class is a wrapper class around PersistenceManagerFactory and its properties
  * These objects are static and need to be carefully modified together such that there are no
@@ -221,7 +219,7 @@ public class PersistenceManagerProvider {
   }
 
   private static void initPMF(Configuration conf) {
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
 
     if (dsp == null) {
       pmf = JDOHelper.getPersistenceManagerFactory(prop);
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java
index 4ff2bb7..60594fe 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java
@@ -17,17 +17,26 @@
  */
 package org.apache.hadoop.hive.metastore.datasource;
 
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import javax.sql.DataSource;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricSet;
 import com.jolbox.bonecp.BoneCPConfig;
 import com.jolbox.bonecp.BoneCPDataSource;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.sql.DataSource;
-import java.sql.SQLException;
-import java.util.Properties;
-
 /**
  * DataSourceProvider for the BoneCP connection pool.
  */
@@ -61,15 +70,16 @@ public class BoneCPDataSourceProvider implements DataSourceProvider {
       throw new SQLException("Cannot create BoneCP configuration: ", e);
     }
     config.setJdbcUrl(driverUrl);
-    //if we are waiting for connection for a long time, something is really wrong
-    //better raise an error than hang forever
-    //see DefaultConnectionStrategy.getConnectionInternal()
+    // if we are waiting for connection for a long time, something is really wrong
+    // better raise an error than hang forever
+    // see DefaultConnectionStrategy.getConnectionInternal()
     config.setConnectionTimeoutInMs(connectionTimeout);
     config.setMaxConnectionsPerPartition(maxPoolSize);
     config.setPartitionCount(Integer.parseInt(partitionCount));
     config.setUser(user);
     config.setPassword(passwd);
-    return new BoneCPDataSource(config);
+
+    return initMetrics(new BoneCPDataSource(config));
   }
 
   @Override
@@ -79,15 +89,80 @@ public class BoneCPDataSourceProvider implements DataSourceProvider {
   }
 
   @Override
-  public boolean supports(Configuration configuration) {
-    String poolingType = MetastoreConf.getVar(configuration,
-            MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
-    if (BONECP.equals(poolingType)) {
-      int boneCpPropsNr = DataSourceProvider.getPrefixedProperties(configuration, BONECP).size();
-      LOG.debug("Found " + boneCpPropsNr + " nr. of bonecp specific configurations");
-      return boneCpPropsNr > 0;
+  public String getPoolingType() {
+    return BONECP;
+  }
+
+  private BoneCPDataSource initMetrics(BoneCPDataSource ds) {
+    final MetricRegistry registry = Metrics.getRegistry();
+    if (registry != null) {
+      registry.registerAll(new BoneCPMetrics(ds));
+    }
+    return ds;
+  }
+
+  private static class BoneCPMetrics implements MetricSet {
+    private BoneCPDataSource ds;
+    private Optional<String> poolName;
+
+    private BoneCPMetrics(final BoneCPDataSource ds) {
+      this.ds = ds;
+      this.poolName = Optional.ofNullable(ds.getPoolName());
+    }
+
+    private String name(final String gaugeName) {
+      return poolName.orElse("BoneCP") + ".pool." + gaugeName;
+    }
+
+    @Override
+    public Map<String, Metric> getMetrics() {
+      final Map<String, Metric> gauges = new HashMap<>();
+
+      gauges.put(name("TotalConnections"), new Gauge<Integer>() {
+        @Override
+        public Integer getValue() {
+          if (ds.getPool() != null) {
+            return ds.getPool().getStatistics().getTotalCreatedConnections();
+          } else {
+            return 0;
+          }
+        }
+      });
+
+      gauges.put(name("IdleConnections"), new Gauge<Integer>() {
+        @Override
+        public Integer getValue() {
+          if (ds.getPool() != null) {
+            return ds.getPool().getStatistics().getTotalFree();
+          } else {
+            return 0;
+          }
+        }
+      });
+
+      gauges.put(name("ActiveConnections"), new Gauge<Integer>() {
+        @Override
+        public Integer getValue() {
+          if (ds.getPool() != null) {
+            return ds.getPool().getStatistics().getTotalLeased();
+          } else {
+            return 0;
+          }
+        }
+      });
+
+      gauges.put(name("WaitTimeAvg"), new Gauge<Double>() {
+        @Override
+        public Double getValue() {
+          if (ds.getPool() != null) {
+            return ds.getPool().getStatistics().getConnectionWaitTimeAvg();
+          } else {
+            return 0.0;
+          }
+        }
+      });
+
+      return Collections.unmodifiableMap(gauges);
     }
-    LOG.debug("Configuration requested " + poolingType + " pooling, BoneCpDSProvider exiting");
-    return false;
   }
 }
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProvider.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProvider.java
index 6dc63fb..d71fa7d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProvider.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProvider.java
@@ -17,14 +17,14 @@
  */
 package org.apache.hadoop.hive.metastore.datasource;
 
-import com.google.common.collect.Iterables;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-
-import javax.sql.DataSource;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.Properties;
+import javax.sql.DataSource;
+
+import com.google.common.collect.Iterables;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 
 public interface DataSourceProvider {
 
@@ -42,11 +42,11 @@ public interface DataSourceProvider {
   boolean mayReturnClosedConnection();
 
   /**
-   * @param configuration Hadoop configuration object
-   * @return factory able to create a connection pool for the implementation
-   * specified in the configuration
+   * Get the declared pooling type string. This is used to check against the constant in
+   * config options.
+   * @return The pooling type string associated with the data source.
    */
-  boolean supports(Configuration configuration);
+  String getPoolingType();
 
   /**
    * @param hdpConfig
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProviderFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProviderFactory.java
index e3c18e3..d7af787 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProviderFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProviderFactory.java
@@ -18,30 +18,34 @@
 package org.apache.hadoop.hive.metastore.datasource;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 
 /**
  * Create a DataSourceProvider for a connectionPool configured in a hadoop
  * Configuration object.
  */
 public abstract  class DataSourceProviderFactory {
-
-  private static final ImmutableList<DataSourceProvider> FACTORIES =
-      ImmutableList.<DataSourceProvider>builder().add(new HikariCPDataSourceProvider(), new BoneCPDataSourceProvider()).build();
+  private static final ImmutableList<DataSourceProvider> FACTORIES = ImmutableList.of(
+    new HikariCPDataSourceProvider(),
+    new BoneCPDataSourceProvider());
 
   /**
+   * The data source providers declare if they are supported or not based on the config.
+   * This function looks through all the data source providers and picks the first one which is
+   * supported. If no data source provider is found, returns a null.
+   *
    * @param hdpConfig hadoop configuration
-   * @return factory for the configured datanucleus.connectionPoolingType
+   * @return factory for the configured datanucleus.connectionPoolingType or null if no supported
+   *         data source providers are found.
    */
-  public static DataSourceProvider getDataSourceProvider(Configuration hdpConfig) {
-
-    for (DataSourceProvider factory : FACTORIES) {
-
-      if (factory.supports(hdpConfig)) {
-        return factory;
-      }
-    }
-    return null;
+  public static DataSourceProvider tryGetDataSourceProviderOrNull(Configuration hdpConfig) {
+    final String configuredPoolingType = MetastoreConf.getVar(hdpConfig,
+        MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE);
+    return Iterables.tryFind(FACTORIES, factory -> {
+      String poolingType = factory.getPoolingType();
+      return poolingType != null && poolingType.equalsIgnoreCase(configuredPoolingType);
+    }).orNull();
   }
-
 }
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/HikariCPDataSourceProvider.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/HikariCPDataSourceProvider.java
index 6ffc24a..6784597 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/HikariCPDataSourceProvider.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/HikariCPDataSourceProvider.java
@@ -17,17 +17,19 @@
  */
 package org.apache.hadoop.hive.metastore.datasource;
 
+import java.sql.SQLException;
+import java.util.Properties;
+import javax.sql.DataSource;
+
+import com.codahale.metrics.MetricRegistry;
 import com.zaxxer.hikari.HikariConfig;
 import com.zaxxer.hikari.HikariDataSource;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.sql.DataSource;
-import java.sql.SQLException;
-import java.util.Properties;
-
 /**
  * DataSourceProvider for the HikariCP connection pool.
  */
@@ -35,7 +37,7 @@ public class HikariCPDataSourceProvider implements DataSourceProvider {
 
   private static final Logger LOG = LoggerFactory.getLogger(HikariCPDataSourceProvider.class);
 
-  public static final String HIKARI = "hikari";
+  static final String HIKARI = "hikari";
   private static final String CONNECTION_TIMEOUT_PROPERTY= "hikari.connectionTimeout";
 
   @Override
@@ -64,7 +66,8 @@ public class HikariCPDataSourceProvider implements DataSourceProvider {
     config.setPassword(passwd);
     //https://github.com/brettwooldridge/HikariCP
     config.setConnectionTimeout(connectionTimeout);
-    return new HikariDataSource(config);
+
+    return new HikariDataSource(initMetrics(config));
   }
 
   @Override
@@ -74,16 +77,8 @@ public class HikariCPDataSourceProvider implements DataSourceProvider {
   }
 
   @Override
-  public boolean supports(Configuration configuration) {
-    String poolingType = MetastoreConf.getVar(configuration,
-            MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
-    if (HIKARI.equals(poolingType)) {
-      int hikariPropsNr = DataSourceProvider.getPrefixedProperties(configuration, HIKARI).size();
-      LOG.debug("Found " + hikariPropsNr + " nr. of hikari specific configurations");
-      return hikariPropsNr > 0;
-    }
-    LOG.debug("Configuration requested " + poolingType + " pooling, HikariCpDSProvider exiting");
-    return false;
+  public String getPoolingType() {
+    return HIKARI;
   }
 
   private Properties replacePrefix(Properties props) {
@@ -92,4 +87,12 @@ public class HikariCPDataSourceProvider implements DataSourceProvider {
         newProps.put(key.toString().replaceFirst(HIKARI + ".", ""), value));
     return newProps;
   }
+
+  private static HikariConfig initMetrics(final HikariConfig config) {
+    final MetricRegistry registry = Metrics.getRegistry();
+    if (registry != null) {
+      config.setMetricRegistry(registry);
+    }
+    return config;
+  }
 }
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/MetricsConstants.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/MetricsConstants.java
index 3b188f8..24c8c4c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/MetricsConstants.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/MetricsConstants.java
@@ -21,6 +21,8 @@ public class MetricsConstants {
   public static final String ACTIVE_CALLS = "active_calls_";
   public static final String API_PREFIX = "api_";
 
+  public static final String TOTAL_API_CALLS = "total_api_calls";
+
   public static final String CREATE_TOTAL_DATABASES = "create_total_count_dbs";
   public static final String CREATE_TOTAL_TABLES = "create_total_count_tables";
   public static final String CREATE_TOTAL_PARTITIONS = "create_total_count_partitions";
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/PerfLogger.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/PerfLogger.java
index a2def26..c111343 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/PerfLogger.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/metrics/PerfLogger.java
@@ -164,15 +164,19 @@ public class PerfLogger {
     return ImmutableMap.copyOf(endTimes);
   }
 
-  //Methods for metrics integration.  Each thread-local PerfLogger will open/close scope during each perf-log method.
-  protected transient Map<String, Timer.Context> timerContexts = new HashMap<>();
+  // Methods for metrics integration.  Each thread-local PerfLogger will open/close scope during each perf-log method.
+  private transient Map<String, Timer.Context> timerContexts = new HashMap<>();
+  private transient Timer.Context totalApiCallsTimerContext = null;
 
   private void beginMetrics(String method) {
     Timer timer = Metrics.getOrCreateTimer(MetricsConstants.API_PREFIX + method);
     if (timer != null) {
       timerContexts.put(method, timer.time());
     }
-
+    timer = Metrics.getOrCreateTimer(MetricsConstants.TOTAL_API_CALLS);
+    if (timer != null) {
+      totalApiCallsTimerContext = timer.time();
+    }
   }
 
   private void endMetrics(String method) {
@@ -180,6 +184,9 @@ public class PerfLogger {
     if (context != null) {
       context.close();
     }
+    if (totalApiCallsTimerContext != null) {
+      totalApiCallsTimerContext.close();
+    }
   }
 
   /**
@@ -190,5 +197,9 @@ public class PerfLogger {
       context.close();
     }
     timerContexts.clear();
+    if (totalApiCallsTimerContext != null) {
+      totalApiCallsTimerContext.close();
+      totalApiCallsTimerContext = null;
+    }
   }
 }
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/datasource/TestDataSourceProviderFactory.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/datasource/TestDataSourceProviderFactory.java
index 2d45c29..2ea94ac 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/datasource/TestDataSourceProviderFactory.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/datasource/TestDataSourceProviderFactory.java
@@ -46,12 +46,12 @@ public class TestDataSourceProviderFactory {
   @Test
   public void testNoDataSourceCreatedWithoutProps() throws SQLException {
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNull(dsp);
 
-    MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, BoneCPDataSourceProvider.BONECP);
+    MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, "dummy");
 
-    dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNull(dsp);
   }
 
@@ -62,7 +62,7 @@ public class TestDataSourceProviderFactory {
     conf.set(BoneCPDataSourceProvider.BONECP + ".firstProp", "value");
     conf.set(BoneCPDataSourceProvider.BONECP + ".secondProp", "value");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);
@@ -75,7 +75,7 @@ public class TestDataSourceProviderFactory {
     MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, BoneCPDataSourceProvider.BONECP);
     conf.set(BoneCPDataSourceProvider.BONECP + ".initSQL", "select 1 from dual");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);
@@ -89,7 +89,7 @@ public class TestDataSourceProviderFactory {
     MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, BoneCPDataSourceProvider.BONECP);
     conf.set(BoneCPDataSourceProvider.BONECP + ".acquireRetryDelayInMs", "599");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);
@@ -103,7 +103,7 @@ public class TestDataSourceProviderFactory {
     MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, BoneCPDataSourceProvider.BONECP);
     conf.set(BoneCPDataSourceProvider.BONECP + ".disableJMX", "true");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);
@@ -118,7 +118,7 @@ public class TestDataSourceProviderFactory {
     // This is needed to prevent the HikariDataSource from trying to connect to the DB
     conf.set(HikariCPDataSourceProvider.HIKARI + ".initializationFailTimeout", "-1");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);
@@ -132,7 +132,7 @@ public class TestDataSourceProviderFactory {
     conf.set(HikariCPDataSourceProvider.HIKARI + ".connectionInitSql", "select 1 from dual");
     conf.set(HikariCPDataSourceProvider.HIKARI + ".initializationFailTimeout", "-1");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);
@@ -147,7 +147,7 @@ public class TestDataSourceProviderFactory {
     conf.set(HikariCPDataSourceProvider.HIKARI + ".idleTimeout", "59999");
     conf.set(HikariCPDataSourceProvider.HIKARI + ".initializationFailTimeout", "-1");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);
@@ -162,7 +162,7 @@ public class TestDataSourceProviderFactory {
     conf.set(HikariCPDataSourceProvider.HIKARI + ".allowPoolSuspension", "false");
     conf.set(HikariCPDataSourceProvider.HIKARI + ".initializationFailTimeout", "-1");
 
-    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
     Assert.assertNotNull(dsp);
 
     DataSource ds = dsp.create(conf);