You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2014/10/01 00:16:54 UTC

svn commit: r1628565 - in /hive/trunk: hcatalog/src/test/e2e/templeton/tests/ ql/src/java/org/apache/hadoop/hive/ql/metadata/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/test/queries/clientnegative/ ql/src/test/results/clientnegative/

Author: thejas
Date: Tue Sep 30 22:16:54 2014
New Revision: 1628565

URL: http://svn.apache.org/r1628565
Log:
HIVE-8287 : Metadata action errors don't have information about cause (Thejas Nair, reviewed by Jason Dere)

Added:
    hive/trunk/ql/src/test/queries/clientnegative/authorization_sba_drop_table.q
    hive/trunk/ql/src/test/results/clientnegative/authorization_sba_drop_table.q.out
Modified:
    hive/trunk/hcatalog/src/test/e2e/templeton/tests/doas.conf
    hive/trunk/hcatalog/src/test/e2e/templeton/tests/hcatperms.conf
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
    hive/trunk/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q
    hive/trunk/ql/src/test/results/clientnegative/alter_partition_with_whitelist.q.out
    hive/trunk/ql/src/test/results/clientnegative/alter_rename_partition_failure2.q.out
    hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on1.q.out
    hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on2.q.out
    hive/trunk/ql/src/test/results/clientnegative/temp_table_rename.q.out

Modified: hive/trunk/hcatalog/src/test/e2e/templeton/tests/doas.conf
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/templeton/tests/doas.conf?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/templeton/tests/doas.conf (original)
+++ hive/trunk/hcatalog/src/test/e2e/templeton/tests/doas.conf Tue Sep 30 22:16:54 2014
@@ -109,7 +109,7 @@ $cfg = 
      'method' => 'GET',
      'url' => ':TEMPLETON_URL:/templeton/v1/ddl/database/default/table/:UNAME:_doastab2/partition?user.name=:UNAME:&doAs=:DOAS:',
      'status_code' => 500,
-     'json_field_substr_match' => {'error' => 'FAILED: AuthorizationException java\.security\.AccessControlException: action READ not permitted on path .* for user :DOAS:'},
+     'json_field_substr_match' => {'error' => 'java\.security\.AccessControlException: Permission denied: user=:DOAS:, access=READ'},
     },
   
     {
@@ -118,7 +118,7 @@ $cfg = 
      'method' => 'DELETE',
      'url' => ':TEMPLETON_URL:/templeton/v1/ddl/database/default/table/:UNAME:_doastab2?user.name=:UNAME:&doAs=:DOAS:',
      'status_code' => 500,
-     'json_field_substr_match' => {'error' => 'java\.security\.AccessControlException: action WRITE not permitted on path .* for user :DOAS:'},
+     'json_field_substr_match' => {'error' => 'java\.security\.AccessControlException: Permission denied: user=:DOAS:, access=READ'},
     },
     {
              #descbe the table....

Modified: hive/trunk/hcatalog/src/test/e2e/templeton/tests/hcatperms.conf
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/templeton/tests/hcatperms.conf?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/templeton/tests/hcatperms.conf (original)
+++ hive/trunk/hcatalog/src/test/e2e/templeton/tests/hcatperms.conf Tue Sep 30 22:16:54 2014
@@ -375,6 +375,13 @@ $cfg = 
                  {
                   'method' => 'DELETE',
                   'format_header' => 'Content-Type: application/json',
+                  'url' => ':TEMPLETON_URL:/templeton/v1/ddl/database/hcatperms_:TNUM:/table/permstable_:TNUM:',
+                  'user_name' => ':UNAME_GROUP:',
+                  'status_code' => 200,
+                 },
+                 {
+                  'method' => 'DELETE',
+                  'format_header' => 'Content-Type: application/json',
                   'url' => ':TEMPLETON_URL:/templeton/v1/ddl/database/hcatperms_:TNUM:?ifExists=true&option=cascade',
                   'user_name' => ':UNAME:',
                   'status_code' => 200,
@@ -677,9 +684,7 @@ $cfg = 
       'format_header' => 'Content-Type: application/json', 
       'user_name' => ':UNAME_OTHER:',
       'status_code' => 500,
-     'json_field_substr_match' => {'error' => 'FAILED: AuthorizationException .*\.security\.AccessControlException: action READ not permitted on path .* for user :UNAME_OTHER:'},
-
-
+     'json_field_substr_match' => {'error' => 'AccessControlException: Permission denied: user=:UNAME_OTHER:, access=READ'},
     },
 
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Tue Sep 30 22:16:54 2014
@@ -109,8 +109,8 @@ import org.apache.hadoop.hive.serde2.laz
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.thrift.TException;
 
 import com.google.common.collect.Sets;
@@ -427,9 +427,9 @@ public class Hive {
       newTbl.checkValidity();
       getMSC().alter_table(names[0], names[1], newTbl.getTTable());
     } catch (MetaException e) {
-      throw new HiveException("Unable to alter table.", e);
+      throw new HiveException("Unable to alter table. " + e.getMessage(), e);
     } catch (TException e) {
-      throw new HiveException("Unable to alter table.", e);
+      throw new HiveException("Unable to alter table. " + e.getMessage(), e);
     }
   }
 
