You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by at...@apache.org on 2009/01/21 01:24:08 UTC

svn commit: r736176 - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/metadata/ ql/src/test/queries/clientnegative/ ql/src/test/results/clientnegative/

Author: athusoo
Date: Tue Jan 20 16:24:07 2009
New Revision: 736176

URL: http://svn.apache.org/viewvc?rev=736176&view=rev
Log:
HIVE-239. Check that replace columns in alter table does not have names
that are same as the partitioning columns (Prasad Chakka via athusoo)


Added:
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/altern1.q
    hadoop/hive/trunk/ql/src/test/results/clientnegative/altern1.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=736176&r1=736175&r2=736176&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Tue Jan 20 16:24:07 2009
@@ -211,3 +211,6 @@
 
     HIVE-232. Metastore.warehouse configuration should use inherited 
     hadoop configuration (Prasad Chakka via dhruba)
+
+    HIVE-239. Check that replace columns in alter table does not have names
+    that are same as the partitioning columns (Prasad Chakka via athusoo)

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=736176&r1=736175&r2=736176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Tue Jan 20 16:24:07 2009
@@ -40,7 +40,6 @@
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.ql.metadata.CheckResult;
@@ -50,8 +49,8 @@
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.plan.MsckDesc;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.MsckDesc;
 import org.apache.hadoop.hive.ql.plan.alterTableDesc;
 import org.apache.hadoop.hive.ql.plan.createTableDesc;
 import org.apache.hadoop.hive.ql.plan.descTableDesc;
@@ -64,8 +63,6 @@
 import org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe;
 import org.apache.hadoop.util.StringUtils;
 
-import com.facebook.thrift.TException;
-
 /**
  * DDLTask implementation
  * 
@@ -249,7 +246,7 @@
     Table tbl = null;
     List<String> parts = null;
 
-    tbl = db.getTable(tabName);
+    tbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tabName);
 
     if (!tbl.isPartitioned()) {
       console.printError("Table " + tabName + " is not a partitioned table");
@@ -340,7 +337,7 @@
         colPath.indexOf('.') == -1 ? colPath.length() : colPath.indexOf('.'));
 
     // describe the table - populate the output stream
-    Table tbl = db.getTable(tableName, false);
+    Table tbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
     Partition part = null;
     try {
       if (tbl == null) {
@@ -383,7 +380,7 @@
           cols = part.getTPartition().getSd().getCols();
         }
       } else {
-        cols = db.getFieldsFromDeserializer(colPath, tbl.getDeserializer());
+        cols = Hive.getFieldsFromDeserializer(colPath, tbl.getDeserializer());
       }
 
       Iterator<FieldSchema> iterCols = cols.iterator();
@@ -456,7 +453,7 @@
    */
   private int alterTable(Hive db, alterTableDesc alterTbl) throws HiveException {
     // alter the table
-    Table tbl = db.getTable(alterTbl.getOldName());
+    Table tbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, alterTbl.getOldName());
     if (alterTbl.getOp() == alterTableDesc.alterTableTypes.RENAME)
       tbl.getTTable().setTableName(alterTbl.getNewName());
     else if (alterTbl.getOp() == alterTableDesc.alterTableTypes.ADDCOLS) {
@@ -526,13 +523,19 @@
         .currentTimeMillis() / 1000));
 
     try {
+      tbl.checkValidity();
+    } catch (HiveException e) {
+      console.printError("Invalid table columns : " + e.getMessage(), StringUtils.stringifyException(e));
+      return 1;
+    }
+
+    try {
       db.alterTable(alterTbl.getOldName(), tbl);
     } catch (InvalidOperationException e) {
+      console.printError("Invalid alter operation: " + e.getMessage());
       LOG.info("alter table: " + StringUtils.stringifyException(e));
       return 1;
-    } catch (MetaException e) {
-      return 1;
-    } catch (TException e) {
+    } catch (HiveException e) {
       return 1;
     }
     return 0;
@@ -549,10 +552,10 @@
   private int dropTable(Hive db, dropTableDesc dropTbl) throws HiveException {
     if (dropTbl.getPartSpecs() == null) {
       // drop the table
-      db.dropTable(dropTbl.getTableName());
+      db.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, dropTbl.getTableName());
     } else {
       // drop partitions in the list
-      Table tbl = db.getTable(dropTbl.getTableName());
+      Table tbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, dropTbl.getTableName());
       List<Partition> parts = new ArrayList<Partition>();
       for (Map<String, String> partSpec : dropTbl.getPartSpecs()) {
         Partition part = db.getPartition(tbl, partSpec, false);

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=736176&r1=736175&r2=736176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Tue Jan 20 16:24:07 2009
@@ -210,13 +210,18 @@
    * @param tblName name of the existing table
    * @param newTbl new name of the table. could be the old name
    * @throws InvalidOperationException if the changes in metadata is not acceptable
-   * @throws MetaException
    * @throws TException
    */
   public void alterTable(String tblName,
       Table newTbl) throws InvalidOperationException,
-      MetaException, TException {
-    getMSC().alter_table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, newTbl.getTTable());
+      HiveException {
+    try {
+      getMSC().alter_table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, newTbl.getTTable());
+    } catch (MetaException e) {
+      throw new HiveException("Unable to alter table.", e);
+    } catch (TException e) {
+      throw new HiveException("Unable to alter table.", e);
+    }
   }
 
   /**

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java?rev=736176&r1=736175&r2=736176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java Tue Jan 20 16:24:07 2009
@@ -184,7 +184,7 @@
       while (partColsIter.hasNext()) {
         String partCol = partColsIter.next().getName();
         if(colNames.contains(partCol.toLowerCase()))
-            throw new HiveException("Partition collumn name " + partCol + " conflicts with table columns.");
+            throw new HiveException("Partition column name " + partCol + " conflicts with table columns.");
       }
     }
     return;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/altern1.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/altern1.q?rev=736176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/altern1.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/altern1.q Tue Jan 20 16:24:07 2009
@@ -0,0 +1,4 @@
+drop table altern1;
+create table altern1(a int, b int) partitioned by (ds string);
+alter table altern1 replace columns(a int, b int, ds string);
+drop table altern1;

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/altern1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/altern1.q.out?rev=736176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/altern1.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/altern1.q.out Tue Jan 20 16:24:07 2009
@@ -0,0 +1,2 @@
+Invalid table columns : Partition column name ds conflicts with table columns.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out?rev=736176&r1=736175&r2=736176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out Tue Jan 20 16:24:07 2009
@@ -1,2 +1,2 @@
-FAILED: Error in metadata: Partition collumn name aint conflicts with table columns.
+FAILED: Error in metadata: Partition column name aint conflicts with table columns.
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask