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 2014/08/11 18:56:31 UTC

svn commit: r1617327 [1/2] - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/ itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/ metastore/src/java/org/apache/hadoop/hive/metastore/ ql/src/...

Author: thejas
Date: Mon Aug 11 16:56:30 2014
New Revision: 1617327

URL: http://svn.apache.org/r1617327
Log:
HIVE-4064 : Handle db qualified names consistently across all HiveQL statements (Navis via Thejas Nair)

Added:
    hive/trunk/ql/src/test/queries/clientpositive/alter_rename_table.q
    hive/trunk/ql/src/test/results/clientpositive/alter_rename_table.q.out
Modified:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
    hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IndexUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterIndexDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableAlterPartDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/PrivilegeObjectDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/RenamePartitionDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowColumnsDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowGrantDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveV1Authorizer.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/PrivilegesTestBase.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV1.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV2.java
    hive/trunk/ql/src/test/results/clientnegative/alter_concatenate_indexed_table.q.out
    hive/trunk/ql/src/test/results/clientnegative/alter_view_failure6.q.out
    hive/trunk/ql/src/test/results/clientnegative/merge_negative_1.q.out
    hive/trunk/ql/src/test/results/clientnegative/merge_negative_2.q.out
    hive/trunk/ql/src/test/results/clientnegative/show_columns3.q.out
    hive/trunk/ql/src/test/results/clientnegative/show_tableproperties1.q.out
    hive/trunk/ql/src/test/results/clientnegative/temp_table_index.q.out
    hive/trunk/ql/src/test/results/clientpositive/drop_multi_partitions.q.out
    hive/trunk/ql/src/test/results/clientpositive/input3.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert2_overwrite_partitions.q.out
    hive/trunk/ql/src/test/results/clientpositive/show_create_table_db_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/show_tblproperties.q.out
    hive/trunk/ql/src/test/results/clientpositive/temp_table_names.q.out
    hive/trunk/ql/src/test/results/clientpositive/temp_table_precedence.q.out

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java Mon Aug 11 16:56:30 2014
@@ -29,7 +29,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -110,7 +109,7 @@ public class TestHiveAuthorizerCheckInvo
         getSortedList(tableObj.getColumns()));
   }
 
