You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/08/30 08:44:53 UTC

svn commit: r1621416 [3/11] - in /hive/branches/spark: ./ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/conf/ contrib/src/test/results/clientnegative/ contrib/src/test/results/clientpositive/ hbase-handler/src/te...

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java Sat Aug 30 06:44:46 2014
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.io.sar
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
+
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.common.type.HiveChar;
@@ -50,6 +51,7 @@ import org.apache.hadoop.hive.serde2.typ
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 import java.math.BigDecimal;
+import java.sql.Timestamp;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -107,6 +109,12 @@ final class SearchArgumentImpl implement
 
     @Override
     public Object getLiteral() {
+      // To get around a kryo 2.22 bug while deserialize a Timestamp into Date
+      // (https://github.com/EsotericSoftware/kryo/issues/88)
+      // When we see a Date, convert back into Timestamp
+      if (literal instanceof java.util.Date) {
+        return new Timestamp(((java.util.Date)literal).getTime());
+      }
       return literal;
     }
 
@@ -317,6 +325,8 @@ final class SearchArgumentImpl implement
             return PredicateLeaf.Type.FLOAT;
           case DATE:
             return PredicateLeaf.Type.DATE;
+          case TIMESTAMP:
+            return PredicateLeaf.Type.TIMESTAMP;
           case DECIMAL:
             return PredicateLeaf.Type.DECIMAL;
           default:
@@ -354,6 +364,7 @@ final class SearchArgumentImpl implement
         case FLOAT:
           return ((Number) lit.getValue()).doubleValue();
         case DATE:
+        case TIMESTAMP:
         case DECIMAL:
           return lit;
         default:
@@ -948,6 +959,7 @@ final class SearchArgumentImpl implement
           literal instanceof Long ||
           literal instanceof Double ||
           literal instanceof DateWritable ||
+          literal instanceof Timestamp ||
           literal instanceof HiveDecimal ||
           literal instanceof BigDecimal) {
         return literal;
@@ -981,7 +993,9 @@ final class SearchArgumentImpl implement
         return PredicateLeaf.Type.FLOAT;
       } else if (literal instanceof DateWritable) {
         return PredicateLeaf.Type.DATE;
-      } else if (literal instanceof HiveDecimal ||
+      } else if (literal instanceof Timestamp) {
+        return PredicateLeaf.Type.TIMESTAMP;
+      }else if (literal instanceof HiveDecimal ||
           literal instanceof BigDecimal) {
         return PredicateLeaf.Type.DECIMAL;
       }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java Sat Aug 30 06:44:46 2014
@@ -5,6 +5,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -13,22 +14,33 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
 import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.TableStatsRequest;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import org.apache.thrift.TException;
 
 public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements IMetaStoreClient {
@@ -71,6 +83,12 @@ public class SessionHiveMetaStoreClient 
     // First try temp table
     org.apache.hadoop.hive.metastore.api.Table table = getTempTable(dbname, name);
     if (table != null) {
+      try {
+        deleteTempTableColumnStatsForTable(dbname, name);
+      } catch (NoSuchObjectException err){
+        // No stats to delete, forgivable error.
+        LOG.info(err);
+      }
       dropTempTable(table, deleteData, envContext);
       return;
     }
@@ -217,6 +235,41 @@ public class SessionHiveMetaStoreClient 
     return super.get_privilege_set(hiveObject, userName, groupNames);
   }
 
+  /** {@inheritDoc} */
+  @Override
+  public boolean updateTableColumnStatistics(ColumnStatistics statsObj)
+      throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+      InvalidInputException {
+    String dbName = statsObj.getStatsDesc().getDbName().toLowerCase();
+    String tableName = statsObj.getStatsDesc().getTableName().toLowerCase();
+    if (getTempTable(dbName, tableName) != null) {
+      return updateTempTableColumnStats(dbName, tableName, statsObj);
+    }
+    return super.updateTableColumnStatistics(statsObj);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tableName,
+      List<String> colNames) throws NoSuchObjectException, MetaException, TException,
+      InvalidInputException, InvalidObjectException {
+    if (getTempTable(dbName, tableName) != null) {
+      return getTempTableColumnStats(dbName, tableName, colNames);
+    }
+    return super.getTableColumnStatistics(dbName, tableName, colNames);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public boolean deleteTableColumnStatistics(String dbName, String tableName, String colName)
+      throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+      InvalidInputException {
+    if (getTempTable(dbName, tableName) != null) {
+      return deleteTempTableColumnStats(dbName, tableName, colName);
+    }
+    return super.deleteTableColumnStatistics(dbName, tableName, colName);
+  }
+
   private void createTempTable(org.apache.hadoop.hive.metastore.api.Table tbl,
       EnvironmentContext envContext) throws AlreadyExistsException, InvalidObjectException,
       MetaException, NoSuchObjectException, TException {
@@ -277,15 +330,19 @@ public class SessionHiveMetaStoreClient 
       org.apache.hadoop.hive.metastore.api.Table oldt,
       org.apache.hadoop.hive.metastore.api.Table newt,
       EnvironmentContext envContext) throws InvalidOperationException, MetaException, TException {
-    Table newTable = new Table(deepCopyAndLowerCaseTable(newt));
     dbname = dbname.toLowerCase();
     tbl_name = tbl_name.toLowerCase();
+    boolean shouldDeleteColStats = false;
 
     // Disallow changing temp table location
     if (!newt.getSd().getLocation().equals(oldt.getSd().getLocation())) {
       throw new MetaException("Temp table location cannot be changed");
     }
 
+    org.apache.hadoop.hive.metastore.api.Table newtCopy = deepCopyAndLowerCaseTable(newt);
+    MetaStoreUtils.updateUnpartitionedTableStatsFast(newtCopy,
+        wh.getFileStatusesForSD(newtCopy.getSd()), false, true);
+    Table newTable = new Table(newtCopy);
     String newDbName = newTable.getDbName();
     String newTableName = newTable.getTableName();
     if (!newDbName.equals(oldt.getDbName()) || !newTableName.equals(oldt.getTableName())) {
@@ -303,6 +360,7 @@ public class SessionHiveMetaStoreClient 
       if (tables == null || tables.remove(tbl_name) == null) {
         throw new MetaException("Could not find temp table entry for " + dbname + "." + tbl_name);
       }
+      shouldDeleteColStats = true;
 
       tables = getTempTablesForDatabase(newDbName);
       if (tables == null) {
@@ -311,8 +369,50 @@ public class SessionHiveMetaStoreClient 
       }
       tables.put(newTableName, newTable);
     } else {
+      if (haveTableColumnsChanged(oldt, newt)) {
+        shouldDeleteColStats = true;
+      }
       getTempTablesForDatabase(dbname).put(tbl_name, newTable);
     }
+
+    if (shouldDeleteColStats) {
+      try {
+        deleteTempTableColumnStatsForTable(dbname, tbl_name);
+      } catch (NoSuchObjectException err){
+        // No stats to delete, forgivable error.
+        LOG.info(err);
+      }
+    }
+  }
+
+  private static boolean haveTableColumnsChanged(org.apache.hadoop.hive.metastore.api.Table oldt,
+      org.apache.hadoop.hive.metastore.api.Table newt) {
+    List<FieldSchema> oldCols = oldt.getSd().getCols();
+    List<FieldSchema> newCols = newt.getSd().getCols();
+    if (oldCols.size() != newCols.size()) {
+      return true;
+    }
+    Iterator<FieldSchema> oldColsIter = oldCols.iterator();
+    Iterator<FieldSchema> newColsIter = newCols.iterator();
+    while (oldColsIter.hasNext()) {
+      // Don't use FieldSchema.equals() since it also compares comments,
+      // which is unnecessary for this method.
+      if (!fieldSchemaEqualsIgnoreComment(oldColsIter.next(), newColsIter.next())) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static boolean fieldSchemaEqualsIgnoreComment(FieldSchema left, FieldSchema right) {
+    // Just check name/type for equality, don't compare comment
+    if (!left.getName().equals(right.getName())) {
+      return true;
+    }
+    if (!left.getType().equals(right.getType())) {
+      return true;
+    }
+    return false;
   }
 
   private void dropTempTable(org.apache.hadoop.hive.metastore.api.Table table, boolean deleteData,
@@ -373,4 +473,102 @@ public class SessionHiveMetaStoreClient 
     }
     return ss.getTempTables().get(dbName);
   }
+
+  private Map<String, ColumnStatisticsObj> getTempTableColumnStatsForTable(String dbName,
+      String tableName) {
+    SessionState ss = SessionState.get();
+    if (ss == null) {
+      LOG.debug("No current SessionState, skipping temp tables");
+      return null;
+    }
+    String lookupName = StatsUtils.getFullyQualifiedTableName(dbName.toLowerCase(),
+        tableName.toLowerCase());
+    return ss.getTempTableColStats().get(lookupName);
+  }
+
+  private static List<ColumnStatisticsObj> copyColumnStatisticsObjList(Map<String, ColumnStatisticsObj> csoMap) {
+    List<ColumnStatisticsObj> retval = new ArrayList<ColumnStatisticsObj>(csoMap.size());
+    for (ColumnStatisticsObj cso : csoMap.values()) {
+      retval.add(new ColumnStatisticsObj(cso));
+    }
+    return retval;
+  }
+
+  private List<ColumnStatisticsObj> getTempTableColumnStats(String dbName, String tableName,
+      List<String> colNames) {
+    Map<String, ColumnStatisticsObj> tableColStats =
+        getTempTableColumnStatsForTable(dbName, tableName);
+    List<ColumnStatisticsObj> retval = new ArrayList<ColumnStatisticsObj>();
+
+    if (tableColStats != null) {
+      for (String colName : colNames) {
+        colName = colName.toLowerCase();
+        if (tableColStats.containsKey(colName)) {
+          retval.add(new ColumnStatisticsObj(tableColStats.get(colName)));
+        }
+      }
+    }
+    return retval;
+  }
+
+  private boolean updateTempTableColumnStats(String dbName, String tableName,
+      ColumnStatistics colStats) throws MetaException {
+    SessionState ss = SessionState.get();
+    if (ss == null) {
+      throw new MetaException("No current SessionState, cannot update temporary table stats for "
+          + dbName + "." + tableName);
+    }
+    Map<String, ColumnStatisticsObj> ssTableColStats =
+        getTempTableColumnStatsForTable(dbName, tableName);
+    if (ssTableColStats == null) {
+      // Add new entry for this table
+      ssTableColStats = new HashMap<String, ColumnStatisticsObj>();
+      ss.getTempTableColStats().put(
+          StatsUtils.getFullyQualifiedTableName(dbName, tableName),
+          ssTableColStats);
+    }
+    mergeColumnStats(ssTableColStats, colStats);
+    return true;
+  }
+
+  private static void mergeColumnStats(Map<String, ColumnStatisticsObj> oldStats,
+      ColumnStatistics newStats) {
+    List<ColumnStatisticsObj> newColList = newStats.getStatsObj();
+    if (newColList != null) {
+      for (ColumnStatisticsObj colStat : newColList) {
+        // This is admittedly a bit simple, StatsObjectConverter seems to allow
+        // old stats attributes to be kept if the new values do not overwrite them.
+        oldStats.put(colStat.getColName().toLowerCase(), colStat);
+      }
+    }
+  }
+
+  private boolean deleteTempTableColumnStatsForTable(String dbName, String tableName)
+      throws NoSuchObjectException {
+    Map<String, ColumnStatisticsObj> deletedEntry =
+        getTempTableColumnStatsForTable(dbName, tableName);
+    if (deletedEntry != null) {
+      SessionState.get().getTempTableColStats().remove(
+          StatsUtils.getFullyQualifiedTableName(dbName, tableName));
+    } else {
+      throw new NoSuchObjectException("Column stats doesn't exist for db=" + dbName +
+          " temp table=" + tableName);
+    }
+    return true;
+  }
+
+  private boolean deleteTempTableColumnStats(String dbName, String tableName, String columnName)
+      throws NoSuchObjectException {
+    ColumnStatisticsObj deletedEntry = null;
+    Map<String, ColumnStatisticsObj> ssTableColStats =
+        getTempTableColumnStatsForTable(dbName, tableName);
+    if (ssTableColStats != null) {
+      deletedEntry = ssTableColStats.remove(columnName.toLowerCase());
+    }
+    if (deletedEntry == null) {
+      throw new NoSuchObjectException("Column stats doesn't exist for db=" + dbName +
+          " temp table=" + tableName);
+    }
+    return true;
+  }
 }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java Sat Aug 30 06:44:46 2014
@@ -815,9 +815,11 @@ public class Vectorizer implements Physi
         ret = validateSelectOperator((SelectOperator) op);
         break;
       case REDUCESINK:
-          ret = validateReduceSinkOperator((ReduceSinkOperator) op);
-          break;
+        ret = validateReduceSinkOperator((ReduceSinkOperator) op);
+        break;
       case FILESINK:
+        ret = validateFileSinkOperator((FileSinkOperator) op);
+        break;
       case LIMIT:
         ret = true;
         break;
@@ -899,6 +901,15 @@ public class Vectorizer implements Physi
     return true;
   }
 