@@ -455,9 +455,9 @@ public class Hive {
     try {
       getMSC().alter_index(dbName, baseTblName, idxName, newIdx);
     } catch (MetaException e) {
-      throw new HiveException("Unable to alter index.", e);
+      throw new HiveException("Unable to alter index. " + e.getMessage(), e);
     } catch (TException e) {
-      throw new HiveException("Unable to alter index.", e);
+      throw new HiveException("Unable to alter index. " + e.getMessage(), e);
     }
   }
 
@@ -502,9 +502,9 @@ public class Hive {
       getMSC().alter_partition(dbName, tblName, newPart.getTPartition());
 
     } catch (MetaException e) {
-      throw new HiveException("Unable to alter partition.", e);
+      throw new HiveException("Unable to alter partition. " + e.getMessage(), e);
     } catch (TException e) {
-      throw new HiveException("Unable to alter partition.", e);
+      throw new HiveException("Unable to alter partition. " + e.getMessage(), e);
     }
   }
 
@@ -534,9 +534,9 @@ public class Hive {
       }
       getMSC().alter_partitions(names[0], names[1], newTParts);
     } catch (MetaException e) {
-      throw new HiveException("Unable to alter partition.", e);
+      throw new HiveException("Unable to alter partition. " + e.getMessage(), e);
     } catch (TException e) {
-      throw new HiveException("Unable to alter partition.", e);
+      throw new HiveException("Unable to alter partition. " + e.getMessage(), e);
     }
   }
   /**
@@ -578,11 +578,11 @@ public class Hive {
           newPart.getTPartition());
 
     } catch (InvalidOperationException e){
-      throw new HiveException("Unable to rename partition.", e);
+      throw new HiveException("Unable to rename partition. " + e.getMessage(), e);
     } catch (MetaException e) {
-      throw new HiveException("Unable to rename partition.", e);
+      throw new HiveException("Unable to rename partition. " + e.getMessage(), e);
     } catch (TException e) {
-      throw new HiveException("Unable to rename partition.", e);
+      throw new HiveException("Unable to rename partition. " + e.getMessage(), e);
     }
   }
 
@@ -591,11 +591,11 @@ public class Hive {
     try {
       getMSC().alterDatabase(dbName, db);
     } catch (MetaException e) {
-      throw new HiveException("Unable to alter database " + dbName, e);
+      throw new HiveException("Unable to alter database " + dbName + ". " + e.getMessage(), e);
     } catch (NoSuchObjectException e) {
       throw new HiveException("Database " + dbName + " does not exists.", e);
     } catch (TException e) {
-      throw new HiveException("Unable to alter database " + dbName, e);
+      throw new HiveException("Unable to alter database " + dbName + ". " + e.getMessage(), e);
     }
   }
   /**
@@ -870,9 +870,9 @@ public class Hive {
     try {
       return getMSC().dropIndex(db_name, tbl_name, index_name, deleteData);
     } catch (NoSuchObjectException e) {
-      throw new HiveException("Partition or table doesn't exist.", e);
+      throw new HiveException("Partition or table doesn't exist. " + e.getMessage(), e);
     } catch (Exception e) {
-      throw new HiveException("Unknown error. Please check logs.", e);
+      throw new HiveException(e.getMessage(), e);
     }
   }
 
@@ -1041,7 +1041,7 @@ public class Hive {
       }
       return null;
     } catch (Exception e) {
-      throw new HiveException("Unable to fetch table " + tableName, e);
+      throw new HiveException("Unable to fetch table " + tableName + ". " + e.getMessage(), e);
     }
 
     // For non-views, we need to do some extra fixes
@@ -1763,7 +1763,7 @@ private void constructOneLBLocationMap(F
     } catch (NoSuchObjectException e) {
       throw new HiveException("Partition or table doesn't exist.", e);
     } catch (Exception e) {
-      throw new HiveException("Unknown error. Please check logs.", e);
+      throw new HiveException(e.getMessage(), e);
     }
   }
 
@@ -1792,7 +1792,7 @@ private void constructOneLBLocationMap(F
     } catch (NoSuchObjectException e) {
       throw new HiveException("Partition or table doesn't exist.", e);
     } catch (Exception e) {
-      throw new HiveException("Unknown error. Please check logs.", e);
+      throw new HiveException(e.getMessage(), e);
     }
   }
 
@@ -2285,7 +2285,7 @@ private void constructOneLBLocationMap(F
         result.add(srcToDest);
       }
     } catch (IOException e) {
-      throw new HiveException("checkPaths: filesystem error in check phase", e);
+      throw new HiveException("checkPaths: filesystem error in check phase. " + e.getMessage(), e);
     }
     return result;
   }
@@ -2352,7 +2352,7 @@ private void constructOneLBLocationMap(F
       try {
         ShimLoader.getHadoopShims().setFullFileStatus(conf, destStatus, fs, destf);
       } catch (IOException e) {
-        LOG.warn("Error setting permission of file " + destf + ": "+ StringUtils.stringifyException(e));
+        LOG.warn("Error setting permission of file " + destf + ": "+ e.getMessage(), e);
       }
     }
     return success;
@@ -2391,7 +2391,7 @@ private void constructOneLBLocationMap(F
       srcs = srcFs.globStatus(srcf);
     } catch (IOException e) {
       LOG.error(StringUtils.stringifyException(e));
-      throw new HiveException("addFiles: filesystem error in check phase", e);
+      throw new HiveException("addFiles: filesystem error in check phase. " + e.getMessage(), e);
     }
     if (srcs == null) {
       LOG.info("No sources specified to move: " + srcf);
@@ -2417,7 +2417,7 @@ private void constructOneLBLocationMap(F
           }
         }
       } catch (IOException e) {
-        throw new HiveException("copyFiles: error while moving files!!!", e);
+        throw new HiveException("copyFiles: error while moving files!!! " + e.getMessage(), e);
       }
     }
   }
@@ -2489,7 +2489,7 @@ private void constructOneLBLocationMap(F
               fs.rename(bucketSrc, bucketDest);
             }
           } catch (IOException e) {
-            throw new HiveException("Error moving acid files", e);
+            throw new HiveException("Error moving acid files " + e.getMessage(), e);
           }
         }
       }
@@ -2721,7 +2721,7 @@ private void constructOneLBLocationMap(F
       throw new HiveException(e);
     }
   }
-  
+
   public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request) throws HiveException {
     try {
       return getMSC().setPartitionColumnStatistics(request);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java Tue Sep 30 22:16:54 2014
@@ -1285,7 +1285,7 @@ public abstract class BaseSemanticAnalyz
     try {
       database = db.getDatabase(dbName);
     } catch (Exception e) {
-      throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(dbName), e);
+      throw new SemanticException(e.getMessage(), e);
     }
     if (database == null && throwException) {
       throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(dbName));
@@ -1315,9 +1315,13 @@ public abstract class BaseSemanticAnalyz
     try {
       tab = database == null ? db.getTable(tblName, false)
           : db.getTable(database, tblName, false);
-    } catch (Exception e) {
+    }
+    catch (InvalidTableException e) {
       throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName), e);
     }
+    catch (Exception e) {
+      throw new SemanticException(e.getMessage(), e);
+    }
     if (tab == null && throwException) {
       throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName));
     }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java Tue Sep 30 22:16:54 2014
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.Context
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 
 /**
@@ -95,8 +96,10 @@ public class ColumnStatsSemanticAnalyzer
     String tableName = getUnescapedName((ASTNode) tree.getChild(0).getChild(0));
     try {
       return db.getTable(tableName);
+    } catch (InvalidTableException e) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName), e);
     } catch (HiveException e) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName));
+      throw new SemanticException(e.getMessage(), e);
     }
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Tue Sep 30 22:16:54 2014
@@ -79,6 +79,7 @@ import org.apache.hadoop.hive.ql.lockmgr
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+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.parse.authorization.AuthorizationParseUtils;
@@ -1718,7 +1719,8 @@ public class DDLSemanticAnalyzer extends
 
     // assume the first component of DOT delimited name is tableName
     // get the attemptTableName
-    static public String getAttemptTableName(Hive db, String qualifiedName, boolean isColumn) {
+    static public String getAttemptTableName(Hive db, String qualifiedName, boolean isColumn)
+        throws SemanticException {
       // check whether the name starts with table
       // DESCRIBE table
       // DESCRIBE table.column
@@ -1739,11 +1741,13 @@ public class DDLSemanticAnalyzer extends
             return tableName;
           }
         }
-      } catch (HiveException e) {
+      } catch (InvalidTableException e) {
         // assume the first DOT delimited component is tableName
         // OK if it is not
         // do nothing when having exception
         return null;
+      } catch (HiveException e) {
+        throw new SemanticException(e.getMessage(), e);
       }
       return null;
     }
@@ -1824,7 +1828,7 @@ public class DDLSemanticAnalyzer extends
       ASTNode parentAst,
       ASTNode ast,
       String tableName,
-      Map<String, String> partSpec) {
+      Map<String, String> partSpec) throws SemanticException {
 
       // if parent has two children
       // it could be DESCRIBE table key
@@ -1880,11 +1884,13 @@ public class DDLSemanticAnalyzer extends
         Table tab = null;
         try {
           tab = db.getTable(tableName);
-        } catch (HiveException e) {
-          // if table not valid
-          // throw semantic exception
+        }
+        catch (InvalidTableException e) {
           throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName), e);
         }
+        catch (HiveException e) {
+          throw new SemanticException(e.getMessage(), e);
+        }
 
         if (partSpec != null) {
           Partition part = null;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Tue Sep 30 22:16:54 2014
@@ -110,6 +110,7 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+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.metadata.VirtualColumn;
@@ -11056,9 +11057,13 @@ public class SemanticAnalyzer extends Ba
     Table tbl;
     try {
       tbl = db.getTable(tableName);
-    } catch (HiveException e) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName));
+    } catch (InvalidTableException e) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName), e);
+    }
+    catch (HiveException e) {
+      throw new SemanticException(e.getMessage(), e);
     }
+
     /* noscan uses hdfs apis to retrieve such information from Namenode.      */
     /* But that will be specific to hdfs. Through storagehandler mechanism,   */
     /* storage of table could be on any storage system: hbase, cassandra etc. */