-  private List<String> getSortedList(Set<String> columns) {
+  private List<String> getSortedList(List<String> columns) {
     List<String> sortedCols = new ArrayList<String>(columns);
     Collections.sort(sortedCols);
     return sortedCols;

Modified: hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java (original)
+++ hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java Mon Aug 11 16:56:30 2014
@@ -18,9 +18,9 @@
 package org.apache.hadoop.hive.ql.hooks;
 
 import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
-import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -34,7 +34,7 @@ import org.mortbay.log.Log;
 /*
  * This hook is used for verifying the column access information
  * that is generated and maintained in the QueryPlan object by the
- * ColumnAccessAnalyer. All the hook does is print out the columns
+ * ColumnAccessAnalyzer. All the hook does is print out the columns
  * accessed from each table as recorded in the ColumnAccessInfo
  * in the QueryPlan.
  */
@@ -58,14 +58,14 @@ public class CheckColumnAccessHook imple
     }
 
     LogHelper console = SessionState.getConsole();
-    Map<String, Set<String>> tableToColumnAccessMap =
+    Map<String, List<String>> tableToColumnAccessMap =
       columnAccessInfo.getTableToColumnAccessMap();
 
     // We need a new map to ensure output is always produced in the same order.
     // This makes tests that use this hook deterministic.
     Map<String, String> outputOrderedMap = new HashMap<String, String>();
 
-    for (Map.Entry<String, Set<String>> tableAccess : tableToColumnAccessMap.entrySet()) {
+    for (Map.Entry<String, List<String>> tableAccess : tableToColumnAccessMap.entrySet()) {
       StringBuilder perTableInfo = new StringBuilder();
       perTableInfo.append("Table:").append(tableAccess.getKey()).append("\n");
       // Sort columns to make output deterministic

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java Mon Aug 11 16:56:30 2014
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Database;
@@ -81,11 +82,11 @@ public class HiveAlterHandler implements
     FileSystem destFs = null;
 
     boolean success = false;
-    String oldTblLoc = null;
-    String newTblLoc = null;
     boolean moveData = false;
     boolean rename = false;
     Table oldt = null;
+    List<ObjectPair<Partition, String>> altps = new ArrayList<ObjectPair<Partition, String>>();
+
     try {
       msdb.openTransaction();
       name = name.toLowerCase();
@@ -132,30 +133,30 @@ public class HiveAlterHandler implements
 
       // if this alter is a rename, the table is not a virtual view, the user
       // didn't change the default location (or new location is empty), and
-      // table is not an external table, that means useris asking metastore to
+      // table is not an external table, that means user is asking metastore to
       // move data to the new location corresponding to the new name
       if (rename
           && !oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())
           && (oldt.getSd().getLocation().compareTo(newt.getSd().getLocation()) == 0
             || StringUtils.isEmpty(newt.getSd().getLocation()))
           && !MetaStoreUtils.isExternalTable(oldt)) {
+
+        srcPath = new Path(oldt.getSd().getLocation());
+        srcFs = wh.getFs(srcPath);
+
         // that means user is asking metastore to move data to new location
         // corresponding to the new name
         // get new location
-        newTblLoc = wh.getTablePath(msdb.getDatabase(newt.getDbName()),
-            newt.getTableName()).toString();
-        Path newTblPath = constructRenamedPath(new Path(newTblLoc),
-            new Path(newt.getSd().getLocation()));
-        newTblLoc = newTblPath.toString();
-        newt.getSd().setLocation(newTblLoc);
-        oldTblLoc = oldt.getSd().getLocation();
+        Path databasePath = constructRenamedPath(
+            wh.getDefaultDatabasePath(newt.getDbName()), srcPath);
+        destPath = new Path(databasePath, newt.getTableName());
+        destFs = wh.getFs(destPath);
+
+        newt.getSd().setLocation(destPath.toString());
         moveData = true;
+
         // check that destination does not exist otherwise we will be
         // overwriting data
-        srcPath = new Path(oldTblLoc);
-        srcFs = wh.getFs(srcPath);
-        destPath = new Path(newTblLoc);
-        destFs = wh.getFs(destPath);
         // check that src and dest are on the same file system
         if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
           throw new InvalidOperationException("table new location " + destPath
@@ -177,22 +178,18 @@ public class HiveAlterHandler implements
               + destPath + " for table " + newt.getDbName() + "."
               + newt.getTableName());
         }
+        String oldTblLocPath = srcPath.toUri().getPath();
+        String newTblLocPath = destPath.toUri().getPath();
+
         // also the location field in partition
         List<Partition> parts = msdb.getPartitions(dbname, name, -1);
         for (Partition part : parts) {
           String oldPartLoc = part.getSd().getLocation();
-          Path oldPartLocPath = new Path(oldPartLoc);
-          String oldTblLocPath = new Path(oldTblLoc).toUri().getPath();
-          String newTblLocPath = new Path(newTblLoc).toUri().getPath();
           if (oldPartLoc.contains(oldTblLocPath)) {
-            Path newPartLocPath = null;
-            URI oldUri = oldPartLocPath.toUri();
-            String newPath = oldUri.getPath().replace(oldTblLocPath,
-                                                      newTblLocPath);
-
-            newPartLocPath = new Path(oldUri.getScheme(),
-                                      oldUri.getAuthority(),
-                                      newPath);
+            URI oldUri = new Path(oldPartLoc).toUri();
+            String newPath = oldUri.getPath().replace(oldTblLocPath, newTblLocPath);
+            Path newPartLocPath = new Path(oldUri.getScheme(), oldUri.getAuthority(), newPath);
+            altps.add(ObjectPair.create(part, part.getSd().getLocation()));
             part.getSd().setLocation(newPartLocPath.toString());
             msdb.alterPartition(dbname, name, part.getValues(), part);
           }
@@ -235,9 +232,23 @@ public class HiveAlterHandler implements
           try {
             msdb.openTransaction();
             msdb.alterTable(dbname, newt.getTableName(), oldt);
+            for (ObjectPair<Partition, String> pair : altps) {
+              Partition part = pair.getFirst();
+              part.getSd().setLocation(pair.getSecond());
+              msdb.alterPartition(dbname, name, part.getValues(), part);
+            }
             revertMetaDataTransaction = msdb.commitTransaction();
           } catch (Exception e1) {
-            LOG.error("Reverting metadata opeation failed During HDFS operation failed", e1);
+            // we should log this for manual rollback by administrator
+            LOG.error("Reverting metadata by HDFS operation failure failed During HDFS operation failed", e1);
+            LOG.error("Table " + Warehouse.getQualifiedName(newt) +
+                " should be renamed to " + Warehouse.getQualifiedName(oldt));
+            LOG.error("Table " + Warehouse.getQualifiedName(newt) +
+                " should have path " + srcPath);
+            for (ObjectPair<Partition, String> pair : altps) {
+              LOG.error("Partition " + Warehouse.getQualifiedName(pair.getFirst()) +
+                  " should have path " + pair.getSecond());
+            }
             if (!revertMetaDataTransaction) {
               msdb.rollbackTransaction();
             }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Mon Aug 11 16:56:30 2014
@@ -3269,6 +3269,9 @@ public class HiveMetaStore extends Thrif
 
       boolean success = false, indexTableCreated = false;
 
+      String[] qualified =
+          MetaStoreUtils.getQualifiedName(index.getDbName(), index.getIndexTableName());
+
       try {
         ms.openTransaction();
         Index old_index = null;
@@ -3291,7 +3294,7 @@ public class HiveMetaStore extends Thrif
         Table indexTbl = indexTable;
         if (indexTbl != null) {
           try {
-            indexTbl = ms.getTable(index.getDbName(), index.getIndexTableName());
+            indexTbl = ms.getTable(qualified[0], qualified[1]);
           } catch (Exception e) {
           }
           if (indexTbl != null) {
@@ -3312,7 +3315,7 @@ public class HiveMetaStore extends Thrif
         if (!success) {
           if (indexTableCreated) {
             try {
-              this.drop_table(index.getDbName(), index.getIndexTableName(), false);
+              drop_table(qualified[0], qualified[1], false);
             } catch (Exception e) {
             }
           }
@@ -3366,8 +3369,8 @@ public class HiveMetaStore extends Thrif
 
         String idxTblName = index.getIndexTableName();
         if (idxTblName != null) {
-          Table tbl = null;
-          tbl = this.get_table(dbName, idxTblName);
+          String[] qualified = MetaStoreUtils.getQualifiedName(index.getDbName(), idxTblName);
+          Table tbl = get_table(qualified[0], qualified[1]);
           if (tbl.getSd() == null) {
             throw new MetaException("Table metadata is corrupted");
           }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Mon Aug 11 16:56:30 2014
@@ -1538,4 +1538,12 @@ public class MetaStoreUtils {
       return part.getValues().size();
     }
   }
+
+  public static String[] getQualifiedName(String defaultDbName, String tableName) {
+    String[] names = tableName.split("\\.");
+    if (names.length == 1) {
+      return new String[] { defaultDbName, tableName};
+    }
+    return new String[] {names[0], names[1]};
+  }
 }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java Mon Aug 11 16:56:30 2014
@@ -2847,7 +2847,8 @@ public class ObjectStore implements RawS
           "Original table does not exist for the given index.");
     }
 
-    MTable indexTable = getMTable(index.getDbName(), index.getIndexTableName());
+    String[] qualified = MetaStoreUtils.getQualifiedName(index.getDbName(), index.getIndexTableName());
+    MTable indexTable = getMTable(qualified[0], qualified[1]);
     if (indexTable == null) {
       throw new InvalidObjectException(
           "Underlying index table does not exist for the given index.");

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java Mon Aug 11 16:56:30 2014
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -186,6 +187,14 @@ public class Warehouse {
     return getDnsPath(new Path(getDatabasePath(db), tableName.toLowerCase()));
   }
 
+  public static String getQualifiedName(Table table) {
+    return table.getDbName() + "." + table.getTableName();
+  }
+
+  public static String getQualifiedName(Partition partition) {
+    return partition.getDbName() + "." + partition.getTableName() + partition.getValues();
+  }
+
   public boolean mkdirs(Path f, boolean inheritPermCandidate) throws MetaException {
     boolean inheritPerms = HiveConf.getBoolVar(conf,
       HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS) && inheritPermCandidate;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Mon Aug 11 16:56:30 2014
@@ -507,7 +507,7 @@ public class Driver implements CommandPr
       // get mapping of tables to columns used
       ColumnAccessInfo colAccessInfo = sem.getColumnAccessInfo();
       // colAccessInfo is set only in case of SemanticAnalyzer
-      Map<String, Set<String>> tab2Cols = colAccessInfo != null ? colAccessInfo
+      Map<String, List<String>> tab2Cols = colAccessInfo != null ? colAccessInfo
           .getTableToColumnAccessMap() : null;
       doAuthorizationV2(ss, op, inputs, outputs, command, tab2Cols);
      return;
@@ -700,7 +700,7 @@ public class Driver implements CommandPr
   }
 
   private static void doAuthorizationV2(SessionState ss, HiveOperation op, HashSet<ReadEntity> inputs,
-      HashSet<WriteEntity> outputs, String command, Map<String, Set<String>> tab2cols) throws HiveException {
+      HashSet<WriteEntity> outputs, String command, Map<String, List<String>> tab2cols) throws HiveException {
 
     HiveAuthzContext.Builder authzContextBuilder = new HiveAuthzContext.Builder();
 
@@ -711,36 +711,14 @@ public class Driver implements CommandPr
     authzContextBuilder.setCommandString(command);
 
     HiveOperationType hiveOpType = getHiveOperationType(op);
-    List<HivePrivilegeObject> inputsHObjs = getHivePrivObjects(inputs);
-    updateInputColumnInfo(inputsHObjs, tab2cols);
+    List<HivePrivilegeObject> inputsHObjs = getHivePrivObjects(inputs, tab2cols);
+    List<HivePrivilegeObject> outputHObjs = getHivePrivObjects(outputs, null);
 
-    List<HivePrivilegeObject> outputHObjs = getHivePrivObjects(outputs);
     ss.getAuthorizerV2().checkPrivileges(hiveOpType, inputsHObjs, outputHObjs, authzContextBuilder.build());
-    return;
   }
 
-  /**
-   * Add column information for input table objects
-   * @param inputsHObjs input HivePrivilegeObject
-   * @param map table to used input columns mapping
-   */
-  private static void updateInputColumnInfo(List<HivePrivilegeObject> inputsHObjs,
-      Map<String, Set<String>> tableName2Cols) {
-    if(tableName2Cols == null) {
-      return;
-    }
-    for(HivePrivilegeObject inputObj : inputsHObjs){
-      if(inputObj.getType() != HivePrivilegeObjectType.TABLE_OR_VIEW){
-        // input columns are relevant only for tables or views
-        continue;
-      }
-      Set<String> cols = tableName2Cols.get(Table.getCompleteName(inputObj.getDbname(),
-          inputObj.getObjectName()));
-      inputObj.setColumns(cols);
-    }
-  }
-
-  private static List<HivePrivilegeObject> getHivePrivObjects(HashSet<? extends Entity> privObjects) {
+  private static List<HivePrivilegeObject> getHivePrivObjects(
+      HashSet<? extends Entity> privObjects, Map<String, List<String>> tableName2Cols) {
     List<HivePrivilegeObject> hivePrivobjs = new ArrayList<HivePrivilegeObject>();
     if(privObjects == null){
       return hivePrivobjs;
@@ -764,13 +742,17 @@ public class Driver implements CommandPr
       //support for authorization on partitions needs to be added
       String dbname = null;
       String objName = null;
+      List<String> partKeys = null;
+      List<String> columns = null;
       switch(privObject.getType()){
       case DATABASE:
-        dbname = privObject.getDatabase() == null ? null : privObject.getDatabase().getName();
+        dbname = privObject.getDatabase().getName();
         break;
       case TABLE:
-        dbname = privObject.getTable() == null ? null : privObject.getTable().getDbName();
-        objName = privObject.getTable() == null ? null : privObject.getTable().getTableName();
+        dbname = privObject.getTable().getDbName();
+        objName = privObject.getTable().getTableName();
+        columns = tableName2Cols == null ? null :
+            tableName2Cols.get(Table.getCompleteName(dbname, objName));
         break;
       case DFS_DIR:
       case LOCAL_DIR:
@@ -788,7 +770,7 @@ public class Driver implements CommandPr
       }
       HivePrivObjectActionType actionType = AuthorizationUtils.getActionType(privObject);
       HivePrivilegeObject hPrivObject = new HivePrivilegeObject(privObjType, dbname, objName,
-          actionType);
+          partKeys, columns, actionType, null);
       hivePrivobjs.add(hPrivObject);
     }
     return hivePrivobjs;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Mon Aug 11 16:56:30 2014
@@ -35,7 +35,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -600,13 +599,9 @@ public class DDLTask extends Task<DDLWor
 
     HiveAuthorizer authorizer = getSessionAuthorizer();
     try {
-      Set<String> colSet = showGrantDesc.getColumns() != null ? new HashSet<String>(
-          showGrantDesc.getColumns()) : null;
       List<HivePrivilegeInfo> privInfos = authorizer.showPrivileges(
           AuthorizationUtils.getHivePrincipal(showGrantDesc.getPrincipalDesc()),
-          AuthorizationUtils.getHivePrivilegeObject(showGrantDesc.getHiveObj(),
-              colSet
-              ));
+          AuthorizationUtils.getHivePrivilegeObject(showGrantDesc.getHiveObj()));
       boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST);
       writeToFile(writeGrantInfo(privInfos, testMode), showGrantDesc.getResFile());
     } catch (IOException e) {
@@ -625,7 +620,7 @@ public class DDLTask extends Task<DDLWor
     //Convert to object types used by the authorization plugin interface
     List<HivePrincipal> hivePrincipals = AuthorizationUtils.getHivePrincipals(principals);
     List<HivePrivilege> hivePrivileges = AuthorizationUtils.getHivePrivileges(privileges);
-    HivePrivilegeObject hivePrivObject = AuthorizationUtils.getHivePrivilegeObject(privSubjectDesc, null);
+    HivePrivilegeObject hivePrivObject = AuthorizationUtils.getHivePrivilegeObject(privSubjectDesc);
 
     HivePrincipal grantorPrincipal = new HivePrincipal(
         grantor, AuthorizationUtils.getHivePrincipalType(grantorType));
@@ -754,8 +749,7 @@ public class DDLTask extends Task<DDLWor
   }
 
   private int dropIndex(Hive db, DropIndexDesc dropIdx) throws HiveException {
-    db.dropIndex(SessionState.get().getCurrentDatabase(), dropIdx.getTableName(),
-        dropIdx.getIndexName(), true);
+    db.dropIndex(dropIdx.getTableName(), dropIdx.getIndexName(), true);
     return 0;
   }
 
@@ -765,11 +759,7 @@ public class DDLTask extends Task<DDLWor
       validateSerDe(crtIndex.getSerde());
     }
 
-    String indexTableName =
-      crtIndex.getIndexTableName() != null ? crtIndex.getIndexTableName() :
-        MetaStoreUtils.getIndexTableName(SessionState.get().getCurrentDatabase(),
-             crtIndex.getTableName(), crtIndex.getIndexName());
-
+    String indexTableName = crtIndex.getIndexTableName();
     if (!Utilities.isDefaultNameNode(conf)) {
       // If location is specified - ensure that it is a full qualified name
       makeLocationQualified(crtIndex, indexTableName);
@@ -792,10 +782,9 @@ public class DDLTask extends Task<DDLWor
   }
 
   private int alterIndex(Hive db, AlterIndexDesc alterIndex) throws HiveException {
-    String dbName = alterIndex.getDbName();
     String baseTableName = alterIndex.getBaseTableName();
     String indexName = alterIndex.getIndexName();
-    Index idx = db.getIndex(dbName, baseTableName, indexName);
+    Index idx = db.getIndex(baseTableName, indexName);
 
     switch(alterIndex.getOp()) {
     case ADDPROPS:
@@ -806,8 +795,7 @@ public class DDLTask extends Task<DDLWor
         Map<String, String> props = new HashMap<String, String>();
         Map<Map<String, String>, Long> basePartTs = new HashMap<Map<String, String>, Long>();
 
-        Table baseTbl = db.getTable(SessionState.get().getCurrentDatabase(),
-            baseTableName);
+        Table baseTbl = db.getTable(baseTableName);
 
         if (baseTbl.isPartitioned()) {
           List<Partition> baseParts;
@@ -854,7 +842,7 @@ public class DDLTask extends Task<DDLWor
     }
 
     try {
-      db.alterIndex(dbName, baseTableName, indexName, idx);
+      db.alterIndex(baseTableName, indexName, idx);
     } catch (InvalidOperationException e) {
       console.printError("Invalid alter operation: " + e.getMessage());
       LOG.info("alter index: " + stringifyException(e));
@@ -896,7 +884,7 @@ public class DDLTask extends Task<DDLWor
    */
   private int renamePartition(Hive db, RenamePartitionDesc renamePartitionDesc) throws HiveException {
 
-    Table tbl = db.getTable(renamePartitionDesc.getDbName(), renamePartitionDesc.getTableName());
+    Table tbl = db.getTable(renamePartitionDesc.getTableName());
 
     Partition oldPart = db.getPartition(tbl, renamePartitionDesc.getOldPartSpec(), false);
     Partition part = db.getPartition(tbl, renamePartitionDesc.getOldPartSpec(), false);
@@ -923,7 +911,7 @@ public class DDLTask extends Task<DDLWor
   private int alterTableAlterPart(Hive db, AlterTableAlterPartDesc alterPartitionDesc)
       throws HiveException {
 
-    Table tbl = db.getTable(alterPartitionDesc.getDbName(), alterPartitionDesc.getTableName());
+    Table tbl = db.getTable(alterPartitionDesc.getTableName(), true);
     String tabName = alterPartitionDesc.getTableName();
 
     // This is checked by DDLSemanticAnalyzer
@@ -1015,14 +1003,11 @@ public class DDLTask extends Task<DDLWor
   private int touch(Hive db, AlterTableSimpleDesc touchDesc)
       throws HiveException {
 
-    String dbName = touchDesc.getDbName();
-    String tblName = touchDesc.getTableName();
-
-    Table tbl = db.getTable(dbName, tblName);
+    Table tbl = db.getTable(touchDesc.getTableName());
 
     if (touchDesc.getPartSpec() == null) {
       try {
-        db.alterTable(tblName, tbl);
+        db.alterTable(touchDesc.getTableName(), tbl);
       } catch (InvalidOperationException e) {
         throw new HiveException("Uable to update table");
       }
@@ -1034,7 +1019,7 @@ public class DDLTask extends Task<DDLWor
         throw new HiveException("Specified partition does not exist");
       }
       try {
-        db.alterPartition(tblName, part);
+        db.alterPartition(touchDesc.getTableName(), part);
       } catch (InvalidOperationException e) {
         throw new HiveException(e);
       }
@@ -1173,10 +1158,8 @@ public class DDLTask extends Task<DDLWor
   private int archive(Hive db, AlterTableSimpleDesc simpleDesc,
       DriverContext driverContext)
           throws HiveException {
-    String dbName = simpleDesc.getDbName();
-    String tblName = simpleDesc.getTableName();
 
-    Table tbl = db.getTable(dbName, tblName);
+    Table tbl = db.getTable(simpleDesc.getTableName());
 
     if (tbl.getTableType() != TableType.MANAGED_TABLE) {
       throw new HiveException("ARCHIVE can only be performed on managed tables");
@@ -1378,7 +1361,7 @@ public class DDLTask extends Task<DDLWor
             authority.toString(),
             harPartitionDir.getPath()); // make in Path to ensure no slash at the end
         setArchived(p, harPath, partSpecInfo.values.size());
-        db.alterPartition(tblName, p);
+        db.alterPartition(simpleDesc.getTableName(), p);
       }
     } catch (Exception e) {
       throw new HiveException("Unable to change the partition info for HAR", e);
@@ -1399,10 +1382,8 @@ public class DDLTask extends Task<DDLWor
 
   private int unarchive(Hive db, AlterTableSimpleDesc simpleDesc)
       throws HiveException {
-    String dbName = simpleDesc.getDbName();
-    String tblName = simpleDesc.getTableName();
 
-    Table tbl = db.getTable(dbName, tblName);
+    Table tbl = db.getTable(simpleDesc.getTableName());
 
     // Means user specified a table, not a partition
     if (simpleDesc.getPartSpec() == null) {
@@ -1587,7 +1568,7 @@ public class DDLTask extends Task<DDLWor
     for(Partition p: partitions) {
       setUnArchived(p);
       try {
-        db.alterPartition(tblName, p);
+        db.alterPartition(simpleDesc.getTableName(), p);
       } catch (InvalidOperationException e) {
         throw new HiveException(e);
       }
@@ -1636,10 +1617,7 @@ public class DDLTask extends Task<DDLWor
 
   private int compact(Hive db, AlterTableSimpleDesc desc) throws HiveException {
 
-    String dbName = desc.getDbName();
-    String tblName = desc.getTableName();
-
-    Table tbl = db.getTable(dbName, tblName);
+    Table tbl = db.getTable(desc.getTableName());
 
     String partName = null;
     if (desc.getPartSpec() == null) {
@@ -2231,15 +2209,7 @@ public class DDLTask extends Task<DDLWor
   public int showColumns(Hive db, ShowColumnsDesc showCols)
       throws HiveException {
 
-    String dbName = showCols.getDbName();
-    String tableName = showCols.getTableName();
-    Table table = null;
-    if (dbName == null) {
-      table = db.getTable(tableName);
-    }
-    else {
-      table = db.getTable(dbName, tableName);
-    }
+    Table table = db.getTable(showCols.getTableName());
 
     // write the results in the file
     DataOutputStream outStream = null;
@@ -3278,7 +3248,8 @@ public class DDLTask extends Task<DDLWor
     Table oldTbl = tbl.copy();
 
     if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.RENAME) {
-      tbl.setTableName(alterTbl.getNewName());
+      tbl.setDbName(Utilities.getDatabaseName(alterTbl.getNewName()));
+      tbl.setTableName(Utilities.getTableName(alterTbl.getNewName()));
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ADDCOLS) {
       List<FieldSchema> newCols = alterTbl.getNewCols();
       List<FieldSchema> oldCols = tbl.getCols();
@@ -4307,10 +4278,12 @@ public class DDLTask extends Task<DDLWor
     if (crtIndex.getLocation() == null) {
       // Location is not set, leave it as-is if index doesn't belong to default DB
       // Currently all indexes are created in current DB only
-      if (db.getDatabaseCurrent().getName().equalsIgnoreCase(MetaStoreUtils.DEFAULT_DATABASE_NAME)) {
+      if (Utilities.getDatabaseName(name).equalsIgnoreCase(MetaStoreUtils.DEFAULT_DATABASE_NAME)) {
         // Default database name path is always ignored, use METASTOREWAREHOUSE and object name
         // instead
-        path = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREWAREHOUSE), name.toLowerCase());
+        String warehouse = HiveConf.getVar(conf, ConfVars.METASTOREWAREHOUSE);
+        String tableName = Utilities.getTableName(name);
+        path = new Path(warehouse, tableName.toLowerCase());
       }
     }
     else {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Mon Aug 11 16:56:30 2014
@@ -1382,9 +1382,8 @@ public final class Utilities {
   public static RCFile.Writer createRCFileWriter(JobConf jc, FileSystem fs, Path file,
       boolean isCompressed, Progressable progressable) throws IOException {
     CompressionCodec codec = null;
-    Class<?> codecClass = null;
     if (isCompressed) {
-      codecClass = FileOutputFormat.getOutputCompressorClass(jc, DefaultCodec.class);
+      Class<?> codecClass = FileOutputFormat.getOutputCompressorClass(jc, DefaultCodec.class);
       codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, jc);
     }
     return new RCFile.Writer(fs, jc, file, progressable, codec);
@@ -2048,19 +2047,53 @@ public final class Utilities {
    * @return String array with two elements, first is db name, second is table name
    * @throws HiveException
    */
-  public static String[] getDbTableName(String dbtable) throws HiveException{
-    if(dbtable == null){
+  public static String[] getDbTableName(String dbtable) throws SemanticException {
+    return getDbTableName(SessionState.get().getCurrentDatabase(), dbtable);
+  }
+
+  public static String[] getDbTableName(String defaultDb, String dbtable) throws SemanticException {
+    if (dbtable == null) {
       return new String[2];
     }
     String[] names =  dbtable.split("\\.");
     switch (names.length) {
-    case 2:
-      return names;
-    case 1:
-      return new String [] {SessionState.get().getCurrentDatabase(), dbtable};
-    default:
-      throw new HiveException(ErrorMsg.INVALID_TABLE_NAME, dbtable);
+      case 2:
+        return names;
+      case 1:
+        return new String [] {defaultDb, dbtable};
+      default:
+        throw new SemanticException(ErrorMsg.INVALID_TABLE_NAME, dbtable);
+    }
+  }
+
+  /**
+   * Accepts qualified name which is in the form of dbname.tablename and returns dbname from it
+   *
+   * @param dbTableName
+   * @return dbname
+   * @throws SemanticException input string is not qualified name
+   */
+  public static String getDatabaseName(String dbTableName) throws SemanticException {
+    String[] split = dbTableName.split("\\.");
+    if (split.length != 2) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE_NAME, dbTableName);
+    }
+    return split[0];
+  }
+
+  /**
+   * Accepts qualified name which is in the form of dbname.tablename and returns tablename from it
+   *
+   * @param dbTableName
+   * @return tablename
+   * @throws SemanticException input string is not qualified name
+   */
+  public static String getTableName(String dbTableName) throws SemanticException {
+    String[] split = dbTableName.split("\\.");
+    if (split.length != 2) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE_NAME, dbTableName);
     }
+    return split[1];
   }
 
   public static void validateColumnNames(List<String> colNames, List<String> checkCols)

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Mon Aug 11 16:56:30 2014
@@ -409,6 +409,12 @@ public class Hive {
     }
   }
 
+  public void alterIndex(String baseTableName, String indexName, Index newIdx)
+      throws InvalidOperationException, HiveException {
+    String[] names = Utilities.getDbTableName(baseTableName);
+    alterIndex(names[0], names[1], indexName, newIdx);
+  }
+
   /**
    * Updates the existing index metadata with the new metadata.
    *
@@ -667,17 +673,16 @@ public class Hive {
       throws HiveException {
 
     try {
-      String dbName = SessionState.get().getCurrentDatabase();
       Index old_index = null;
       try {
-        old_index = getIndex(dbName, tableName, indexName);
+        old_index = getIndex(tableName, indexName);
       } catch (Exception e) {
       }
       if (old_index != null) {
-        throw new HiveException("Index " + indexName + " already exists on table " + tableName + ", db=" + dbName);
+        throw new HiveException("Index " + indexName + " already exists on table " + tableName);
       }
 
-      org.apache.hadoop.hive.metastore.api.Table baseTbl = getMSC().getTable(dbName, tableName);
+      org.apache.hadoop.hive.metastore.api.Table baseTbl = getTable(tableName).getTTable();
       if (baseTbl.getTableType() == TableType.VIRTUAL_VIEW.toString()) {
         throw new HiveException("tableName="+ tableName +" is a VIRTUAL VIEW. Index on VIRTUAL VIEW is not supported.");
       }
@@ -686,17 +691,13 @@ public class Hive {
             + " is a TEMPORARY TABLE. Index on TEMPORARY TABLE is not supported.");
       }
 
-      if (indexTblName == null) {
-        indexTblName = MetaStoreUtils.getIndexTableName(dbName, tableName, indexName);
-      } else {
-        org.apache.hadoop.hive.metastore.api.Table temp = null;
-        try {
-          temp = getMSC().getTable(dbName, indexTblName);
-        } catch (Exception e) {
-        }
-        if (temp != null) {
-          throw new HiveException("Table name " + indexTblName + " already exists. Choose another name.");
-        }
+      org.apache.hadoop.hive.metastore.api.Table temp = null;
+      try {
+        temp = getTable(indexTblName).getTTable();
+      } catch (Exception e) {
+      }
+      if (temp != null) {
+        throw new HiveException("Table name " + indexTblName + " already exists. Choose another name.");
       }
 
       org.apache.hadoop.hive.metastore.api.StorageDescriptor storageDescriptor = baseTbl.getSd().deepCopy();
@@ -774,7 +775,9 @@ public class Hive {
       HiveIndexHandler indexHandler = HiveUtils.getIndexHandler(this.getConf(), indexHandlerClass);
 
       if (indexHandler.usesIndexTable()) {
-        tt = new org.apache.hadoop.hive.ql.metadata.Table(dbName, indexTblName).getTTable();
+        String idname = Utilities.getDatabaseName(indexTblName);
+        String itname = Utilities.getTableName(indexTblName);
+        tt = new org.apache.hadoop.hive.ql.metadata.Table(idname, itname).getTTable();
         List<FieldSchema> partKeys = baseTbl.getPartitionKeys();
         tt.setPartitionKeys(partKeys);
         tt.setTableType(TableType.INDEX_TABLE.toString());
@@ -798,7 +801,9 @@ public class Hive {
         throw new RuntimeException("Please specify deferred rebuild using \" WITH DEFERRED REBUILD \".");
       }
 
-      Index indexDesc = new Index(indexName, indexHandlerClass, dbName, tableName, time, time, indexTblName,
+      String tdname = Utilities.getDatabaseName(tableName);
+      String ttname = Utilities.getTableName(tableName);
+      Index indexDesc = new Index(indexName, indexHandlerClass, tdname, ttname, time, time, indexTblName,
           storageDescriptor, params, deferredRebuild);
       if (indexComment != null) {
         indexDesc.getParameters().put("comment", indexComment);
@@ -818,19 +823,6 @@ public class Hive {
     }
   }
 
-  public Index getIndex(String qualifiedIndexName) throws HiveException {
-    String[] names = getQualifiedNames(qualifiedIndexName);
-    switch (names.length) {
-    case 3:
-      return getIndex(names[0], names[1], names[2]);
-    case 2:
-      return getIndex(SessionState.get().getCurrentDatabase(),
-          names[0], names[1]);
-    default:
-      throw new HiveException("Invalid index name:" + qualifiedIndexName);
-    }
-  }
-
   public Index getIndex(String baseTableName, String indexName) throws HiveException {
     String[] names = Utilities.getDbTableName(baseTableName);
     return this.getIndex(names[0], names[1], indexName);
@@ -845,6 +837,11 @@ public class Hive {
     }
   }
 
+  public boolean dropIndex(String baseTableName, String index_name, boolean deleteData) throws HiveException {
+    String[] names = Utilities.getDbTableName(baseTableName);
+    return dropIndex(names[0], names[1], index_name, deleteData);
+  }
+
   public boolean dropIndex(String db_name, String tbl_name, String index_name, boolean deleteData) throws HiveException {
     try {
       return getMSC().dropIndex(db_name, tbl_name, index_name, deleteData);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IndexUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IndexUtils.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IndexUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IndexUtils.java Mon Aug 11 16:56:30 2014
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.index.IndexMetadataChangeTask;
@@ -99,7 +100,10 @@ public final class IndexUtils {
     }
 
     for (Index index : indexes) {
-      Table indexTable = hive.getTable(index.getIndexTableName());
+      // index.getDbName() is used as a default database, which is database of target table,
+      // if index.getIndexTableName() does not contain database name
+      String[] qualified = Utilities.getDbTableName(index.getDbName(), index.getIndexTableName());
+      Table indexTable = hive.getTable(qualified[0], qualified[1]);
       // get partitions that match the spec
       Partition matchingPartition = hive.getPartition(indexTable, partSpec, false);
       if (matchingPartition == null) {
@@ -180,8 +184,8 @@ public final class IndexUtils {
   public static List<Index> getIndexes(Table baseTableMetaData, List<String> matchIndexTypes)
     throws SemanticException {
     List<Index> matchingIndexes = new ArrayList<Index>();
-    List<Index> indexesOnTable = null;
 
+    List<Index> indexesOnTable;
     try {
       indexesOnTable = baseTableMetaData.getAllIndexes((short) -1); // get all indexes
     } catch (HiveException e) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java Mon Aug 11 16:56:30 2014
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.index.AggregateIndexHandler;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -336,8 +337,9 @@ public class RewriteGBUsingIndex impleme
       // index is changed.
       List<String> idxTblColNames = new ArrayList<String>();
       try {
-        Table idxTbl = hiveInstance.getTable(index.getDbName(),
+        String[] qualified = Utilities.getDbTableName(index.getDbName(),
             index.getIndexTableName());
+        Table idxTbl = hiveInstance.getTable(qualified[0], qualified[1]);
         for (FieldSchema idxTblCol : idxTbl.getCols()) {
           idxTblColNames.add(idxTblCol.getName());
         }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java Mon Aug 11 16:56:30 2014
@@ -34,6 +34,7 @@ import java.util.Map.Entry;
 
 import org.antlr.runtime.tree.CommonTree;
 import org.antlr.runtime.tree.Tree;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -305,6 +306,28 @@ public abstract class BaseSemanticAnalyz
     return unescapeIdentifier(tableOrColumnNode.getText());
   }
 
+  public static String[] getQualifiedTableName(ASTNode tabNameNode) throws SemanticException {
+    if (tabNameNode.getType() != HiveParser.TOK_TABNAME ||
+        (tabNameNode.getChildCount() != 1 && tabNameNode.getChildCount() != 2)) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE_NAME.getMsg(tabNameNode));
+    }
+    if (tabNameNode.getChildCount() == 2) {
+      String dbName = unescapeIdentifier(tabNameNode.getChild(0).getText());
+      String tableName = unescapeIdentifier(tabNameNode.getChild(1).getText());
+      return new String[] {dbName, tableName};
+    }
+    String tableName = unescapeIdentifier(tabNameNode.getChild(0).getText());
+    return new String[]{SessionState.get().getCurrentDatabase(), tableName};
+  }
+
+  public static String getDotName(String[] qname) throws SemanticException {
+    String genericName = StringUtils.join(qname, ".");
+    if (qname.length != 2) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE_NAME, genericName);
+    }
+    return genericName;
+  }
+
   /**
    * Get the unqualified name from a table node.
    *
@@ -817,9 +840,9 @@ public abstract class BaseSemanticAnalyz
     this.columnAccessInfo = columnAccessInfo;
   }
 
-  protected HashMap<String, String> extractPartitionSpecs(Tree partspec)
+  protected LinkedHashMap<String, String> extractPartitionSpecs(Tree partspec)
       throws SemanticException {
-    HashMap<String, String> partSpec = new LinkedHashMap<String, String>();
+    LinkedHashMap<String, String> partSpec = new LinkedHashMap<String, String>();
     for (int i = 0; i < partspec.getChildCount(); ++i) {
       CommonTree partspec_val = (CommonTree) partspec.getChild(i);
       String val = stripQuotes(partspec_val.getChild(1).getText());
@@ -1176,23 +1199,16 @@ public abstract class BaseSemanticAnalyz
     }
   }
 
+  protected Table getTable(String[] qualified) throws SemanticException {
+    return getTable(qualified[0], qualified[1], true);
+  }
+
   protected Table getTable(String tblName) throws SemanticException {
     return getTable(null, tblName, true);
   }
 
   protected Table getTable(String tblName, boolean throwException) throws SemanticException {
-    String currentDb = SessionState.get().getCurrentDatabase();
-    return getTable(currentDb, tblName, throwException);
-  }
-
-  // qnName : possibly contains database name (dot separated)
-  protected Table getTableWithQN(String qnName, boolean throwException) throws SemanticException {
-    int dot = qnName.indexOf('.');
-    if (dot < 0) {
-      String currentDb = SessionState.get().getCurrentDatabase();
-      return getTable(currentDb, qnName, throwException);
-    }
-    return getTable(qnName.substring(0, dot), qnName.substring(dot + 1), throwException);
+    return getTable(null, tblName, throwException);
   }
 
   protected Table getTable(String database, String tblName, boolean throwException)

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java Mon Aug 11 16:56:30 2014
@@ -18,8 +18,11 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -42,7 +45,13 @@ public class ColumnAccessInfo {
     tableColumns.add(col);
   }
 
-  public Map<String, Set<String>> getTableToColumnAccessMap() {
-    return tableToColumnAccessMap;
+  public Map<String, List<String>> getTableToColumnAccessMap() {
+    Map<String, List<String>> mapping = new HashMap<String, List<String>>();
+    for (Map.Entry<String, Set<String>> entry : tableToColumnAccessMap.entrySet()) {
+      List<String> sortedCols = new ArrayList<String>(entry.getValue());
+      Collections.sort(sortedCols);
+      mapping.put(entry.getKey(), sortedCols);
+    }
+    return mapping;
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Mon Aug 11 16:56:30 2014
@@ -217,7 +217,7 @@ public class DDLSemanticAnalyzer extends
     }
 
     public TablePartition(ASTNode tblPart) throws SemanticException {
-      tableName = unescapeIdentifier(tblPart.getChild(0).getText());
+      tableName = getDotName((getQualifiedTableName((ASTNode) tblPart.getChild(0))));
       if (tblPart.getChildCount() > 1) {
         ASTNode part = (ASTNode) tblPart.getChild(1);
         if (part.getToken().getType() == HiveParser.TOK_PARTSPEC) {
@@ -1015,7 +1015,7 @@ public class DDLSemanticAnalyzer extends
   private void analyzeCreateIndex(ASTNode ast) throws SemanticException {
     String indexName = unescapeIdentifier(ast.getChild(0).getText());
     String typeName = unescapeSQLString(ast.getChild(1).getText());
-    String tableName = getUnescapedName((ASTNode) ast.getChild(2));
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(2));
     List<String> indexedCols = getColumnNames((ASTNode) ast.getChild(3));
 
     IndexType indexType = HiveIndex.getIndexType(typeName);
@@ -1081,8 +1081,14 @@ public class DDLSemanticAnalyzer extends
 
     storageFormat.fillDefaultStorageFormat();
 
+    if (indexTableName == null) {
+      indexTableName = MetaStoreUtils.getIndexTableName(qualified[0], qualified[1], indexName);
+      indexTableName = qualified[0] + "." + indexTableName; // on same database with base table
+    } else {
+      indexTableName = getDotName(Utilities.getDbTableName(indexTableName));
+    }
 
-    CreateIndexDesc crtIndexDesc = new CreateIndexDesc(tableName, indexName,
+    CreateIndexDesc crtIndexDesc = new CreateIndexDesc(getDotName(qualified), indexName,
         indexedCols, indexTableName, deferredRebuild, storageFormat.getInputFormat(),
         storageFormat.getOutputFormat(),
         storageFormat.getStorageHandler(), typeName, location, idxProps, tblProps,
@@ -1116,21 +1122,20 @@ public class DDLSemanticAnalyzer extends
   }
 
   private void analyzeAlterIndexRebuild(ASTNode ast) throws SemanticException {
-    String baseTableName = unescapeIdentifier(ast.getChild(0).getText());
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     String indexName = unescapeIdentifier(ast.getChild(1).getText());
     HashMap<String, String> partSpec = null;
     Tree part = ast.getChild(2);
     if (part != null) {
       partSpec = extractPartitionSpecs(part);
     }
-    List<Task<?>> indexBuilder = getIndexBuilderMapRed(baseTableName, indexName, partSpec);
+    List<Task<?>> indexBuilder = getIndexBuilderMapRed(qualified, indexName, partSpec);
     rootTasks.addAll(indexBuilder);
 
     // Handle updating index timestamps
     AlterIndexDesc alterIdxDesc = new AlterIndexDesc(AlterIndexTypes.UPDATETIMESTAMP);
     alterIdxDesc.setIndexName(indexName);
-    alterIdxDesc.setBaseTableName(baseTableName);
-    alterIdxDesc.setDbName(SessionState.get().getCurrentDatabase());
+    alterIdxDesc.setBaseTableName(getDotName(qualified));
     alterIdxDesc.setSpec(partSpec);
 
     Task<?> tsTask = TaskFactory.get(new DDLWork(alterIdxDesc), conf);
@@ -1142,27 +1147,28 @@ public class DDLSemanticAnalyzer extends
   private void analyzeAlterIndexProps(ASTNode ast)
       throws SemanticException {
 
-    String baseTableName = getUnescapedName((ASTNode) ast.getChild(0));
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     String indexName = unescapeIdentifier(ast.getChild(1).getText());
     HashMap<String, String> mapProp = getProps((ASTNode) (ast.getChild(2))
         .getChild(0));
 
-    AlterIndexDesc alterIdxDesc =
-        new AlterIndexDesc(AlterIndexTypes.ADDPROPS);
+    AlterIndexDesc alterIdxDesc = new AlterIndexDesc(AlterIndexTypes.ADDPROPS);
     alterIdxDesc.setProps(mapProp);
     alterIdxDesc.setIndexName(indexName);
-    alterIdxDesc.setBaseTableName(baseTableName);
-    alterIdxDesc.setDbName(SessionState.get().getCurrentDatabase());
+    alterIdxDesc.setBaseTableName(getDotName(qualified));
 
     rootTasks.add(TaskFactory.get(new DDLWork(alterIdxDesc), conf));
   }
 
-  private List<Task<?>> getIndexBuilderMapRed(String baseTableName, String indexName,
+  private List<Task<?>> getIndexBuilderMapRed(String[] names, String indexName,
       HashMap<String, String> partSpec) throws SemanticException {
     try {
-      String dbName = SessionState.get().getCurrentDatabase();
-      Index index = db.getIndex(dbName, baseTableName, indexName);
-      Table indexTbl = getTable(index.getIndexTableName());
+      Index index = db.getIndex(names[0], names[1], indexName);
+      Table indexTbl = null;
+      String indexTableName = index.getIndexTableName();
+      if (indexTableName != null) {
+        indexTbl = getTable(Utilities.getDbTableName(index.getDbName(), indexTableName));
+      }
       String baseTblName = index.getOrigTableName();
       Table baseTbl = getTable(baseTblName);
 
@@ -1474,7 +1480,7 @@ public class DDLSemanticAnalyzer extends
       boolean checkIndex = HiveConf.getBoolVar(conf,
           HiveConf.ConfVars.HIVE_CONCATENATE_CHECK_INDEX);
       if (checkIndex) {
-        List<Index> indexes = db.getIndexes(tblObj.getDbName(), tableName,
+        List<Index> indexes = db.getIndexes(tblObj.getDbName(), tblObj.getTableName(),
             Short.MAX_VALUE);
         if (indexes != null && indexes.size() > 0) {
           throw new SemanticException("can not do merge because source table "
@@ -1633,7 +1639,7 @@ public class DDLSemanticAnalyzer extends
     LinkedHashMap<String, String> newPartSpec = null;
     if (partSpec != null) newPartSpec = new LinkedHashMap<String, String>(partSpec);
 
-    AlterTableSimpleDesc desc = new AlterTableSimpleDesc(SessionState.get().getCurrentDatabase(),
+    AlterTableSimpleDesc desc = new AlterTableSimpleDesc(
         tableName, newPartSpec, type);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
@@ -2098,25 +2104,17 @@ public class DDLSemanticAnalyzer extends
   }
 
   private void analyzeShowColumns(ASTNode ast) throws SemanticException {
-    ShowColumnsDesc showColumnsDesc;
-    String dbName = null;
-    String tableName = null;
-    switch (ast.getChildCount()) {
-    case 1:
-      tableName = getUnescapedName((ASTNode) ast.getChild(0));
-      break;
-    case 2:
-      dbName = getUnescapedName((ASTNode) ast.getChild(0));
-      tableName = getUnescapedName((ASTNode) ast.getChild(1));
-      break;
-    default:
-      break;
+    String tableName = getUnescapedName((ASTNode) ast.getChild(0));
+    if (ast.getChildCount() > 1) {
+      if (tableName.contains(".")) {
+        throw new SemanticException("Duplicates declaration for database name");
+      }
+      tableName = getUnescapedName((ASTNode) ast.getChild(1)) + "." + tableName;
     }
-
-    Table tab = getTable(dbName, tableName, true);
+    Table tab = getTable(tableName);
     inputs.add(new ReadEntity(tab));
 
-    showColumnsDesc = new ShowColumnsDesc(ctx.getResFile(), dbName, tableName);
+    ShowColumnsDesc showColumnsDesc = new ShowColumnsDesc(ctx.getResFile(), tableName);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         showColumnsDesc), conf));
     setFetchTask(createFetchTask(showColumnsDesc.getSchema()));
@@ -2157,13 +2155,13 @@ public class DDLSemanticAnalyzer extends
 
   private void analyzeShowTableProperties(ASTNode ast) throws SemanticException {
     ShowTblPropertiesDesc showTblPropertiesDesc;
-    String tableNames = getUnescapedName((ASTNode) ast.getChild(0));
-    String dbName = SessionState.get().getCurrentDatabase();
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     String propertyName = null;
     if (ast.getChildCount() > 1) {
       propertyName = unescapeSQLString(ast.getChild(1).getText());
     }
 
+    String tableNames = getDotName(qualified);
     validateTable(tableNames, null);
 
     showTblPropertiesDesc = new ShowTblPropertiesDesc(ctx.getResFile().toString(), tableNames,
@@ -2437,17 +2435,20 @@ public class DDLSemanticAnalyzer extends
 
 
   private void analyzeAlterTableRename(ASTNode ast, boolean expectView) throws SemanticException {
-    String tblName = getUnescapedName((ASTNode) ast.getChild(0));
-    AlterTableDesc alterTblDesc = new AlterTableDesc(tblName,
-        getUnescapedName((ASTNode) ast.getChild(1)), expectView);
+    String[] source = getQualifiedTableName((ASTNode) ast.getChild(0));
+    String[] target = getQualifiedTableName((ASTNode) ast.getChild(1));
 
-    addInputsOutputsAlterTable(tblName, null, alterTblDesc);
+    String sourceName = getDotName(source);
+    String targetName = getDotName(target);
+
+    AlterTableDesc alterTblDesc = new AlterTableDesc(sourceName, targetName, expectView);
+    addInputsOutputsAlterTable(sourceName, null, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         alterTblDesc), conf));
   }
 
   private void analyzeAlterTableRenameCol(ASTNode ast) throws SemanticException {
-    String tblName = getUnescapedName((ASTNode) ast.getChild(0));
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     String newComment = null;
     String newType = null;
     newType = getTypeStringFromAST((ASTNode) ast.getChild(3));
@@ -2477,7 +2478,7 @@ public class DDLSemanticAnalyzer extends
     String newColName = ast.getChild(2).getText();
 
     /* Validate the operation of renaming a column name. */
-    Table tab = getTable(tblName);
+    Table tab = getTable(qualified);
 
     SkewedInfo skewInfo = tab.getTTable().getSd().getSkewedInfo();
     if ((null != skewInfo)
@@ -2487,6 +2488,7 @@ public class DDLSemanticAnalyzer extends
           + ErrorMsg.ALTER_TABLE_NOT_ALLOWED_RENAME_SKEWED_COLUMN.getMsg());
     }
 
+    String tblName = getDotName(qualified);
     AlterTableDesc alterTblDesc = new AlterTableDesc(tblName,
         unescapeIdentifier(oldColName), unescapeIdentifier(newColName),
         newType, newComment, first, flagCol);
@@ -2511,9 +2513,8 @@ public class DDLSemanticAnalyzer extends
     List<Map<String, String>> partSpecs = new ArrayList<Map<String, String>>();
     partSpecs.add(oldPartSpec);
     partSpecs.add(newPartSpec);
-    addTablePartsOutputs(tblName, partSpecs, WriteEntity.WriteType.DDL_EXCLUSIVE);
-    RenamePartitionDesc renamePartitionDesc = new RenamePartitionDesc(
-        SessionState.get().getCurrentDatabase(), tblName, oldPartSpec, newPartSpec);
+    addTablePartsOutputs(tab, partSpecs, WriteEntity.WriteType.DDL_EXCLUSIVE);
+    RenamePartitionDesc renamePartitionDesc = new RenamePartitionDesc(tblName, oldPartSpec, newPartSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         renamePartitionDesc), conf));
   }
@@ -2536,7 +2537,9 @@ public class DDLSemanticAnalyzer extends
 
   private void analyzeAlterTableModifyCols(ASTNode ast,
       AlterTableTypes alterType) throws SemanticException {
-    String tblName = getUnescapedName((ASTNode) ast.getChild(0));
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
+
+    String tblName = getDotName(qualified);
     List<FieldSchema> newCols = getColumns((ASTNode) ast.getChild(1));
     AlterTableDesc alterTblDesc = new AlterTableDesc(tblName, newCols,
         alterType);
@@ -2559,8 +2562,8 @@ public class DDLSemanticAnalyzer extends
     // popular case but that's kinda hacky. Let's not do it for now.
     boolean canGroupExprs = ifExists;
 
-    String tblName = getUnescapedName((ASTNode) ast.getChild(0));
-    Table tab = getTable(tblName, true);
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
+    Table tab = getTable(qualified);
     Map<Integer, List<ExprNodeGenericFuncDesc>> partSpecs =
         getFullPartitionSpecs(ast, tab, canGroupExprs);
     if (partSpecs.isEmpty()) return; // nothing to do
@@ -2574,24 +2577,19 @@ public class DDLSemanticAnalyzer extends
     addTableDropPartsOutputs(tab, partSpecs.values(), !ifExists, ignoreProtection);
 
     DropTableDesc dropTblDesc =
-        new DropTableDesc(tblName, partSpecs, expectView, ignoreProtection);
+        new DropTableDesc(getDotName(qualified), partSpecs, expectView, ignoreProtection);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTblDesc), conf));
   }
 
   private void analyzeAlterTablePartColType(ASTNode ast)
       throws SemanticException {
     // get table name
-    String tblName = getUnescapedName((ASTNode)ast.getChild(0));
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
 
-    Table tab = null;
 
     // check if table exists.
-    try {
-      tab = getTable(tblName, true);
-      inputs.add(new ReadEntity(tab));
-    } catch (HiveException e) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName));
-    }
+    Table tab = getTable(qualified);
+    inputs.add(new ReadEntity(tab));
 
     // validate the DDL is a valid operation on the table.
     validateAlterTableType(tab, AlterTableTypes.ALTERPARTITION, false);
@@ -2625,7 +2623,7 @@ public class DDLSemanticAnalyzer extends
     }
 
     AlterTableAlterPartDesc alterTblAlterPartDesc =
-            new AlterTableAlterPartDesc(SessionState.get().getCurrentDatabase(), tblName, newCol);
+            new AlterTableAlterPartDesc(getDotName(qualified), newCol);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
             alterTblAlterPartDesc), conf));
@@ -2648,10 +2646,10 @@ public class DDLSemanticAnalyzer extends
       throws SemanticException {
 
     // ^(TOK_ALTERTABLE_ADDPARTS identifier ifNotExists? alterStatementSuffixAddPartitionsElement+)
-    String tblName = getUnescapedName((ASTNode)ast.getChild(0));
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     boolean ifNotExists = ast.getChild(1).getType() == HiveParser.TOK_IFNOTEXISTS;
 
-    Table tab = getTable(tblName, true);
+    Table tab = getTable(qualified);
     boolean isView = tab.isView();
     validateAlterTableType(tab, AlterTableTypes.ADDPARTITION, expectView);
     outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_SHARED));
@@ -2662,7 +2660,8 @@ public class DDLSemanticAnalyzer extends
     String currentLocation = null;
     Map<String, String> currentPart = null;
     // Parser has done some verification, so the order of tokens doesn't need to be verified here.
-    AddPartitionDesc addPartitionDesc = new AddPartitionDesc(tab.getDbName(), tblName, ifNotExists);
+    AddPartitionDesc addPartitionDesc =
+        new AddPartitionDesc(tab.getDbName(), tab.getTableName(), ifNotExists);
     for (int num = start; num < numCh; num++) {
       ASTNode child = (ASTNode) ast.getChild(num);
       switch (child.getToken().getType()) {
@@ -2683,7 +2682,7 @@ public class DDLSemanticAnalyzer extends
         currentLocation = unescapeSQLString(child.getChild(0).getText());
         boolean isLocal = false;
         try {
-          // do best effor to determine if this is a local file
+          // do best effort to determine if this is a local file
           String scheme = new URI(currentLocation).getScheme();
           if (scheme != null) {
             isLocal = FileUtils.isLocalFile(conf, currentLocation);
@@ -2714,7 +2713,7 @@ public class DDLSemanticAnalyzer extends
       // Compile internal query to capture underlying table partition dependencies
       StringBuilder cmd = new StringBuilder();
       cmd.append("SELECT * FROM ");
-      cmd.append(HiveUtils.unparseIdentifier(tblName));
+      cmd.append(HiveUtils.unparseIdentifier(getDotName(qualified)));
       cmd.append(" WHERE ");
       boolean firstOr = true;
       for (int i = 0; i < addPartitionDesc.getPartitionCount(); ++i) {
@@ -2775,9 +2774,9 @@ public class DDLSemanticAnalyzer extends
    */
   private void analyzeAlterTableTouch(CommonTree ast)
       throws SemanticException {
+    String[] qualified = getQualifiedTableName((ASTNode)ast.getChild(0));
 
-    String tblName = getUnescapedName((ASTNode)ast.getChild(0));
-    Table tab = getTable(tblName, true);
+    Table tab = getTable(qualified);
     validateAlterTableType(tab, AlterTableTypes.TOUCH);
     inputs.add(new ReadEntity(tab));
 
@@ -2786,16 +2785,16 @@ public class DDLSemanticAnalyzer extends
 
     if (partSpecs.size() == 0) {
       AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc(
-          SessionState.get().getCurrentDatabase(), tblName, null,
+          getDotName(qualified), null,
           AlterTableDesc.AlterTableTypes.TOUCH);
       outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_NO_LOCK));
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           touchDesc), conf));
     } else {
-      addTablePartsOutputs(tblName, partSpecs, WriteEntity.WriteType.DDL_NO_LOCK);
+      addTablePartsOutputs(tab, partSpecs, WriteEntity.WriteType.DDL_NO_LOCK);
       for (Map<String, String> partSpec : partSpecs) {
         AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc(
-            SessionState.get().getCurrentDatabase(), tblName, partSpec,
+            getDotName(qualified), partSpec,
             AlterTableDesc.AlterTableTypes.TOUCH);
         rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
             touchDesc), conf));
@@ -2810,12 +2809,12 @@ public class DDLSemanticAnalyzer extends
       throw new SemanticException(ErrorMsg.ARCHIVE_METHODS_DISABLED.getMsg());
 
     }
-    String tblName = getUnescapedName((ASTNode) ast.getChild(0));
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     // partition name to value
     List<Map<String, String>> partSpecs = getPartitionSpecs(ast);
 
-    Table tab = getTable(tblName, true);
-    addTablePartsOutputs(tblName, partSpecs, true, WriteEntity.WriteType.DDL_NO_LOCK);
+    Table tab = getTable(qualified);
+    addTablePartsOutputs(tab, partSpecs, true, WriteEntity.WriteType.DDL_NO_LOCK);
     validateAlterTableType(tab, AlterTableTypes.ARCHIVE);
     inputs.add(new ReadEntity(tab));
 
@@ -2835,7 +2834,7 @@ public class DDLSemanticAnalyzer extends
       throw new SemanticException(e.getMessage(), e);
     }
     AlterTableSimpleDesc archiveDesc = new AlterTableSimpleDesc(
-        SessionState.get().getCurrentDatabase(), tblName, partSpec,
+        getDotName(qualified), partSpec,
         (isUnArchive ? AlterTableTypes.UNARCHIVE : AlterTableTypes.ARCHIVE));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         archiveDesc), conf));