+  private boolean validateFileSinkOperator(FileSinkOperator op) {
+    // HIVE-7557: For now, turn off dynamic partitioning to give more time to 
+    // figure out how to make VectorFileSink work correctly with it...
+   if (op.getConf().getDynPartCtx() != null) {
+     return false;
+   }
+   return true;
+  }
+
   private boolean validateExprNodeDesc(List<ExprNodeDesc> descs) {
     return validateExprNodeDesc(descs, VectorExpressionDescriptor.Mode.PROJECTION);
   }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java Sat Aug 30 06:44:46 2014
@@ -104,7 +104,8 @@ public class StatsRulesProcFactory {
         tsop.setStatistics(stats.clone());
 
         if (LOG.isDebugEnabled()) {
-          LOG.debug("[0] STATS-" + tsop.toString() + ": " + stats.extendedToString());
+          LOG.debug("[0] STATS-" + tsop.toString() + " (" + table.getTableName()
+              + "): " + stats.extendedToString());
         }
       } catch (CloneNotSupportedException e) {
         throw new SemanticException(ErrorMsg.STATISTICS_CLONING_FAILED.getMsg());
@@ -1092,7 +1093,9 @@ public class StatsRulesProcFactory {
             String key = entry.getValue().get(joinColIdx);
             key = StatsUtils.stripPrefixFromColumnName(key);
             ColStatistics cs = joinedColStats.get(key);
-            cs.setCountDistint(minNDV);
+            if (cs != null) {
+              cs.setCountDistint(minNDV);
+            }
           }
         }
 

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java Sat Aug 30 06:44:46 2014
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.ql.QueryPr
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.LineageInfo;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
@@ -61,7 +62,6 @@ import org.apache.hadoop.hive.ql.optimiz
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
 import org.apache.hadoop.hive.ql.plan.PlanUtils;
-import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
@@ -317,7 +317,7 @@ public abstract class BaseSemanticAnalyz
       return new String[] {dbName, tableName};
     }
     String tableName = unescapeIdentifier(tabNameNode.getChild(0).getText());
-    return new String[]{SessionState.get().getCurrentDatabase(), tableName};
+    return Utilities.getDbTableName(tableName);
   }
 
   public static String getDotName(String[] qname) throws SemanticException {
@@ -646,6 +646,20 @@ public abstract class BaseSemanticAnalyz
       this(db, conf, ast, true, false);
     }
 
+    public tableSpec(Hive db, HiveConf conf, String tableName, Map<String, String> partSpec)
+        throws HiveException {
+      this.tableName = tableName;
+      this.partSpec = partSpec;
+      this.tableHandle = db.getTable(tableName);
+      if (partSpec != null) {
+        this.specType = SpecType.STATIC_PARTITION;
+        this.partHandle = db.getPartition(tableHandle, partSpec, false);
+        this.partitions = Arrays.asList(partHandle);
+      } else {
+        this.specType = SpecType.TABLE_ONLY;
+      }
+    }
+
     public tableSpec(Hive db, HiveConf conf, ASTNode ast, boolean allowDynamicPartitionsSpec,
         boolean allowPartialPartitionsSpec) throws SemanticException {
       assert (ast.getToken().getType() == HiveParser.TOK_TAB
@@ -1188,21 +1202,26 @@ public abstract class BaseSemanticAnalyz
   }
 
   protected Database getDatabase(String dbName, boolean throwException) throws SemanticException {
+    Database database;
     try {
-      Database database = db.getDatabase(dbName);
-      if (database == null && throwException) {
-        throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(dbName));
-      }
-      return database;
-    } catch (HiveException e) {
+      database = db.getDatabase(dbName);
+    } catch (Exception e) {
       throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(dbName), e);
     }
+    if (database == null && throwException) {
+      throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(dbName));
+    }
+    return database;
   }
 
   protected Table getTable(String[] qualified) throws SemanticException {
     return getTable(qualified[0], qualified[1], true);
   }
 
+  protected Table getTable(String[] qualified, boolean throwException) throws SemanticException {
+    return getTable(qualified[0], qualified[1], throwException);
+  }
+
   protected Table getTable(String tblName) throws SemanticException {
     return getTable(null, tblName, true);
   }
@@ -1213,43 +1232,46 @@ public abstract class BaseSemanticAnalyz
 
   protected Table getTable(String database, String tblName, boolean throwException)
       throws SemanticException {
+    Table tab;
     try {
-      Table tab = database == null ? db.getTable(tblName, false)
+      tab = database == null ? db.getTable(tblName, false)
           : db.getTable(database, tblName, false);
-      if (tab == null && throwException) {
-        throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName));
-      }
-      return tab;
-    } catch (HiveException e) {
+    } catch (Exception e) {
       throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName), e);
     }
+    if (tab == null && throwException) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName));
+    }
+    return tab;
   }
 
   protected Partition getPartition(Table table, Map<String, String> partSpec,
       boolean throwException) throws SemanticException {
+    Partition partition;
     try {
-      Partition partition = db.getPartition(table, partSpec, false);
-      if (partition == null && throwException) {
-        throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec));
-      }
-      return partition;
-    } catch (HiveException e) {
+      partition = db.getPartition(table, partSpec, false);
+    } catch (Exception e) {
       throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec), e);
     }
+    if (partition == null && throwException) {
+      throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec));
+    }
+    return partition;
   }
 
   protected List<Partition> getPartitions(Table table, Map<String, String> partSpec,
       boolean throwException) throws SemanticException {
+    List<Partition> partitions;
     try {
-      List<Partition> partitions = partSpec == null ? db.getPartitions(table) :
+      partitions = partSpec == null ? db.getPartitions(table) :
           db.getPartitions(table, partSpec);
-      if (partitions.isEmpty() && throwException) {
-        throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec));
-      }
-      return partitions;
-    } catch (HiveException e) {
+    } catch (Exception e) {
       throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec), e);
     }
+    if (partitions.isEmpty() && throwException) {
+      throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec));
+    }
+    return partitions;
   }
 
   protected String toMessage(ErrorMsg message, Object detail) {

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java Sat Aug 30 06:44:46 2014
@@ -363,7 +363,6 @@ public class ColumnStatsSemanticAnalyzer
       originalTree = tree;
       boolean isPartitionStats = isPartitionLevelStats(tree);
       Map<String,String> partSpec = null;
-      checkIfTemporaryTable();
       checkForPartitionColumns(colNames, Utilities.getColumnNamesFromFieldSchema(tbl.getPartitionKeys()));
       validateSpecifiedColumnNames(colNames);
       if (conf.getBoolVar(ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS) && tbl.isPartitioned()) {
@@ -414,13 +413,6 @@ public class ColumnStatsSemanticAnalyzer
     }
   }
 
-  private void checkIfTemporaryTable() throws SemanticException {
-    if (tbl.isTemporary()) {
-      throw new SemanticException(tbl.getTableName()
-          + " is a temporary table.  Column statistics are not supported on temporary tables.");
-    }
-  }
-
   @Override
   public void analyze(ASTNode ast, Context origCtx) throws SemanticException {
     QB qb;

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Sat Aug 30 06:44:46 2014
@@ -249,39 +249,67 @@ public class DDLSemanticAnalyzer extends
   }
 
   @Override
