Repository: hive
Updated Branches:
  refs/heads/branch-2.3 ee57fa1c5 -> 6cdfaaedd


HIVE-16321 Possible deadlock in metastore with Acid enabled (Eugene Koifman, 
reviewed by Wei Zheng)


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

Branch: refs/heads/branch-2.3
Commit: 6cdfaaedd7d2b22fe32fa0f6686d3533012bf982
Parents: ee57fa1
Author: Eugene Koifman <ekoif...@hortonworks.com>
Authored: Sat Apr 22 15:33:26 2017 -0700
Committer: Eugene Koifman <ekoif...@hortonworks.com>
Committed: Sat Apr 22 15:33:26 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  7 ++
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 92 ++++++++++++--------
 2 files changed, 62 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6cdfaaed/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 08e7617..ebbb8d9 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -737,6 +737,13 @@ public class HiveConf extends Configuration {
     METASTORE_CONNECTION_POOLING_TYPE("datanucleus.connectionPoolingType", 
"BONECP", new StringSet("BONECP", "DBCP",
       "HikariCP", "NONE"),
         "Specify connection pool library for datanucleus"),
+    
METASTORE_CONNECTION_POOLING_MAX_CONNECTIONS("datanucleus.connectionPool.maxPoolSize",
 10,
+      "Specify the maximum number of connections in the connection pool. Note: 
The configured size will be used by\n" +
+        " 2 connection pools (TxnHandler and ObjectStore). When configuring 
the max connection pool size, it is \n" +
+        "recommended to take into account the number of metastore instances 
and the number of HiveServer2 instances \n" +
+        "configured with embedded metastore. To get optimal performance, set 
config to meet the following condition\n"+
+        "(2 * pool_size * metastore_instances + 2 * pool_size * 
HS2_instances_with_embedded_metastore) = \n" +
+        "(2 * physical_core_count + hard_disk_count)."),
     // Workaround for DN bug on Postgres:
     // http://www.datanucleus.org/servlet/forum/viewthread_thread,7985_offset
     
METASTORE_DATANUCLEUS_INIT_COL_INFO("datanucleus.rdbms.initializeColumnInfo", 
"NONE",

http://git-wip-us.apache.org/repos/asf/hive/blob/6cdfaaed/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java 
b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index d378d06..0e8253b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -147,6 +147,7 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
   static final private Logger LOG = 
LoggerFactory.getLogger(TxnHandler.class.getName());
 
   static private DataSource connPool;
+  private static DataSource connPoolMutex;
   static private boolean doRetryOnConnPool = false;
   
   private enum OpertaionType {
@@ -203,8 +204,8 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
   private int deadlockCnt;
   private long deadlockRetryInterval;
   protected HiveConf conf;
-  protected DatabaseProduct dbProduct;
-  private SQLGenerator sqlGenerator;
+  private static DatabaseProduct dbProduct;
+  private static SQLGenerator sqlGenerator;
 
   // (End user) Transaction timeout, in milliseconds.
   private long timeout;
@@ -223,7 +224,6 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
    */
   private final static ConcurrentHashMap<String, Semaphore> derbyKey2Lock = 
new ConcurrentHashMap<>();
   private static final String hostname = ServerUtils.hostname();
-  private static volatile boolean dumpConfig = true;
 
   // Private methods should never catch SQLException and then throw 
MetaException.  The public
   // methods depend on SQLException coming back so they can detect and handle 
deadlocks.  Private
@@ -247,20 +247,36 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
 
     checkQFileTestHack();
 
-    Connection dbConn = null;
-    // Set up the JDBC connection pool
-    try {
-      setupJdbcConnectionPool(conf);
-      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);
-      throw new RuntimeException(e);
-    }
-    finally {
-      closeDbConn(dbConn);
+    synchronized (TxnHandler.class) {
+      if (connPool == null) {
+        //only do this once per JVM; useful for support
+        LOG.info(HiveConfUtil.dumpConfig(conf).toString());
+
+        Connection dbConn = null;
+        // Set up the JDBC connection pool
+        try {
+          int maxPoolSize = 
conf.getIntVar(HiveConf.ConfVars.METASTORE_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
+           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);
+          determineDatabaseProduct(dbConn);
+          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);
+        }
+      }
     }
 
     timeout = HiveConf.getTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 
TimeUnit.MILLISECONDS);
@@ -270,11 +286,6 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
     retryLimit = HiveConf.getIntVar(conf, 
HiveConf.ConfVars.HMSHANDLERATTEMPTS);
     deadlockRetryInterval = retryInterval / 10;
     maxOpenTxns = HiveConf.getIntVar(conf, 
HiveConf.ConfVars.HIVE_MAX_OPEN_TXNS);
-    if(dumpConfig) {
-      LOG.info(HiveConfUtil.dumpConfig(conf).toString());
-      //only do this once per JVM; useful for support
-      dumpConfig = false;
-    }
   }
   @Override
   @RetrySemantics.ReadOnly