@@ -3003,20 +3002,20 @@ public class DDLSemanticAnalyzer extends
    * Add the table partitions to be modified in the output, so that it is available for the
    * pre-execution hook. If the partition does not exist, no error is thrown.
    */
-  private void addTablePartsOutputs(String tblName, List<Map<String, String>> partSpecs,
+  private void addTablePartsOutputs(Table table, List<Map<String, String>> partSpecs,
                                     WriteEntity.WriteType writeType)
       throws SemanticException {
-    addTablePartsOutputs(tblName, partSpecs, false, false, null, writeType);
+    addTablePartsOutputs(table, partSpecs, false, false, null, writeType);
   }
 
   /**
    * Add the table partitions to be modified in the output, so that it is available for the
    * pre-execution hook. If the partition does not exist, no error is thrown.
    */
-  private void addTablePartsOutputs(String tblName, List<Map<String, String>> partSpecs,
+  private void addTablePartsOutputs(Table table, List<Map<String, String>> partSpecs,
       boolean allowMany, WriteEntity.WriteType writeType)
       throws SemanticException {
-    addTablePartsOutputs(tblName, partSpecs, false, allowMany, null, writeType);
+    addTablePartsOutputs(table, partSpecs, false, allowMany, null, writeType);
   }
 
   /**
@@ -3024,10 +3023,9 @@ public class DDLSemanticAnalyzer extends
    * pre-execution hook. If the partition does not exist, throw an error if
    * throwIfNonExistent is true, otherwise ignore it.
    */
-  private void addTablePartsOutputs(String tblName, List<Map<String, String>> partSpecs,
+  private void addTablePartsOutputs(Table table, List<Map<String, String>> partSpecs,
       boolean throwIfNonExistent, boolean allowMany, ASTNode ast, WriteEntity.WriteType writeType)
       throws SemanticException {
-    Table tab = getTable(tblName);
 
     Iterator<Map<String, String>> i;
     int index;
@@ -3036,7 +3034,7 @@ public class DDLSemanticAnalyzer extends
       List<Partition> parts = null;
       if (allowMany) {
         try {
-          parts = db.getPartitions(tab, partSpec);
+          parts = db.getPartitions(table, partSpec);
         } catch (HiveException e) {
           LOG.error("Got HiveException during obtaining list of partitions"
               + StringUtils.stringifyException(e));
@@ -3045,7 +3043,7 @@ public class DDLSemanticAnalyzer extends
       } else {
         parts = new ArrayList<Partition>();
         try {
-          Partition p = db.getPartition(tab, partSpec, false);
+          Partition p = db.getPartition(table, partSpec, false);
           if (p != null) {
             parts.add(p);
           }
@@ -3125,14 +3123,15 @@ public class DDLSemanticAnalyzer extends
      */
     HiveConf hiveConf = SessionState.get().getConf();
 
-    String tableName = getUnescapedName((ASTNode) ast.getChild(0));
-    Table tab = getTable(tableName, true);
+    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
+    Table tab = getTable(qualified);
 
     inputs.add(new ReadEntity(tab));
     outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_EXCLUSIVE));
 
     validateAlterTableType(tab, AlterTableTypes.ADDSKEWEDBY);
 
+    String tableName = getDotName(qualified);
     if (ast.getChildCount() == 1) {
       /* Convert a skewed table to non-skewed table. */
       AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, true,

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g Mon Aug 11 16:56:30 2014
@@ -953,8 +953,8 @@ alterTableStatementSuffix
 alterStatementPartitionKeyType
 @init {msgs.push("alter partition key type"); }
 @after {msgs.pop();}
-	: identifier KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
-	-> ^(TOK_ALTERTABLE_PARTCOLTYPE identifier columnNameType)
+	: tableName KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
+	-> ^(TOK_ALTERTABLE_PARTCOLTYPE tableName columnNameType)
 	;
 
 alterViewStatementSuffix
@@ -974,16 +974,14 @@ alterViewStatementSuffix
 alterIndexStatementSuffix
 @init { pushMsg("alter index statement", state); }
 @after { popMsg(state); }
-    : indexName=identifier
-      (KW_ON tableNameId=identifier)
-      partitionSpec?
+    : indexName=identifier KW_ON tableName partitionSpec?
     (
       KW_REBUILD
-      ->^(TOK_ALTERINDEX_REBUILD $tableNameId $indexName partitionSpec?)
+      ->^(TOK_ALTERINDEX_REBUILD tableName $indexName partitionSpec?)
     |
       KW_SET KW_IDXPROPERTIES
       indexProperties
-      ->^(TOK_ALTERINDEX_PROPERTIES $tableNameId $indexName indexProperties)
+      ->^(TOK_ALTERINDEX_PROPERTIES tableName $indexName indexProperties)
     )
     ;
 
@@ -1011,23 +1009,23 @@ alterDatabaseSuffixSetOwner
 alterStatementSuffixRename
 @init { pushMsg("rename statement", state); }
 @after { popMsg(state); }
-    : oldName=identifier KW_RENAME KW_TO newName=identifier
+    : oldName=tableName KW_RENAME KW_TO newName=tableName
     -> ^(TOK_ALTERTABLE_RENAME $oldName $newName)
     ;
 
 alterStatementSuffixAddCol
 @init { pushMsg("add column statement", state); }
 @after { popMsg(state); }
-    : identifier (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN
-    -> {$add != null}? ^(TOK_ALTERTABLE_ADDCOLS identifier columnNameTypeList)
-    ->                 ^(TOK_ALTERTABLE_REPLACECOLS identifier columnNameTypeList)
+    : tableName (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN
+    -> {$add != null}? ^(TOK_ALTERTABLE_ADDCOLS tableName columnNameTypeList)
+    ->                 ^(TOK_ALTERTABLE_REPLACECOLS tableName columnNameTypeList)
     ;
 
 alterStatementSuffixRenameCol
 @init { pushMsg("rename column name", state); }
 @after { popMsg(state); }
-    : identifier KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition?
-    ->^(TOK_ALTERTABLE_RENAMECOL identifier $oldName $newName colType $comment? alterStatementChangeColPosition?)
+    : tableName KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition?
+    ->^(TOK_ALTERTABLE_RENAMECOL tableName $oldName $newName colType $comment? alterStatementChangeColPosition?)
     ;
 
 alterStatementChangeColPosition
@@ -1039,8 +1037,8 @@ alterStatementChangeColPosition
 alterStatementSuffixAddPartitions
 @init { pushMsg("add partition statement", state); }
 @after { popMsg(state); }
-    : identifier KW_ADD ifNotExists? alterStatementSuffixAddPartitionsElement+
-    -> ^(TOK_ALTERTABLE_ADDPARTS identifier ifNotExists? alterStatementSuffixAddPartitionsElement+)
+    : tableName KW_ADD ifNotExists? alterStatementSuffixAddPartitionsElement+
+    -> ^(TOK_ALTERTABLE_ADDPARTS tableName ifNotExists? alterStatementSuffixAddPartitionsElement+)
     ;
 
 alterStatementSuffixAddPartitionsElement
@@ -1050,22 +1048,22 @@ alterStatementSuffixAddPartitionsElement
 alterStatementSuffixTouch
 @init { pushMsg("touch statement", state); }
 @after { popMsg(state); }
-    : identifier KW_TOUCH (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_TOUCH identifier (partitionSpec)*)
+    : tableName KW_TOUCH (partitionSpec)*
+    -> ^(TOK_ALTERTABLE_TOUCH tableName (partitionSpec)*)
     ;
 
 alterStatementSuffixArchive
 @init { pushMsg("archive statement", state); }
 @after { popMsg(state); }
-    : identifier KW_ARCHIVE (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_ARCHIVE identifier (partitionSpec)*)
+    : tableName KW_ARCHIVE (partitionSpec)*
+    -> ^(TOK_ALTERTABLE_ARCHIVE tableName (partitionSpec)*)
     ;
 
 alterStatementSuffixUnArchive
 @init { pushMsg("unarchive statement", state); }
 @after { popMsg(state); }
-    : identifier KW_UNARCHIVE (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_UNARCHIVE identifier (partitionSpec)*)
+    : tableName KW_UNARCHIVE (partitionSpec)*
+    -> ^(TOK_ALTERTABLE_UNARCHIVE tableName (partitionSpec)*)
     ;
 
 partitionLocation
@@ -1078,26 +1076,26 @@ partitionLocation
 alterStatementSuffixDropPartitions
 @init { pushMsg("drop partition statement", state); }
 @after { popMsg(state); }
-    : identifier KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
-    -> ^(TOK_ALTERTABLE_DROPPARTS identifier dropPartitionSpec+ ifExists? ignoreProtection?)
+    : tableName KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
+    -> ^(TOK_ALTERTABLE_DROPPARTS tableName dropPartitionSpec+ ifExists? ignoreProtection?)
     ;
 
 alterStatementSuffixProperties
 @init { pushMsg("alter properties statement", state); }
 @after { popMsg(state); }
-    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
-    -> ^(TOK_ALTERTABLE_PROPERTIES $name tableProperties)
-    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    -> ^(TOK_DROPTABLE_PROPERTIES $name tableProperties ifExists?)
+    : tableName KW_SET KW_TBLPROPERTIES tableProperties
+    -> ^(TOK_ALTERTABLE_PROPERTIES tableName tableProperties)
+    | tableName KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    -> ^(TOK_DROPTABLE_PROPERTIES tableName tableProperties ifExists?)
     ;
 
 alterViewSuffixProperties
 @init { pushMsg("alter view properties statement", state); }
 @after { popMsg(state); }
-    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
-    -> ^(TOK_ALTERVIEW_PROPERTIES $name tableProperties)
-    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    -> ^(TOK_DROPVIEW_PROPERTIES $name tableProperties ifExists?)
+    : tableName KW_SET KW_TBLPROPERTIES tableProperties
+    -> ^(TOK_ALTERVIEW_PROPERTIES tableName tableProperties)
+    | tableName KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    -> ^(TOK_DROPVIEW_PROPERTIES tableName tableProperties ifExists?)
     ;
 
 alterStatementSuffixSerdeProperties
@@ -1112,8 +1110,8 @@ alterStatementSuffixSerdeProperties
 tablePartitionPrefix
 @init {pushMsg("table partition prefix", state);}
 @after {popMsg(state);}
-  :name=identifier partitionSpec?
-  ->^(TOK_TABLE_PARTITION $name partitionSpec?)
+  : tableName partitionSpec?
+  ->^(TOK_TABLE_PARTITION tableName partitionSpec?)
   ;
 
 alterTblPartitionStatement
@@ -1192,21 +1190,21 @@ alterStatementSuffixLocation
 alterStatementSuffixSkewedby
 @init {pushMsg("alter skewed by statement", state);}
 @after{popMsg(state);}
-	:name=identifier tableSkewed
-	->^(TOK_ALTERTABLE_SKEWED $name tableSkewed)
+	: tableName tableSkewed
+	->^(TOK_ALTERTABLE_SKEWED tableName tableSkewed)
 	|
-	name=identifier KW_NOT KW_SKEWED
-	->^(TOK_ALTERTABLE_SKEWED $name)
+	 tableName KW_NOT KW_SKEWED
+	->^(TOK_ALTERTABLE_SKEWED tableName)
 	|
-	name=identifier KW_NOT storedAsDirs
-	->^(TOK_ALTERTABLE_SKEWED $name storedAsDirs)
+	 tableName KW_NOT storedAsDirs
+	->^(TOK_ALTERTABLE_SKEWED tableName storedAsDirs)
 	;
 
 alterStatementSuffixExchangePartition
 @init {pushMsg("alter exchange partition", state);}
 @after{popMsg(state);}
-    : name=tableName KW_EXCHANGE partitionSpec KW_WITH KW_TABLE exchangename=tableName
-    -> ^(TOK_EXCHANGEPARTITION $name partitionSpec $exchangename)
+    : tableName KW_EXCHANGE partitionSpec KW_WITH KW_TABLE exchangename=tableName
+    -> ^(TOK_EXCHANGEPARTITION tableName partitionSpec $exchangename)
     ;
 
 alterStatementSuffixProtectMode
@@ -1315,14 +1313,14 @@ showStatement
 @after { popMsg(state); }
     : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? -> ^(TOK_SHOWDATABASES showStmtIdentifier?)
     | KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  -> ^(TOK_SHOWTABLES (TOK_FROM $db_name)? showStmtIdentifier?)
-    | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tabname=tableName ((KW_FROM|KW_IN) db_name=identifier)? 
-    -> ^(TOK_SHOWCOLUMNS $db_name? $tabname)
+    | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tableName ((KW_FROM|KW_IN) db_name=identifier)?
+    -> ^(TOK_SHOWCOLUMNS tableName $db_name?)
     | KW_SHOW KW_FUNCTIONS showFunctionIdentifier?  -> ^(TOK_SHOWFUNCTIONS showFunctionIdentifier?)
     | KW_SHOW KW_PARTITIONS tabName=tableName partitionSpec? -> ^(TOK_SHOWPARTITIONS $tabName partitionSpec?) 
     | KW_SHOW KW_CREATE KW_TABLE tabName=tableName -> ^(TOK_SHOW_CREATETABLE $tabName)
     | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec?
     -> ^(TOK_SHOW_TABLESTATUS showStmtIdentifier $db_name? partitionSpec?)
-    | KW_SHOW KW_TBLPROPERTIES tblName=identifier (LPAREN prptyName=StringLiteral RPAREN)? -> ^(TOK_SHOW_TBLPROPERTIES $tblName $prptyName?)
+    | KW_SHOW KW_TBLPROPERTIES tableName (LPAREN prptyName=StringLiteral RPAREN)? -> ^(TOK_SHOW_TBLPROPERTIES tableName $prptyName?)
     | KW_SHOW KW_LOCKS (parttype=partTypeExpr)? (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWLOCKS $parttype? $isExtended?)
     | KW_SHOW KW_LOCKS (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?)
     | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
@@ -1455,26 +1453,25 @@ privilegeIncludeColObject
 @init {pushMsg("privilege object including columns", state);}
 @after {popMsg(state);}
     : KW_ALL -> ^(TOK_RESOURCE_ALL)
-    | privObjectType identifier (LPAREN cols=columnNameList RPAREN)? partitionSpec?
-    -> ^(TOK_PRIV_OBJECT_COL identifier privObjectType $cols? partitionSpec?)
+    | privObjectCols -> ^(TOK_PRIV_OBJECT_COL privObjectCols)
     ;
 
 privilegeObject
-@init {pushMsg("privilege subject", state);}
+@init {pushMsg("privilege object", state);}
 @after {popMsg(state);}
-    : KW_ON privObjectType identifier partitionSpec?
-    -> ^(TOK_PRIV_OBJECT identifier privObjectType partitionSpec?)
+    : KW_ON privObject -> ^(TOK_PRIV_OBJECT privObject)
     ;
 
-
 // database or table type. Type is optional, default type is table
-privObjectType
-@init {pushMsg("privilege object type type", state);}
-@after {popMsg(state);}
-    : (KW_DATABASE|KW_SCHEMA) -> ^(TOK_DB_TYPE)
-    | KW_TABLE? -> ^(TOK_TABLE_TYPE)
+privObject
+    : (KW_DATABASE|KW_SCHEMA) identifier -> ^(TOK_DB_TYPE identifier)
+    | KW_TABLE? tableName partitionSpec? -> ^(TOK_TABLE_TYPE tableName partitionSpec?)
     ;
 
+privObjectCols
+    : (KW_DATABASE|KW_SCHEMA) identifier -> ^(TOK_DB_TYPE identifier)
+    | KW_TABLE? tableName (LPAREN cols=columnNameList RPAREN)? partitionSpec? -> ^(TOK_TABLE_TYPE tableName $cols? partitionSpec?)
+    ;
 
 privilegeList
 @init {pushMsg("grant privilege list", state);}
@@ -1551,8 +1548,8 @@ withAdminOption
 metastoreCheck
 @init { pushMsg("metastore check statement", state); }
 @after { popMsg(state); }
-    : KW_MSCK (repair=KW_REPAIR)? (KW_TABLE table=identifier partitionSpec? (COMMA partitionSpec)*)?
-    -> ^(TOK_MSCK $repair? ($table partitionSpec*)?)
+    : KW_MSCK (repair=KW_REPAIR)? (KW_TABLE tableName partitionSpec? (COMMA partitionSpec)*)?
+    -> ^(TOK_MSCK $repair? (tableName partitionSpec*)?)
     ;
 
 resourceList

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java?rev=1617327&r1=1617326&r2=1617327&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java Mon Aug 11 16:56:30 2014
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -134,10 +135,10 @@ public class IndexUpdater {
     inputs.addAll(driver.getPlan().getInputs());
   }
 
-
   private boolean containsPartition(Index index, Map<String, String> partSpec)
-    throws HiveException {
-    Table indexTable = hive.getTable(index.getIndexTableName());
+      throws HiveException {
+    String[] qualified = Utilities.getDbTableName(index.getDbName(), index.getIndexTableName());
+    Table indexTable = hive.getTable(qualified[0], qualified[1]);
     List<Partition> parts = hive.getPartitions(indexTable, partSpec);
     return (parts == null || parts.size() == 0);
   }