You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pv...@apache.org on 2018/05/30 11:42:03 UTC

hive git commit: HIVE-17317: Make Dbcp configurable using hive properties in hive-site.xml (Barna Zsombor Klara, reviewed by Antal Sinkovits, Peter Vary)

Repository: hive
Updated Branches:
  refs/heads/master c2ba457bb -> 16e6b837d


HIVE-17317: Make Dbcp configurable using hive properties in hive-site.xml (Barna Zsombor Klara, reviewed by Antal Sinkovits, Peter Vary)


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

Branch: refs/heads/master
Commit: 16e6b837d96050b0837086c8dd171c71d9c0bcee
Parents: c2ba457
Author: Peter Vary <pv...@cloudera.com>
Authored: Wed May 30 13:40:49 2018 +0200
Committer: Peter Vary <pv...@cloudera.com>
Committed: Wed May 30 13:40:49 2018 +0200

----------------------------------------------------------------------
 .../hadoop/hive/metastore/ObjectStore.java      |   8 +-
 .../datasource/BoneCPDataSourceProvider.java    |  14 +--
 .../datasource/DataSourceProviderFactory.java   |  21 +++-
 .../datasource/DbCPDataSourceProvider.java      | 117 +++++++++++++++++++
 .../datasource/HikariCPDataSourceProvider.java  |  14 +--
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  44 ++-----
 .../TestDataSourceProviderFactory.java          |  83 ++++++++++++-
 7 files changed, 242 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/16e6b837/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 446fe7d..b15d89d 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
@@ -217,6 +217,7 @@ import org.datanucleus.AbstractNucleusContext;
 import org.datanucleus.ClassLoaderResolver;
 import org.datanucleus.ClassLoaderResolverImpl;
 import org.datanucleus.NucleusContext;
+import org.datanucleus.PropertyNames;
 import org.datanucleus.api.jdo.JDOPersistenceManager;
 import org.datanucleus.api.jdo.JDOPersistenceManagerFactory;
 import org.datanucleus.store.rdbms.exceptions.MissingTableException;
