HIVE-25522: NullPointerException in TxnHandler (#2647)
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 b2a22f1..38db1c9 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
@@ -294,14 +294,15 @@
checkQFileTestHack();
synchronized (TxnHandler.class) {
- if (connPool == null) {
- Connection dbConn = null;
- // Set up the JDBC connection pool
- try {
- int maxPoolSize = MetastoreConf.getIntVar(conf, ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS);
- long getConnectionTimeoutMs = 30000;
+ int maxPoolSize = MetastoreConf.getIntVar(conf, ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS);
+ long getConnectionTimeoutMs = 30000;
+ synchronized (TxnHandler.class) {
+ if (connPool == null) {
connPool = setupJdbcConnectionPool(conf, maxPoolSize, getConnectionTimeoutMs);
- /*the mutex pools should ideally be somewhat larger since some operations require 1
+ }
+
+ if (connPoolMutex == null) {
+ /*the mutex pools should ideally be somewhat larger since some operations require 1
connection from each pool and we want to avoid taking a connection from primary pool
and then blocking because mutex pool is empty. There is only 1 thread in any HMS trying
to mutex on each MUTEX_KEY except MUTEX_KEY.CheckLock. The CheckLock operation gets a
@@ -309,15 +310,19 @@
order (not very elegant...). So number of connection requests for connPoolMutex cannot
exceed (size of connPool + MUTEX_KEY.values().length - 1).*/
connPoolMutex = setupJdbcConnectionPool(conf, maxPoolSize + MUTEX_KEY.values().length, getConnectionTimeoutMs);
- dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
- determineDatabaseProduct(dbConn);
+ }
+
+ if (dbProduct == null) {
+ try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) {
+ determineDatabaseProduct(dbConn);
+ } catch (SQLException e) {
+ LOG.error("Unable to determine database product", e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ if (sqlGenerator == null) {
sqlGenerator = new SQLGenerator(dbProduct, conf);
- } catch (SQLException e) {
- String msg = "Unable to instantiate JDBC connection pooling, " + e.getMessage();
- LOG.error(msg);
- throw new RuntimeException(e);
- } finally {
- closeDbConn(dbConn);
}
}
}
@@ -4489,33 +4494,38 @@
}
}
- 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();
+ private static synchronized DataSource setupJdbcConnectionPool(Configuration conf, int maxPoolSize, long getConnectionTimeoutMs) {
+ try {
+ 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);
- } else {
- throw new RuntimeException("Unknown JDBC connection pooling " + connectionPooler);
+ 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);
+ } else {
+ throw new RuntimeException("Unknown JDBC connection pooling " + connectionPooler);
+ }
+ } catch (SQLException e) {
+ LOG.error("Unable to instantiate JDBC connection pooling", e);
+ throw new RuntimeException(e);
}
}