You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by su...@apache.org on 2021/11/06 19:53:00 UTC
[hive] branch master updated: HIVE-25522: NullPointerException in
TxnHandler (#2647)
This is an automated email from the ASF dual-hosted git repository.
sunchao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new c8a0b6f HIVE-25522: NullPointerException in TxnHandler (#2647)
c8a0b6f is described below
commit c8a0b6f4887edc2f33ed5d2c77c8baf9c0bbd11c
Author: Szehon Ho <sz...@gmail.com>
AuthorDate: Sat Nov 6 12:52:39 2021 -0700
HIVE-25522: NullPointerException in TxnHandler (#2647)
---
.../hadoop/hive/metastore/txn/TxnHandler.java | 41 +++++++++++++---------
1 file changed, 24 insertions(+), 17 deletions(-)
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 9abe615..8b05e93 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -368,33 +368,36 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
public void setConf(Configuration conf){
this.conf = conf;
+ int maxPoolSize = MetastoreConf.getIntVar(conf, ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS);
+ long getConnectionTimeoutMs = 30000;
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;
- connPool = setupJdbcConnectionPool(conf, maxPoolSize, getConnectionTimeoutMs);
- /*the mutex pools should ideally be somewhat larger since some operations require 1
+ connPool = setupJdbcConnectionPool(conf, maxPoolSize, getConnectionTimeoutMs);
+ }
+
+ 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
connection from connPool first, then connPoolMutex. All others, go in the opposite
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);
+ connPoolMutex = setupJdbcConnectionPool(conf, maxPoolSize + MUTEX_KEY.values().length, getConnectionTimeoutMs);
+ }
+
+ if (dbProduct == null) {
+ try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) {
determineDatabaseProduct(dbConn);
- sqlGenerator = new SQLGenerator(dbProduct, conf);
} catch (SQLException e) {
- String msg = "Unable to instantiate JDBC connection pooling, " + e.getMessage();
- LOG.error(msg);
+ LOG.error("Unable to determine database product", e);
throw new RuntimeException(e);
- } finally {
- closeDbConn(dbConn);
}
}
+
+ if (sqlGenerator == null) {
+ sqlGenerator = new SQLGenerator(dbProduct, conf);
+ }
}
numOpenTxns = Metrics.getOrCreateGauge(MetricsConstants.NUM_OPEN_TXNS);
@@ -4611,7 +4614,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
private void determineDatabaseProduct(Connection conn) {
- if (dbProduct != null) return;
try {
String s = conn.getMetaData().getDatabaseProductName();
dbProduct = DatabaseProduct.determineDatabaseProduct(s, conf);
@@ -5575,10 +5577,15 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
}
}
- private static synchronized DataSource setupJdbcConnectionPool(Configuration conf, int maxPoolSize, long getConnectionTimeoutMs) throws SQLException {
+ private synchronized static DataSource setupJdbcConnectionPool(Configuration conf, int maxPoolSize, long getConnectionTimeoutMs) {
DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf);
if (dsp != null) {
- return dsp.create(conf);
+ try {
+ return dsp.create(conf);
+ } catch (SQLException e) {
+ LOG.error("Unable to instantiate JDBC connection pooling", e);
+ throw new RuntimeException(e);
+ }
} else {
String connectionPooler = MetastoreConf.getVar(conf, ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
if ("none".equals(connectionPooler)) {