You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/03/02 02:41:16 UTC

hive git commit: HIVE-13002 : Hive object is not thread safe, is shared via a threadlocal and thus should not be passed around too much - part 1 (Sergey Shelukhin, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 86386473c -> 01fd68443


HIVE-13002 : Hive object is not thread safe, is shared via a threadlocal and thus should not be passed around too much - part 1 (Sergey Shelukhin, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 01fd68443d41a95362920c5a526d13e41b7684c8
Parents: 8638647
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Mar 1 17:41:07 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Mar 1 17:41:07 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/ColumnStatsTask.java    | 17 +++--
 .../hive/ql/exec/ColumnStatsUpdateTask.java     | 12 +--
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java | 77 +++++++++++---------
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |  8 +-
 .../hadoop/hive/ql/exec/StatsNoJobTask.java     | 10 ++-
 .../apache/hadoop/hive/ql/exec/StatsTask.java   | 14 ++--
 .../org/apache/hadoop/hive/ql/exec/Task.java    | 12 ++-
 .../ql/hooks/UpdateInputAccessTimeHook.java     | 20 ++---
 .../hive/ql/metadata/PartitionIterable.java     |  2 +-
 .../ql/optimizer/index/RewriteGBUsingIndex.java |  1 +
 .../RewriteQueryUsingAggregateIndexCtx.java     |  1 +
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  1 +
 .../hadoop/hive/ql/parse/IndexUpdater.java      |  1 +
 .../hadoop/hive/ql/parse/TaskCompiler.java      |  1 +
 .../HiveAuthorizationTaskFactoryImpl.java       |  1 +
 .../HiveAuthorizationProviderBase.java          | 16 ++--
 .../authorization/plugin/HiveV1Authorizer.java  | 21 +++++-
 .../service/cli/session/HiveSessionImpl.java    |  2 +-
 18 files changed, 124 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
index f9a9fd2..9059928 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsWork;
@@ -314,7 +315,8 @@ public class ColumnStatsTask extends Task<ColumnStatsWork> implements Serializab
     }
   }
 