@@ -1934,7 +1945,10 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
 
   }
 
-  protected Connection getDbConn(int isolationLevel) throws SQLException {
+  Connection getDbConn(int isolationLevel) throws SQLException {
+    return getDbConn(isolationLevel, connPool);
+  }
+  private Connection getDbConn(int isolationLevel, DataSource connPool) throws 
SQLException {
     int rc = doRetryOnConnPool ? 10 : 1;
     Connection dbConn = null;
     while (true) {
@@ -3131,9 +3145,7 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
     }
   }
 
-  private static synchronized void setupJdbcConnectionPool(HiveConf conf) 
throws SQLException {
-    if (connPool != null) return;
-
+  private static synchronized DataSource setupJdbcConnectionPool(HiveConf 
conf, int maxPoolSize, long getConnectionTimeoutMs) throws SQLException {
     String driverUrl = HiveConf.getVar(conf, 
HiveConf.ConfVars.METASTORECONNECTURLKEY);
     String user = getMetastoreJdbcUser(conf);
     String passwd = getMetastoreJdbcPasswd(conf);
@@ -3143,38 +3155,44 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
     if ("bonecp".equals(connectionPooler)) {
       BoneCPConfig config = new BoneCPConfig();
       config.setJdbcUrl(driverUrl);
-      //if we are waiting for connection for 60s, something is really wrong
+      //if we are waiting for connection for a long time, something is really 
wrong
       //better raise an error than hang forever
-      config.setConnectionTimeoutInMs(60000);
-      config.setMaxConnectionsPerPartition(10);
+      //see DefaultConnectionStrategy.getConnectionInternal()
+      config.setConnectionTimeoutInMs(getConnectionTimeoutMs);
+      config.setMaxConnectionsPerPartition(maxPoolSize);
       config.setPartitionCount(1);
       config.setUser(user);
       config.setPassword(passwd);
-      connPool = new BoneCPDataSource(config);
       doRetryOnConnPool = true;  // Enable retries to work around BONECP bug.
+      return new BoneCPDataSource(config);
     } else if ("dbcp".equals(connectionPooler)) {
-      ObjectPool objectPool = new GenericObjectPool();
+      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);
-      connPool = new PoolingDataSource(objectPool);
+        new PoolableConnectionFactory(connFactory, objectPool, null, null, 
false, true);
+      return new PoolingDataSource(objectPool);
     } else if ("hikaricp".equals(connectionPooler)) {
       HikariConfig config = new HikariConfig();
+      config.setMaximumPoolSize(maxPoolSize);
       config.setJdbcUrl(driverUrl);
       config.setUsername(user);
       config.setPassword(passwd);
+      //https://github.com/brettwooldridge/HikariCP
+      config.setConnectionTimeout(getConnectionTimeoutMs);
 
-      connPool = new HikariDataSource(config);
+      return new HikariDataSource(config);
     } else if ("none".equals(connectionPooler)) {
       LOG.info("Choosing not to pool JDBC connections");
-      connPool = new NoPoolConnectionPool(conf);
+      return new NoPoolConnectionPool(conf);
     } else {
       throw new RuntimeException("Unknown JDBC connection pooling " + 
connectionPooler);
     }
   }
-
   private static synchronized void buildJumpTable() {
     if (jumpTable != null) return;
 
@@ -3427,7 +3445,7 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
       try {
         String sqlStmt = sqlGenerator.addForUpdateClause("select MT_COMMENT 
from AUX_TABLE where MT_KEY1=" + quoteString(key) + " and MT_KEY2=0");
         lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, 
connPoolMutex);
         stmt = dbConn.createStatement();
         if(LOG.isDebugEnabled()) {
           LOG.debug("About to execute SQL: " + sqlStmt);

Reply via email to