You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2012/08/02 07:09:03 UTC

svn commit: r1368352 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/parse/ test/queries/clientnegative/ test/queries/clientpositive/ test/results/clientnegative/ test/results/clientpositive/

Author: namit
Date: Thu Aug  2 05:09:02 2012
New Revision: 1368352

URL: http://svn.apache.org/viewvc?rev=1368352&view=rev
Log:
HIVE-3282 Convert runtime exceptions to semantic exceptions for missing partitions/tables
in show/describe statements (Sambavi Muthukrishnan via namit)


Added:
    hive/trunk/ql/src/test/queries/clientnegative/desc_failure1.q
    hive/trunk/ql/src/test/queries/clientnegative/desc_failure2.q
    hive/trunk/ql/src/test/queries/clientnegative/show_partitions1.q
    hive/trunk/ql/src/test/queries/clientnegative/show_tableproperties1.q
    hive/trunk/ql/src/test/results/clientnegative/desc_failure1.q.out
    hive/trunk/ql/src/test/results/clientnegative/desc_failure2.q.out
    hive/trunk/ql/src/test/results/clientnegative/show_partitions1.q.out
    hive/trunk/ql/src/test/results/clientnegative/show_tableproperties1.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/trunk/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q
    hive/trunk/ql/src/test/queries/clientpositive/inputddl6.q
    hive/trunk/ql/src/test/queries/clientpositive/show_tblproperties.q
    hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db1.q.out
    hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db2.q.out
    hive/trunk/ql/src/test/results/clientnegative/show_tablestatus.q.out
    hive/trunk/ql/src/test/results/clientnegative/show_tablestatus_not_existing_part.q.out
    hive/trunk/ql/src/test/results/clientpositive/desc_non_existent_tbl.q.out
    hive/trunk/ql/src/test/results/clientpositive/inputddl6.q.out
    hive/trunk/ql/src/test/results/clientpositive/show_tblproperties.q.out

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=1368352&r1=1368351&r2=1368352&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 Thu Aug  2 05:09:02 2012
@@ -1395,6 +1395,39 @@ public class DDLSemanticAnalyzer extends
     return (FetchTask) TaskFactory.get(fetch, conf);
   }
 
+  private void validateDatabase(String databaseName) throws SemanticException {
+    try {
+      if (!db.databaseExists(databaseName)) {
+        throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(databaseName));
+      }
+    } catch (HiveException e) {
+      throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(databaseName), e);
+    }
+  }
+
+  private void validateTable(String tableName, Map<String, String> partSpec)
+      throws SemanticException {
+    Table tab = null;
+    try {
+      tab = db.getTable(tableName);
+    } catch (HiveException e) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName), e);
+    }
+
+    if (partSpec != null) {
+      Partition part = null;
+      try {
+        part = db.getPartition(tab, partSpec, false);
+      } catch (HiveException e) {
+        throw new SemanticException(ErrorMsg.INVALID_PARTITION.getMsg(partSpec.toString()), e);
+      }
+
+      if (part == null) {
+        throw new SemanticException(ErrorMsg.INVALID_PARTITION.getMsg(partSpec.toString()));
+      }
+    }
+  }
+
   private void analyzeDescribeTable(ASTNode ast) throws SemanticException {
     ASTNode tableTypeExpr = (ASTNode) ast.getChild(0);
     String tableName = getFullyQualifiedName((ASTNode) tableTypeExpr
@@ -1407,6 +1440,11 @@ public class DDLSemanticAnalyzer extends
       partSpec = getPartSpec(partspec);
     }
 
+    // Handle xpath correctly
+    String actualTableName = tableName.substring(0,
+        tableName.indexOf('.') == -1 ? tableName.length() : tableName.indexOf('.'));
+    validateTable(actualTableName, partSpec);
+
     DescTableDesc descTblDesc = new DescTableDesc(ctx.getResFile(), tableName, partSpec);
     if (ast.getChildCount() == 2) {
       int descOptions = ast.getChild(1).getType();
@@ -1466,6 +1504,9 @@ public class DDLSemanticAnalyzer extends
     if(partSpecs.size() > 0) {
       partSpec = partSpecs.get(0);
     }
+
+    validateTable(tableName, null);
+
     showPartsDesc = new ShowPartitionsDesc(tableName, ctx.getResFile(), partSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         showPartsDesc), conf));
@@ -1501,12 +1542,14 @@ public class DDLSemanticAnalyzer extends
     case 2: // Specifies a DB
       assert(ast.getChild(0).getType() == HiveParser.TOK_FROM);
       dbName = unescapeIdentifier(ast.getChild(1).getText());