@@ -645,7 +646,9 @@ public class ObjectStore implements RawStore, Configurable {
     if (pmf == null) {
 
       Configuration conf = MetastoreConf.newMetastoreConf();
-      DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+      DataSourceProvider dsp = DataSourceProviderFactory.hasProviderSpecificConfigurations(conf) ?
+              DataSourceProviderFactory.getDataSourceProvider(conf) : null;
+
       if (dsp == null) {
         pmf = JDOHelper.getPersistenceManagerFactory(prop);
       } else {
@@ -654,7 +657,8 @@ public class ObjectStore implements RawStore, Configurable {
           Map<Object, Object> dsProperties = new HashMap<>();
           //Any preexisting datanucleus property should be passed along
           dsProperties.putAll(prop);
-          dsProperties.put("datanucleus.ConnectionFactory", ds);
+          dsProperties.put(PropertyNames.PROPERTY_CONNECTION_FACTORY, ds);
+          dsProperties.put(PropertyNames.PROPERTY_CONNECTION_FACTORY2, ds);
           dsProperties.put("javax.jdo.PersistenceManagerFactoryClass",
               "org.datanucleus.api.jdo.JDOPersistenceManagerFactory");
           pmf = JDOHelper.getPersistenceManagerFactory(dsProperties);

http://git-wip-us.apache.org/repos/asf/hive/blob/16e6b837/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java
----------------------------------------------------------------------
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..7e33c51 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
@@ -36,8 +36,8 @@ public class BoneCPDataSourceProvider implements DataSourceProvider {
   private static final Logger LOG = LoggerFactory.getLogger(BoneCPDataSourceProvider.class);
 
   public static final String BONECP = "bonecp";
-  private static final String CONNECTION_TIMEOUT_PROPERTY= "bonecp.connectionTimeoutInMs";
-  private static final String PARTITION_COUNT_PROPERTY= "bonecp.partitionCount";
+  private static final String CONNECTION_TIMEOUT_PROPERTY= BONECP + ".connectionTimeoutInMs";
+  private static final String PARTITION_COUNT_PROPERTY= BONECP + ".partitionCount";
 
   @Override
   public DataSource create(Configuration hdpConfig) throws SQLException {
@@ -81,13 +81,7 @@ 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;
-    }
-    LOG.debug("Configuration requested " + poolingType + " pooling, BoneCpDSProvider exiting");
-    return false;
+            MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE);
+    return BONECP.equalsIgnoreCase(poolingType);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/16e6b837/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DataSourceProviderFactory.java
----------------------------------------------------------------------
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..5a92e10 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,7 +18,9 @@
 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
@@ -27,7 +29,8 @@ import org.apache.hadoop.conf.Configuration;
 public abstract  class DataSourceProviderFactory {
 
   private static final ImmutableList<DataSourceProvider> FACTORIES =
-      ImmutableList.<DataSourceProvider>builder().add(new HikariCPDataSourceProvider(), new BoneCPDataSourceProvider()).build();
+      ImmutableList.<DataSourceProvider>builder().add(new HikariCPDataSourceProvider(), new BoneCPDataSourceProvider(),
+              new DbCPDataSourceProvider()).build();
 
   /**
    * @param hdpConfig hadoop configuration
@@ -44,4 +47,20 @@ public abstract  class DataSourceProviderFactory {
     return null;
   }
 
+  /**
+   * @param hdpConfig hadoop configuration
+   * @return true if the configuration contains settings specifically aimed for one
+   * of the supported conntection pool implementations.
+   */
+  public static boolean hasProviderSpecificConfigurations(Configuration hdpConfig) {
+
+    String poolingType = MetastoreConf.getVar(hdpConfig, MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
+
+    return Iterables.any(hdpConfig, entry ->
+    {
+      String key = entry.getKey();
+      return key != null && (key.startsWith(poolingType));
+    });
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/16e6b837/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DbCPDataSourceProvider.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DbCPDataSourceProvider.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DbCPDataSourceProvider.java
new file mode 100644
index 0000000..7fe487b
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/DbCPDataSourceProvider.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.datasource;
+
+import org.apache.commons.dbcp.ConnectionFactory;
+import org.apache.commons.dbcp.DriverManagerConnectionFactory;
+import org.apache.commons.dbcp.PoolableConnectionFactory;
+import org.apache.commons.dbcp.PoolingDataSource;
+import org.apache.commons.pool.impl.GenericObjectPool;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.sql.DataSource;
+import java.sql.SQLException;
+
+/**
+ * DataSourceProvider for the dbcp connection pool.
+ */
+public class DbCPDataSourceProvider implements DataSourceProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbCPDataSourceProvider.class);
+
+  static final String DBCP = "dbcp";
+  private static final String CONNECTION_TIMEOUT_PROPERTY = DBCP + ".maxWait";
+  private static final String CONNECTION_MAX_IDLE_PROPERTY = DBCP + ".maxIdle";
+  private static final String CONNECTION_MIN_IDLE_PROPERTY = DBCP + ".minIdle";
+  private static final String CONNECTION_TEST_BORROW_PROPERTY = DBCP + ".testOnBorrow";
+  private static final String CONNECTION_MIN_EVICT_MILLIS_PROPERTY = DBCP + ".minEvictableIdleTimeMillis";
+  private static final String CONNECTION_TEST_IDLEPROPERTY = DBCP + ".testWhileIdle";
+  private static final String CONNECTION_TIME_BETWEEN_EVICTION_RUNS_MILLIS = DBCP + ".timeBetweenEvictionRunsMillis";
+  private static final String CONNECTION_NUM_TESTS_PER_EVICTION_RUN = DBCP + ".numTestsPerEvictionRun";
+  private static final String CONNECTION_TEST_ON_RETURN = DBCP + ".testOnReturn";
+  private static final String CONNECTION_SOFT_MIN_EVICTABLE_IDLE_TIME = DBCP + ".softMinEvictableIdleTimeMillis";
+  private static final String CONNECTION_LIFO = DBCP + ".lifo";
+
+  @Override
+  public DataSource create(Configuration hdpConfig) throws SQLException {
+
+    LOG.debug("Creating dbcp connection pool for the MetaStore");
+
+    String driverUrl = DataSourceProvider.getMetastoreJdbcDriverUrl(hdpConfig);
+    String user = DataSourceProvider.getMetastoreJdbcUser(hdpConfig);
+    String passwd = DataSourceProvider.getMetastoreJdbcPasswd(hdpConfig);
+    int maxPoolSize = hdpConfig.getInt(
+            MetastoreConf.ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS.getVarname(),
+            ((Long) MetastoreConf.ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS.getDefaultVal()).intValue());
+    long connectionTimeout = hdpConfig.getLong(CONNECTION_TIMEOUT_PROPERTY, 30000L);
+    int connectionMaxIlde = hdpConfig.getInt(CONNECTION_MAX_IDLE_PROPERTY, GenericObjectPool.DEFAULT_MAX_IDLE);
+    int connectionMinIlde = hdpConfig.getInt(CONNECTION_MIN_IDLE_PROPERTY, GenericObjectPool.DEFAULT_MIN_IDLE);
+    boolean testOnBorrow = hdpConfig.getBoolean(CONNECTION_TEST_BORROW_PROPERTY,
+            GenericObjectPool.DEFAULT_TEST_ON_BORROW);
+    long evictionTimeMillis = hdpConfig.getLong(CONNECTION_MIN_EVICT_MILLIS_PROPERTY,
+            GenericObjectPool.DEFAULT_MIN_EVICTABLE_IDLE_TIME_MILLIS);
+    boolean testWhileIdle = hdpConfig.getBoolean(CONNECTION_TEST_IDLEPROPERTY,
+            GenericObjectPool.DEFAULT_TEST_WHILE_IDLE);
+    long timeBetweenEvictionRuns = hdpConfig.getLong(CONNECTION_TIME_BETWEEN_EVICTION_RUNS_MILLIS,
+            GenericObjectPool.DEFAULT_TIME_BETWEEN_EVICTION_RUNS_MILLIS);
+    int numTestsPerEvictionRun = hdpConfig.getInt(CONNECTION_NUM_TESTS_PER_EVICTION_RUN,
+            GenericObjectPool.DEFAULT_NUM_TESTS_PER_EVICTION_RUN);
+    boolean testOnReturn = hdpConfig.getBoolean(CONNECTION_TEST_ON_RETURN, GenericObjectPool.DEFAULT_TEST_ON_RETURN);
+    long softMinEvictableIdleTimeMillis = hdpConfig.getLong(CONNECTION_SOFT_MIN_EVICTABLE_IDLE_TIME,
+            GenericObjectPool.DEFAULT_SOFT_MIN_EVICTABLE_IDLE_TIME_MILLIS);
+    boolean lifo = hdpConfig.getBoolean(CONNECTION_LIFO, GenericObjectPool.DEFAULT_LIFO);
+
+    GenericObjectPool objectPool = new GenericObjectPool();
+    objectPool.setMaxActive(maxPoolSize);
+    objectPool.setMaxWait(connectionTimeout);
+    objectPool.setMaxIdle(connectionMaxIlde);
+    objectPool.setMinIdle(connectionMinIlde);
+    objectPool.setTestOnBorrow(testOnBorrow);
+    objectPool.setTestWhileIdle(testWhileIdle);
+    objectPool.setMinEvictableIdleTimeMillis(evictionTimeMillis);
+    objectPool.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRuns);
+    objectPool.setNumTestsPerEvictionRun(numTestsPerEvictionRun);
+    objectPool.setTestOnReturn(testOnReturn);
+    objectPool.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+    objectPool.setLifo(lifo);
+
+    ConnectionFactory connFactory = new DriverManagerConnectionFactory(driverUrl, user, passwd);
+    // This doesn't get used, but it's still necessary, see
+    // https://git1-us-west.apache.org/repos/asf?p=commons-dbcp.git;a=blob;f=doc/ManualPoolingDataSourceExample.java;
+    // h=f45af2b8481f030b27364e505984c0eef4f35cdb;hb=refs/heads/DBCP_1_5_x_BRANCH
+    new PoolableConnectionFactory(connFactory, objectPool, null, null, false, true);
+
+    return new PoolingDataSource(objectPool);
+  }
+
+  @Override
+  public boolean mayReturnClosedConnection() {
+    // Only BoneCP should return true
+    return false;
+  }
+
+  @Override
+  public boolean supports(Configuration configuration) {
+    String poolingType = MetastoreConf.getVar(configuration,
+            MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE);
+    return DBCP.equalsIgnoreCase(poolingType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/16e6b837/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/HikariCPDataSourceProvider.java
----------------------------------------------------------------------
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..8f6ae57 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
@@ -35,8 +35,8 @@ public class HikariCPDataSourceProvider implements DataSourceProvider {
 
   private static final Logger LOG = LoggerFactory.getLogger(HikariCPDataSourceProvider.class);
 
-  public static final String HIKARI = "hikari";
-  private static final String CONNECTION_TIMEOUT_PROPERTY= "hikari.connectionTimeout";
+  public static final String HIKARI = "hikaricp";
+  private static final String CONNECTION_TIMEOUT_PROPERTY= HIKARI + ".connectionTimeout";
 
   @Override
   public DataSource create(Configuration hdpConfig) throws SQLException {
@@ -76,14 +76,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;
+            MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE);
+    return HIKARI.equalsIgnoreCase(poolingType);
   }
 
   private Properties replacePrefix(Properties props) {

http://git-wip-us.apache.org/repos/asf/hive/blob/16e6b837/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 469e699..565fb89 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -51,13 +51,8 @@ import java.util.regex.Pattern;
 
 import javax.sql.DataSource;
 
-import org.apache.commons.dbcp.ConnectionFactory;
-import org.apache.commons.dbcp.DriverManagerConnectionFactory;
-import org.apache.commons.dbcp.PoolableConnectionFactory;
-import org.apache.commons.dbcp.PoolingDataSource;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.commons.pool.impl.GenericObjectPool;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -122,9 +117,8 @@ import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
-import org.apache.hadoop.hive.metastore.datasource.BoneCPDataSourceProvider;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
-import org.apache.hadoop.hive.metastore.datasource.HikariCPDataSourceProvider;
+import org.apache.hadoop.hive.metastore.datasource.DataSourceProviderFactory;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
@@ -4334,32 +4328,18 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   }
 
   private static synchronized DataSource setupJdbcConnectionPool(Configuration conf, int maxPoolSize, long getConnectionTimeoutMs) throws SQLException {
-    String driverUrl = DataSourceProvider.getMetastoreJdbcDriverUrl(conf);
-    String user = DataSourceProvider.getMetastoreJdbcUser(conf);
-    String passwd = DataSourceProvider.getMetastoreJdbcPasswd(conf);
-    String connectionPooler = MetastoreConf.getVar(conf, ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
-
-    if ("bonecp".equals(connectionPooler)) {
-      doRetryOnConnPool = true;  // Enable retries to work around BONECP bug.
-      return new BoneCPDataSourceProvider().create(conf);
-    } else if ("dbcp".equals(connectionPooler)) {
-      GenericObjectPool objectPool = new GenericObjectPool();
-      //https://commons.apache.org/proper/commons-pool/api-1.6/org/apache/commons/pool/impl/GenericObjectPool.html#setMaxActive(int)
-      objectPool.setMaxActive(maxPoolSize);
-      objectPool.setMaxWait(getConnectionTimeoutMs);
-      ConnectionFactory connFactory = new DriverManagerConnectionFactory(driverUrl, user, passwd);
-      // This doesn't get used, but it's still necessary, see
-      // http://svn.apache.org/viewvc/commons/proper/dbcp/branches/DBCP_1_4_x_BRANCH/doc/ManualPoolingDataSourceExample.java?view=markup
-      PoolableConnectionFactory poolConnFactory =
-          new PoolableConnectionFactory(connFactory, objectPool, null, null, false, true);
-      return new PoolingDataSource(objectPool);
-    } else if ("hikaricp".equals(connectionPooler)) {
-      return new HikariCPDataSourceProvider().create(conf);
-    } else if ("none".equals(connectionPooler)) {
-      LOG.info("Choosing not to pool JDBC connections");
-      return new NoPoolConnectionPool(conf);
+    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    if (dsp != null) {
+      doRetryOnConnPool = dsp.mayReturnClosedConnection();
+      return dsp.create(conf);
     } else {
-      throw new RuntimeException("Unknown JDBC connection pooling " + connectionPooler);
+      String connectionPooler = MetastoreConf.getVar(conf, ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
+      if ("none".equals(connectionPooler)) {
+        LOG.info("Choosing not to pool JDBC connections");
+        return new NoPoolConnectionPool(conf);
+      } else {
+        throw new RuntimeException("Unknown JDBC connection pooling " + connectionPooler);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/16e6b837/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/datasource/TestDataSourceProviderFactory.java
----------------------------------------------------------------------
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..6ae7f50 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.metastore.datasource;
 
 import com.jolbox.bonecp.BoneCPDataSource;
 import com.zaxxer.hikari.HikariDataSource;
+import org.apache.commons.dbcp.PoolingDataSource;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
@@ -41,26 +42,35 @@ public class TestDataSourceProviderFactory {
     conf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setVar(conf, ConfVars.CONNECTION_USER_NAME, "dummyUser");
     MetastoreConf.setVar(conf, ConfVars.PWD, "dummyPass");
+    conf.unset(ConfVars.CONNECTION_POOLING_TYPE.getVarname());
   }
 
   @Test
   public void testNoDataSourceCreatedWithoutProps() throws SQLException {
 
+    MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, "dummy");
+
     DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
     Assert.assertNull(dsp);
+  }
+
+  @Test
+  public void testCanCreateDataSourceForSpecificProp() throws SQLException {
+
+    Assert.assertFalse(
+            DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
 
     MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, BoneCPDataSourceProvider.BONECP);
+    conf.set(BoneCPDataSourceProvider.BONECP + ".dummy.var", "dummy");
 
-    dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
-    Assert.assertNull(dsp);
+    Assert.assertTrue(
+            DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
   }
 
   @Test
   public void testCreateBoneCpDataSource() throws SQLException {
 
     MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, BoneCPDataSourceProvider.BONECP);
-    conf.set(BoneCPDataSourceProvider.BONECP + ".firstProp", "value");
-    conf.set(BoneCPDataSourceProvider.BONECP + ".secondProp", "value");
 
     DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
     Assert.assertNotNull(dsp);
@@ -170,4 +180,69 @@ public class TestDataSourceProviderFactory {
     Assert.assertEquals(false, ((HikariDataSource)ds).isAllowPoolSuspension());
   }
 
+  @Test
+  public void testCreateDbCpDataSource() throws SQLException {
+
+    MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, DbCPDataSourceProvider.DBCP);
+
+    DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf);
+    Assert.assertNotNull(dsp);
+
+    DataSource ds = dsp.create(conf);
+    Assert.assertTrue(ds instanceof PoolingDataSource);
+  }
+
+  @Test
+  public void testHasProviderSpecificConfigurationBonecp() throws SQLException {
+
+    MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, BoneCPDataSourceProvider.BONECP);
+
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("dbcp.dummyConf", "dummyValue");
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("hikaricp.dummyConf", "dummyValue");
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("bonecp.dummyConf", "dummyValue");
+    Assert.assertTrue(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+  }
+
+  @Test
+  public void testHasProviderSpecificConfigurationHikaricp() throws SQLException {
+
+    MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, HikariCPDataSourceProvider.HIKARI);
+
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("dbcp.dummyConf", "dummyValue");
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("bonecp.dummyConf", "dummyValue");
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("hikaricp.dummyConf", "dummyValue");
+    Assert.assertTrue(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+  }
+
+  @Test
+  public void testHasProviderSpecificConfigurationDbcp() throws SQLException {
+
+    MetastoreConf.setVar(conf, ConfVars.CONNECTION_POOLING_TYPE, DbCPDataSourceProvider.DBCP);
+
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("hikaricp.dummyConf", "dummyValue");
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("bonecp.dummyConf", "dummyValue");
+    Assert.assertFalse(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+    conf.set("dbcp.dummyConf", "dummyValue");
+    Assert.assertTrue(DataSourceProviderFactory.hasProviderSpecificConfigurations(conf));
+
+  }
 }