You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2013/09/13 23:02:13 UTC

svn commit: r1523102 - in /hive/branches/branch-0.12: cli/src/java/org/apache/hadoop/hive/cli/ cli/src/test/org/apache/hadoop/hive/cli/ hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/ hcatalog/core/src/main/java/org/apache/hive/hc...

Author: thejas
Date: Fri Sep 13 21:02:13 2013
New Revision: 1523102

URL: http://svn.apache.org/r1523102
Log:
HIVE-4171 : Current database in metastore.Hive is not consistent with SessionState (Thejas Nair via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/session/
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java
Modified:
    hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
    hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliSessionState.java
    hive/branches/branch-0.12/cli/src/test/org/apache/hadoop/hive/cli/TestCliSessionState.java
    hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
    hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java
    hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
    hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java

Modified: hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Fri Sep 13 21:02:13 2013
@@ -806,7 +806,8 @@ public class CliDriver {
     if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIPRINTCURRENTDB)) {
       return "";
     }
-    String currDb = ss.getCurrentDbName();
+    //BUG: This will not work in remote mode - HIVE-5153
+    String currDb = SessionState.get().getCurrentDatabase();
 
     if (currDb == null) {
       return "";

Modified: hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliSessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliSessionState.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliSessionState.java (original)
+++ hive/branches/branch-0.12/cli/src/java/org/apache/hadoop/hive/cli/CliSessionState.java Fri Sep 13 21:02:13 2013
@@ -24,8 +24,6 @@ import java.util.List;
 import java.util.Properties;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.service.HiveClient;
 import org.apache.thrift.TException;
@@ -36,7 +34,7 @@ import org.apache.thrift.transport.TTran
 import org.apache.thrift.transport.TTransportException;
 
 /**
- * CliSessionState.
+ * SessionState for hive cli.
  *
  */
 public class CliSessionState extends SessionState {
@@ -76,8 +74,6 @@ public class CliSessionState extends Ses
   private TTransport transport;
   private HiveClient client;
 
-  private Hive hive; // currently only used (and init'ed) in getCurrentDbName
-
   public CliSessionState(HiveConf conf) {
     super(conf);
     remoteMode = false;
@@ -106,6 +102,7 @@ public class CliSessionState extends Ses
     return port;
   }
 
+  @Override
   public void close() {
     try {
       super.close();
@@ -128,18 +125,4 @@ public class CliSessionState extends Ses
     return client;
   }
 
-  /**
-   * Return the name of the current database
-   * @return the name of the current database or, if an error, null
-   */
-  public String getCurrentDbName() {
-    if (hive == null) {
-      try {
-        hive = Hive.get(conf);
-      } catch (HiveException e) {
-        return null;
-      }
-    }
-    return hive.getCurrentDatabase();
-  }
 }

Modified: hive/branches/branch-0.12/cli/src/test/org/apache/hadoop/hive/cli/TestCliSessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/cli/src/test/org/apache/hadoop/hive/cli/TestCliSessionState.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/cli/src/test/org/apache/hadoop/hive/cli/TestCliSessionState.java (original)
+++ hive/branches/branch-0.12/cli/src/test/org/apache/hadoop/hive/cli/TestCliSessionState.java Fri Sep 13 21:02:13 2013
@@ -30,6 +30,7 @@ import java.net.Socket;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -84,10 +85,9 @@ public class TestCliSessionState {
    */
   @Test
   public void testgetDbName() throws Exception {
-    HiveConf configuration = new HiveConf();
-    CliSessionState sessionState = new CliSessionState(configuration);
-    assertEquals(MetaStoreUtils.DEFAULT_DATABASE_NAME, sessionState.getCurrentDbName());
-
+    SessionState.start(new HiveConf());
+    assertEquals(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        SessionState.get().getCurrentDatabase());
   }
 
   /**

Modified: hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java (original)
+++ hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java Fri Sep 13 21:02:13 2013
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.ShowTablesDesc;
 import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc;
 import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hcatalog.common.ErrorType;
 import org.apache.hcatalog.common.HCatException;
 
@@ -290,14 +291,14 @@ public class HCatSemanticAnalyzer extend
 
     ShowTablesDesc showTables = work.getShowTblsDesc();
     if (showTables != null) {
-      String dbName = showTables.getDbName() == null ? cntxt.getHive().getCurrentDatabase()
+      String dbName = showTables.getDbName() == null ? SessionState.get().getCurrentDatabase()
         : showTables.getDbName();
       authorize(cntxt.getHive().getDatabase(dbName), Privilege.SELECT);
     }
 
     ShowTableStatusDesc showTableStatus = work.getShowTblStatusDesc();
     if (showTableStatus != null) {
-      String dbName = showTableStatus.getDbName() == null ? cntxt.getHive().getCurrentDatabase()
+      String dbName = showTableStatus.getDbName() == null ? SessionState.get().getCurrentDatabase()
         : showTableStatus.getDbName();
       authorize(cntxt.getHive().getDatabase(dbName), Privilege.SELECT);
     }
@@ -315,7 +316,7 @@ public class HCatSemanticAnalyzer extend
         //this is actually a ALTER TABLE DROP PARITITION statement
         for (PartitionSpec partSpec : dropTable.getPartSpecs()) {
           // partitions are not added as write entries in drop partitions in Hive
-          Table table = hive.getTable(hive.getCurrentDatabase(), dropTable.getTableName());
+          Table table = hive.getTable(SessionState.get().getCurrentDatabase(), dropTable.getTableName());
           List<Partition> partitions = null;
           try {
             partitions = hive.getPartitionsByFilter(table, partSpec.toString());
@@ -332,7 +333,8 @@ public class HCatSemanticAnalyzer extend
 
     AlterTableDesc alterTable = work.getAlterTblDesc();
     if (alterTable != null) {
-      Table table = hive.getTable(hive.getCurrentDatabase(), alterTable.getOldName(), false);
+      Table table = hive.getTable(SessionState.get().getCurrentDatabase(),
+         alterTable.getOldName(), false);
 
       Partition part = null;
       if (alterTable.getPartSpec() != null) {

Modified: hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java (original)
+++ hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java Fri Sep 13 21:02:13 2013
@@ -47,10 +47,6 @@ public class HCatSemanticAnalyzerBase ex
 
   private HiveAuthorizationProvider authProvider;
 
-  protected String getDbName(Hive hive, String dbName) {
-    return dbName == null ? hive.getCurrentDatabase() : dbName;
-  }
-
   public HiveAuthorizationProvider getAuthProvider() {
     if (authProvider == null) {
       authProvider = SessionState.get().getAuthorizer();

Modified: hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java (original)
+++ hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java Fri Sep 13 21:02:13 2013
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.ShowTablesDesc;
 import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc;
 import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.ErrorType;
 import org.apache.hive.hcatalog.common.HCatException;
 
@@ -287,14 +288,14 @@ public class HCatSemanticAnalyzer extend
 
     ShowTablesDesc showTables = work.getShowTblsDesc();
     if (showTables != null) {
-      String dbName = showTables.getDbName() == null ? cntxt.getHive().getCurrentDatabase()
+      String dbName = showTables.getDbName() == null ? SessionState.get().getCurrentDatabase()
         : showTables.getDbName();
       authorize(cntxt.getHive().getDatabase(dbName), Privilege.SELECT);
     }
 
     ShowTableStatusDesc showTableStatus = work.getShowTblStatusDesc();
     if (showTableStatus != null) {
-      String dbName = showTableStatus.getDbName() == null ? cntxt.getHive().getCurrentDatabase()
+      String dbName = showTableStatus.getDbName() == null ? SessionState.get().getCurrentDatabase()
         : showTableStatus.getDbName();
       authorize(cntxt.getHive().getDatabase(dbName), Privilege.SELECT);
     }
@@ -312,14 +313,13 @@ public class HCatSemanticAnalyzer extend
         //this is actually a ALTER TABLE DROP PARITITION statement
         for (PartitionSpec partSpec : dropTable.getPartSpecs()) {
           // partitions are not added as write entries in drop partitions in Hive
-          Table table = hive.getTable(hive.getCurrentDatabase(), dropTable.getTableName());
+          Table table = hive.getTable(SessionState.get().getCurrentDatabase(), dropTable.getTableName());
           List<Partition> partitions = null;
           try {
             partitions = hive.getPartitionsByFilter(table, partSpec.toString());
           } catch (Exception e) {
             throw new HiveException(e);
           }
-
           for (Partition part : partitions) {
             authorize(part, Privilege.DROP);
           }
@@ -329,7 +329,8 @@ public class HCatSemanticAnalyzer extend
 
     AlterTableDesc alterTable = work.getAlterTblDesc();
     if (alterTable != null) {
-      Table table = hive.getTable(hive.getCurrentDatabase(), alterTable.getOldName(), false);
+      Table table = hive.getTable(SessionState.get().getCurrentDatabase(),
+        alterTable.getOldName(), false);
 
       Partition part = null;
       if (alterTable.getPartSpec() != null) {

Modified: hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java (original)
+++ hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java Fri Sep 13 21:02:13 2013
@@ -46,10 +46,6 @@ public class HCatSemanticAnalyzerBase ex
 
   private HiveAuthorizationProvider authProvider;
 
-  protected String getDbName(Hive hive, String dbName) {
-    return dbName == null ? hive.getCurrentDatabase() : dbName;
-  }
-
   public HiveAuthorizationProvider getAuthProvider() {
     if (authProvider == null) {
       authProvider = SessionState.get().getAuthorizer();

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Fri Sep 13 21:02:13 2013
@@ -540,14 +540,14 @@ public class Driver implements CommandPr
       if (op.equals(HiveOperation.CREATETABLE_AS_SELECT)
           || op.equals(HiveOperation.CREATETABLE)) {
         ss.getAuthorizer().authorize(
-            db.getDatabase(db.getCurrentDatabase()), null,
+            db.getDatabase(SessionState.get().getCurrentDatabase()), null,
             HiveOperation.CREATETABLE_AS_SELECT.getOutputRequiredPrivileges());
       } else {
         if (op.equals(HiveOperation.IMPORT)) {
           ImportSemanticAnalyzer isa = (ImportSemanticAnalyzer) sem;
           if (!isa.existsTable()) {
             ss.getAuthorizer().authorize(
-                db.getDatabase(db.getCurrentDatabase()), null,
+                db.getDatabase(SessionState.get().getCurrentDatabase()), null,
                 HiveOperation.CREATETABLE_AS_SELECT.getOutputRequiredPrivileges());
           }
         }
@@ -831,14 +831,13 @@ public class Driver implements CommandPr
                                plan.getQueryStr());
 
       // Lock the database also
-      try {
-        Hive db = Hive.get(conf);
-        lockObjects.add(new HiveLockObj(
-                                        new HiveLockObject(db.getCurrentDatabase(), lockData),
-                                        HiveLockMode.SHARED));
-      } catch (HiveException e) {
-        throw new SemanticException(e.getMessage());
-      }
+      String currentDb = SessionState.get().getCurrentDatabase();
+      lockObjects.add(
+          new HiveLockObj(
+              new HiveLockObject(currentDb, lockData),
+              HiveLockMode.SHARED
+              )
+          );
 
       List<HiveLock> hiveLocks = ctx.getHiveLockMgr().lock(lockObjects, false);
 

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java Fri Sep 13 21:02:13 2013
@@ -36,13 +36,12 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsWork;
-import org.apache.hadoop.hive.ql.plan.FetchWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
@@ -234,7 +233,7 @@ public class ColumnStatsTask extends Tas
       throw new HiveException("Unexpected object type encountered while unpacking row");
     }
 
-    String dbName = db.getCurrentDatabase();
+    String dbName = SessionState.get().getCurrentDatabase();
     String tableName = work.getColStats().getTableName();
     String partName = null;
     List<String> colName = work.getColStats().getColName();

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Fri Sep 13 21:02:13 2013
@@ -40,10 +40,10 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.Map.Entry;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.lang.StringUtils;
@@ -112,8 +112,9 @@ import org.apache.hadoop.hive.ql.plan.Al
 import org.apache.hadoop.hive.ql.plan.AlterIndexDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableAlterPartDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
-import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
+import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.AlterTableExchangePartition;
+import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
 import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
@@ -151,9 +152,9 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.TruncateTableDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
-import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
@@ -535,7 +536,7 @@ public class DDLTask extends Task<DDLWor
             dbName = dbTab[0];
             tableName = dbTab[1];
           } else {
-            dbName = db.getCurrentDatabase();
+            dbName = SessionState.get().getCurrentDatabase();
             tableName = obj;
           }
           dbObj = db.getDatabase(dbName);
@@ -701,7 +702,7 @@ public class DDLTask extends Task<DDLWor
             dbName = dbTab[0];
             tableName = dbTab[1];
           } else {
-            dbName = db.getCurrentDatabase();
+            dbName = SessionState.get().getCurrentDatabase();
             tableName = obj;
           }
           dbObj = db.getDatabase(dbName);
@@ -874,7 +875,7 @@ public class DDLTask extends Task<DDLWor
   }
 
   private int dropIndex(Hive db, DropIndexDesc dropIdx) throws HiveException {
-    db.dropIndex(db.getCurrentDatabase(), dropIdx.getTableName(),
+    db.dropIndex(SessionState.get().getCurrentDatabase(), dropIdx.getTableName(),
         dropIdx.getIndexName(), true);
     return 0;
   }
@@ -897,7 +898,7 @@ public class DDLTask extends Task<DDLWor
     if (HiveUtils.getIndexHandler(conf, crtIndex.getIndexTypeHandlerClass()).usesIndexTable()) {
         String indexTableName =
             crtIndex.getIndexTableName() != null ? crtIndex.getIndexTableName() :
-            MetaStoreUtils.getIndexTableName(db.getCurrentDatabase(),
+            MetaStoreUtils.getIndexTableName(SessionState.get().getCurrentDatabase(),
             crtIndex.getTableName(), crtIndex.getIndexName());
         Table indexTable = db.getTable(indexTableName);
         work.getOutputs().add(new WriteEntity(indexTable));
@@ -919,7 +920,10 @@ public class DDLTask extends Task<DDLWor
         try {
           Map<String, String> props = new HashMap<String, String>();
           Map<Map<String, String>, Long> basePartTs = new HashMap<Map<String, String>, Long>();
-          Table baseTbl = db.getTable(db.getCurrentDatabase(), baseTableName);
+
+          Table baseTbl = db.getTable(SessionState.get().getCurrentDatabase(),
+              baseTableName);
+
           if (baseTbl.isPartitioned()) {
             List<Partition> baseParts;
             if (alterIndex.getSpec() != null) {
@@ -3475,7 +3479,7 @@ public class DDLTask extends Task<DDLWor
     if (!db.databaseExists(dbName)) {
       throw new HiveException(ErrorMsg.DATABASE_NOT_EXISTS, dbName);
     }
-    db.setCurrentDatabase(dbName);
+    SessionState.get().setCurrentDatabase(dbName);
 
     // set database specific parameters
     Database database = db.getDatabase(dbName);

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Fri Sep 13 21:02:13 2013
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
 import static org.apache.hadoop.hive.serde.serdeConstants.COLLECTION_DELIM;
 import static org.apache.hadoop.hive.serde.serdeConstants.ESCAPE_CHAR;
@@ -94,10 +93,12 @@ import org.apache.thrift.TException;
 import com.google.common.collect.Sets;
 
 /**
- * The Hive class contains information about this instance of Hive. An instance
- * of Hive represents a set of data in a file system (usually HDFS) organized
- * for easy query processing
+ * This class has functions that implement meta data/DDL operations using calls
+ * to the metastore.
+ * It has a metastore client instance it uses to communicate with the metastore.
  *
+ * It is a thread local variable, and the instances is accessed using static
+ * get methods in this class.
  */
 
 public class Hive {
@@ -106,7 +107,6 @@ public class Hive {
 
   private HiveConf conf = null;
   private IMetaStoreClient metaStoreClient;
-  private String currentDatabase;
 
   private static ThreadLocal<Hive> hiveDB = new ThreadLocal<Hive>() {
     @Override
@@ -167,9 +167,6 @@ public class Hive {
       closeCurrent();
       c.set("fs.scheme.class", "dfs");
       Hive newdb = new Hive(c);
-      if (db != null && db.getCurrentDatabase() != null){
-        newdb.setCurrentDatabase(db.getCurrentDatabase());
-      }
       hiveDB.set(newdb);
       return newdb;
     }
@@ -575,7 +572,7 @@ public class Hive {
   public void createTable(Table tbl, boolean ifNotExists) throws HiveException {
     try {
       if (tbl.getDbName() == null || "".equals(tbl.getDbName().trim())) {
-        tbl.setDbName(getCurrentDatabase());
+        tbl.setDbName(SessionState.get().getCurrentDatabase());
       }
       if (tbl.getCols().size() == 0) {
         tbl.setFields(MetaStoreUtils.getFieldsFromDeserializer(tbl.getTableName(),
@@ -651,7 +648,7 @@ public class Hive {
       throws HiveException {
 
     try {
-      String dbName = getCurrentDatabase();
+      String dbName = SessionState.get().getCurrentDatabase();
       Index old_index = null;
       try {
         old_index = getIndex(dbName, tableName, indexName);
@@ -795,7 +792,8 @@ public class Hive {
     case 3:
       return getIndex(names[0], names[1], names[2]);
     case 2:
-      return getIndex(getCurrentDatabase(), names[0], names[1]);
+      return getIndex(SessionState.get().getCurrentDatabase(),
+          names[0], names[1]);
     default:
       throw new HiveException("Invalid index name:" + qualifiedIndexName);
     }
@@ -1001,7 +999,7 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getAllTables() throws HiveException {
-    return getAllTables(getCurrentDatabase());
+    return getAllTables(SessionState.get().getCurrentDatabase());
   }
 
   /**
@@ -1024,7 +1022,8 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getTablesByPattern(String tablePattern) throws HiveException {
-    return getTablesByPattern(getCurrentDatabase(), tablePattern);
+    return getTablesByPattern(SessionState.get().getCurrentDatabase(),
+        tablePattern);
   }
 
   /**
@@ -1147,6 +1146,16 @@ public class Hive {
   }
 
   /**
+   * Get the Database object for current database
+   * @return a Database object if this database exists, null otherwise.
+   * @throws HiveException
+   */
+  public Database getDatabaseCurrent() throws HiveException {
+    String currentDb = SessionState.get().getCurrentDatabase();
+    return getDatabase(currentDb);
+  }
+
+  /**
    * Load a directory into a Hive Table Partition - Alters existing content of
    * the partition with the contents of loadPath. - If the partition does not
    * exist - one is created - files in loadPath are moved into Hive. But the
@@ -1911,25 +1920,6 @@ private void constructOneLBLocationMap(F
     }
   }
 
-  /**
-   * Get the name of the current database
-   * @return the current database name
-   */
-  public String getCurrentDatabase() {
-    if (null == currentDatabase) {
-      currentDatabase = DEFAULT_DATABASE_NAME;
-    }
-    return currentDatabase;
-  }
-
-  /**
-   * Set the name of the current database
-   * @param currentDatabase
-   */
-  public void setCurrentDatabase(String currentDatabase) {
-    this.currentDatabase = currentDatabase;
-  }
-
   public void createRole(String roleName, String ownerName)
       throws HiveException {
     try {
@@ -2503,7 +2493,7 @@ private void constructOneLBLocationMap(F
     case 2:
       return new Table(names[0], names[1]);
     case 1:
-      return new Table(getCurrentDatabase(), names[0]);
+      return new Table(SessionState.get().getCurrentDatabase(), names[0]);
     default:
       try{
         throw new HiveException("Invalid table name: " + tableName);

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Fri Sep 13 21:02:13 2013
@@ -1120,7 +1120,7 @@ public class DDLSemanticAnalyzer extends
     AlterIndexDesc alterIdxDesc = new AlterIndexDesc(AlterIndexTypes.UPDATETIMESTAMP);
     alterIdxDesc.setIndexName(indexName);
     alterIdxDesc.setBaseTableName(baseTableName);
-    alterIdxDesc.setDbName(db.getCurrentDatabase());
+    alterIdxDesc.setDbName(SessionState.get().getCurrentDatabase());
     alterIdxDesc.setSpec(partSpec);
 
     Task<?> tsTask = TaskFactory.get(new DDLWork(alterIdxDesc), conf);
@@ -1142,7 +1142,7 @@ public class DDLSemanticAnalyzer extends
     alterIdxDesc.setProps(mapProp);
     alterIdxDesc.setIndexName(indexName);
     alterIdxDesc.setBaseTableName(baseTableName);
-    alterIdxDesc.setDbName(db.getCurrentDatabase());
+    alterIdxDesc.setDbName(SessionState.get().getCurrentDatabase());
 
     rootTasks.add(TaskFactory.get(new DDLWork(alterIdxDesc), conf));
   }
@@ -1150,7 +1150,7 @@ public class DDLSemanticAnalyzer extends
   private List<Task<?>> getIndexBuilderMapRed(String baseTableName, String indexName,
       HashMap<String, String> partSpec) throws SemanticException {
     try {
-      String dbName = db.getCurrentDatabase();
+      String dbName = SessionState.get().getCurrentDatabase();
       Index index = db.getIndex(dbName, baseTableName, indexName);
       Table indexTbl = getTable(index.getIndexTableName());
       String baseTblName = index.getOrigTableName();
@@ -2056,7 +2056,7 @@ public class DDLSemanticAnalyzer extends
 
   private void analyzeShowTables(ASTNode ast) throws SemanticException {
     ShowTablesDesc showTblsDesc;
-    String dbName = db.getCurrentDatabase();
+    String dbName = SessionState.get().getCurrentDatabase();
     String tableNames = null;
 
     if (ast.getChildCount() > 3) {
@@ -2119,7 +2119,7 @@ public class DDLSemanticAnalyzer extends
   private void analyzeShowTableStatus(ASTNode ast) throws SemanticException {
     ShowTableStatusDesc showTblStatusDesc;
     String tableNames = getUnescapedName((ASTNode) ast.getChild(0));
-    String dbName = db.getCurrentDatabase();
+    String dbName = SessionState.get().getCurrentDatabase();
     int children = ast.getChildCount();
     HashMap<String, String> partSpec = null;
     if (children >= 2) {
@@ -2152,7 +2152,7 @@ public class DDLSemanticAnalyzer extends
   private void analyzeShowTableProperties(ASTNode ast) throws SemanticException {
     ShowTblPropertiesDesc showTblPropertiesDesc;
     String tableNames = getUnescapedName((ASTNode) ast.getChild(0));
-    String dbName = db.getCurrentDatabase();
+    String dbName = SessionState.get().getCurrentDatabase();
     String propertyName = null;
     if (ast.getChildCount() > 1) {
       propertyName = unescapeSQLString(ast.getChild(1).getText());
@@ -2414,7 +2414,7 @@ public class DDLSemanticAnalyzer extends
     partSpecs.add(newPartSpec);
     addTablePartsOutputs(tblName, partSpecs);
     RenamePartitionDesc renamePartitionDesc = new RenamePartitionDesc(
-        db.getCurrentDatabase(), tblName, oldPartSpec, newPartSpec);
+        SessionState.get().getCurrentDatabase(), tblName, oldPartSpec, newPartSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         renamePartitionDesc), conf));
   }
@@ -2504,7 +2504,7 @@ public class DDLSemanticAnalyzer extends
 
     // check if table exists.
     try {
-      tab = db.getTable(db.getCurrentDatabase(), tblName, true);
+      tab = db.getTable(SessionState.get().getCurrentDatabase(), tblName, true);
       inputs.add(new ReadEntity(tab));
     } catch (HiveException e) {
       throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName));
@@ -2542,7 +2542,7 @@ public class DDLSemanticAnalyzer extends
     }
 
     AlterTableAlterPartDesc alterTblAlterPartDesc =
-            new AlterTableAlterPartDesc(db.getCurrentDatabase(), tblName, newCol);
+            new AlterTableAlterPartDesc(SessionState.get().getCurrentDatabase(), tblName, newCol);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
             alterTblAlterPartDesc), conf));
@@ -2592,7 +2592,7 @@ public class DDLSemanticAnalyzer extends
         if (currentPart != null) {
           validatePartitionValues(currentPart);
           AddPartitionDesc addPartitionDesc = new AddPartitionDesc(
-              db.getCurrentDatabase(), tblName, currentPart,
+              SessionState.get().getCurrentDatabase(), tblName, currentPart,
               currentLocation, ifNotExists, expectView);
           partitionDescs.add(addPartitionDesc);
         }
@@ -2613,7 +2613,7 @@ public class DDLSemanticAnalyzer extends
     if (currentPart != null) {
       validatePartitionValues(currentPart);
       AddPartitionDesc addPartitionDesc = new AddPartitionDesc(
-          db.getCurrentDatabase(), tblName, currentPart,
+          SessionState.get().getCurrentDatabase(), tblName, currentPart,
           currentLocation, ifNotExists, expectView);
       partitionDescs.add(addPartitionDesc);
     }
@@ -2698,7 +2698,7 @@ public class DDLSemanticAnalyzer extends
 
     if (partSpecs.size() == 0) {
       AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc(
-          db.getCurrentDatabase(), tblName, null,
+          SessionState.get().getCurrentDatabase(), tblName, null,
           AlterTableDesc.AlterTableTypes.TOUCH);
       outputs.add(new WriteEntity(tab));
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
@@ -2707,7 +2707,7 @@ public class DDLSemanticAnalyzer extends
       addTablePartsOutputs(tblName, partSpecs);
       for (Map<String, String> partSpec : partSpecs) {
         AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc(
-            db.getCurrentDatabase(), tblName, partSpec,
+            SessionState.get().getCurrentDatabase(), tblName, partSpec,
             AlterTableDesc.AlterTableTypes.TOUCH);
         rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
             touchDesc), conf));
@@ -2747,7 +2747,7 @@ public class DDLSemanticAnalyzer extends
       throw new SemanticException(e.getMessage(), e);
     }
     AlterTableSimpleDesc archiveDesc = new AlterTableSimpleDesc(
-        db.getCurrentDatabase(), tblName, partSpec,
+        SessionState.get().getCurrentDatabase(), tblName, partSpec,
         (isUnArchive ? AlterTableTypes.UNARCHIVE : AlterTableTypes.ARCHIVE));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         archiveDesc), conf));
@@ -3240,7 +3240,7 @@ public class DDLSemanticAnalyzer extends
   }
 
   private Table getTable(String tblName, boolean throwException) throws SemanticException {
-    return getTable(db.getCurrentDatabase(), tblName, throwException);
+    return getTable(SessionState.get().getCurrentDatabase(), tblName, throwException);
   }
 
   private Table getTable(String database, String tblName, boolean throwException)

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java Fri Sep 13 21:02:13 2013
@@ -56,6 +56,7 @@ import org.apache.hadoop.hive.ql.plan.Cr
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
 import org.apache.hadoop.hive.ql.plan.MoveWork;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 
 /**
@@ -94,7 +95,7 @@ public class ImportSemanticAnalyzer exte
         Path metadataPath = new Path(fromPath, METADATA_NAME);
         Map.Entry<org.apache.hadoop.hive.metastore.api.Table,
         List<Partition>> rv =  EximUtil.readMetaData(fs, metadataPath);
-        dbname = db.getCurrentDatabase();
+        dbname = SessionState.get().getCurrentDatabase();
         org.apache.hadoop.hive.metastore.api.Table table = rv.getKey();
         tblDesc =  new CreateTableDesc(
             table.getTableName(),
@@ -241,8 +242,9 @@ public class ImportSemanticAnalyzer exte
         Task<?> t = TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
             tblDesc), conf);
         Table table = new Table(dbname, tblDesc.getTableName());
+        String currentDb = SessionState.get().getCurrentDatabase();
         conf.set("import.destination.dir",
-            wh.getTablePath(db.getDatabase(db.getCurrentDatabase()),
+            wh.getTablePath(db.getDatabaseCurrent(),
                 tblDesc.getTableName()).toString());
         if ((tblDesc.getPartCols() != null) && (tblDesc.getPartCols().size() != 0)) {
           for (AddPartitionDesc addPartitionDesc : partitionDescs) {
@@ -260,7 +262,7 @@ public class ImportSemanticAnalyzer exte
             if (tblDesc.getLocation() != null) {
               tablePath = new Path(tblDesc.getLocation());
             } else {
-              tablePath = wh.getTablePath(db.getDatabase(db.getCurrentDatabase()), tblDesc.getTableName());
+              tablePath = wh.getTablePath(db.getDatabaseCurrent(), tblDesc.getTableName());
             }
             checkTargetLocationEmpty(fs, tablePath);
             t.addDependentTask(loadTable(fromURI, table));
@@ -312,7 +314,7 @@ public class ImportSemanticAnalyzer exte
               Warehouse.makePartPath(addPartitionDesc.getPartSpec()));
         } else {
           tgtPath = new Path(wh.getTablePath(
-              db.getDatabase(db.getCurrentDatabase()), tblDesc.getTableName()),
+              db.getDatabaseCurrent(), tblDesc.getTableName()),
               Warehouse.makePartPath(addPartitionDesc.getPartSpec()));
         }
       } else {

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Fri Sep 13 21:02:13 2013
@@ -603,7 +603,7 @@ public class SemanticAnalyzer extends Ba
 
     qb.getParseInfo().setSrcForAlias(alias, tableTree);
 
-    unparseTranslator.addTableNameTranslation(tableTree, db.getCurrentDatabase());
+    unparseTranslator.addTableNameTranslation(tableTree, SessionState.get().getCurrentDatabase());
     if (aliasIndex != 0) {
       unparseTranslator.addIdentifierTranslation((ASTNode) tabref
           .getChild(aliasIndex));
@@ -802,7 +802,7 @@ public class SemanticAnalyzer extends Ba
         break;
 
       case HiveParser.TOK_INSERT_INTO:
-        String currentDatabase = db.getCurrentDatabase();
+        String currentDatabase = SessionState.get().getCurrentDatabase();
         String tab_name = getUnescapedName((ASTNode) ast.getChild(0).getChild(0), currentDatabase);
         qbp.addInsertIntoTable(tab_name);
 
@@ -8327,7 +8327,7 @@ public class SemanticAnalyzer extends Ba
       }
       viewSelect = child;
       // prevent view from referencing itself
-      viewsExpanded.add(db.getCurrentDatabase() + "." + createVwDesc.getViewName());
+      viewsExpanded.add(SessionState.get().getCurrentDatabase() + "." + createVwDesc.getViewName());
     }
 
     // continue analyzing from the child ASTNode.

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Fri Sep 13 21:02:13 2013
@@ -17,6 +17,7 @@
  */
 
 package org.apache.hadoop.hive.ql.session;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 
 import java.io.File;
 import java.io.IOException;
@@ -133,6 +134,8 @@ public class SessionState {
 
   private Map<String, List<String>> localMapRedErrors;
 
+  private String currentDatabase;
+
   /**
    * Lineage state.
    */
@@ -767,6 +770,17 @@ public class SessionState {
     this.localMapRedErrors = localMapRedErrors;
   }
 
+  public String getCurrentDatabase() {
+    if (currentDatabase == null) {
+      currentDatabase = DEFAULT_DATABASE_NAME;
+    }
+    return currentDatabase;
+  }
+
+  public void setCurrentDatabase(String currentDatabase) {
+    this.currentDatabase = currentDatabase;
+  }
+
   public void close() throws IOException {
     File resourceDir =
       new File(getConf().getVar(HiveConf.ConfVars.DOWNLOADED_RESOURCES_DIR));

Modified: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Fri Sep 13 21:02:13 2013
@@ -470,7 +470,7 @@ public class QTestUtil {
     // Delete any tables other than the source tables
     // and any databases other than the default database.
     for (String dbName : db.getAllDatabases()) {
-      db.setCurrentDatabase(dbName);
+      SessionState.get().setCurrentDatabase(dbName);
       for (String tblName : db.getAllTables()) {
         if (!DEFAULT_DATABASE_NAME.equals(dbName) || !srcTables.contains(tblName)) {
           Table tblObj = db.getTable(tblName);
@@ -494,7 +494,7 @@ public class QTestUtil {
         db.dropDatabase(dbName);
       }
     }
-    Hive.get().setCurrentDatabase(DEFAULT_DATABASE_NAME);
+    SessionState.get().setCurrentDatabase(DEFAULT_DATABASE_NAME);
 
     List<String> roleNames = db.getAllRoleNames();
       for (String roleName : roleNames) {
@@ -617,7 +617,8 @@ public class QTestUtil {
     db.createTable("src_sequencefile", cols, null,
         SequenceFileInputFormat.class, SequenceFileOutputFormat.class);
 
-    Table srcThrift = new Table(db.getCurrentDatabase(), "src_thrift");
+    Table srcThrift =
+        new Table(SessionState.get().getCurrentDatabase(), "src_thrift");
     srcThrift.setInputFormatClass(SequenceFileInputFormat.class.getName());
     srcThrift.setOutputFormatClass(SequenceFileOutputFormat.class.getName());
     srcThrift.setSerializationLib(ThriftDeserializer.class.getName());

Modified: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java (original)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java Fri Sep 13 21:02:13 2013
@@ -55,6 +55,7 @@ import org.apache.hadoop.hive.ql.plan.Re
 import org.apache.hadoop.hive.ql.plan.ReduceWork;
 import org.apache.hadoop.hive.ql.plan.ScriptDesc;
 import org.apache.hadoop.hive.ql.plan.SelectDesc;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.mapred.TextInputFormat;
@@ -78,7 +79,8 @@ public class TestExecDriver extends Test
   static {
     try {
       conf = new HiveConf(ExecDriver.class);
-
+      SessionState.start(conf);
+      
       fs = FileSystem.get(conf);
       if (fs.exists(tmppath) && !fs.getFileStatus(tmppath).isDir()) {
         throw new RuntimeException(tmpdir + " exists but is not a directory");

Modified: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/history/TestHiveHistory.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/history/TestHiveHistory.java (original)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/history/TestHiveHistory.java Fri Sep 13 21:02:13 2013
@@ -64,6 +64,7 @@ public class TestHiveHistory extends Tes
   protected void setUp() {
     try {
       conf = new HiveConf(HiveHistory.class);
+      SessionState.start(conf);
 
       fs = FileSystem.get(conf);
       if (fs.exists(tmppath) && !fs.getFileStatus(tmppath).isDir()) {

Modified: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Fri Sep 13 21:02:13 2013
@@ -30,6 +30,7 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.index.HiveIndex;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer;
@@ -61,6 +63,7 @@ public class TestHive extends TestCase {
   protected void setUp() throws Exception {
     super.setUp();
     hiveConf = new HiveConf(this.getClass());
+    SessionState.start(hiveConf);
     try {
       hm = Hive.get(hiveConf);
     } catch (Exception e) {
@@ -96,6 +99,7 @@ public class TestHive extends TestCase {
         e1.printStackTrace();
         assertTrue("Unable to drop table", false);
       }
+
       Table tbl = new Table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       List<FieldSchema> fields = tbl.getCols();
 
@@ -142,6 +146,7 @@ public class TestHive extends TestCase {
       tbl.setStoredAsSubDirectories(false);
 
       // create table
+      setNullCreateTableGrants();
       try {
         hm.createTable(tbl);
       } catch (HiveException e) {
@@ -168,6 +173,14 @@ public class TestHive extends TestCase {
     }
   }
 
+  private void setNullCreateTableGrants() {
+    //having a non null create table grants privileges causes problems in
+    // the tests that compares underlying thrift Table object of created
+    // table with a table object that was fetched from metastore.
+    // This is because the fetch does not populate the privileges field in Table
+    SessionState.get().setCreateTableGrants(null);
+  }
+
   /**
    * Tests create and fetch of a thrift based table.
    *
@@ -190,6 +203,8 @@ public class TestHive extends TestCase {
       tbl.setSerdeParam(serdeConstants.SERIALIZATION_FORMAT, TBinaryProtocol.class
           .getName());
       tbl.setStoredAsSubDirectories(false);
+
+      setNullCreateTableGrants();
       try {
         hm.createTable(tbl);
       } catch (HiveException e) {
@@ -487,13 +502,26 @@ public class TestHive extends TestCase {
     }
   }
 
-  public void testHiveRefreshDatabase() throws Throwable{
-    String testDatabaseName = "test_database";
-    Database testDatabase = new Database();
-    testDatabase.setName(testDatabaseName);
-    hm.createDatabase(testDatabase, true);
-    hm.setCurrentDatabase(testDatabaseName);
-    hm = Hive.get(hiveConf, true); //refresh Hive instance
-    assertEquals(testDatabaseName, hm.getCurrentDatabase());
+  public void testHiveRefreshOnConfChange() throws Throwable{
+    Hive prevHiveObj = Hive.get();
+    Hive newHiveObj;
+
+    //if HiveConf has not changed, same object should be returned
+    HiveConf newHconf = new HiveConf(hiveConf);
+    newHiveObj = Hive.get(newHconf);
+    assertTrue(prevHiveObj == newHiveObj);
+
+    //if needs refresh param is passed, it should return new object
+    newHiveObj = Hive.get(newHconf, true);
+    assertTrue(prevHiveObj != newHiveObj);
+
+    //if HiveConf has changed, new object should be returned
+    prevHiveObj = Hive.get();
+    //change value of a metavar config param in new hive conf
+    newHconf = new HiveConf(hiveConf);
+    newHconf.setIntVar(ConfVars.METASTORETHRIFTCONNECTIONRETRIES,
+        newHconf.getIntVar(ConfVars.METASTORETHRIFTCONNECTIONRETRIES) + 1);
+    newHiveObj = Hive.get(newHconf);
+    assertTrue(prevHiveObj != newHiveObj);
   }
 }

Modified: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java?rev=1523102&r1=1523101&r2=1523102&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java (original)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java Fri Sep 13 21:02:13 2013
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro;
 import org.junit.Before;
 import org.junit.Test;
@@ -44,6 +45,7 @@ public class TestMacroSemanticAnalyzer {
     context = new Context(conf);
     parseDriver = new ParseDriver();
     analyzer = new MacroSemanticAnalyzer(conf);
+    SessionState.start(conf);
   }
 
   private ASTNode parse(String command) throws Exception {

Added: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java?rev=1523102&view=auto
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java (added)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java Fri Sep 13 21:02:13 2013
@@ -0,0 +1,61 @@
+/**
+ * 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.ql.session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test SessionState
+ */
+public class TestSessionState {
+
+
+  @Before
+  public void setup(){
+    SessionState.start(new HiveConf());
+  }
+
+  /**
+   * test set and get db
+   */
+  @Test
+  public void testgetDbName() throws Exception {
+    //check that we start with default db
+    assertEquals(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        SessionState.get().getCurrentDatabase());
+    final String newdb = "DB_2";
+
+    //set new db and verify get
+    SessionState.get().setCurrentDatabase(newdb);
+    assertEquals(newdb,
+        SessionState.get().getCurrentDatabase());
+
+    //verify that a new sessionstate has default db
+    SessionState.start(new HiveConf());
+    assertEquals(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        SessionState.get().getCurrentDatabase());
+
+  }
+
+
+}