+      validateDatabase(dbName);
       showTblsDesc = new ShowTablesDesc(ctx.getResFile(), dbName);
       break;
     case 3: // Uses a pattern and specifies a DB
       assert(ast.getChild(0).getType() == HiveParser.TOK_FROM);
       dbName = unescapeIdentifier(ast.getChild(1).getText());
       tableNames = unescapeSQLString(ast.getChild(2).getText());
+      validateDatabase(dbName);
       showTblsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, tableNames);
       break;
     default: // No pattern or DB
@@ -1575,6 +1618,11 @@ public class DDLSemanticAnalyzer extends
         }
       }
     }
+
+    if (partSpec != null) {
+      validateTable(tableNames, partSpec);
+    }
+
     showTblStatusDesc = new ShowTableStatusDesc(ctx.getResFile().toString(), dbName,
         tableNames, partSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
@@ -1590,6 +1638,9 @@ public class DDLSemanticAnalyzer extends
     if (ast.getChildCount() > 1) {
       propertyName = unescapeSQLString(ast.getChild(1).getText());
     }
+
+    validateTable(tableNames, null);
+
     showTblPropertiesDesc = new ShowTblPropertiesDesc(ctx.getResFile().toString(), tableNames,
         propertyName);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),

Added: hive/trunk/ql/src/test/queries/clientnegative/desc_failure1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/desc_failure1.q?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/desc_failure1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/desc_failure1.q Thu Aug  2 05:09:02 2012
@@ -0,0 +1 @@
+DESC NonExistentTable;

Added: hive/trunk/ql/src/test/queries/clientnegative/desc_failure2.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/desc_failure2.q?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/desc_failure2.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/desc_failure2.q Thu Aug  2 05:09:02 2012
@@ -0,0 +1,2 @@
+DESC srcpart;
+DESC srcpart PARTITION(ds='2012-04-08', hr='15');

Added: hive/trunk/ql/src/test/queries/clientnegative/show_partitions1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/show_partitions1.q?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/show_partitions1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/show_partitions1.q Thu Aug  2 05:09:02 2012
@@ -0,0 +1 @@
+SHOW PARTITIONS NonExistentTable;

Added: hive/trunk/ql/src/test/queries/clientnegative/show_tableproperties1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/show_tableproperties1.q?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/show_tableproperties1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/show_tableproperties1.q Thu Aug  2 05:09:02 2012
@@ -0,0 +1 @@
+SHOW TBLPROPERTIES NonExistentTable; 

Modified: hive/trunk/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q Thu Aug  2 05:09:02 2012
@@ -1 +0,0 @@
-desc nonexistenttbl;

Modified: hive/trunk/ql/src/test/queries/clientpositive/inputddl6.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/inputddl6.q?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/inputddl6.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/inputddl6.q Thu Aug  2 05:09:02 2012
@@ -9,7 +9,7 @@ DESCRIBE EXTENDED INPUTDDL6 PARTITION (d
 SHOW PARTITIONS INPUTDDL6;
 ALTER TABLE INPUTDDL6 DROP PARTITION (ds='2008-04-08');
 SHOW PARTITIONS INPUTDDL6;
-DROP TABLE INPUTDDL6;
 EXPLAIN
 DESCRIBE EXTENDED INPUTDDL6 PARTITION (ds='2008-04-09');
+DROP TABLE INPUTDDL6;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/show_tblproperties.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/show_tblproperties.q?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/show_tblproperties.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/show_tblproperties.q Thu Aug  2 05:09:02 2012
@@ -1,4 +1,3 @@
-show tblproperties tmpfoo;
 
 create table tmpfoo (a String);
 show tblproperties tmpfoo("bar");

Added: hive/trunk/ql/src/test/results/clientnegative/desc_failure1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/desc_failure1.q.out?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/desc_failure1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/desc_failure1.q.out Thu Aug  2 05:09:02 2012
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10001]: Table not found NonExistentTable

Added: hive/trunk/ql/src/test/results/clientnegative/desc_failure2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/desc_failure2.q.out?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/desc_failure2.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/desc_failure2.q.out Thu Aug  2 05:09:02 2012
@@ -0,0 +1,9 @@
+PREHOOK: query: DESC srcpart
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: DESC srcpart
+POSTHOOK: type: DESCTABLE
+key	string	default
+value	string	default
+ds	string	
+hr	string	
+FAILED: SemanticException [Error 10006]: Partition not found {ds=2012-04-08, hr=15}

Added: hive/trunk/ql/src/test/results/clientnegative/show_partitions1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/show_partitions1.q.out?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/show_partitions1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/show_partitions1.q.out Thu Aug  2 05:09:02 2012
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10001]: Table not found NonExistentTable

Added: hive/trunk/ql/src/test/results/clientnegative/show_tableproperties1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/show_tableproperties1.q.out?rev=1368352&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/show_tableproperties1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/show_tableproperties1.q.out Thu Aug  2 05:09:02 2012
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10001]: Table not found NonExistentTable