@@ -11081,8 +11086,10 @@ public class SemanticAnalyzer extends Ba
     Table tbl;
     try {
       tbl = db.getTable(tableName);
+    } catch (InvalidTableException e) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName), e);
     } catch (HiveException e) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName));
+      throw new SemanticException(e.getMessage(), e);
     }
     /* partialscan uses hdfs apis to retrieve such information from Namenode.      */
     /* But that will be specific to hdfs. Through storagehandler mechanism,   */

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java Tue Sep 30 22:16:54 2014
@@ -17,6 +17,13 @@
  */
 package org.apache.hadoop.hive.ql.parse;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.Context;
@@ -27,19 +34,12 @@ import org.apache.hadoop.hive.ql.hooks.W
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-
 /**
  * A subclass of the {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer} that just handles
  * update and delete statements.  It works by rewriting the updates and deletes into insert
@@ -128,11 +128,16 @@ public class UpdateDeleteSemanticAnalyze
     Table mTable;
     try {
       mTable = db.getTable(tableName[0], tableName[1]);
+    } catch (InvalidTableException e) {
+      LOG.error("Failed to find table " + getDotName(tableName) + " got exception "
+          + e.getMessage());
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(getDotName(tableName)), e);
     } catch (HiveException e) {
-      LOG.error("Failed to find table " + getDotName(tableName) + " got exception " +
-          e.getMessage());
-      throw new SemanticException(ErrorMsg.INVALID_TABLE, getDotName(tableName));
+      LOG.error("Failed to find table " + getDotName(tableName) + " got exception "
+          + e.getMessage());
+      throw new SemanticException(e.getMessage(), e);
     }
+
     List<FieldSchema> partCols = mTable.getPartCols();
 
     rewrittenQueryStr.append("insert into table ");

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_sba_drop_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_sba_drop_table.q?rev=1628565&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_sba_drop_table.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_sba_drop_table.q Tue Sep 30 22:16:54 2014
@@ -0,0 +1,9 @@
+set hive.metastore.pre.event.listeners=org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
+set hive.security.metastore.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_sba_droptab1;
+
+create table t1(i int) location '${system:test.tmp.dir}/a_sba_droptab1';
+dfs -chmod 555 ${system:test.tmp.dir}/a_sba_droptab1;
+-- Attempt to drop table without having write permissions on table dir should result in error
+drop table t1;

Modified: hive/trunk/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q (original)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q Tue Sep 30 22:16:54 2014
@@ -7,6 +7,6 @@ dfs ${system:test.dfs.mkdir} ${system:te
 dfs -touchz ${system:test.tmp.dir}/a_uri_crtab1/1.txt;
 dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab1/1.txt;
 
-create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext';
+create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab1';
 
 -- Attempt to create table with dir that does not have write permission should fail

Modified: hive/trunk/ql/src/test/results/clientnegative/alter_partition_with_whitelist.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/alter_partition_with_whitelist.q.out?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/alter_partition_with_whitelist.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/alter_partition_with_whitelist.q.out Tue Sep 30 22:16:54 2014
@@ -27,4 +27,4 @@ PREHOOK: query: ALTER TABLE part_whiteli
 PREHOOK: type: ALTERTABLE_RENAMEPART
 PREHOOK: Input: default@part_whitelist_test
 PREHOOK: Output: default@part_whitelist_test@ds=1
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to rename partition.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to rename partition. Partition value '1,2,3' contains a character not matched by whitelist pattern '[\\x20-\\x7E&&[^,]]*'.  (configure with hive.metastore.partition.name.whitelist.pattern)

Modified: hive/trunk/ql/src/test/results/clientnegative/alter_rename_partition_failure2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/alter_rename_partition_failure2.q.out?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/alter_rename_partition_failure2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/alter_rename_partition_failure2.q.out Tue Sep 30 22:16:54 2014
@@ -35,4 +35,4 @@ PREHOOK: query: alter table alter_rename
 PREHOOK: type: ALTERTABLE_RENAMEPART
 PREHOOK: Input: default@alter_rename_partition
 PREHOOK: Output: default@alter_rename_partition@pcol1=old_part1%3A/pcol2=old_part2%3A
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to rename partition.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to rename partition. Partition already exists:default.alter_rename_partition.[old_part1:, old_part2:]

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_sba_drop_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_sba_drop_table.q.out?rev=1628565&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_sba_drop_table.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_sba_drop_table.q.out Tue Sep 30 22:16:54 2014
@@ -0,0 +1,16 @@
+#### A masked pattern was here ####
+PREHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+#### A masked pattern was here ####
+POSTHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: -- Attempt to drop table without having write permissions on table dir should result in error
+drop table t1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t1
+PREHOOK: Output: default@t1
+#### A masked pattern was here ####

Modified: hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on1.q.out?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on1.q.out Tue Sep 30 22:16:54 2014
@@ -107,4 +107,5 @@ ALTER TABLE test_table123 REPLACE COLUMN
 PREHOOK: type: ALTERTABLE_REPLACECOLS
 PREHOOK: Input: default@test_table123
 PREHOOK: Output: default@test_table123
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. The following columns have types incompatible with the existing columns in their respective positions :
+b

Modified: hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on2.q.out?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/disallow_incompatible_type_change_on2.q.out Tue Sep 30 22:16:54 2014
@@ -40,4 +40,5 @@ PREHOOK: query: ALTER TABLE test_table12
 PREHOOK: type: ALTERTABLE_RENAMECOL
 PREHOOK: Input: default@test_table123
 PREHOOK: Output: default@test_table123
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. The following columns have types incompatible with the existing columns in their respective positions :
+b

Modified: hive/trunk/ql/src/test/results/clientnegative/temp_table_rename.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/temp_table_rename.q.out?rev=1628565&r1=1628564&r2=1628565&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/temp_table_rename.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/temp_table_rename.q.out Tue Sep 30 22:16:54 2014
@@ -18,4 +18,4 @@ PREHOOK: query: alter table tmp2 rename 
 PREHOOK: type: ALTERTABLE_RENAME
 PREHOOK: Input: default@tmp2
 PREHOOK: Output: default@tmp2
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. Cannot rename temporary table to tmp1 - temporary table already exists with the same name