-  public void analyzeInternal(ASTNode ast) throws SemanticException {
+  public void analyzeInternal(ASTNode input) throws SemanticException {
 
-    switch (ast.getToken().getType()) {
-    case HiveParser.TOK_ALTERTABLE_PARTITION: {
-      ASTNode tablePart = (ASTNode) ast.getChild(0);
-      TablePartition tblPart = new TablePartition(tablePart);
-      String tableName = tblPart.tableName;
-      HashMap<String, String> partSpec = tblPart.partSpec;
-      ast = (ASTNode) ast.getChild(1);
-      if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_FILEFORMAT) {
+    ASTNode ast = input;
+    switch (ast.getType()) {
+    case HiveParser.TOK_ALTERTABLE: {
+      ast = (ASTNode) input.getChild(1);
+      String[] qualified = getQualifiedTableName((ASTNode) input.getChild(0));
+      String tableName = getDotName(qualified);
+      HashMap<String, String> partSpec = DDLSemanticAnalyzer.getPartSpec((ASTNode) input.getChild(2));
+      if (ast.getType() == HiveParser.TOK_ALTERTABLE_RENAME) {
+        analyzeAlterTableRename(qualified, ast, false);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_TOUCH) {
+        analyzeAlterTableTouch(qualified, ast);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_ARCHIVE) {
+        analyzeAlterTableArchive(qualified, ast, false);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_UNARCHIVE) {
+        analyzeAlterTableArchive(qualified, ast, true);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_ADDCOLS) {
+        analyzeAlterTableModifyCols(qualified, ast, AlterTableTypes.ADDCOLS);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_REPLACECOLS) {
+        analyzeAlterTableModifyCols(qualified, ast, AlterTableTypes.REPLACECOLS);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_RENAMECOL) {
+        analyzeAlterTableRenameCol(qualified, ast);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_ADDPARTS) {
+        analyzeAlterTableAddParts(qualified, ast, false);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_DROPPARTS) {
+        analyzeAlterTableDropParts(qualified, ast, false);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_PARTCOLTYPE) {
+        analyzeAlterTablePartColType(qualified, ast);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_PROPERTIES) {
+        analyzeAlterTableProps(qualified, ast, false, false);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_DROPPROPERTIES) {
+        analyzeAlterTableProps(qualified, ast, false, true);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_SKEWED) {
+        analyzeAltertableSkewedby(qualified, ast);
+      } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION) {
+        analyzeExchangePartition(qualified, ast);
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_FILEFORMAT) {
         analyzeAlterTableFileFormat(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_PROTECTMODE) {
         analyzeAlterTableProtectMode(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_LOCATION) {
         analyzeAlterTableLocation(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_MERGEFILES) {
-        analyzeAlterTablePartMergeFiles(tablePart, ast, tableName, partSpec);
+        analyzeAlterTablePartMergeFiles(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_SERIALIZER) {
         analyzeAlterTableSerde(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES) {
         analyzeAlterTableSerdeProps(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_RENAMEPART) {
         analyzeAlterTableRenamePart(ast, tableName, partSpec);
-      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTBLPART_SKEWED_LOCATION) {
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_SKEWED_LOCATION) {
         analyzeAlterTableSkewedLocation(ast, tableName, partSpec);
-      } else if (ast.getToken().getType() == HiveParser.TOK_TABLEBUCKETS) {
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_BUCKETS) {
         analyzeAlterTableBucketNum(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_CLUSTER_SORT) {
         analyzeAlterTableClusterSort(ast, tableName, partSpec);
-      } else if (ast.getToken().getType() == HiveParser.TOK_COMPACT) {
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_COMPACT) {
         analyzeAlterTableCompact(ast, tableName, partSpec);
       } else if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_UPDATECOLSTATS){
-        analyzeAlterTableUpdateStats(ast,tblPart);
+        analyzeAlterTableUpdateStats(ast, tableName, partSpec);
       }
       break;
     }
@@ -360,66 +388,22 @@ public class DDLSemanticAnalyzer extends
     case HiveParser.TOK_DROPVIEW:
       analyzeDropTable(ast, true);
       break;
-    case HiveParser.TOK_ALTERVIEW_PROPERTIES:
-      analyzeAlterTableProps(ast, true, false);
-      break;
-    case HiveParser.TOK_DROPVIEW_PROPERTIES:
-      analyzeAlterTableProps(ast, true, true);
-      break;
-    case HiveParser.TOK_ALTERVIEW_ADDPARTS:
-      // for ALTER VIEW ADD PARTITION, we wrapped the ADD to discriminate
-      // view from table; unwrap it now
-      analyzeAlterTableAddParts((ASTNode) ast.getChild(0), true);
-      break;
-    case HiveParser.TOK_ALTERVIEW_DROPPARTS:
-      // for ALTER VIEW DROP PARTITION, we wrapped the DROP to discriminate
-      // view from table; unwrap it now
-      analyzeAlterTableDropParts((ASTNode) ast.getChild(0), true);
-      break;
-    case HiveParser.TOK_ALTERVIEW_RENAME:
-      // for ALTER VIEW RENAME, we wrapped the RENAME to discriminate
-      // view from table; unwrap it now
-      analyzeAlterTableRename(((ASTNode) ast.getChild(0)), true);
-      break;
-    case HiveParser.TOK_ALTERTABLE_RENAME:
-      analyzeAlterTableRename(ast, false);
-      break;
-    case HiveParser.TOK_ALTERTABLE_UPDATECOLSTATS:
-      analyzeAlterTableUpdateStats(ast, null);
-      break;
-    case HiveParser.TOK_ALTERTABLE_TOUCH:
-      analyzeAlterTableTouch(ast);
-      break;
-    case HiveParser.TOK_ALTERTABLE_ARCHIVE:
-      analyzeAlterTableArchive(ast, false);
-      break;
-    case HiveParser.TOK_ALTERTABLE_UNARCHIVE:
-      analyzeAlterTableArchive(ast, true);
-      break;
-    case HiveParser.TOK_ALTERTABLE_ADDCOLS:
-      analyzeAlterTableModifyCols(ast, AlterTableTypes.ADDCOLS);
-      break;
-    case HiveParser.TOK_ALTERTABLE_REPLACECOLS:
-      analyzeAlterTableModifyCols(ast, AlterTableTypes.REPLACECOLS);
-      break;
-    case HiveParser.TOK_ALTERTABLE_RENAMECOL:
-      analyzeAlterTableRenameCol(ast);
-      break;
-    case HiveParser.TOK_ALTERTABLE_ADDPARTS:
-      analyzeAlterTableAddParts(ast, false);
-      break;
-    case HiveParser.TOK_ALTERTABLE_DROPPARTS:
-      analyzeAlterTableDropParts(ast, false);
-      break;
-    case HiveParser.TOK_ALTERTABLE_PARTCOLTYPE:
-      analyzeAlterTablePartColType(ast);
-      break;
-    case HiveParser.TOK_ALTERTABLE_PROPERTIES:
-      analyzeAlterTableProps(ast, false, false);
-      break;
-    case HiveParser.TOK_DROPTABLE_PROPERTIES:
-      analyzeAlterTableProps(ast, false, true);
+    case HiveParser.TOK_ALTERVIEW: {
+      String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
+      ast = (ASTNode) ast.getChild(1);
+      if (ast.getType() == HiveParser.TOK_ALTERVIEW_PROPERTIES) {
+        analyzeAlterTableProps(qualified, ast, true, false);
+      } else if (ast.getType() == HiveParser.TOK_ALTERVIEW_DROPPROPERTIES) {
+        analyzeAlterTableProps(qualified, ast, true, true);
+      } else if (ast.getType() == HiveParser.TOK_ALTERVIEW_ADDPARTS) {
+        analyzeAlterTableAddParts(qualified, ast, true);
+      } else if (ast.getType() == HiveParser.TOK_ALTERVIEW_DROPPARTS) {
+        analyzeAlterTableDropParts(qualified, ast, true);
+      } else if (ast.getType() == HiveParser.TOK_ALTERVIEW_RENAME) {
+        analyzeAlterTableRename(qualified, ast, true);
+      }
       break;
+    }
     case HiveParser.TOK_ALTERINDEX_REBUILD:
       analyzeAlterIndexRebuild(ast);
       break;
@@ -499,12 +483,6 @@ public class DDLSemanticAnalyzer extends
     case HiveParser.TOK_REVOKE:
       analyzeRevoke(ast);
       break;
-    case HiveParser.TOK_ALTERTABLE_SKEWED:
-      analyzeAltertableSkewedby(ast);
-      break;
-   case HiveParser.TOK_EXCHANGEPARTITION:
-      analyzeExchangePartition(ast);
-      break;
    case HiveParser.TOK_SHOW_SET_ROLE:
      analyzeSetShowRole(ast);
      break;
@@ -516,20 +494,14 @@ public class DDLSemanticAnalyzer extends
     }
   }
 
-  private void analyzeAlterTableUpdateStats(ASTNode ast, TablePartition tblPart)
+  private void analyzeAlterTableUpdateStats(ASTNode ast, String tblName, Map<String, String> partSpec)
       throws SemanticException {
-    String tblName = null;
-    String colName = null;
-    Map<String, String> mapProp = null;
-    Map<String, String> partSpec = null;
+    String colName = getUnescapedName((ASTNode) ast.getChild(0));
+    Map<String, String> mapProp = getProps((ASTNode) (ast.getChild(1)).getChild(0));
+
+    Table tbl = getTable(tblName);
     String partName = null;
-    if (tblPart == null) {
-      tblName = getUnescapedName((ASTNode) ast.getChild(0));
-      colName = getUnescapedName((ASTNode) ast.getChild(1));
-      mapProp = getProps((ASTNode) (ast.getChild(2)).getChild(0));
-    } else {
-      tblName = tblPart.tableName;
-      partSpec = tblPart.partSpec;
+    if (partSpec != null) {
       try {
         partName = Warehouse.makePartName(partSpec, false);
       } catch (MetaException e) {
@@ -537,15 +509,6 @@ public class DDLSemanticAnalyzer extends
         throw new SemanticException("partition " + partSpec.toString()
             + " not found");
       }
-      colName = getUnescapedName((ASTNode) ast.getChild(0));
-      mapProp = getProps((ASTNode) (ast.getChild(1)).getChild(0));
-    }
-
-    Table tbl = null;
-    try {
-      tbl = db.getTable(tblName);
-    } catch (HiveException e) {
-      throw new SemanticException("table " + tbl + " not found");
     }
 
     String colType = null;
@@ -711,12 +674,12 @@ public class DDLSemanticAnalyzer extends
     addAlterDbDesc(alterDesc);
   }
 
-  private void analyzeExchangePartition(ASTNode ast) throws SemanticException {
-    Table destTable =  getTable(getUnescapedName((ASTNode)ast.getChild(0)));
-    Table sourceTable = getTable(getUnescapedName((ASTNode)ast.getChild(2)));
+  private void analyzeExchangePartition(String[] qualified, ASTNode ast) throws SemanticException {
+    Table destTable = getTable(qualified);
+    Table sourceTable = getTable(getUnescapedName((ASTNode)ast.getChild(1)));
 
     // Get the partition specs
-    Map<String, String> partSpecs = getPartSpec((ASTNode) ast.getChild(1));
+    Map<String, String> partSpecs = getPartSpec((ASTNode) ast.getChild(0));
     validatePartitionValues(partSpecs);
     boolean sameColumns = MetaStoreUtils.compareFieldColumns(
         destTable.getAllCols(), sourceTable.getAllCols());
@@ -1237,8 +1200,7 @@ public class DDLSemanticAnalyzer extends
       if (indexTableName != null) {
         indexTbl = getTable(Utilities.getDbTableName(index.getDbName(), indexTableName));
       }
-      String baseTblName = index.getOrigTableName();
-      Table baseTbl = getTable(baseTblName);
+      Table baseTbl = getTable(new String[] {index.getDbName(), index.getOrigTableName()});
 
       String handlerCls = index.getIndexHandlerClass();
       HiveIndexHandler handler = HiveUtils.getIndexHandler(conf, handlerCls);
@@ -1331,16 +1293,16 @@ public class DDLSemanticAnalyzer extends
     }
   }
 
-  private void analyzeAlterTableProps(ASTNode ast, boolean expectView, boolean isUnset)
+  private void analyzeAlterTableProps(String[] qualified, ASTNode ast, boolean expectView, boolean isUnset)
       throws SemanticException {
 
-    String tableName = getUnescapedName((ASTNode) ast.getChild(0));
-    HashMap<String, String> mapProp = getProps((ASTNode) (ast.getChild(1))
+    String tableName = getDotName(qualified);
+    HashMap<String, String> mapProp = getProps((ASTNode) (ast.getChild(0))
         .getChild(0));
     AlterTableDesc alterTblDesc = null;
     if (isUnset == true) {
       alterTblDesc = new AlterTableDesc(AlterTableTypes.DROPPROPS, expectView);
-      if (ast.getChild(2) != null) {
+      if (ast.getChild(1) != null) {
         alterTblDesc.setDropIfExists(true);
       }
     } else {
@@ -1527,7 +1489,7 @@ public class DDLSemanticAnalyzer extends
         alterTblDesc), conf));
   }
 
-  private void analyzeAlterTablePartMergeFiles(ASTNode tablePartAST, ASTNode ast,
+  private void analyzeAlterTablePartMergeFiles(ASTNode ast,
       String tableName, HashMap<String, String> partSpec)
       throws SemanticException {
     AlterTablePartMergeFilesDesc mergeDesc = new AlterTablePartMergeFilesDesc(
@@ -1639,7 +1601,7 @@ public class DDLSemanticAnalyzer extends
         StatsWork statDesc;
         if (oldTblPartLoc.equals(newTblPartLoc)) {
           // If we're merging to the same location, we can avoid some metastore calls
-          tableSpec tablepart = new tableSpec(this.db, conf, tablePartAST);
+          tableSpec tablepart = new tableSpec(db, conf, tableName, partSpec);
           statDesc = new StatsWork(tablepart);
         } else {
           statDesc = new StatsWork(ltd);
@@ -1672,7 +1634,7 @@ public class DDLSemanticAnalyzer extends
       alterTblDesc = new AlterTableDesc(tableName, true, partSpec);
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc), conf));
       break;
-    case HiveParser.TOK_TABLEBUCKETS:
+    case HiveParser.TOK_ALTERTABLE_BUCKETS:
       ASTNode buckets = (ASTNode) ast.getChild(0);
       List<String> bucketCols = getColumnNames((ASTNode) buckets.getChild(0));
       List<Order> sortCols = new ArrayList<Order>();
@@ -2502,9 +2464,9 @@ public class DDLSemanticAnalyzer extends
   }
 
 
-  private void analyzeAlterTableRename(ASTNode ast, boolean expectView) throws SemanticException {
-    String[] source = getQualifiedTableName((ASTNode) ast.getChild(0));
-    String[] target = getQualifiedTableName((ASTNode) ast.getChild(1));
+  private void analyzeAlterTableRename(String[] source, ASTNode ast, boolean expectView)
+      throws SemanticException {
+    String[] target = getQualifiedTableName((ASTNode) ast.getChild(0));
 
     String sourceName = getDotName(source);
     String targetName = getDotName(target);
@@ -2515,22 +2477,21 @@ public class DDLSemanticAnalyzer extends
         alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableRenameCol(ASTNode ast) throws SemanticException {
-    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
+  private void analyzeAlterTableRenameCol(String[] qualified, ASTNode ast) throws SemanticException {
     String newComment = null;
     String newType = null;
-    newType = getTypeStringFromAST((ASTNode) ast.getChild(3));
+    newType = getTypeStringFromAST((ASTNode) ast.getChild(2));
     boolean first = false;
     String flagCol = null;
     ASTNode positionNode = null;
-    if (ast.getChildCount() == 6) {
-      newComment = unescapeSQLString(ast.getChild(4).getText());
-      positionNode = (ASTNode) ast.getChild(5);
-    } else if (ast.getChildCount() == 5) {
-      if (ast.getChild(4).getType() == HiveParser.StringLiteral) {
-        newComment = unescapeSQLString(ast.getChild(4).getText());
+    if (ast.getChildCount() == 5) {
+      newComment = unescapeSQLString(ast.getChild(3).getText());
+      positionNode = (ASTNode) ast.getChild(4);
+    } else if (ast.getChildCount() == 4) {
+      if (ast.getChild(3).getType() == HiveParser.StringLiteral) {
+        newComment = unescapeSQLString(ast.getChild(3).getText());
       } else {
-        positionNode = (ASTNode) ast.getChild(4);
+        positionNode = (ASTNode) ast.getChild(3);
       }
     }
 
@@ -2542,8 +2503,8 @@ public class DDLSemanticAnalyzer extends
       }
     }
 
-    String oldColName = ast.getChild(1).getText();
-    String newColName = ast.getChild(2).getText();
+    String oldColName = ast.getChild(0).getText();
+    String newColName = ast.getChild(1).getText();
 
     /* Validate the operation of renaming a column name. */
     Table tab = getTable(qualified);
@@ -2603,12 +2564,11 @@ public class DDLSemanticAnalyzer extends
         alterBucketNum), conf));
   }
 
-  private void analyzeAlterTableModifyCols(ASTNode ast,
+  private void analyzeAlterTableModifyCols(String[] qualified, ASTNode ast,
       AlterTableTypes alterType) throws SemanticException {
-    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
 
     String tblName = getDotName(qualified);
-    List<FieldSchema> newCols = getColumns((ASTNode) ast.getChild(1));
+    List<FieldSchema> newCols = getColumns((ASTNode) ast.getChild(0));
     AlterTableDesc alterTblDesc = new AlterTableDesc(tblName, newCols,
         alterType);
 
@@ -2617,7 +2577,7 @@ public class DDLSemanticAnalyzer extends
         alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableDropParts(ASTNode ast, boolean expectView)
+  private void analyzeAlterTableDropParts(String[] qualified, ASTNode ast, boolean expectView)
       throws SemanticException {
 
     boolean ifExists = (ast.getFirstChildWithType(HiveParser.TOK_IFEXISTS) != null)
@@ -2630,7 +2590,6 @@ public class DDLSemanticAnalyzer extends
     // popular case but that's kinda hacky. Let's not do it for now.
     boolean canGroupExprs = ifExists;
 
-    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     Table tab = getTable(qualified);
     Map<Integer, List<ExprNodeGenericFuncDesc>> partSpecs =
         getFullPartitionSpecs(ast, tab, canGroupExprs);
@@ -2649,10 +2608,8 @@ public class DDLSemanticAnalyzer extends
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTblDesc), conf));
   }
 
-  private void analyzeAlterTablePartColType(ASTNode ast)
+  private void analyzeAlterTablePartColType(String[] qualified, ASTNode ast)
       throws SemanticException {
-    // get table name
-    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
 
 
     // check if table exists.
@@ -2664,7 +2621,7 @@ public class DDLSemanticAnalyzer extends
 
     // Alter table ... partition column ( column newtype) only takes one column at a time.
     // It must have a column name followed with type.
-    ASTNode colAst = (ASTNode) ast.getChild(1);
+    ASTNode colAst = (ASTNode) ast.getChild(0);
     assert(colAst.getChildCount() == 2);
 
     FieldSchema newCol = new FieldSchema();
@@ -2710,12 +2667,11 @@ public class DDLSemanticAnalyzer extends
    * @throws SemanticException
    *           Parsing failed
    */
-  private void analyzeAlterTableAddParts(CommonTree ast, boolean expectView)
+  private void analyzeAlterTableAddParts(String[] qualified, CommonTree ast, boolean expectView)
       throws SemanticException {
 
     // ^(TOK_ALTERTABLE_ADDPARTS identifier ifNotExists? alterStatementSuffixAddPartitionsElement+)
-    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
-    boolean ifNotExists = ast.getChild(1).getType() == HiveParser.TOK_IFNOTEXISTS;
+    boolean ifNotExists = ast.getChild(0).getType() == HiveParser.TOK_IFNOTEXISTS;
 
     Table tab = getTable(qualified);
     boolean isView = tab.isView();
@@ -2723,7 +2679,7 @@ public class DDLSemanticAnalyzer extends
     outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_SHARED));
 
     int numCh = ast.getChildCount();
-    int start = ifNotExists ? 2 : 1;
+    int start = ifNotExists ? 1 : 0;
 
     String currentLocation = null;
     Map<String, String> currentPart = null;
@@ -2840,9 +2796,8 @@ public class DDLSemanticAnalyzer extends
    * @throws SemanticException
    *           Parsin failed
    */
-  private void analyzeAlterTableTouch(CommonTree ast)
+  private void analyzeAlterTableTouch(String[] qualified, CommonTree ast)
       throws SemanticException {
-    String[] qualified = getQualifiedTableName((ASTNode)ast.getChild(0));
 
     Table tab = getTable(qualified);
     validateAlterTableType(tab, AlterTableTypes.TOUCH);
@@ -2870,14 +2825,13 @@ public class DDLSemanticAnalyzer extends
     }
   }
 
-  private void analyzeAlterTableArchive(CommonTree ast, boolean isUnArchive)
+  private void analyzeAlterTableArchive(String[] qualified, CommonTree ast, boolean isUnArchive)
       throws SemanticException {
 
     if (!conf.getBoolVar(HiveConf.ConfVars.HIVEARCHIVEENABLED)) {
       throw new SemanticException(ErrorMsg.ARCHIVE_METHODS_DISABLED.getMsg());
 
     }
-    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     // partition name to value
     List<Map<String, String>> partSpecs = getPartitionSpecs(ast);
 
@@ -2948,7 +2902,7 @@ public class DDLSemanticAnalyzer extends
     List<Map<String, String>> partSpecs = new ArrayList<Map<String, String>>();
     int childIndex = 0;
     // get partition metadata if partition specified
-    for (childIndex = 1; childIndex < ast.getChildCount(); childIndex++) {
+    for (childIndex = 0; childIndex < ast.getChildCount(); childIndex++) {
       Tree partspec = ast.getChild(childIndex);
       // sanity check
       if (partspec.getType() == HiveParser.TOK_PARTSPEC) {
@@ -2976,7 +2930,7 @@ public class DDLSemanticAnalyzer extends
 
     Map<Integer, List<ExprNodeGenericFuncDesc>> result =
         new HashMap<Integer, List<ExprNodeGenericFuncDesc>>();
-    for (int childIndex = 1; childIndex < ast.getChildCount(); childIndex++) {
+    for (int childIndex = 0; childIndex < ast.getChildCount(); childIndex++) {
       Tree partSpecTree = ast.getChild(childIndex);
       if (partSpecTree.getType() != HiveParser.TOK_PARTSPEC) continue;
       ExprNodeGenericFuncDesc expr = null;
@@ -3184,14 +3138,13 @@ public class DDLSemanticAnalyzer extends
    *          node
    * @throws SemanticException
    */
-  private void analyzeAltertableSkewedby(ASTNode ast) throws SemanticException {
+  private void analyzeAltertableSkewedby(String[] qualified, ASTNode ast) throws SemanticException {
     /**
      * Throw an error if the user tries to use the DDL with
      * hive.internal.ddl.list.bucketing.enable set to false.
      */
     HiveConf hiveConf = SessionState.get().getConf();
 
-    String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
     Table tab = getTable(qualified);
 
     inputs.add(new ReadEntity(tab));
@@ -3200,7 +3153,7 @@ public class DDLSemanticAnalyzer extends
     validateAlterTableType(tab, AlterTableTypes.ADDSKEWEDBY);
 
     String tableName = getDotName(qualified);
-    if (ast.getChildCount() == 1) {
+    if (ast.getChildCount() == 0) {
       /* Convert a skewed table to non-skewed table. */
       AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, true,
           new ArrayList<String>(), new ArrayList<List<String>>());
@@ -3208,7 +3161,7 @@ public class DDLSemanticAnalyzer extends
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           alterTblDesc), conf));
     } else {
-      switch (((ASTNode) ast.getChild(1)).getToken().getType()) {
+      switch (((ASTNode) ast.getChild(0)).getToken().getType()) {
       case HiveParser.TOK_TABLESKEWED:
         handleAlterTableSkewedBy(ast, tableName, tab);
         break;
@@ -3255,7 +3208,7 @@ public class DDLSemanticAnalyzer extends
     List<String> skewedColNames = new ArrayList<String>();
     List<List<String>> skewedValues = new ArrayList<List<String>>();
     /* skewed column names. */
-    ASTNode skewedNode = (ASTNode) ast.getChild(1);
+    ASTNode skewedNode = (ASTNode) ast.getChild(0);
     skewedColNames = analyzeSkewedTablDDLColNames(skewedColNames, skewedNode);
     /* skewed value. */
     analyzeDDLSkewedValues(skewedValues, skewedNode);

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g Sat Aug 30 06:44:46 2014
@@ -126,11 +126,10 @@ TOK_CREATEINDEX;
 TOK_CREATEINDEX_INDEXTBLNAME;
 TOK_DEFERRED_REBUILDINDEX;
 TOK_DROPINDEX;
-TOK_DROPTABLE_PROPERTIES;
 TOK_LIKETABLE;
 TOK_DESCTABLE;
 TOK_DESCFUNCTION;
-TOK_ALTERTABLE_PARTITION;
+TOK_ALTERTABLE;
 TOK_ALTERTABLE_RENAME;
 TOK_ALTERTABLE_ADDCOLS;
 TOK_ALTERTABLE_RENAMECOL;
@@ -152,6 +151,13 @@ TOK_ALTERTABLE_FILEFORMAT;
 TOK_ALTERTABLE_LOCATION;
 TOK_ALTERTABLE_PROPERTIES;
 TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION;
+TOK_ALTERTABLE_DROPPROPERTIES;
+TOK_ALTERTABLE_SKEWED;
+TOK_ALTERTABLE_EXCHANGEPARTITION;
+TOK_ALTERTABLE_SKEWED_LOCATION;
+TOK_ALTERTABLE_BUCKETS;
+TOK_ALTERTABLE_CLUSTER_SORT;
+TOK_ALTERTABLE_COMPACT;
 TOK_ALTERINDEX_REBUILD;
 TOK_ALTERINDEX_PROPERTIES;
 TOK_MSCK;
@@ -177,7 +183,6 @@ TOK_TABCOLLIST;
 TOK_TABCOL;
 TOK_TABLECOMMENT;
 TOK_TABLEPARTCOLS;
-TOK_TABLEBUCKETS;
 TOK_TABLEROWFORMAT;
 TOK_TABLEROWFORMATFIELD;
 TOK_TABLEROWFORMATCOLLITEMS;
@@ -192,7 +197,6 @@ TOK_DISABLE;
 TOK_READONLY;
 TOK_NO_DROP;
 TOK_STORAGEHANDLER;
-TOK_ALTERTABLE_CLUSTER_SORT;
 TOK_NOT_CLUSTERED;
 TOK_NOT_SORTED;
 TOK_TABCOLNAME;
@@ -215,9 +219,9 @@ TOK_DROPMACRO;
 TOK_TEMPORARY;
 TOK_CREATEVIEW;
 TOK_DROPVIEW;
-TOK_ALTERVIEW_AS;
+TOK_ALTERVIEW;
 TOK_ALTERVIEW_PROPERTIES;
-TOK_DROPVIEW_PROPERTIES;
+TOK_ALTERVIEW_DROPPROPERTIES;
 TOK_ALTERVIEW_ADDPARTS;
 TOK_ALTERVIEW_DROPPARTS;
 TOK_ALTERVIEW_RENAME;
@@ -302,8 +306,6 @@ TOK_TABLESKEWED;
 TOK_TABCOLVALUE;
 TOK_TABCOLVALUE_PAIR;
 TOK_TABCOLVALUES;
-TOK_ALTERTABLE_SKEWED;
-TOK_ALTERTBLPART_SKEWED_LOCATION;
 TOK_SKEWED_LOCATIONS;
 TOK_SKEWED_LOCATION_LIST;
 TOK_SKEWED_LOCATION_MAP;
@@ -315,7 +317,6 @@ TOK_WINDOWSPEC;
 TOK_WINDOWVALUES;
 TOK_WINDOWRANGE;
 TOK_IGNOREPROTECTION;
-TOK_EXCHANGEPARTITION;
 TOK_SUBQUERY_EXPR;
 TOK_SUBQUERY_OP;
 TOK_SUBQUERY_OP_NOTIN;
@@ -328,7 +329,6 @@ TOK_FILE;
 TOK_JAR;
 TOK_RESOURCE_URI;
 TOK_RESOURCE_LIST;
-TOK_COMPACT;
 TOK_SHOW_COMPACTIONS;
 TOK_SHOW_TRANSACTIONS;
 TOK_DELETE_FROM;
@@ -935,56 +935,62 @@ dropTableStatement
 alterStatement
 @init { pushMsg("alter statement", state); }
 @after { popMsg(state); }
-    : KW_ALTER!
-        (
-            KW_TABLE! alterTableStatementSuffix
-        |
-            KW_VIEW! alterViewStatementSuffix
-        |
-            KW_INDEX! alterIndexStatementSuffix
-        |
-            (KW_DATABASE|KW_SCHEMA)! alterDatabaseStatementSuffix
-        )
+    : KW_ALTER KW_TABLE tableName alterTableStatementSuffix -> ^(TOK_ALTERTABLE tableName alterTableStatementSuffix)
+    | KW_ALTER KW_VIEW tableName KW_AS? alterViewStatementSuffix -> ^(TOK_ALTERVIEW tableName alterViewStatementSuffix)
+    | KW_ALTER KW_INDEX alterIndexStatementSuffix -> alterIndexStatementSuffix
+    | KW_ALTER (KW_DATABASE|KW_SCHEMA) alterDatabaseStatementSuffix -> alterDatabaseStatementSuffix
     ;
 
 alterTableStatementSuffix
 @init { pushMsg("alter table statement", state); }
 @after { popMsg(state); }
-    : alterStatementSuffixRename
+    : alterStatementSuffixRename[true]
     | alterStatementSuffixAddCol
     | alterStatementSuffixRenameCol
     | alterStatementSuffixUpdateStatsCol
-    | alterStatementSuffixDropPartitions
-    | alterStatementSuffixAddPartitions
+    | alterStatementSuffixDropPartitions[true]
+    | alterStatementSuffixAddPartitions[true]
     | alterStatementSuffixTouch
     | alterStatementSuffixArchive
     | alterStatementSuffixUnArchive
     | alterStatementSuffixProperties
-    | alterTblPartitionStatement
     | alterStatementSuffixSkewedby
     | alterStatementSuffixExchangePartition
     | alterStatementPartitionKeyType
+    | partitionSpec? alterTblPartitionStatementSuffix -> alterTblPartitionStatementSuffix partitionSpec?
     ;
 
+alterTblPartitionStatementSuffix
+@init {pushMsg("alter table partition statement suffix", state);}
+@after {popMsg(state);}
+  : alterStatementSuffixFileFormat
+  | alterStatementSuffixLocation
+  | alterStatementSuffixProtectMode
+  | alterStatementSuffixMergeFiles
+  | alterStatementSuffixSerdeProperties
+  | alterStatementSuffixRenamePart
+  | alterStatementSuffixBucketNum
+  | alterTblPartitionStatementSuffixSkewedLocation
+  | alterStatementSuffixClusterbySortby
+  | alterStatementSuffixCompact
+  | alterStatementSuffixUpdateStatsCol
+  ;
+
 alterStatementPartitionKeyType
 @init {msgs.push("alter partition key type"); }
 @after {msgs.pop();}
-	: tableName KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
-	-> ^(TOK_ALTERTABLE_PARTCOLTYPE tableName columnNameType)
+	: KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
+	-> ^(TOK_ALTERTABLE_PARTCOLTYPE columnNameType)
 	;
 
 alterViewStatementSuffix
 @init { pushMsg("alter view statement", state); }
 @after { popMsg(state); }
     : alterViewSuffixProperties
-    | alterStatementSuffixRename
-        -> ^(TOK_ALTERVIEW_RENAME alterStatementSuffixRename)
-    | alterStatementSuffixAddPartitions
-        -> ^(TOK_ALTERVIEW_ADDPARTS alterStatementSuffixAddPartitions)
-    | alterStatementSuffixDropPartitions
-        -> ^(TOK_ALTERVIEW_DROPPARTS alterStatementSuffixDropPartitions)
-    | name=tableName KW_AS selectStatementWithCTE
-        -> ^(TOK_ALTERVIEW_AS $name selectStatementWithCTE)
+    | alterStatementSuffixRename[false]
+    | alterStatementSuffixAddPartitions[false]
+    | alterStatementSuffixDropPartitions[false]
+    | selectStatementWithCTE
     ;
 
 alterIndexStatementSuffix
@@ -1022,33 +1028,34 @@ alterDatabaseSuffixSetOwner
     -> ^(TOK_ALTERDATABASE_OWNER $dbName principalName)
     ;
 
-alterStatementSuffixRename
+alterStatementSuffixRename[boolean table]
 @init { pushMsg("rename statement", state); }
 @after { popMsg(state); }
-    : oldName=tableName KW_RENAME KW_TO newName=tableName
-    -> ^(TOK_ALTERTABLE_RENAME $oldName $newName)
+    : KW_RENAME KW_TO tableName
+    -> { table }? ^(TOK_ALTERTABLE_RENAME tableName)
+    ->            ^(TOK_ALTERVIEW_RENAME tableName)
     ;
 
 alterStatementSuffixAddCol
 @init { pushMsg("add column statement", state); }
 @after { popMsg(state); }
-    : tableName (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN
-    -> {$add != null}? ^(TOK_ALTERTABLE_ADDCOLS tableName columnNameTypeList)
-    ->                 ^(TOK_ALTERTABLE_REPLACECOLS tableName columnNameTypeList)
+    : (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN
+    -> {$add != null}? ^(TOK_ALTERTABLE_ADDCOLS columnNameTypeList)
+    ->                 ^(TOK_ALTERTABLE_REPLACECOLS columnNameTypeList)
     ;
 
 alterStatementSuffixRenameCol
 @init { pushMsg("rename column name", state); }
 @after { popMsg(state); }
-    : tableName KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition?
-    ->^(TOK_ALTERTABLE_RENAMECOL tableName $oldName $newName colType $comment? alterStatementChangeColPosition?)
+    : KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition?
+    ->^(TOK_ALTERTABLE_RENAMECOL $oldName $newName colType $comment? alterStatementChangeColPosition?)
     ;
 
 alterStatementSuffixUpdateStatsCol
 @init { pushMsg("update column statistics", state); }
 @after { popMsg(state); }
-    : identifier KW_UPDATE KW_STATISTICS KW_FOR KW_COLUMN? colName=identifier KW_SET tableProperties (KW_COMMENT comment=StringLiteral)?
-    ->^(TOK_ALTERTABLE_UPDATECOLSTATS identifier $colName tableProperties $comment?)
+    : KW_UPDATE KW_STATISTICS KW_FOR KW_COLUMN? colName=identifier KW_SET tableProperties (KW_COMMENT comment=StringLiteral)?
+    ->^(TOK_ALTERTABLE_UPDATECOLSTATS $colName tableProperties $comment?)
     ;
 
 alterStatementChangeColPosition
@@ -1057,11 +1064,12 @@ alterStatementChangeColPosition
     -> ^(TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION $afterCol)
     ;
 
-alterStatementSuffixAddPartitions
+alterStatementSuffixAddPartitions[boolean table]
 @init { pushMsg("add partition statement", state); }
 @after { popMsg(state); }
-    : tableName KW_ADD ifNotExists? alterStatementSuffixAddPartitionsElement+
-    -> ^(TOK_ALTERTABLE_ADDPARTS tableName ifNotExists? alterStatementSuffixAddPartitionsElement+)
+    : KW_ADD ifNotExists? alterStatementSuffixAddPartitionsElement+
+    -> { table }? ^(TOK_ALTERTABLE_ADDPARTS ifNotExists? alterStatementSuffixAddPartitionsElement+)
+    ->            ^(TOK_ALTERVIEW_ADDPARTS ifNotExists? alterStatementSuffixAddPartitionsElement+)
     ;
 
 alterStatementSuffixAddPartitionsElement
@@ -1071,22 +1079,22 @@ alterStatementSuffixAddPartitionsElement
 alterStatementSuffixTouch
 @init { pushMsg("touch statement", state); }
 @after { popMsg(state); }
-    : tableName KW_TOUCH (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_TOUCH tableName (partitionSpec)*)
+    : KW_TOUCH (partitionSpec)*
+    -> ^(TOK_ALTERTABLE_TOUCH (partitionSpec)*)
     ;
 
 alterStatementSuffixArchive
 @init { pushMsg("archive statement", state); }
 @after { popMsg(state); }
-    : tableName KW_ARCHIVE (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_ARCHIVE tableName (partitionSpec)*)
+    : KW_ARCHIVE (partitionSpec)*
+    -> ^(TOK_ALTERTABLE_ARCHIVE (partitionSpec)*)
     ;
 
 alterStatementSuffixUnArchive
 @init { pushMsg("unarchive statement", state); }
 @after { popMsg(state); }
-    : tableName KW_UNARCHIVE (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_UNARCHIVE tableName (partitionSpec)*)
+    : KW_UNARCHIVE (partitionSpec)*
+    -> ^(TOK_ALTERTABLE_UNARCHIVE (partitionSpec)*)
     ;
 
 partitionLocation
@@ -1096,29 +1104,30 @@ partitionLocation
       KW_LOCATION locn=StringLiteral -> ^(TOK_PARTITIONLOCATION $locn)
     ;
 
-alterStatementSuffixDropPartitions
+alterStatementSuffixDropPartitions[boolean table]
 @init { pushMsg("drop partition statement", state); }
 @after { popMsg(state); }
-    : tableName KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
-    -> ^(TOK_ALTERTABLE_DROPPARTS tableName dropPartitionSpec+ ifExists? ignoreProtection?)
+    : KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
+    -> { table }? ^(TOK_ALTERTABLE_DROPPARTS dropPartitionSpec+ ifExists? ignoreProtection?)
+    ->            ^(TOK_ALTERVIEW_DROPPARTS dropPartitionSpec+ ifExists? ignoreProtection?)
     ;
 
 alterStatementSuffixProperties
 @init { pushMsg("alter properties statement", state); }
 @after { popMsg(state); }
-    : tableName KW_SET KW_TBLPROPERTIES tableProperties
-    -> ^(TOK_ALTERTABLE_PROPERTIES tableName tableProperties)
-    | tableName KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    -> ^(TOK_DROPTABLE_PROPERTIES tableName tableProperties ifExists?)
+    : KW_SET KW_TBLPROPERTIES tableProperties
+    -> ^(TOK_ALTERTABLE_PROPERTIES tableProperties)
+    | KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    -> ^(TOK_ALTERTABLE_DROPPROPERTIES tableProperties ifExists?)
     ;
 
 alterViewSuffixProperties
 @init { pushMsg("alter view properties statement", state); }
 @after { popMsg(state); }
-    : tableName KW_SET KW_TBLPROPERTIES tableProperties
-    -> ^(TOK_ALTERVIEW_PROPERTIES tableName tableProperties)
-    | tableName KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    -> ^(TOK_DROPVIEW_PROPERTIES tableName tableProperties ifExists?)
+    : KW_SET KW_TBLPROPERTIES tableProperties
+    -> ^(TOK_ALTERVIEW_PROPERTIES tableProperties)
+    | KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    -> ^(TOK_ALTERVIEW_DROPPROPERTIES tableProperties ifExists?)
     ;
 
 alterStatementSuffixSerdeProperties
@@ -1137,29 +1146,6 @@ tablePartitionPrefix
   ->^(TOK_TABLE_PARTITION tableName partitionSpec?)
   ;
 
-alterTblPartitionStatement
-@init {pushMsg("alter table partition statement", state);}
-@after {popMsg(state);}
-  : tablePartitionPrefix alterTblPartitionStatementSuffix
-  -> ^(TOK_ALTERTABLE_PARTITION tablePartitionPrefix alterTblPartitionStatementSuffix)
-  ;
-
-alterTblPartitionStatementSuffix
-@init {pushMsg("alter table partition statement suffix", state);}
-@after {popMsg(state);}
-  : alterStatementSuffixFileFormat
-  | alterStatementSuffixLocation
-  | alterStatementSuffixProtectMode
-  | alterStatementSuffixMergeFiles
-  | alterStatementSuffixSerdeProperties
-  | alterStatementSuffixRenamePart
-  | alterStatementSuffixStatsPart
-  | alterStatementSuffixBucketNum
-  | alterTblPartitionStatementSuffixSkewedLocation
-  | alterStatementSuffixClusterbySortby
-  | alterStatementSuffixCompact
-  ;
-
 alterStatementSuffixFileFormat
 @init {pushMsg("alter fileformat statement", state); }
 @after {popMsg(state);}
@@ -1179,7 +1165,7 @@ alterTblPartitionStatementSuffixSkewedLo
 @init {pushMsg("alter partition skewed location", state);}
 @after {popMsg(state);}
   : KW_SET KW_SKEWED KW_LOCATION skewedLocations
-  -> ^(TOK_ALTERTBLPART_SKEWED_LOCATION skewedLocations)
+  -> ^(TOK_ALTERTABLE_SKEWED_LOCATION skewedLocations)
   ;
   
 skewedLocations
@@ -1214,21 +1200,21 @@ alterStatementSuffixLocation
 alterStatementSuffixSkewedby
 @init {pushMsg("alter skewed by statement", state);}
 @after{popMsg(state);}
-	: tableName tableSkewed
-	->^(TOK_ALTERTABLE_SKEWED tableName tableSkewed)
+	: tableSkewed
+	->^(TOK_ALTERTABLE_SKEWED tableSkewed)
 	|
-	 tableName KW_NOT KW_SKEWED
-	->^(TOK_ALTERTABLE_SKEWED tableName)
+	 KW_NOT KW_SKEWED
+	->^(TOK_ALTERTABLE_SKEWED)
 	|
-	 tableName KW_NOT storedAsDirs
-	->^(TOK_ALTERTABLE_SKEWED tableName storedAsDirs)
+	 KW_NOT storedAsDirs
+	->^(TOK_ALTERTABLE_SKEWED storedAsDirs)
 	;
 
 alterStatementSuffixExchangePartition
 @init {pushMsg("alter exchange partition", state);}
 @after{popMsg(state);}
-    : tableName KW_EXCHANGE partitionSpec KW_WITH KW_TABLE exchangename=tableName
-    -> ^(TOK_EXCHANGEPARTITION tableName partitionSpec $exchangename)
+    : KW_EXCHANGE partitionSpec KW_WITH KW_TABLE exchangename=tableName
+    -> ^(TOK_ALTERTABLE_EXCHANGEPARTITION partitionSpec $exchangename)
     ;
 
 alterStatementSuffixProtectMode
@@ -1278,14 +1264,14 @@ alterStatementSuffixBucketNum
 @init { pushMsg("", state); }
 @after { popMsg(state); }
     : KW_INTO num=Number KW_BUCKETS
-    -> ^(TOK_TABLEBUCKETS $num)
+    -> ^(TOK_ALTERTABLE_BUCKETS $num)
     ;
 
 alterStatementSuffixCompact
 @init { msgs.push("compaction request"); }
 @after { msgs.pop(); }
     : KW_COMPACT compactType=StringLiteral
-    -> ^(TOK_COMPACT $compactType)
+    -> ^(TOK_ALTERTABLE_COMPACT $compactType)
     ;
 
 
@@ -1707,7 +1693,7 @@ tableBuckets
 @after { popMsg(state); }
     :
       KW_CLUSTERED KW_BY LPAREN bucketCols=columnNameList RPAREN (KW_SORTED KW_BY LPAREN sortCols=columnNameOrderList RPAREN)? KW_INTO num=Number KW_BUCKETS
-    -> ^(TOK_TABLEBUCKETS $bucketCols $sortCols? $num)
+    -> ^(TOK_ALTERTABLE_BUCKETS $bucketCols $sortCols? $num)
     ;
 
 tableSkewed

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java Sat Aug 30 06:44:46 2014
@@ -88,6 +88,7 @@ public class IndexUpdater {
       sb.append("ALTER INDEX ");
       sb.append(idx.getIndexName());
       sb.append(" ON ");
+      sb.append(idx.getDbName()).append('.');
       sb.append(idx.getOrigTableName());
       sb.append(" REBUILD");
       driver.compile(sb.toString(), false);
@@ -125,6 +126,7 @@ public class IndexUpdater {
     sb.append("ALTER INDEX ");
     sb.append(index.getIndexName());
     sb.append(" ON ");
+    sb.append(index.getDbName()).append('.');
     sb.append(index.getOrigTableName());
     sb.append(" PARTITION ");
     sb.append(ps.toString());

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Sat Aug 30 06:44:46 2014
@@ -9408,7 +9408,7 @@ public class SemanticAnalyzer extends Ba
 
     // analyze create view command
     if (ast.getToken().getType() == HiveParser.TOK_CREATEVIEW ||
-        ast.getToken().getType() == HiveParser.TOK_ALTERVIEW_AS) {
+        (ast.getToken().getType() == HiveParser.TOK_ALTERVIEW && ast.getChild(1).getType() == HiveParser.TOK_QUERY)) {
       child = analyzeCreateView(ast, qb);
       SessionState.get().setCommandType(HiveOperation.CREATEVIEW);
       if (child == null) {
@@ -9416,7 +9416,7 @@ public class SemanticAnalyzer extends Ba
       }
       viewSelect = child;
       // prevent view from referencing itself
-      viewsExpanded.add(SessionState.get().getCurrentDatabase() + "." + createVwDesc.getViewName());
+      viewsExpanded.add(createVwDesc.getViewName());
     }
 
     // continue analyzing from the child ASTNode.
@@ -9517,6 +9517,11 @@ public class SemanticAnalyzer extends Ba
 
     LOG.info("Completed plan generation");
 
+    // put accessed columns to readEntity
+    if (HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_SCANCOLS)) {
+      putAccessedColumnsToReadEntity(inputs, columnAccessInfo);
+    }
+
     if (!ctx.getExplain()) {
       // if desired check we're not going over partition scan limits
       enforceScanLimits(pCtx, origFetchTask);
@@ -9525,6 +9530,26 @@ public class SemanticAnalyzer extends Ba
     return;
   }
 
+  private void putAccessedColumnsToReadEntity(HashSet<ReadEntity> inputs, ColumnAccessInfo columnAccessInfo) {
+    Map<String, List<String>> tableToColumnAccessMap = columnAccessInfo.getTableToColumnAccessMap();
+    if (tableToColumnAccessMap != null && !tableToColumnAccessMap.isEmpty()) {
+      for(ReadEntity entity: inputs) {
+        switch (entity.getType()) {
+          case TABLE:
+            entity.getAccessedColumns().addAll(
+                tableToColumnAccessMap.get(entity.getTable().getCompleteName()));
+            break;
+          case PARTITION:
+            entity.getAccessedColumns().addAll(
+                tableToColumnAccessMap.get(entity.getPartition().getTable().getCompleteName()));
+            break;
+          default:
+            // no-op
+        }
+      }
+    }
+  }
+
   private void enforceScanLimits(ParseContext pCtx, FetchTask fTask)
       throws SemanticException {
     int scanLimit = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVELIMITTABLESCANPARTITION);
@@ -9998,7 +10023,9 @@ public class SemanticAnalyzer extends Ba
    */
   private ASTNode analyzeCreateTable(ASTNode ast, QB qb)
       throws SemanticException {
-    String tableName = getUnescapedName((ASTNode) ast.getChild(0));
+    String[] qualifiedTabName = getQualifiedTableName((ASTNode) ast.getChild(0));
+    String dbDotTab = getDotName(qualifiedTabName);
+
     String likeTableName = null;
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     List<FieldSchema> partCols = new ArrayList<FieldSchema>();
@@ -10024,7 +10051,7 @@ public class SemanticAnalyzer extends Ba
     RowFormatParams rowFormatParams = new RowFormatParams();
     StorageFormat storageFormat = new StorageFormat(conf);
 
-    LOG.info("Creating table " + tableName + " position="
+    LOG.info("Creating table " + dbDotTab + " position="
         + ast.getCharPositionInLine());
     int numCh = ast.getChildCount();
 
@@ -10096,7 +10123,7 @@ public class SemanticAnalyzer extends Ba
       case HiveParser.TOK_TABLEPARTCOLS:
         partCols = getColumns((ASTNode) child.getChild(0), false);
         break;
-      case HiveParser.TOK_TABLEBUCKETS:
+      case HiveParser.TOK_ALTERTABLE_BUCKETS:
         bucketCols = getColumnNames((ASTNode) child.getChild(0));
         if (child.getChildCount() == 2) {
           numBuckets = (Integer.valueOf(child.getChild(1).getText()))
@@ -10155,7 +10182,7 @@ public class SemanticAnalyzer extends Ba
     // check for existence of table
     if (ifNotExists) {
       try {
-        Table table = getTable(tableName, false);
+        Table table = getTable(qualifiedTabName, false);
         if (table != null) { // table exists
           return null;
         }
@@ -10165,11 +10192,7 @@ public class SemanticAnalyzer extends Ba
       }
     }
 
-    String[] qualified = Hive.getQualifiedNames(tableName);
-    String dbName = qualified.length == 1 ? SessionState.get().getCurrentDatabase() : qualified[0];
-    Database database  = getDatabase(dbName);
-    outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_SHARED));
-    outputs.add(new WriteEntity(new Table(dbName, tableName), WriteEntity.WriteType.DDL_NO_LOCK));
+    addDbAndTabToOutputs(qualifiedTabName);
 
     if (isTemporary) {
       if (partCols.size() > 0) {
@@ -10198,7 +10221,7 @@ public class SemanticAnalyzer extends Ba
     case CREATE_TABLE: // REGULAR CREATE TABLE DDL
       tblProps = addDefaultProperties(tblProps);
 
-      crtTblDesc = new CreateTableDesc(tableName, isExt, isTemporary, cols, partCols,
+      crtTblDesc = new CreateTableDesc(dbDotTab, isExt, isTemporary, cols, partCols,
           bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim,
           rowFormatParams.fieldEscape,
           rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim,
@@ -10227,7 +10250,7 @@ public class SemanticAnalyzer extends Ba
               + "and source table in CREATE TABLE LIKE is partitioned.");
         }
       }
-      CreateTableLikeDesc crtTblLikeDesc = new CreateTableLikeDesc(tableName, isExt, isTemporary,
+      CreateTableLikeDesc crtTblLikeDesc = new CreateTableLikeDesc(dbDotTab, isExt, isTemporary,
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(), location,
           storageFormat.getSerde(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
           likeTableName);
@@ -10240,9 +10263,9 @@ public class SemanticAnalyzer extends Ba
 
       // Verify that the table does not already exist
       try {
-        Table dumpTable = db.newTable(tableName);
+        Table dumpTable = db.newTable(dbDotTab);
         if (null != db.getTable(dumpTable.getDbName(), dumpTable.getTableName(), false)) {
-          throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(tableName));
+          throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(dbDotTab));
         }
       } catch (HiveException e) {
         throw new SemanticException(e);
@@ -10250,11 +10273,10 @@ public class SemanticAnalyzer extends Ba
 
       tblProps = addDefaultProperties(tblProps);
 
-      crtTblDesc = new CreateTableDesc(dbName, tableName, isExt, isTemporary, cols, partCols,
-          bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim,
-          rowFormatParams.fieldEscape,
-          rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim,
-          comment, storageFormat.getInputFormat(),
+      crtTblDesc = new CreateTableDesc(qualifiedTabName[0], dbDotTab, isExt, isTemporary, cols,
+          partCols, bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim,
+          rowFormatParams.fieldEscape, rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim,
+          rowFormatParams.lineDelim, comment, storageFormat.getInputFormat(),
           storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
           storageFormat.getStorageHandler(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
           skewedColNames, skewedValues);
@@ -10271,9 +10293,17 @@ public class SemanticAnalyzer extends Ba
     return null;
   }
 
+  private void addDbAndTabToOutputs(String[] qualifiedTabName) throws SemanticException {
+    Database database  = getDatabase(qualifiedTabName[0]);
+    outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_SHARED));
+    outputs.add(new WriteEntity(new Table(qualifiedTabName[0], qualifiedTabName[1]),
+        WriteEntity.WriteType.DDL_NO_LOCK));
+  }
+
   private ASTNode analyzeCreateView(ASTNode ast, QB qb)
       throws SemanticException {
-    String tableName = getUnescapedName((ASTNode) ast.getChild(0));
+    String[] qualTabName = getQualifiedTableName((ASTNode) ast.getChild(0));
+    String dbDotTable = getDotName(qualTabName);
     List<FieldSchema> cols = null;
     boolean ifNotExists = false;
     boolean orReplace = false;
@@ -10283,7 +10313,7 @@ public class SemanticAnalyzer extends Ba
     Map<String, String> tblProps = null;
     List<String> partColNames = null;
 
-    LOG.info("Creating view " + tableName + " position="
+    LOG.info("Creating view " + dbDotTable + " position="
         + ast.getCharPositionInLine());
     int numCh = ast.getChildCount();
     for (int num = 1; num < numCh; num++) {
@@ -10319,19 +10349,21 @@ public class SemanticAnalyzer extends Ba
       throw new SemanticException("Can't combine IF NOT EXISTS and OR REPLACE.");
     }
 
-    if (ast.getToken().getType() == HiveParser.TOK_ALTERVIEW_AS) {
+    if (ast.getToken().getType() == HiveParser.TOK_ALTERVIEW &&
+        ast.getChild(1).getType() == HiveParser.TOK_QUERY) {
       isAlterViewAs = true;
       orReplace = true;
     }
 
     createVwDesc = new CreateViewDesc(
-      tableName, cols, comment, tblProps, partColNames,
+      dbDotTable, cols, comment, tblProps, partColNames,
       ifNotExists, orReplace, isAlterViewAs);
 
     unparseTranslator.enable();
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         createVwDesc), conf));
 
+    addDbAndTabToOutputs(qualTabName);
     return selectStmt;
   }
 

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java Sat Aug 30 06:44:46 2014
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.parse;
 
 import java.util.HashMap;
 
+import org.antlr.runtime.tree.Tree;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -57,7 +58,7 @@ public final class SemanticAnalyzerFacto
     commandType.put(HiveParser.TOK_ALTERTABLE_ARCHIVE, HiveOperation.ALTERTABLE_ARCHIVE);
     commandType.put(HiveParser.TOK_ALTERTABLE_UNARCHIVE, HiveOperation.ALTERTABLE_UNARCHIVE);
     commandType.put(HiveParser.TOK_ALTERTABLE_PROPERTIES, HiveOperation.ALTERTABLE_PROPERTIES);
-    commandType.put(HiveParser.TOK_DROPTABLE_PROPERTIES, HiveOperation.ALTERTABLE_PROPERTIES);
+    commandType.put(HiveParser.TOK_ALTERTABLE_DROPPROPERTIES, HiveOperation.ALTERTABLE_PROPERTIES);
     commandType.put(HiveParser.TOK_SHOWDATABASES, HiveOperation.SHOWDATABASES);
     commandType.put(HiveParser.TOK_SHOWTABLES, HiveOperation.SHOWTABLES);
     commandType.put(HiveParser.TOK_SHOWCOLUMNS, HiveOperation.SHOWCOLUMNS);
@@ -81,9 +82,11 @@ public final class SemanticAnalyzerFacto
     commandType.put(HiveParser.TOK_ALTERINDEX_REBUILD, HiveOperation.ALTERINDEX_REBUILD);
     commandType.put(HiveParser.TOK_ALTERINDEX_PROPERTIES, HiveOperation.ALTERINDEX_PROPS);
     commandType.put(HiveParser.TOK_ALTERVIEW_PROPERTIES, HiveOperation.ALTERVIEW_PROPERTIES);
-    commandType.put(HiveParser.TOK_DROPVIEW_PROPERTIES, HiveOperation.ALTERVIEW_PROPERTIES);
+    commandType.put(HiveParser.TOK_ALTERVIEW_DROPPROPERTIES, HiveOperation.ALTERVIEW_PROPERTIES);
     commandType.put(HiveParser.TOK_ALTERVIEW_ADDPARTS, HiveOperation.ALTERTABLE_ADDPARTS);
     commandType.put(HiveParser.TOK_ALTERVIEW_DROPPARTS, HiveOperation.ALTERTABLE_DROPPARTS);
+    commandType.put(HiveParser.TOK_ALTERVIEW_RENAME, HiveOperation.ALTERVIEW_RENAME);
+    commandType.put(HiveParser.TOK_ALTERVIEW, HiveOperation.ALTERVIEW_AS);
     commandType.put(HiveParser.TOK_QUERY, HiveOperation.QUERY);
     commandType.put(HiveParser.TOK_LOCKTABLE, HiveOperation.LOCKTABLE);
     commandType.put(HiveParser.TOK_UNLOCKTABLE, HiveOperation.UNLOCKTABLE);
@@ -105,11 +108,9 @@ public final class SemanticAnalyzerFacto
     commandType.put(HiveParser.TOK_DESCDATABASE, HiveOperation.DESCDATABASE);
     commandType.put(HiveParser.TOK_ALTERTABLE_SKEWED, HiveOperation.ALTERTABLE_SKEWED);
     commandType.put(HiveParser.TOK_ANALYZE, HiveOperation.ANALYZE_TABLE);
-    commandType.put(HiveParser.TOK_ALTERVIEW_RENAME, HiveOperation.ALTERVIEW_RENAME);
     commandType.put(HiveParser.TOK_ALTERTABLE_PARTCOLTYPE, HiveOperation.ALTERTABLE_PARTCOLTYPE);
     commandType.put(HiveParser.TOK_SHOW_COMPACTIONS, HiveOperation.SHOW_COMPACTIONS);
     commandType.put(HiveParser.TOK_SHOW_TRANSACTIONS, HiveOperation.SHOW_TRANSACTIONS);
-    commandType.put(HiveParser.TOK_ALTERTABLE_UPDATECOLSTATS, HiveOperation.ALTERTABLE_UPDATETABLESTATS);
   }
 
   static {
@@ -134,17 +135,20 @@ public final class SemanticAnalyzerFacto
             HiveOperation.ALTERPARTITION_SERDEPROPERTIES });
     tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_RENAMEPART,
         new HiveOperation[] {null, HiveOperation.ALTERTABLE_RENAMEPART});
-    tablePartitionCommandType.put(HiveParser.TOK_COMPACT,
+    tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_COMPACT,
         new HiveOperation[] {HiveOperation.ALTERTABLE_COMPACT, HiveOperation.ALTERTABLE_COMPACT});
-    tablePartitionCommandType.put(HiveParser.TOK_ALTERTBLPART_SKEWED_LOCATION,
+    tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_SKEWED_LOCATION,
         new HiveOperation[] {HiveOperation.ALTERTBLPART_SKEWED_LOCATION,
             HiveOperation.ALTERTBLPART_SKEWED_LOCATION });
-    tablePartitionCommandType.put(HiveParser.TOK_TABLEBUCKETS,
+    tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_BUCKETS,
         new HiveOperation[] {HiveOperation.ALTERTABLE_BUCKETNUM,
             HiveOperation.ALTERPARTITION_BUCKETNUM});
     tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_CLUSTER_SORT,
         new HiveOperation[] {HiveOperation.ALTERTABLE_CLUSTER_SORT,
             HiveOperation.ALTERTABLE_CLUSTER_SORT});
+    tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_UPDATECOLSTATS,
+        new HiveOperation[] {HiveOperation.ALTERTABLE_UPDATETABLESTATS,
+            HiveOperation.ALTERTABLE_UPDATEPARTSTATS});
   }
 
   public static BaseSemanticAnalyzer get(HiveConf conf, ASTNode tree)
@@ -152,9 +156,9 @@ public final class SemanticAnalyzerFacto
     if (tree.getToken() == null) {
       throw new RuntimeException("Empty Syntax Tree");
     } else {
-      setSessionCommandType(commandType.get(tree.getToken().getType()));
+      setSessionCommandType(commandType.get(tree.getType()));
 
-      switch (tree.getToken().getType()) {
+      switch (tree.getType()) {
       case HiveParser.TOK_EXPLAIN:
         return new ExplainSemanticAnalyzer(conf);
       case HiveParser.TOK_EXPLAIN_SQ_REWRITE:
@@ -165,6 +169,47 @@ public final class SemanticAnalyzerFacto
         return new ExportSemanticAnalyzer(conf);
       case HiveParser.TOK_IMPORT:
         return new ImportSemanticAnalyzer(conf);
+      case HiveParser.TOK_ALTERTABLE: {
+        Tree child = tree.getChild(1);
+        switch (child.getType()) {
+          case HiveParser.TOK_ALTERTABLE_RENAME:
+          case HiveParser.TOK_ALTERTABLE_TOUCH:
+          case HiveParser.TOK_ALTERTABLE_ARCHIVE:
+          case HiveParser.TOK_ALTERTABLE_UNARCHIVE:
+          case HiveParser.TOK_ALTERTABLE_ADDCOLS:
+          case HiveParser.TOK_ALTERTABLE_RENAMECOL:
+          case HiveParser.TOK_ALTERTABLE_REPLACECOLS:
+          case HiveParser.TOK_ALTERTABLE_DROPPARTS:
+          case HiveParser.TOK_ALTERTABLE_ADDPARTS:
+          case HiveParser.TOK_ALTERTABLE_PARTCOLTYPE:
+          case HiveParser.TOK_ALTERTABLE_PROPERTIES:
+          case HiveParser.TOK_ALTERTABLE_DROPPROPERTIES:
+          case HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION:
+          case HiveParser.TOK_ALTERTABLE_SKEWED:
+          setSessionCommandType(commandType.get(child.getType()));
+          return new DDLSemanticAnalyzer(conf);
+        }
+        HiveOperation commandType =
+            tablePartitionCommandType.get(child.getType())[tree.getChildCount() > 2 ? 1 : 0];
+        setSessionCommandType(commandType);
+        return new DDLSemanticAnalyzer(conf);
+      }
+      case HiveParser.TOK_ALTERVIEW: {
+        Tree child = tree.getChild(1);
+        switch (child.getType()) {
+          case HiveParser.TOK_ALTERVIEW_PROPERTIES:
+          case HiveParser.TOK_ALTERVIEW_DROPPROPERTIES:
+          case HiveParser.TOK_ALTERVIEW_ADDPARTS:
+          case HiveParser.TOK_ALTERVIEW_DROPPARTS:
+          case HiveParser.TOK_ALTERVIEW_RENAME:
+            setSessionCommandType(commandType.get(child.getType()));
+            return new DDLSemanticAnalyzer(conf);
+        }
+        // TOK_ALTERVIEW_AS
+        assert child.getType() == HiveParser.TOK_QUERY;
+        setSessionCommandType(HiveOperation.ALTERVIEW_AS);
+        return new SemanticAnalyzer(conf);
+      }
       case HiveParser.TOK_CREATEDATABASE:
       case HiveParser.TOK_DROPDATABASE:
       case HiveParser.TOK_SWITCHDATABASE:
@@ -174,24 +219,8 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_DESCTABLE:
       case HiveParser.TOK_DESCFUNCTION:
       case HiveParser.TOK_MSCK:
-      case HiveParser.TOK_ALTERTABLE_ADDCOLS:
-      case HiveParser.TOK_ALTERTABLE_RENAMECOL:
-      case HiveParser.TOK_ALTERTABLE_REPLACECOLS:
-      case HiveParser.TOK_ALTERTABLE_RENAME:
-      case HiveParser.TOK_ALTERTABLE_DROPPARTS:
-      case HiveParser.TOK_ALTERTABLE_ADDPARTS:
-      case HiveParser.TOK_ALTERTABLE_PROPERTIES:
-      case HiveParser.TOK_DROPTABLE_PROPERTIES:
-      case HiveParser.TOK_ALTERTABLE_SERIALIZER:
-      case HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES:
-      case HiveParser.TOK_ALTERTABLE_PARTCOLTYPE:
       case HiveParser.TOK_ALTERINDEX_REBUILD:
       case HiveParser.TOK_ALTERINDEX_PROPERTIES:
-      case HiveParser.TOK_ALTERVIEW_PROPERTIES:
-      case HiveParser.TOK_DROPVIEW_PROPERTIES:
-      case HiveParser.TOK_ALTERVIEW_ADDPARTS:
-      case HiveParser.TOK_ALTERVIEW_DROPPARTS:
-      case HiveParser.TOK_ALTERVIEW_RENAME:
       case HiveParser.TOK_SHOWDATABASES:
       case HiveParser.TOK_SHOWTABLES:
       case HiveParser.TOK_SHOWCOLUMNS:
@@ -209,9 +238,6 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_CREATEINDEX:
       case HiveParser.TOK_DROPINDEX:
       case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT:
-      case HiveParser.TOK_ALTERTABLE_TOUCH:
-      case HiveParser.TOK_ALTERTABLE_ARCHIVE:
-      case HiveParser.TOK_ALTERTABLE_UNARCHIVE:
       case HiveParser.TOK_LOCKTABLE:
       case HiveParser.TOK_UNLOCKTABLE:
       case HiveParser.TOK_LOCKDB:
@@ -228,23 +254,8 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_SHOW_ROLES:
       case HiveParser.TOK_ALTERDATABASE_PROPERTIES:
       case HiveParser.TOK_ALTERDATABASE_OWNER:
-      case HiveParser.TOK_ALTERTABLE_SKEWED:
       case HiveParser.TOK_TRUNCATETABLE:
-      case HiveParser.TOK_EXCHANGEPARTITION:
       case HiveParser.TOK_SHOW_SET_ROLE:
-      case HiveParser.TOK_ALTERTABLE_UPDATECOLSTATS:
-        return new DDLSemanticAnalyzer(conf);
-      case HiveParser.TOK_ALTERTABLE_PARTITION:
-        HiveOperation commandType = null;
-        Integer type = ((ASTNode) tree.getChild(1)).getToken().getType();
-        if (type == HiveParser.TOK_ALTERTABLE_UPDATECOLSTATS) {
-          commandType = HiveOperation.ALTERTABLE_UPDATEPARTSTATS;
-        } else if (tree.getChild(0).getChildCount() > 1) {
-          commandType = tablePartitionCommandType.get(type)[1];
-        } else {
-          commandType = tablePartitionCommandType.get(type)[0];
-        }
-        setSessionCommandType(commandType);
         return new DDLSemanticAnalyzer(conf);
 
       case HiveParser.TOK_CREATEFUNCTION: