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 2010/10/06 18:37:51 UTC

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

Author: namit
Date: Wed Oct  6 16:37:51 2010
New Revision: 1005114

URL: http://svn.apache.org/viewvc?rev=1005114&view=rev
Log:
HIVE-1691 Validate partition spec in analyze
(Ning Zhang via namit)


Added:
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze1.q
    hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze1.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/test/results/clientnegative/dyn_part1.q.out

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=1005114&r1=1005113&r2=1005114&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Wed Oct  6 16:37:51 2010
@@ -321,6 +321,9 @@ Trunk -  Unreleased
     HIVE-1678 Bug in mapjoin followed by another mapjoin
     (Amareshwari Sriramadasu via namit)
 
+    HIVE-1691 Validate partition spec in analyze
+    (Ning Zhang via namit)
+
   TESTS
 
     HIVE-1464. improve  test query performance

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1005114&r1=1005113&r2=1005114&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Wed Oct  6 16:37:51 2010
@@ -1549,4 +1549,19 @@ public final class Utilities {
       Constants.LIST_COLUMNS,
       columnNamesString);
   }
+
+  public static void validatePartSpec(Table tbl, Map<String, String> partSpec)
+      throws SemanticException {
+
+    List<FieldSchema> parts = tbl.getPartitionKeys();
+    Set<String> partCols = new HashSet<String>(parts.size());
+    for (FieldSchema col: parts) {
+      partCols.add(col.getName());
+    }
+    for (String col: partSpec.keySet()) {
+      if (!partCols.contains(col)) {
+        throw new SemanticException(ErrorMsg.NONEXISTPARTCOL.getMsg(col));
+      }
+    }
+  }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java?rev=1005114&r1=1005113&r2=1005114&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java Wed Oct  6 16:37:51 2010
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.ql.Context;
 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;
@@ -577,6 +578,7 @@ public abstract class BaseSemanticAnalyz
         throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg(ast
             .getChild(childIndex), e.getMessage()), e);
       }
+
       // get partition metadata if partition specified
       if (ast.getChildCount() == 2) {
         childIndex = 1;
@@ -586,13 +588,17 @@ public abstract class BaseSemanticAnalyz
         for (int i = 0; i < partspec.getChildCount(); ++i) {
           ASTNode partspec_val = (ASTNode) partspec.getChild(i);
           String val = null;
+          String colName = unescapeIdentifier(partspec_val.getChild(0).getText().toLowerCase());
           if (partspec_val.getChildCount() < 2) { // DP in the form of T partition (ds, hr)
             ++numDynParts;
           } else { // in the form of T partition (ds="2010-03-03")
             val = stripQuotes(partspec_val.getChild(1).getText());
           }
-          partSpec.put(unescapeIdentifier(partspec_val.getChild(0).getText().toLowerCase()), val);
+          partSpec.put(colName, val);
         }
+
+        Utilities.validatePartSpec(tableHandle, partSpec);
+
         // check if the partition spec is valid
         if (numDynParts > 0) {
           List<FieldSchema> parts = tableHandle.getPartitionKeys();
@@ -671,5 +677,4 @@ public abstract class BaseSemanticAnalyz
     }
     return partSpec;
   }
-
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java?rev=1005114&r1=1005113&r2=1005114&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java Wed Oct  6 16:37:51 2010
@@ -145,7 +145,7 @@ public enum ErrorMsg {
   DYNAMIC_PARTITION_MERGE("Dynamic partition does not support merging mapfiles/mapredfiles yet."
       + "Please set hive.merge.mapfiles and hive.merge.mapredfiles to false or use static "
       +	"partitions"),
-  NONEXISTPARTCOL("Partition column in the partition specification does not exist"),
+  NONEXISTPARTCOL("Non-Partition column appears in the partition specification: "),
   UNSUPPORTED_TYPE("DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use "
       + "STRING instead."),
   CREATE_NON_NATIVE_AS("CREATE TABLE AS SELECT cannot be used for a non-native table"),
@@ -323,15 +323,15 @@ public enum ErrorMsg {
     sb.append(getCharPositionInLine(tree));
   }
 
-  String getMsg(Tree tree) {
+  public String getMsg(Tree tree) {
     return getMsg((ASTNode) tree);
   }
 
-  String getMsg(ASTNode tree, String reason) {
+  public String getMsg(ASTNode tree, String reason) {
     return getMsg(tree) + ": " + reason;
   }
 
-  String getMsg(Tree tree, String reason) {
+  public String getMsg(Tree tree, String reason) {
     return getMsg((ASTNode) tree, reason);
   }
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1005114&r1=1005113&r2=1005114&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Wed Oct  6 16:37:51 2010
@@ -3379,7 +3379,7 @@ public class SemanticAnalyzer extends Ba
         }
         dpCtx = qbm.getDPCtx(dest);
         if (dpCtx == null) {
-          validatePartSpec(dest_tab, partSpec);
+          // validatePartSpec(dest_tab, partSpec);
           dpCtx = new DynamicPartitionCtx(dest_tab, partSpec,
               conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME),
               conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTSPERNODE));
@@ -3678,19 +3678,6 @@ public class SemanticAnalyzer extends Ba
     return output;
   }
 
-  private void validatePartSpec(Table tbl, Map<String, String> partSpec)
-      throws SemanticException {
-    List<FieldSchema> parts = tbl.getPartitionKeys();
-    Set<String> partCols = new HashSet<String>(parts.size());
-    for (FieldSchema col: parts) {
-      partCols.add(col.getName());
-    }
-    for (String col: partSpec.keySet()) {
-      if (!partCols.contains(col)) {
-        throw new SemanticException(ErrorMsg.NONEXISTPARTCOL.getMsg());
-      }
-    }
-  }
 
   /**
    * Generate the conversion SelectOperator that converts the columns into the

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze.q?rev=1005114&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze.q Wed Oct  6 16:37:51 2010
@@ -0,0 +1 @@
+analyze table srcpart compute statistics;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze1.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze1.q?rev=1005114&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze1.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/analyze1.q Wed Oct  6 16:37:51 2010
@@ -0,0 +1 @@
+analyze table srcpart partition (key) compute statistics;

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze.q.out?rev=1005114&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze.q.out Wed Oct  6 16:37:51 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: Table is partitioned and partition specification is needed

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze1.q.out?rev=1005114&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze1.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/analyze1.q.out Wed Oct  6 16:37:51 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: Non-Partition column appears in the partition specification:  key

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/dyn_part1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/dyn_part1.q.out?rev=1005114&r1=1005113&r2=1005114&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/dyn_part1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/dyn_part1.q.out Wed Oct  6 16:37:51 2010
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table dynamic_partition (key string) partitioned by (value string)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@dynamic_partition
-FAILED: Error in semantic analysis: Partition column in the partition specification does not exist
+FAILED: Error in semantic analysis: Non-Partition column appears in the partition specification:  hr