Modified: hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db1.q.out?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db1.q.out Thu Aug  2 05:09:02 2012
@@ -1,4 +1 @@
-PREHOOK: query: SHOW TABLES FROM nonexistent
-PREHOOK: type: SHOWTABLES
-FAILED: Error in metadata: ERROR: The database nonexistent does not exist.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask
+FAILED: SemanticException [Error 10072]: Database does not exist: nonexistent

Modified: hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db2.q.out?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/show_tables_bad_db2.q.out Thu Aug  2 05:09:02 2012
@@ -1,4 +1 @@
-PREHOOK: query: SHOW TABLES FROM nonexistent LIKE 'test'
-PREHOOK: type: SHOWTABLES
-FAILED: Error in metadata: ERROR: The database nonexistent does not exist.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask
+FAILED: SemanticException [Error 10072]: Database does not exist: nonexistent

Modified: hive/trunk/ql/src/test/results/clientnegative/show_tablestatus.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/show_tablestatus.q.out?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/show_tablestatus.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/show_tablestatus.q.out Thu Aug  2 05:09:02 2012
@@ -1,4 +1 @@
-PREHOOK: query: SHOW TABLE EXTENDED LIKE `srcpar*` PARTITION(ds='2008-04-08', hr=11)
-PREHOOK: type: SHOW_TABLESTATUS
-Table srcpar* does not exist
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask
+FAILED: SemanticException [Error 10001]: Table not found srcpar*

Modified: hive/trunk/ql/src/test/results/clientnegative/show_tablestatus_not_existing_part.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/show_tablestatus_not_existing_part.q.out?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/show_tablestatus_not_existing_part.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/show_tablestatus_not_existing_part.q.out Thu Aug  2 05:09:02 2012
@@ -1,4 +1 @@
-PREHOOK: query: SHOW TABLE EXTENDED LIKE `srcpart` PARTITION(ds='2008-14-08', hr=11)
-PREHOOK: type: SHOW_TABLESTATUS
-FAILED: Error in metadata: Partition {ds=2008-14-08, hr=11} for table srcpart does not exist.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask
+FAILED: SemanticException [Error 10006]: Partition not found {ds=2008-14-08, hr=11}

Modified: hive/trunk/ql/src/test/results/clientpositive/desc_non_existent_tbl.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/desc_non_existent_tbl.q.out?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/desc_non_existent_tbl.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/desc_non_existent_tbl.q.out Thu Aug  2 05:09:02 2012
@@ -1,5 +0,0 @@
-PREHOOK: query: desc nonexistenttbl
-PREHOOK: type: DESCTABLE
-POSTHOOK: query: desc nonexistenttbl
-POSTHOOK: type: DESCTABLE
-Table nonexistenttbl does not exist	 	 

Modified: hive/trunk/ql/src/test/results/clientpositive/inputddl6.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/inputddl6.q.out?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/inputddl6.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/inputddl6.q.out Thu Aug  2 05:09:02 2012
@@ -60,14 +60,6 @@ PREHOOK: type: SHOWPARTITIONS
 POSTHOOK: query: SHOW PARTITIONS INPUTDDL6
 POSTHOOK: type: SHOWPARTITIONS
 ds=2008-04-09
-PREHOOK: query: DROP TABLE INPUTDDL6
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@inputddl6
-PREHOOK: Output: default@inputddl6
-POSTHOOK: query: DROP TABLE INPUTDDL6
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@inputddl6
-POSTHOOK: Output: default@inputddl6
 PREHOOK: query: EXPLAIN
 DESCRIBE EXTENDED INPUTDDL6 PARTITION (ds='2008-04-09')
 PREHOOK: type: DESCTABLE
@@ -94,3 +86,11 @@ STAGE PLANS:
       limit: -1
 
 
+PREHOOK: query: DROP TABLE INPUTDDL6
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@inputddl6
+PREHOOK: Output: default@inputddl6
+POSTHOOK: query: DROP TABLE INPUTDDL6
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@inputddl6
+POSTHOOK: Output: default@inputddl6

Modified: hive/trunk/ql/src/test/results/clientpositive/show_tblproperties.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/show_tblproperties.q.out?rev=1368352&r1=1368351&r2=1368352&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/show_tblproperties.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/show_tblproperties.q.out Thu Aug  2 05:09:02 2012
@@ -1,8 +1,3 @@
-PREHOOK: query: show tblproperties tmpfoo
-PREHOOK: type: SHOW_TBLPROPERTIES
-POSTHOOK: query: show tblproperties tmpfoo
-POSTHOOK: type: SHOW_TBLPROPERTIES
-Table tmpfoo does not exist	 
 PREHOOK: query: create table tmpfoo (a String)
 PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table tmpfoo (a String)