-  private List<ColumnStatistics> constructColumnStatsFromPackedRows() throws HiveException, MetaException, IOException {
+  private List<ColumnStatistics> constructColumnStatsFromPackedRows(
+      Hive db) throws HiveException, MetaException, IOException {
 
     String currentDb = SessionState.get().getCurrentDatabase();
     String tableName = work.getColStats().getTableName();
@@ -389,20 +391,20 @@ public class ColumnStatsTask extends Task<ColumnStatsWork> implements Serializab
     return statsDesc;
   }
 
-  private int persistPartitionStats() throws HiveException, MetaException, IOException {
+  private int persistPartitionStats(Hive db) throws HiveException, MetaException, IOException {
 
     // Fetch result of the analyze table partition (p1=c1).. compute statistics for columns ..
     // Construct a column statistics object from the result
-    List<ColumnStatistics> colStats = constructColumnStatsFromPackedRows();
+    List<ColumnStatistics> colStats = constructColumnStatsFromPackedRows(db);
     // Persist the column statistics object to the metastore
     db.setPartitionColumnStatistics(new SetPartitionsStatsRequest(colStats));
     return 0;
   }
 
-  private int persistTableStats() throws HiveException, MetaException, IOException {
+  private int persistTableStats(Hive db) throws HiveException, MetaException, IOException {
     // Fetch result of the analyze table .. compute statistics for columns ..
     // Construct a column statistics object from the result
-    ColumnStatistics colStats = constructColumnStatsFromPackedRows().get(0);
+    ColumnStatistics colStats = constructColumnStatsFromPackedRows(db).get(0);
     // Persist the column statistics object to the metastore
     db.updateTableColumnStatistics(colStats);
     return 0;
@@ -411,10 +413,11 @@ public class ColumnStatsTask extends Task<ColumnStatsWork> implements Serializab
   @Override
   public int execute(DriverContext driverContext) {
     try {
+      Hive db = getHive();
       if (work.getColStats().isTblLevel()) {
-        return persistTableStats();
+        return persistTableStats(db);
       } else {
-        return persistPartitionStats();
+        return persistPartitionStats(db);
       }
     } catch (Exception e) {
       LOG.error("Failed to run column stats task", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
index a1b98f4..bca8a6c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.exec;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -29,7 +28,6 @@ import java.util.Map.Entry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
@@ -47,6 +45,7 @@ import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsDesc;
@@ -288,7 +287,7 @@ public class ColumnStatsUpdateTask extends Task<ColumnStatsUpdateWork> {
     return statsDesc;
   }
 
-  private int persistTableStats() throws HiveException, MetaException,
+  private int persistTableStats(Hive db) throws HiveException, MetaException,
       IOException {
     // Construct a column statistics object from user input
     ColumnStatistics colStats = constructColumnStatsFromInput();
@@ -297,7 +296,7 @@ public class ColumnStatsUpdateTask extends Task<ColumnStatsUpdateWork> {
     return 0;
   }
 
-  private int persistPartitionStats() throws HiveException, MetaException,
+  private int persistPartitionStats(Hive db) throws HiveException, MetaException,
       IOException {
     // Construct a column statistics object from user input
     ColumnStatistics colStats = constructColumnStatsFromInput();
@@ -309,10 +308,11 @@ public class ColumnStatsUpdateTask extends Task<ColumnStatsUpdateWork> {
   @Override
   public int execute(DriverContext driverContext) {
     try {
+      Hive db = getHive();
       if (work.getColStats().isTblLevel()) {
-        return persistTableStats();
+        return persistTableStats(db);
       } else {
-        return persistPartitionStats();
+        return persistPartitionStats(db);
       }
     } catch (Exception e) {
       LOG.info("Failed to persist stats in metastore", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 6fca9f7..c51cfd6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -293,12 +293,12 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       LockDatabaseDesc lockDatabaseDesc = work.getLockDatabaseDesc();
       if (lockDatabaseDesc != null) {
-        return lockDatabase(lockDatabaseDesc);
+        return lockDatabase(db, lockDatabaseDesc);
       }
 
       UnlockDatabaseDesc unlockDatabaseDesc = work.getUnlockDatabaseDesc();
       if (unlockDatabaseDesc != null) {
-        return unlockDatabase(unlockDatabaseDesc);
+        return unlockDatabase(db, unlockDatabaseDesc);
       }
 
       SwitchDatabaseDesc switchDatabaseDesc = work.getSwitchDatabaseDesc();
@@ -308,12 +308,12 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       DescDatabaseDesc descDatabaseDesc = work.getDescDatabaseDesc();
       if (descDatabaseDesc != null) {
-        return descDatabase(descDatabaseDesc);
+        return descDatabase(db, descDatabaseDesc);
       }
 
       AlterDatabaseDesc alterDatabaseDesc = work.getAlterDatabaseDesc();
       if (alterDatabaseDesc != null) {
-        return alterDatabase(alterDatabaseDesc);
+        return alterDatabase(db, alterDatabaseDesc);
       }
 
       CreateTableDesc crtTbl = work.getCreateTblDesc();
@@ -392,7 +392,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       DescFunctionDesc descFunc = work.getDescFunctionDesc();
       if (descFunc != null) {
-        return describeFunction(descFunc);
+        return describeFunction(db, descFunc);
       }
 
       ShowDatabasesDesc showDatabases = work.getShowDatabasesDesc();
@@ -422,32 +422,32 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       ShowFunctionsDesc showFuncs = work.getShowFuncsDesc();
       if (showFuncs != null) {
-        return showFunctions(showFuncs);
+        return showFunctions(db, showFuncs);
       }
 
       ShowLocksDesc showLocks = work.getShowLocksDesc();
       if (showLocks != null) {
-        return showLocks(showLocks);
+        return showLocks(db, showLocks);
       }
 
       ShowCompactionsDesc compactionsDesc = work.getShowCompactionsDesc();
       if (compactionsDesc != null) {
-        return showCompactions(compactionsDesc);
+        return showCompactions(db, compactionsDesc);
       }
 
       ShowTxnsDesc txnsDesc = work.getShowTxnsDesc();
       if (txnsDesc != null) {
-        return showTxns(txnsDesc);
+        return showTxns(db, txnsDesc);
       }
 
        LockTableDesc lockTbl = work.getLockTblDesc();
       if (lockTbl != null) {
-        return lockTable(lockTbl);
+        return lockTable(db, lockTbl);
       }
 
       UnlockTableDesc unlockTbl = work.getUnlockTblDesc();
       if (unlockTbl != null) {
-        return unlockTable(unlockTbl);
+        return unlockTable(db, unlockTbl);
       }
 
       ShowPartitionsDesc showParts = work.getShowPartsDesc();
@@ -472,31 +472,31 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       RoleDDLDesc roleDDLDesc = work.getRoleDDLDesc();
       if (roleDDLDesc != null) {
-        return roleDDL(roleDDLDesc);
+        return roleDDL(db, roleDDLDesc);
       }
 
       GrantDesc grantDesc = work.getGrantDesc();
       if (grantDesc != null) {
-        return grantOrRevokePrivileges(grantDesc.getPrincipals(), grantDesc
+        return grantOrRevokePrivileges(db, grantDesc.getPrincipals(), grantDesc
             .getPrivileges(), grantDesc.getPrivilegeSubjectDesc(), grantDesc.getGrantor(),
             grantDesc.getGrantorType(), grantDesc.isGrantOption(), true);
       }
 
       RevokeDesc revokeDesc = work.getRevokeDesc();
       if (revokeDesc != null) {
-        return grantOrRevokePrivileges(revokeDesc.getPrincipals(), revokeDesc
+        return grantOrRevokePrivileges(db, revokeDesc.getPrincipals(), revokeDesc
             .getPrivileges(), revokeDesc.getPrivilegeSubjectDesc(), null, null,
             revokeDesc.isGrantOption(), false);
       }
 
       ShowGrantDesc showGrantDesc = work.getShowGrantDesc();
       if (showGrantDesc != null) {
-        return showGrants(showGrantDesc);
+        return showGrants(db, showGrantDesc);
       }
 
       GrantRevokeRoleDDL grantOrRevokeRoleDDL = work.getGrantRevokeRoleDDL();
       if (grantOrRevokeRoleDDL != null) {
-        return grantOrRevokeRole(grantOrRevokeRoleDDL);
+        return grantOrRevokeRole(db, grantOrRevokeRoleDDL);
       }
 
       ShowIndexesDesc showIndexes = work.getShowIndexesDesc();
@@ -667,7 +667,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return ret;
   }
 
-  private HiveAuthorizer getSessionAuthorizer() {
+  private HiveAuthorizer getSessionAuthorizer(Hive db) {
     HiveAuthorizer authorizer = SessionState.get().getAuthorizerV2();
     if (authorizer == null) {
       authorizer = new HiveV1Authorizer(conf, db);
@@ -675,9 +675,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return authorizer;
   }
 
-  private int grantOrRevokeRole(GrantRevokeRoleDDL grantOrRevokeRoleDDL)
+  private int grantOrRevokeRole(Hive db, GrantRevokeRoleDDL grantOrRevokeRoleDDL)
       throws HiveException {
-    HiveAuthorizer authorizer = getSessionAuthorizer();
+    HiveAuthorizer authorizer = getSessionAuthorizer(db);
     //convert to the types needed for plugin api
     HivePrincipal grantorPrinc = null;
     if(grantOrRevokeRoleDDL.getGrantor() != null){
@@ -706,9 +706,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
   }
 
-  private int showGrants(ShowGrantDesc showGrantDesc) throws HiveException {
+  private int showGrants(Hive db, ShowGrantDesc showGrantDesc) throws HiveException {
 
-    HiveAuthorizer authorizer = getSessionAuthorizer();
+    HiveAuthorizer authorizer = getSessionAuthorizer(db);
     try {
       List<HivePrivilegeInfo> privInfos = authorizer.showPrivileges(
           getAuthorizationTranslator(authorizer).getHivePrincipal(showGrantDesc.getPrincipalDesc()),
@@ -721,12 +721,12 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private int grantOrRevokePrivileges(List<PrincipalDesc> principals,
+  private int grantOrRevokePrivileges(Hive db, List<PrincipalDesc> principals,
       List<PrivilegeDesc> privileges, PrivilegeObjectDesc privSubjectDesc,
       String grantor, PrincipalType grantorType, boolean grantOption, boolean isGrant)
           throws HiveException {
 
-    HiveAuthorizer authorizer = getSessionAuthorizer();
+    HiveAuthorizer authorizer = getSessionAuthorizer(db);
 
     //Convert to object types used by the authorization plugin interface
     List<HivePrincipal> hivePrincipals = AuthorizationUtils.getHivePrincipals(
@@ -750,8 +750,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private int roleDDL(RoleDDLDesc roleDDLDesc) throws Exception {
-    HiveAuthorizer authorizer = getSessionAuthorizer();
+  private int roleDDL(Hive db, RoleDDLDesc roleDDLDesc) throws Exception {
+    HiveAuthorizer authorizer = getSessionAuthorizer(db);
     RoleDDLDesc.RoleOperation operation = roleDDLDesc.getOperation();
     //call the appropriate hive authorizer function
     switch(operation){
@@ -826,7 +826,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     writeToFile(sb.toString(), resFile);
   }
 
-  private int alterDatabase(AlterDatabaseDesc alterDbDesc) throws HiveException {
+  private int alterDatabase(Hive db, AlterDatabaseDesc alterDbDesc) throws HiveException {
 
     String dbName = alterDbDesc.getDatabaseName();
     Database database = db.getDatabase(dbName);
@@ -2380,6 +2380,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
   /**
    * Write a list of the user defined functions to a file.
+   * @param db 
    *
    * @param showFuncs
    *          are the functions we're interested in.
@@ -2387,7 +2388,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int showFunctions(ShowFunctionsDesc showFuncs) throws HiveException {
+  private int showFunctions(Hive db, ShowFunctionsDesc showFuncs) throws HiveException {
     // get the tables for the desired patten - populate the output stream
     Set<String> funcs = null;
     if (showFuncs.getPattern() != null) {
@@ -2432,6 +2433,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
   /**
    * Write a list of the current locks to a file.
+   * @param db 
    *
    * @param showLocks
    *          the locks we're interested in.
@@ -2439,7 +2441,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int showLocks(ShowLocksDesc showLocks) throws HiveException {
+  private int showLocks(Hive db, ShowLocksDesc showLocks) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
     HiveLockManager lockMgr = txnManager.getLockManager();
@@ -2605,7 +2607,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private int showCompactions(ShowCompactionsDesc desc) throws HiveException {
+  private int showCompactions(Hive db, ShowCompactionsDesc desc) throws HiveException {
     // Call the metastore to get the currently queued and running compactions.
     ShowCompactResponse rsp = db.showCompactions();
 
@@ -2657,7 +2659,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private int showTxns(ShowTxnsDesc desc) throws HiveException {
+  private int showTxns(Hive db, ShowTxnsDesc desc) throws HiveException {
     // Call the metastore to get the currently queued and running compactions.
     GetOpenTxnsInfoResponse rsp = db.showTransactions();
 
@@ -2695,6 +2697,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
    /**
    * Lock the table/partition specified
+   * @param db 
    *
    * @param lockTbl
    *          the table/partition to be locked along with the mode
@@ -2702,7 +2705,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int lockTable(LockTableDesc lockTbl) throws HiveException {
+  private int lockTable(Hive db, LockTableDesc lockTbl) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
     return txnManager.lockTable(db, lockTbl);
@@ -2717,7 +2720,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int lockDatabase(LockDatabaseDesc lockDb) throws HiveException {
+  private int lockDatabase(Hive db, LockDatabaseDesc lockDb) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
     return txnManager.lockDatabase(db, lockDb);
@@ -2732,7 +2735,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int unlockDatabase(UnlockDatabaseDesc unlockDb) throws HiveException {
+  private int unlockDatabase(Hive db, UnlockDatabaseDesc unlockDb) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
     return txnManager.unlockDatabase(db, unlockDb);
@@ -2740,6 +2743,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
   /**
    * Unlock the table/partition specified
+   * @param db 
    *
    * @param unlockTbl
    *          the table/partition to be unlocked
@@ -2747,7 +2751,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int unlockTable(UnlockTableDesc unlockTbl) throws HiveException {
+  private int unlockTable(Hive db, UnlockTableDesc unlockTbl) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
     return txnManager.unlockTable(db, unlockTbl);
@@ -2755,12 +2759,13 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
   /**
    * Shows a description of a function.
+   * @param db 
    *
    * @param descFunc
    *          is the function we are describing
    * @throws HiveException
    */
-  private int describeFunction(DescFunctionDesc descFunc) throws HiveException, SQLException {
+  private int describeFunction(Hive db, DescFunctionDesc descFunc) throws HiveException, SQLException {
     String funcName = descFunc.getName();
 
     // write the results in the file
@@ -2812,7 +2817,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private int descDatabase(DescDatabaseDesc descDatabase) throws HiveException {
+  private int descDatabase(Hive db, DescDatabaseDesc descDatabase) throws HiveException {
     DataOutputStream outStream = getOutputStream(descDatabase.getResFile());
     try {
       Database database = db.getDatabase(descDatabase.getDatabaseName());

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index 27e7b43..074da6b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -231,6 +231,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
   public int execute(DriverContext driverContext) {
 
     try {
+      Hive db = getHive();
 
       // Do any hive related operations like moving tables and files
       // to appropriate locations
@@ -441,7 +442,8 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
               Partition partn = entry.getValue();
 
               if (bucketCols != null || sortCols != null) {
-                updatePartitionBucketSortColumns(table, partn, bucketCols, numBuckets, sortCols);
+                updatePartitionBucketSortColumns(
+                    db, table, partn, bucketCols, numBuckets, sortCols);
               }
 
               WriteEntity enty = new WriteEntity(partn,
@@ -486,7 +488,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
             Partition partn = db.getPartition(table, tbd.getPartitionSpec(), false);
 
             if (bucketCols != null || sortCols != null) {
-              updatePartitionBucketSortColumns(table, partn, bucketCols,
+              updatePartitionBucketSortColumns(db, table, partn, bucketCols,
                   numBuckets, sortCols);
             }
 
@@ -548,7 +550,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
    * @throws InvalidOperationException
    * @throws HiveException
    */
-  private void updatePartitionBucketSortColumns(Table table, Partition partn,
+  private void updatePartitionBucketSortColumns(Hive db, Table table, Partition partn,
       List<BucketCol> bucketCols, int numBuckets, List<SortCol> sortCols)
           throws IOException, InvalidOperationException, HiveException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
index 175dbdb..3199ee1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.io.StatsProvidingRecordReader;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -96,6 +97,7 @@ public class StatsNoJobTask extends Task<StatsNoJobWork> implements Serializable
 
     String tableName = "";
     ExecutorService threadPool = null;
+    Hive db = getHive();
     try {
       tableName = work.getTableSpecs().tableName;
       table = db.getTable(tableName);
@@ -111,7 +113,7 @@ public class StatsNoJobTask extends Task<StatsNoJobWork> implements Serializable
       console.printError("Cannot get table " + tableName, e.toString());
     }
 
-    return aggregateStats(threadPool);
+    return aggregateStats(threadPool, db);
   }
 
   @Override
@@ -219,7 +221,7 @@ public class StatsNoJobTask extends Task<StatsNoJobWork> implements Serializable
 
   }
 
-  private int aggregateStats(ExecutorService threadPool) {
+  private int aggregateStats(ExecutorService threadPool, Hive db) {
     int ret = 0;
 
     try {
@@ -302,7 +304,7 @@ public class StatsNoJobTask extends Task<StatsNoJobWork> implements Serializable
         shutdownAndAwaitTermination(threadPool);
         LOG.debug("Stats collection threadpool shutdown successful.");
 
-        ret = updatePartitions();
+        ret = updatePartitions(db);
       }
 
     } catch (Exception e) {
@@ -317,7 +319,7 @@ public class StatsNoJobTask extends Task<StatsNoJobWork> implements Serializable
     return ret;
   }
 
-  private int updatePartitions() throws InvalidOperationException, HiveException {
+  private int updatePartitions(Hive db) throws InvalidOperationException, HiveException {
     if (!partUpdates.isEmpty()) {
       List<Partition> updatedParts = Lists.newArrayList(partUpdates.values());
       if (updatedParts.contains(null) && work.isStatsReliable()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
index 28bc9a7..87a7667 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -102,6 +103,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
     assert (workComponentsPresent == 1);
 
     String tableName = "";
+    Hive hive = getHive();
     try {
       if (work.getLoadTableDesc() != null) {
         tableName = work.getLoadTableDesc().getTable().getTableName();
@@ -111,14 +113,14 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
         tableName = work.getLoadFileDesc().getDestinationCreateTable();
       }
 
-      table = db.getTable(tableName);
+      table = hive.getTable(tableName);
 
     } catch (HiveException e) {
       LOG.error("Cannot get table " + tableName, e);
       console.printError("Cannot get table " + tableName, e.toString());
     }
 
-    return aggregateStats();
+    return aggregateStats(hive);
 
   }
 
@@ -132,7 +134,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
     return "STATS";
   }
 
-  private int aggregateStats() {
+  private int aggregateStats(Hive db) {
 
     StatsAggregator statsAggregator = null;
     int ret = 0;
@@ -153,7 +155,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
         }
       }
 
-      List<Partition> partitions = getPartitionsList();
+      List<Partition> partitions = getPartitionsList(db);
       boolean atomic = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_ATOMIC);
 
       String tableFullName = table.getDbName() + "." + table.getTableName();
@@ -185,7 +187,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
           environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED, StatsSetupConst.TASK);
         }
 
-        db.alterTable(tableFullName, new Table(tTable), environmentContext);
+        getHive().alterTable(tableFullName, new Table(tTable), environmentContext);
         if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) {
           console.printInfo("Table " + tableFullName + " stats: [" + toString(parameters) + ']');
         }
@@ -374,7 +376,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
    * @return a list of partitions that need to update statistics.
    * @throws HiveException
    */
-  private List<Partition> getPartitionsList() throws HiveException {
+  private List<Partition> getPartitionsList(Hive db) throws HiveException {
     if (work.getLoadFileDesc() != null) {
       return null; //we are in CTAS, so we know there are no partitions
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
index f2b15c5..e199e5e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
@@ -57,7 +57,6 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
   protected transient boolean isdone;
   protected transient boolean queued;
   protected transient HiveConf conf;
-  protected transient Hive db;
   protected transient LogHelper console;
   protected transient QueryPlan queryPlan;
   protected transient DriverContext driverContext;
@@ -128,18 +127,17 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
     started = false;
     setInitialized();
     this.conf = conf;
+    this.driverContext = driverContext;
+    console = new LogHelper(LOG);
+  }
 
+  protected Hive getHive() {
     try {
-      db = Hive.get(conf);
+      return Hive.getWithFastCheck(conf);
     } catch (HiveException e) {
-      // Bail out ungracefully - we should never hit
-      // this here - but would have hit it in SemanticAnalyzer
       LOG.error(StringUtils.stringifyException(e));
       throw new RuntimeException(e);
     }
-    this.driverContext = driverContext;
-
-    console = new LogHelper(LOG);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
index 48f3b28..8a9a551 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
@@ -18,10 +18,7 @@
 package org.apache.hadoop.hive.ql.hooks;
 
 import java.util.Set;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -38,20 +35,17 @@ public class UpdateInputAccessTimeHook {
   private static final String LAST_ACCESS_TIME = "lastAccessTime";
 
   public static class PreExec implements PreExecute {
-    Hive db;
-
     public void run(SessionState sess, Set<ReadEntity> inputs,
                     Set<WriteEntity> outputs, UserGroupInformation ugi)
       throws Exception {
 
-      if (db == null) {
-        try {
-          db = Hive.get(sess.getConf());
-        } catch (HiveException e) {
-          // ignore
-          db = null;
-          return;
-        }
+      Hive db;
+      try {
+        db = Hive.get(sess.getConf());
+      } catch (HiveException e) {
+        // ignore
+        db = null;
+        return;
       }
 
       int lastAccessTime = (int) (System.currentTimeMillis()/1000);

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
index b956987..25a7b57 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
@@ -123,7 +123,7 @@ public class PartitionIterable implements Iterable<Partition> {
   private List<Partition> ptnsProvided = null;
 
   // used for LAZY_FETCH_PARTITIONS cases
-  private Hive db = null;
+  private Hive db = null; // Assumes one instance of this + single-threaded compilation for each query.
   private Table table = null;
   private Map<String, String> partialPartitionSpec = null;
   private List<String> partitionNames = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java
index 9f8fb03..d709e21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 
 public class RewriteGBUsingIndex extends Transform {
   private ParseContext parseContext;
+  // Assumes one instance of this + single-threaded compilation for each query.
   private Hive hiveDb;
   private HiveConf hiveConf;
   private static final Logger LOG = LoggerFactory.getLogger(RewriteGBUsingIndex.class.getName());

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
index a8ba4d7..74bedcb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
@@ -81,6 +81,7 @@ public final class RewriteQueryUsingAggregateIndexCtx  implements NodeProcessorC
         parseContext, hiveDb, canApplyCtx);
   }
 
+  // Assumes one instance of this + single-threaded compilation for each query.
   private final Hive hiveDb;
   private final ParseContext parseContext;
   private final RewriteCanApplyCtx canApplyCtx;

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index 28c8fdb..6523288 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -84,6 +84,7 @@ import com.google.common.annotations.VisibleForTesting;
  */
 public abstract class BaseSemanticAnalyzer {
   protected static final Logger STATIC_LOG = LoggerFactory.getLogger(BaseSemanticAnalyzer.class.getName());
+  // Assumes one instance of this + single-threaded compilation for each query.
   protected final Hive db;
   protected final HiveConf conf;
   protected List<Task<? extends Serializable>> rootTasks;

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
index f323b05..653b657 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
@@ -43,6 +43,7 @@ import java.util.Set;
 public class IndexUpdater {
   private List<LoadTableDesc> loadTableWork;
   private HiveConf conf;
+  // Assumes one instance of this + single-threaded compilation for each query.
   private Hive hive;
   private List<Task<? extends Serializable>> tasks;
   private Set<ReadEntity> inputs;

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index 7415078..8e64a0b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -71,6 +71,7 @@ public abstract class TaskCompiler {
 
   protected final Logger LOG = LoggerFactory.getLogger(TaskCompiler.class);
 
+  // Assumes one instance of this + single-threaded compilation for each query.
   protected Hive db;
   protected LogHelper console;
   protected HiveConf conf;

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
index dc34507..706459a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFactory {
 
   private final HiveConf conf;
+  // Assumes one instance of this + single-threaded compilation for each query.
   private final Hive db;
 
   public HiveAuthorizationTaskFactoryImpl(HiveConf conf, Hive db) {

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
index a1299a4..191426c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
@@ -40,16 +41,19 @@ public abstract class HiveAuthorizationProviderBase implements
 
   protected class HiveProxy {
 
-    private final Hive hiveClient;
+    private final boolean hasHiveClient;
+    private final HiveConf conf;
     private HMSHandler handler;
 
     public HiveProxy(Hive hive) {
-      this.hiveClient = hive;
+      this.hasHiveClient = hive != null;
+      this.conf = hive.getConf();
       this.handler = null;
     }
 
     public HiveProxy() {
-      this.hiveClient = null;
+      this.hasHiveClient = false;
+      this.conf = null;
       this.handler = null;
     }
 
@@ -58,14 +62,14 @@ public abstract class HiveAuthorizationProviderBase implements
     }
 
     public boolean isRunFromMetaStore(){
-      return (this.hiveClient == null);
+      return !hasHiveClient;
     }
 
     public PrincipalPrivilegeSet get_privilege_set(HiveObjectType column, String dbName,
         String tableName, List<String> partValues, String col, String userName,
         List<String> groupNames) throws HiveException {
       if (!isRunFromMetaStore()) {
-        return hiveClient.get_privilege_set(
+        return Hive.getWithFastCheck(conf).get_privilege_set(
             column, dbName, tableName, partValues, col, userName, groupNames);
       } else {
         HiveObjectRef hiveObj = new HiveObjectRef(column, dbName,
@@ -82,7 +86,7 @@ public abstract class HiveAuthorizationProviderBase implements
 
     public Database getDatabase(String dbName) throws HiveException {
       if (!isRunFromMetaStore()) {
-        return hiveClient.getDatabase(dbName);
+        return Hive.getWithFastCheck(conf).getDatabase(dbName);
       } else {
         try {
           return handler.get_database_core(dbName);

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveV1Authorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveV1Authorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveV1Authorizer.java
index 8e60757..8a03989 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveV1Authorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveV1Authorizer.java
@@ -45,11 +45,16 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 public class HiveV1Authorizer extends AbstractHiveAuthorizer {
 
   private final HiveConf conf;
-  private final Hive hive;
 
-  public HiveV1Authorizer(HiveConf conf, Hive hive) {
+  public HiveV1Authorizer(HiveConf conf) {
     this.conf = conf;
-    this.hive = hive;
+  }
+
+
+  // Leave this ctor around for backward compat.
+  @Deprecated
+  public HiveV1Authorizer(HiveConf conf, Hive hive) {
+    this(conf);
   }
 
   @Override
@@ -98,6 +103,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
         priv.setPrincipalName(principal.getName());
         priv.setPrincipalType(type);
       }
+      Hive hive = Hive.getWithFastCheck(this.conf);
       if (isGrant) {
         hive.grantPrivileges(privBag);
       } else {
@@ -136,6 +142,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
     if (privObject.getPartKeys() != null && grantOption) {
       throw new HiveException("Grant does not support partition level.");
     }
+    Hive hive = Hive.getWithFastCheck(this.conf);
     Database dbObj = hive.getDatabase(privObject.getDbname());
     if (dbObj == null) {
       throw new HiveException("Database " + privObject.getDbname() + " does not exists");
@@ -202,6 +209,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
   @Override
   public void createRole(String roleName, HivePrincipal adminGrantor) throws HiveAuthzPluginException, HiveAccessControlException {
     try {
+      Hive hive = Hive.getWithFastCheck(this.conf);
       hive.createRole(roleName, adminGrantor == null ? null : adminGrantor.getName());
     } catch (HiveException e) {
       throw new HiveAuthzPluginException(e);
@@ -211,6 +219,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
   @Override
   public void dropRole(String roleName) throws HiveAuthzPluginException, HiveAccessControlException {
     try {
+      Hive hive = Hive.getWithFastCheck(this.conf);
       hive.dropRole(roleName);
     } catch (HiveException e) {
       throw new HiveAuthzPluginException(e);
@@ -220,6 +229,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
   @Override
   public List<HiveRoleGrant> getPrincipalGrantInfoForRole(String roleName) throws HiveAuthzPluginException, HiveAccessControlException {
     try {
+      Hive hive = Hive.getWithFastCheck(this.conf);
       return SQLStdHiveAccessController.getHiveRoleGrants(hive.getMSC(), roleName);
     } catch (Exception e) {
       throw new HiveAuthzPluginException(e);
@@ -231,6 +241,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
     PrincipalType type = AuthorizationUtils.getThriftPrincipalType(principal.getType());
     try {
       List<HiveRoleGrant> grants = new ArrayList<HiveRoleGrant>();
+      Hive hive = Hive.getWithFastCheck(this.conf);
       for (RolePrincipalGrant grant : hive.getRoleGrantInfoForPrincipal(principal.getName(), type)) {
         grants.add(new HiveRoleGrant(grant));
       }
@@ -263,6 +274,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
   private void grantOrRevokeRole(List<HivePrincipal> principals, List<String> roles,
       boolean grantOption, HivePrincipal grantor, boolean isGrant) throws HiveException {
     PrincipalType grantorType = AuthorizationUtils.getThriftPrincipalType(grantor.getType());
+    Hive hive = Hive.getWithFastCheck(this.conf);
     for (HivePrincipal principal : principals) {
       PrincipalType principalType = AuthorizationUtils.getThriftPrincipalType(principal.getType());
       String userName = principal.getName();
@@ -280,6 +292,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
   @Override
   public List<String> getAllRoles() throws HiveAuthzPluginException, HiveAccessControlException {
     try {
+      Hive hive = Hive.getWithFastCheck(this.conf);
       return hive.getAllRoleNames();
     } catch (HiveException e) {
       throw new HiveAuthzPluginException(e);
@@ -295,6 +308,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
 
     List<HiveObjectPrivilege> privs = new ArrayList<HiveObjectPrivilege>();
     try {
+      Hive hive = Hive.getWithFastCheck(this.conf);
       if (privObj == null) {
         // show user level privileges
         privs.addAll(hive.showPrivilegeGrant(HiveObjectType.GLOBAL, name, type,
@@ -358,6 +372,7 @@ public class HiveV1Authorizer extends AbstractHiveAuthorizer {
       throw new HiveAuthzPluginException("Cannot resolve current user name");
     }
     try {
+      Hive hive = Hive.getWithFastCheck(this.conf);
       List<String> roleNames = new ArrayList<String>();
       for (Role role : hive.listRoles(userName, PrincipalType.USER)) {
         roleNames.add(role.getRoleName());

http://git-wip-us.apache.org/repos/asf/hive/blob/01fd6844/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
index 4aa1875..1ddf0a4 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
@@ -105,7 +105,7 @@ public class HiveSessionImpl implements HiveSession {
   private final Set<OperationHandle> opHandleSet = new HashSet<OperationHandle>();
   private boolean isOperationLogEnabled;
   private File sessionLogDir;
-
+  // TODO: the control flow for this needs to be defined. Hive is supposed to be thread-local.
   private Hive sessionHive;
 
   private volatile long lastAccessTime;