You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jv...@apache.org on 2011/05/17 03:29:17 UTC

svn commit: r1103980 [1/2] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/test/queries/clientnegative/ ql/src/test/results/clientnegative/ ql/src/test/results/compiler/errors/

Author: jvs
Date: Tue May 17 01:29:14 2011
New Revision: 1103980

URL: http://svn.apache.org/viewvc?rev=1103980&view=rev
Log:
HIVE-2155. Improve error messages emitted during semantic analysis
(Syed Albiz via jvs)


Added:
    hive/trunk/ql/src/test/queries/clientnegative/union3.q
    hive/trunk/ql/src/test/results/clientnegative/union3.q.out
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.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/ErrorMsg.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/TypeCheckProcFactory.java
    hive/trunk/ql/src/test/results/clientnegative/analyze_view.q.out
    hive/trunk/ql/src/test/results/clientnegative/clusterbydistributeby.q.out
    hive/trunk/ql/src/test/results/clientnegative/clusterbysortby.q.out
    hive/trunk/ql/src/test/results/clientnegative/clustern3.q.out
    hive/trunk/ql/src/test/results/clientnegative/clustern4.q.out
    hive/trunk/ql/src/test/results/clientnegative/create_view_failure3.q.out
    hive/trunk/ql/src/test/results/clientnegative/ctas.q.out
    hive/trunk/ql/src/test/results/clientnegative/ddltime.q.out
    hive/trunk/ql/src/test/results/clientnegative/drop_partition_failure.q.out
    hive/trunk/ql/src/test/results/clientnegative/dyn_part2.q.out
    hive/trunk/ql/src/test/results/clientnegative/dyn_part_merge.q.out
    hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out
    hive/trunk/ql/src/test/results/clientnegative/groupby_key.q.out
    hive/trunk/ql/src/test/results/clientnegative/input1.q.out
    hive/trunk/ql/src/test/results/clientnegative/input2.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter1.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter2.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalid_t_create1.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalid_t_create2.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalid_t_create3.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalid_t_transform.q.out
    hive/trunk/ql/src/test/results/clientnegative/invalidate_view1.q.out
    hive/trunk/ql/src/test/results/clientnegative/joinneg.q.out
    hive/trunk/ql/src/test/results/clientnegative/line_terminator.q.out
    hive/trunk/ql/src/test/results/clientnegative/load_part_nospec.q.out
    hive/trunk/ql/src/test/results/clientnegative/load_wrong_noof_part.q.out
    hive/trunk/ql/src/test/results/clientnegative/nopart_insert.q.out
    hive/trunk/ql/src/test/results/clientnegative/nopart_load.q.out
    hive/trunk/ql/src/test/results/clientnegative/notable_alias3.q.out
    hive/trunk/ql/src/test/results/clientnegative/orderbysortby.q.out
    hive/trunk/ql/src/test/results/clientnegative/regex_col_1.q.out
    hive/trunk/ql/src/test/results/clientnegative/regex_col_2.q.out
    hive/trunk/ql/src/test/results/clientnegative/regex_col_groupby.q.out
    hive/trunk/ql/src/test/results/clientnegative/semijoin1.q.out
    hive/trunk/ql/src/test/results/clientnegative/semijoin2.q.out
    hive/trunk/ql/src/test/results/clientnegative/semijoin3.q.out
    hive/trunk/ql/src/test/results/clientnegative/semijoin4.q.out
    hive/trunk/ql/src/test/results/clientnegative/split_sample_out_of_range.q.out
    hive/trunk/ql/src/test/results/clientnegative/split_sample_wrong_format.q.out
    hive/trunk/ql/src/test/results/clientnegative/strict_orderby.q.out
    hive/trunk/ql/src/test/results/clientnegative/subq_insert.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong1.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong2.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong2.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong3.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_coalesce.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_args_len.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_type.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_args_len.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_type.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_if_not_bool.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_if_wrong_args_len.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_in.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_args_len.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_type.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_args_len.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_type.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_args_len.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_type.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong2.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong3.q.out
    hive/trunk/ql/src/test/results/clientnegative/udtf_not_supported1.q.out
    hive/trunk/ql/src/test/results/clientnegative/udtf_not_supported2.q.out
    hive/trunk/ql/src/test/results/clientnegative/union.q.out
    hive/trunk/ql/src/test/results/clientnegative/union2.q.out
    hive/trunk/ql/src/test/results/compiler/errors/ambiguous_table_col.q.out
    hive/trunk/ql/src/test/results/compiler/errors/duplicate_alias.q.out
    hive/trunk/ql/src/test/results/compiler/errors/insert_wrong_number_columns.q.out
    hive/trunk/ql/src/test/results/compiler/errors/invalid_dot.q.out
    hive/trunk/ql/src/test/results/compiler/errors/invalid_function_param2.q.out
    hive/trunk/ql/src/test/results/compiler/errors/invalid_index.q.out
    hive/trunk/ql/src/test/results/compiler/errors/invalid_list_index.q.out
    hive/trunk/ql/src/test/results/compiler/errors/invalid_list_index2.q.out
    hive/trunk/ql/src/test/results/compiler/errors/invalid_map_index.q.out
    hive/trunk/ql/src/test/results/compiler/errors/invalid_map_index2.q.out
    hive/trunk/ql/src/test/results/compiler/errors/nonkey_groupby.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_column1.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_column2.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_column3.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_column4.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_column5.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_column6.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_function1.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_function2.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_function3.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_function4.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_table1.q.out
    hive/trunk/ql/src/test/results/compiler/errors/unknown_table2.q.out
    hive/trunk/ql/src/test/results/compiler/errors/wrong_distinct1.q.out

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue May 17 01:29:14 2011
@@ -752,4 +752,13 @@ public class HiveConf extends Configurat
     return "_col" + pos;
   }
 
+  public static int getPositionFromInternalName(String internalName) {
+    char pos = internalName.charAt(internalName.length()-1);
+    if (Character.isDigit(pos)) {
+      return Character.digit(pos, 10);
+    } else{
+      return -1;
+    }
+  }
+
 }

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=1103980&r1=1103979&r2=1103980&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 May 17 01:29:14 2011
@@ -136,8 +136,8 @@ public abstract class BaseSemanticAnalyz
               .getText());
           if (!lineDelim.equals("\n")
               && !lineDelim.equals("10")) {
-            throw new SemanticException(
-                ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg());
+            throw new SemanticException(SemanticAnalyzer.generateErrorMessage(rowChild,
+                ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg()));
           }
           break;
         default:

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java Tue May 17 01:29:14 2011
@@ -105,19 +105,19 @@ public enum ErrorMsg {
   BUCKETED_NUMBERATOR_BIGGER_DENOMINATOR("Numberator should not be bigger than "
       + "denaminator in sample clause for table"),
   NEED_PARTITION_ERROR("Need to specify partition columns because the destination "
-      + "table is partitioned."),
+      + "table is partitioned"),
   CTAS_CTLT_COEXISTENCE("Create table command does not allow LIKE and AS-SELECT in "
       + "the same command"),
   LINES_TERMINATED_BY_NON_NEWLINE("LINES TERMINATED BY only supports newline '\\n' right now"),
   CTAS_COLLST_COEXISTENCE("CREATE TABLE AS SELECT command cannot specify the list of columns "
-      + "for the target table."),
+      + "for the target table"),
   CTLT_COLLST_COEXISTENCE("CREATE TABLE LIKE command cannot specify the list of columns for "
-      + "the target table."),
-  INVALID_SELECT_SCHEMA("Cannot derive schema from the select-clause."),
+      + "the target table"),
+  INVALID_SELECT_SCHEMA("Cannot derive schema from the select-clause"),
   CTAS_PARCOL_COEXISTENCE("CREATE-TABLE-AS-SELECT does not support partitioning in the target "
-      + "table."),
-  CTAS_MULTI_LOADFILE("CREATE-TABLE-AS-SELECT results in multiple file load."),
-  CTAS_EXTTBL_COEXISTENCE("CREATE-TABLE-AS-SELECT cannot create external table."),
+      + "table"),
+  CTAS_MULTI_LOADFILE("CREATE-TABLE-AS-SELECT results in multiple file load"),
+  CTAS_EXTTBL_COEXISTENCE("CREATE-TABLE-AS-SELECT cannot create external table"),
   TABLE_ALREADY_EXISTS("Table already exists:", "42S02"),
   COLUMN_ALIAS_ALREADY_EXISTS("Column alias already exists:", "42S02"),
   UDTF_MULTIPLE_EXPR("Only a single expression in the SELECT clause is supported with UDTF's"),
@@ -147,12 +147,12 @@ public enum ErrorMsg {
       + "hive.exec.dynamic.partition=true or specify partition column values"),
   DYNAMIC_PARTITION_STRICT_MODE("Dynamic partition strict mode requires at least one "
       + "static partition column. To turn this off set hive.exec.dynamic.partition.mode=nonstrict"),
-  DYNAMIC_PARTITION_MERGE("Dynamic partition does not support merging using non-CombineHiveInputFormat."
+  DYNAMIC_PARTITION_MERGE("Dynamic partition does not support merging using non-CombineHiveInputFormat"
       + "Please check your hive.input.format setting and make sure your Hadoop version support "
-      + "CombineFileInputFormat."),
+      + "CombineFileInputFormat"),
   NONEXISTPARTCOL("Non-Partition column appears in the partition specification: "),
   UNSUPPORTED_TYPE("DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use "
-      + "STRING instead."),
+      + "STRING instead"),
   CREATE_NON_NATIVE_AS("CREATE TABLE AS SELECT cannot be used for a non-native table"),
   LOAD_INTO_NON_NATIVE("A non-native table cannot be used as target for LOAD"),
   LOCKMGR_NOT_SPECIFIED("Lock manager not specified correctly, set hive.lock.manager"),
@@ -160,9 +160,9 @@ public enum ErrorMsg {
   LOCK_CANNOT_BE_ACQUIRED("Locks on the underlying objects cannot be acquired. retry after some time"),
   ZOOKEEPER_CLIENT_COULD_NOT_BE_INITIALIZED("Check hive.zookeeper.quorum and hive.zookeeper.client.port"),
   OVERWRITE_ARCHIVED_PART("Cannot overwrite an archived partition. " +
-      "Unarchive before running this command."),
+      "Unarchive before running this command"),
   ARCHIVE_METHODS_DISABLED("Archiving methods are currently disabled. " +
-      "Please see the Hive wiki for more information about enabling archiving."),
+      "Please see the Hive wiki for more information about enabling archiving"),
   ARCHIVE_ON_MULI_PARTS("ARCHIVE can only be run on a single partition"),
   UNARCHIVE_ON_MULI_PARTS("ARCHIVE can only be run on a single partition"),
   ARCHIVE_ON_TABLE("ARCHIVE can only be run on partitions"),
@@ -289,11 +289,10 @@ public enum ErrorMsg {
 
   // Dirty hack as this will throw away spaces and other things - find a better
   // way!
-  private String getText(ASTNode tree) {
+  public static String getText(ASTNode tree) {
     if (tree.getChildCount() == 0) {
       return tree.getText();
     }
-
     return getText((ASTNode) tree.getChild(tree.getChildCount() - 1));
   }
 
@@ -302,8 +301,9 @@ public enum ErrorMsg {
     renderPosition(sb, tree);
     sb.append(" ");
     sb.append(mesg);
-    sb.append(" ");
+    sb.append(" '");
     sb.append(getText(tree));
+    sb.append("'");
     renderOrigin(sb, tree.getOrigin());
     return sb.toString();
   }

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=1103980&r1=1103979&r2=1103980&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 May 17 01:29:14 2011
@@ -421,6 +421,19 @@ public class SemanticAnalyzer extends Ba
     return exprs;
   }
 
+  public static String generateErrorMessage(ASTNode ast, String message) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(ast.getLine());
+    sb.append(":");
+    sb.append(ast.getCharPositionInLine());
+    sb.append(" ");
+    sb.append(message);
+    sb.append(". Error encountered near token '");
+    sb.append(ErrorMsg.getText(ast));
+    sb.append("'");
+    return sb.toString();
+  }
+
   /**
    * Goes though the tabref tree and finds the alias for the table. Once found,
    * it records the table name-> alias association in aliasToTabs. It also makes
@@ -481,8 +494,9 @@ public class SemanticAnalyzer extends Ba
       // TODO: For now only support sampling on up to two columns
       // Need to change it to list of columns
       if (sampleCols.size() > 2) {
-        throw new SemanticException(ErrorMsg.SAMPLE_RESTRICTION.getMsg(tabref
-            .getChild(0)));
+        throw new SemanticException(generateErrorMessage(
+              (ASTNode) tabref.getChild(0),
+              ErrorMsg.SAMPLE_RESTRICTION.getMsg()));
       }
       qb.getParseInfo().setTabSample(
           alias,
@@ -501,15 +515,16 @@ public class SemanticAnalyzer extends Ba
       String inputFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEINPUTFORMAT);
       if (!inputFormat.equals(
         CombineHiveInputFormat.class.getName())) {
-        throw new SemanticException(
-            "Percentage sampling is not supported in " + inputFormat);
+        throw new SemanticException(generateErrorMessage((ASTNode) tabref.getChild(1),
+            "Percentage sampling is not supported in " + inputFormat));
       }
       ASTNode sampleClause = (ASTNode) tabref.getChild(1);
       String alias_id = getAliasId(alias, qb);
       String strPercentage = unescapeIdentifier(sampleClause.getChild(0).getText());
       Double percent = Double.valueOf(strPercentage).doubleValue();
       if (percent < 0  || percent > 100) {
-        throw new SemanticException("Sampling percentage should be between 0 and 100.");
+        throw new SemanticException(generateErrorMessage(sampleClause,
+                    "Sampling percentage should be between 0 and 100"));
       }
       nameToSplitSample.put(alias_id, new SplitSample(
           percent, conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM)));
@@ -583,7 +598,8 @@ public class SemanticAnalyzer extends Ba
     int numChildren = join.getChildCount();
     if ((numChildren != 2) && (numChildren != 3)
         && join.getToken().getType() != HiveParser.TOK_UNIQUEJOIN) {
-      throw new SemanticException("Join with multiple children");
+      throw new SemanticException(generateErrorMessage(join,
+                  "Join with multiple children"));
     }
 
     for (int num = 0; num < numChildren; num++) {
@@ -713,7 +729,8 @@ public class SemanticAnalyzer extends Ba
       case HiveParser.TOK_FROM:
         int child_count = ast.getChildCount();
         if (child_count != 1) {
-          throw new SemanticException("Multiple Children " + child_count);
+          throw new SemanticException(generateErrorMessage(ast,
+                      "Multiple Children " + child_count));
         }
 
         // Check if this is a subquery / lateral view
@@ -742,11 +759,11 @@ public class SemanticAnalyzer extends Ba
         // select list
         qbp.setDistributeByExprForClause(ctx_1.dest, ast);
         if (qbp.getClusterByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(ErrorMsg.CLUSTERBY_DISTRIBUTEBY_CONFLICT
-              .getMsg(ast));
+          throw new SemanticException(generateErrorMessage(ast,
+                ErrorMsg.CLUSTERBY_DISTRIBUTEBY_CONFLICT.getMsg()));
         } else if (qbp.getOrderByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(ErrorMsg.ORDERBY_DISTRIBUTEBY_CONFLICT
-              .getMsg(ast));
+          throw new SemanticException(generateErrorMessage(ast,
+                ErrorMsg.ORDERBY_DISTRIBUTEBY_CONFLICT.getMsg()));
         }
         break;
 
@@ -755,11 +772,11 @@ public class SemanticAnalyzer extends Ba
         // select list
         qbp.setSortByExprForClause(ctx_1.dest, ast);
         if (qbp.getClusterByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(ErrorMsg.CLUSTERBY_SORTBY_CONFLICT
-              .getMsg(ast));
+          throw new SemanticException(generateErrorMessage(ast,
+                ErrorMsg.CLUSTERBY_SORTBY_CONFLICT.getMsg()));
         } else if (qbp.getOrderByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(ErrorMsg.ORDERBY_SORTBY_CONFLICT
-              .getMsg(ast));
+          throw new SemanticException(generateErrorMessage(ast,
+                ErrorMsg.ORDERBY_SORTBY_CONFLICT.getMsg()));
         }
 
         break;
@@ -769,8 +786,8 @@ public class SemanticAnalyzer extends Ba
         // select list
         qbp.setOrderByExprForClause(ctx_1.dest, ast);
         if (qbp.getClusterByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(ErrorMsg.CLUSTERBY_ORDERBY_CONFLICT
-              .getMsg(ast));
+          throw new SemanticException(generateErrorMessage(ast,
+                ErrorMsg.CLUSTERBY_ORDERBY_CONFLICT.getMsg()));
         }
         break;
 
@@ -778,8 +795,8 @@ public class SemanticAnalyzer extends Ba
         // Get the groupby aliases - these are aliased to the entries in the
         // select list
         if (qbp.getSelForClause(ctx_1.dest).getToken().getType() == HiveParser.TOK_SELECTDI) {
-          throw new SemanticException(ErrorMsg.SELECT_DISTINCT_WITH_GROUPBY
-              .getMsg(ast));
+          throw new SemanticException(generateErrorMessage(ast,
+                ErrorMsg.SELECT_DISTINCT_WITH_GROUPBY.getMsg()));
         }
         qbp.setGroupByExprForClause(ctx_1.dest, ast);
         skipRecursion = true;
@@ -811,7 +828,8 @@ public class SemanticAnalyzer extends Ba
         // explicitly say:
         // select * from (subq1 union subq2) subqalias
         if (!qbp.getIsSubQ()) {
-          throw new SemanticException(ErrorMsg.UNION_NOTIN_SUBQ.getMsg());
+          throw new SemanticException(generateErrorMessage(ast,
+                                        ErrorMsg.UNION_NOTIN_SUBQ.getMsg()));
         }
 
       default:
@@ -886,8 +904,9 @@ public class SemanticAnalyzer extends Ba
         }
 
         if (!InputFormat.class.isAssignableFrom(tab.getInputFormatClass())) {
-          throw new SemanticException(ErrorMsg.INVALID_INPUT_FORMAT_TYPE
-              .getMsg(qb.getParseInfo().getSrcForAlias(alias)));
+          throw new SemanticException(generateErrorMessage(
+                qb.getParseInfo().getSrcForAlias(alias),
+                ErrorMsg.INVALID_INPUT_FORMAT_TYPE.getMsg()));
         }
 
         qb.getMetaData().setSrcForAlias(alias, tab);
@@ -898,7 +917,8 @@ public class SemanticAnalyzer extends Ba
             try {
               ts.partitions = db.getPartitionsByNames(ts.tableHandle, ts.partSpec);
             } catch (HiveException e) {
-              throw new SemanticException("Cannot get partitions for " + ts.partSpec, e);
+              throw new SemanticException(generateErrorMessage(qb.getParseInfo().getSrcForAlias(alias),
+                          "Cannot get partitions for " + ts.partSpec), e);
             }
           }
           qb.getParseInfo().addTableSpec(alias, ts);
@@ -973,7 +993,8 @@ public class SemanticAnalyzer extends Ba
                 fname = ctx.getExternalTmpFileURI(
                     FileUtils.makeQualified(new Path(location), conf).toUri());
               } catch (Exception e) {
-                throw new SemanticException("Error creating temporary folder on: " + location, e);
+                throw new SemanticException(generateErrorMessage(ast,
+                      "Error creating temporary folder on: " + location), e);
               }
             } else {
               qb.setIsQuery(true);
@@ -986,8 +1007,8 @@ public class SemanticAnalyzer extends Ba
           break;
         }
         default:
-          throw new SemanticException("Unknown Token Type "
-              + ast.getToken().getType());
+          throw new SemanticException(generateErrorMessage(ast,
+                "Unknown Token Type " + ast.getToken().getType()));
         }
       }
     } catch (HiveException e) {
@@ -1536,6 +1557,10 @@ public class SemanticAnalyzer extends Ba
     return (end == -1) ? "" : cmd.substring(end, cmd.length());
   }
 
+  private static int getPositionFromInternalName(String internalName) {
+    return HiveConf.getPositionFromInternalName(internalName);
+  }
+
   private String fetchFilesNotInLocalFilesystem(String cmd) {
     SessionState ss = SessionState.get();
     String progName = getScriptProgName(cmd);
@@ -1615,8 +1640,8 @@ public class SemanticAnalyzer extends Ba
           String lineDelim = unescapeSQLString(rowChild.getChild(0).getText());
           tblDesc.getProperties().setProperty(Constants.LINE_DELIM, lineDelim);
           if (!lineDelim.equals("\n") && !lineDelim.equals("10")) {
-            throw new SemanticException(
-                ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg());
+            throw new SemanticException(generateErrorMessage(rowChild,
+                    ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg()));
           }
           break;
         default:
@@ -2017,12 +2042,15 @@ public class SemanticAnalyzer extends Ba
     if (isUDTF) {
       // Only support a single expression when it's a UDTF
       if (selExprList.getChildCount() > 1) {
-        throw new SemanticException(ErrorMsg.UDTF_MULTIPLE_EXPR.getMsg());
+        throw new SemanticException(generateErrorMessage(
+                    (ASTNode) selExprList.getChild(1),
+                    ErrorMsg.UDTF_MULTIPLE_EXPR.getMsg()));
       }
       // Require an AS for UDTFs for column aliases
       ASTNode selExpr = (ASTNode) selExprList.getChild(posn);
       if (selExpr.getChildCount() < 2) {
-        throw new SemanticException(ErrorMsg.UDTF_REQUIRE_AS.getMsg());
+        throw new SemanticException(generateErrorMessage(udtfExpr,
+                    ErrorMsg.UDTF_REQUIRE_AS.getMsg()));
       }
       // Get the column / table aliases from the expression. Start from 1 as
       // 0 is the TOK_FUNCTION
@@ -2082,7 +2110,9 @@ public class SemanticAnalyzer extends Ba
       // This check is not needed and invalid when there is a transform b/c the
       // AST's are slightly different.
       if (!isInTransform && !isUDTF && child.getChildCount() > 2) {
-        throw new SemanticException(ErrorMsg.INVALID_AS.getMsg());
+        throw new SemanticException(generateErrorMessage(
+                    (ASTNode) child.getChild(2),
+                    ErrorMsg.INVALID_AS.getMsg()));
       }
 
       // The real expression
@@ -3610,12 +3640,16 @@ public class SemanticAnalyzer extends Ba
       List<FieldSchema> parts = dest_tab.getPartitionKeys();
       if (parts != null && parts.size() > 0) { // table is partitioned
         if (partSpec== null || partSpec.size() == 0) { // user did NOT specify partition
-          throw new SemanticException(ErrorMsg.NEED_PARTITION_ERROR.getMsg());
+          throw new SemanticException(generateErrorMessage(
+                qb.getParseInfo().getDestForClause(dest),
+                ErrorMsg.NEED_PARTITION_ERROR.getMsg()));
         }
         // the HOLD_DDLTIIME hint should not be used with dynamic partition since the
         // newly generated partitions should always update their DDLTIME
         if (holdDDLTime) {
-          throw new SemanticException(ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg());
+          throw new SemanticException(generateErrorMessage(
+                qb.getParseInfo().getDestForClause(dest),
+                ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg()));
         }
         dpCtx = qbm.getDPCtx(dest);
         if (dpCtx == null) {
@@ -3640,7 +3674,9 @@ public class SemanticAnalyzer extends Ba
           HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVEJOBPROGRESS, true);
 
         } else { // QBMetaData.DEST_PARTITION capture the all-SP case
-          throw new SemanticException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg());
+          throw new SemanticException(generateErrorMessage(
+                qb.getParseInfo().getDestForClause(dest),
+                ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg()));
         }
         if (dpCtx.getSPPath() != null) {
           dest_path = new Path(dest_tab.getPath(), dpCtx.getSPPath());
@@ -3729,7 +3765,7 @@ public class SemanticAnalyzer extends Ba
 
       if ("har".equalsIgnoreCase(dest_path.toUri().getScheme())) {
         throw new SemanticException(ErrorMsg.OVERWRITE_ARCHIVED_PART
-            .getMsg());
+            .getMsg(qb.getParseInfo().getDestForClause(dest)));
       }
       queryTmpdir = ctx.getExternalTmpFileURI(dest_path.toUri());
       table_desc = Utilities.getTableDesc(dest_tab);
@@ -3747,7 +3783,9 @@ public class SemanticAnalyzer extends Ba
         try {
           Partition part = db.getPartition(dest_tab, dest_part.getSpec(), false);
           if (part == null) {
-            throw new SemanticException(ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg());
+            throw new SemanticException(generateErrorMessage(
+                  qb.getParseInfo().getDestForClause(dest),
+                  ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg()));
           }
         } catch (HiveException e) {
           throw new SemanticException(e);
@@ -4394,8 +4432,8 @@ public class SemanticAnalyzer extends Ba
         if (conf.getVar(HiveConf.ConfVars.HIVEMAPREDMODE).equalsIgnoreCase(
             "strict")
             && limit == null) {
-          throw new SemanticException(ErrorMsg.NO_LIMIT_WITH_ORDERBY
-              .getMsg(sortExprs));
+          throw new SemanticException(generateErrorMessage(sortExprs,
+                ErrorMsg.NO_LIMIT_WITH_ORDERBY.getMsg()));
         }
       }
     }
@@ -5852,6 +5890,8 @@ public class SemanticAnalyzer extends Ba
     HashMap<String, ColumnInfo> leftmap = leftRR.getFieldMap(leftalias);
     HashMap<String, ColumnInfo> rightmap = rightRR.getFieldMap(rightalias);
     // make sure the schemas of both sides are the same
+    ASTNode tabref = qb.getAliases().isEmpty() ? null :
+                       qb.getParseInfo().getSrcForAlias(qb.getAliases().get(0));
     if (leftmap.size() != rightmap.size()) {
       throw new SemanticException("Schema of both sides of union should match.");
     }
@@ -5860,26 +5900,30 @@ public class SemanticAnalyzer extends Ba
       ColumnInfo lInfo = lEntry.getValue();
       ColumnInfo rInfo = rightmap.get(field);
       if (rInfo == null) {
-        throw new SemanticException(
+        throw new SemanticException(generateErrorMessage(tabref,
             "Schema of both sides of union should match. " + rightalias
-            + " does not have the field " + field);
+            + " does not have the field " + field));
       }
       if (lInfo == null) {
-        throw new SemanticException(
+        throw new SemanticException(generateErrorMessage(tabref,
             "Schema of both sides of union should match. " + leftalias
-            + " does not have the field " + field);
+            + " does not have the field " + field));
       }
       if (!lInfo.getInternalName().equals(rInfo.getInternalName())) {
-        throw new SemanticException(
-            "Schema of both sides of union should match: " + field + ":"
-            + lInfo.getInternalName() + " " + rInfo.getInternalName());
+        throw new SemanticException(generateErrorMessage(tabref,
+            "Schema of both sides of union should match: field " + field + ":"
+            + " appears on the left side of the UNION at column position: " +
+            getPositionFromInternalName(lInfo.getInternalName())
+            + ", and on the right side of the UNION at column position: " +
+            getPositionFromInternalName(rInfo.getInternalName())
+            + ". Column positions should match for a UNION"));
       }
       if (!lInfo.getType().getTypeName().equals(rInfo.getType().getTypeName())) {
-        throw new SemanticException(
+        throw new SemanticException(generateErrorMessage(tabref,
             "Schema of both sides of union should match: Column " + field
             + " is of type " + lInfo.getType().getTypeName()
             + " on first table and type " + rInfo.getType().getTypeName()
-            + " on second table");
+            + " on second table"));
       }
     }
 
@@ -7135,8 +7179,9 @@ public class SemanticAnalyzer extends Ba
       int explicitColCount = imposedSchema.size();
       int derivedColCount = derivedSchema.size();
       if (explicitColCount != derivedColCount) {
-        throw new SemanticException(ErrorMsg.VIEW_COL_MISMATCH
-            .getMsg(viewSelect));
+        throw new SemanticException(generateErrorMessage(
+                    viewSelect,
+                    ErrorMsg.VIEW_COL_MISMATCH.getMsg()));
       }
     }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java Tue May 17 01:29:14 2011
@@ -384,7 +384,8 @@ public final class TypeCheckProcFactory 
         if (colInfo == null) {
           // It's not a column or a table alias.
           if (input.getIsExprResolver()) {
-            ctx.setError(ErrorMsg.NON_KEY_EXPR_IN_GROUPBY.getMsg(expr), expr);
+            stack.pop();
+            ctx.setError(ErrorMsg.NON_KEY_EXPR_IN_GROUPBY.getMsg((ASTNode) stack.peek()), expr);
             return null;
           } else {
             ctx.setError(ErrorMsg.INVALID_TABLE_OR_COLUMN.getMsg(expr
@@ -580,8 +581,9 @@ public final class TypeCheckProcFactory 
           if (!(children.get(1) instanceof ExprNodeConstantDesc)
               || !(((ExprNodeConstantDesc) children.get(1)).getTypeInfo()
               .equals(TypeInfoFactory.intTypeInfo))) {
-            throw new SemanticException(ErrorMsg.INVALID_ARRAYINDEX_CONSTANT
-                .getMsg(expr));
+            throw new SemanticException(SemanticAnalyzer.generateErrorMessage(
+                  expr,
+                  ErrorMsg.INVALID_ARRAYINDEX_CONSTANT.getMsg()));
           }
 
           // Calculate TypeInfo
@@ -591,8 +593,9 @@ public final class TypeCheckProcFactory 
         } else if (myt.getCategory() == Category.MAP) {
           // Only allow constant map key for now
           if (!(children.get(1) instanceof ExprNodeConstantDesc)) {
-            throw new SemanticException(ErrorMsg.INVALID_MAPINDEX_CONSTANT
-                .getMsg(expr));
+            throw new SemanticException(SemanticAnalyzer.generateErrorMessage(
+                  expr,
+                  ErrorMsg.INVALID_MAPINDEX_CONSTANT.getMsg()));
           }
           if (!(((ExprNodeConstantDesc) children.get(1)).getTypeInfo()
               .equals(((MapTypeInfo) myt).getMapKeyTypeInfo()))) {

Added: hive/trunk/ql/src/test/queries/clientnegative/union3.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/union3.q?rev=1103980&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/union3.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/union3.q Tue May 17 01:29:14 2011
@@ -0,0 +1,5 @@
+-- Ensure that UNION ALL columns are in the correct order on both sides
+-- Ensure that the appropriate error message is propagated
+CREATE TABLE IF NOT EXISTS union3  (bar int, baz int);
+SELECT * FROM ( SELECT f.bar, f.baz FROM union3 f UNION ALL SELECT b.baz, b.bar FROM union3 b ) c;
+DROP TABLE union3;

Modified: hive/trunk/ql/src/test/results/clientnegative/analyze_view.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/analyze_view.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/analyze_view.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/analyze_view.q.out Tue May 17 01:29:14 2011
@@ -4,9 +4,9 @@ POSTHOOK: query: DROP VIEW av
 POSTHOOK: type: DROPVIEW
 PREHOOK: query: CREATE VIEW av AS SELECT * FROM src
 PREHOOK: type: CREATEVIEW
-PREHOOK: Output: file:/var/folders/7P/7PeC14kXFIWq0PIYyexGbmKuXUk/-Tmp-/jsichi/hive_2011-02-01_17-50-22_779_44083551773069928/-mr-10000
+PREHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-10_11-05-15_170_7006434946918241536/-mr-10000
 POSTHOOK: query: CREATE VIEW av AS SELECT * FROM src
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Output: default@av
-POSTHOOK: Output: file:/var/folders/7P/7PeC14kXFIWq0PIYyexGbmKuXUk/-Tmp-/jsichi/hive_2011-02-01_17-50-22_779_44083551773069928/-mr-10000
+POSTHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-10_11-05-15_170_7006434946918241536/-mr-10000
 FAILED: Error in semantic analysis: ANALYZE is not supported for views

Modified: hive/trunk/ql/src/test/results/clientnegative/clusterbydistributeby.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/clusterbydistributeby.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/clusterbydistributeby.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/clusterbydistributeby.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@dest1
-FAILED: Error in semantic analysis: Line 8:14 Cannot have both CLUSTER BY and DISTRIBUTE BY clauses tkey
+FAILED: Error in semantic analysis: 8:14 Cannot have both CLUSTER BY and DISTRIBUTE BY clauses. Error encountered near token 'tkey'

Modified: hive/trunk/ql/src/test/results/clientnegative/clusterbysortby.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/clusterbysortby.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/clusterbysortby.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/clusterbysortby.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@dest1
-FAILED: Error in semantic analysis: Line 8:8 Cannot have both CLUSTER BY and SORT BY clauses one
+FAILED: Error in semantic analysis: 8:8 Cannot have both CLUSTER BY and SORT BY clauses. Error encountered near token 'one'

Modified: hive/trunk/ql/src/test/results/clientnegative/clustern3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/clustern3.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/clustern3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/clustern3.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:52 Invalid column reference key
+FAILED: Error in semantic analysis: Line 2:52 Invalid column reference 'key'

Modified: hive/trunk/ql/src/test/results/clientnegative/clustern4.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/clustern4.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/clustern4.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/clustern4.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:50 Invalid table alias or column reference key
+FAILED: Error in semantic analysis: Line 2:50 Invalid table alias or column reference 'key'

Modified: hive/trunk/ql/src/test/results/clientnegative/create_view_failure3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/create_view_failure3.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/create_view_failure3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/create_view_failure3.q.out Tue May 17 01:29:14 2011
@@ -2,4 +2,4 @@ PREHOOK: query: DROP VIEW xxx13
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: DROP VIEW xxx13
 POSTHOOK: type: DROPVIEW
-FAILED: Error in semantic analysis: Line 5:16 The number of columns produced by the SELECT clause does not match the number of column names specified by CREATE VIEW key
+FAILED: Error in semantic analysis: 5:16 The number of columns produced by the SELECT clause does not match the number of column names specified by CREATE VIEW. Error encountered near token 'key'

Modified: hive/trunk/ql/src/test/results/clientnegative/ctas.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ctas.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ctas.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/ctas.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: CREATE-TABLE-AS-SELECT cannot create external table.
+FAILED: Error in semantic analysis: CREATE-TABLE-AS-SELECT cannot create external table

Modified: hive/trunk/ql/src/test/results/clientnegative/ddltime.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ddltime.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ddltime.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/ddltime.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table T2 like srcpart
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@T2
-FAILED: Error in semantic analysis: org.apache.hadoop.hive.ql.parse.SemanticException: HOLD_DDLTIME hint cannot be applied to dynamic partitions or non-existent partitions
+FAILED: Error in semantic analysis: org.apache.hadoop.hive.ql.parse.SemanticException: 3:23 HOLD_DDLTIME hint cannot be applied to dynamic partitions or non-existent partitions. Error encountered near token ''1''

Modified: hive/trunk/ql/src/test/results/clientnegative/drop_partition_failure.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/drop_partition_failure.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/drop_partition_failure.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/drop_partition_failure.q.out Tue May 17 01:29:14 2011
@@ -31,4 +31,4 @@ POSTHOOK: type: SHOWPARTITIONS
 b=1/c=1
 b=1/c=2
 b=2/c=2
-FAILED: Error in semantic analysis: Line 3:31 Partition not found '3'
+FAILED: Error in semantic analysis: Line 3:31 Partition not found ''3''

Modified: hive/trunk/ql/src/test/results/clientnegative/dyn_part2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/dyn_part2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/dyn_part2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/dyn_part2.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table nzhang_part1 (key string, value string) partitioned by (ds string, hr string)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@nzhang_part1
-FAILED: Error in semantic analysis: Line 3:23 Cannot insert into target table because column number/types are different hr: Table insclause-0 has 3 columns, but query has 2 columns.
+FAILED: Error in semantic analysis: Line 3:23 Cannot insert into target table because column number/types are different 'hr': Table insclause-0 has 3 columns, but query has 2 columns.

Modified: hive/trunk/ql/src/test/results/clientnegative/dyn_part_merge.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/dyn_part_merge.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/dyn_part_merge.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/dyn_part_merge.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table dyn_merge(key string, value string) partitioned by (ds string)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@dyn_merge
-FAILED: Error in semantic analysis: Dynamic partition does not support merging using non-CombineHiveInputFormat.Please check your hive.input.format setting and make sure your Hadoop version support CombineFileInputFormat.
+FAILED: Error in semantic analysis: Dynamic partition does not support merging using non-CombineHiveInputFormatPlease check your hive.input.format setting and make sure your Hadoop version support CombineFileInputFormat

Modified: hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out Tue May 17 01:29:14 2011
@@ -19,4 +19,4 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: default@dest1
 POSTHOOK: Lineage: dest1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-FAILED: Error in semantic analysis: Line 3:20 Input format must implement InputFormat dest1
+FAILED: Error in semantic analysis: 3:20 Input format must implement InputFormat. Error encountered near token 'dest1'

Modified: hive/trunk/ql/src/test/results/clientnegative/groupby_key.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/groupby_key.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/groupby_key.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/groupby_key.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:14 Expression not in GROUP BY key value
+FAILED: Error in semantic analysis: Line 1:7 Expression not in GROUP BY key 'value'

Modified: hive/trunk/ql/src/test/results/clientnegative/input1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/input1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/input1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/input1.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:7 Invalid table alias a
+FAILED: Error in semantic analysis: Line 1:7 Invalid table alias 'a'

Modified: hive/trunk/ql/src/test/results/clientnegative/input2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/input2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/input2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/input2.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:7 Invalid table alias or column reference a
+FAILED: Error in semantic analysis: Line 1:7 Invalid table alias or column reference 'a'

Modified: hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalid_create_tbl1.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead

Modified: hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter1.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE alter_test (d STRING)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@alter_test
-FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead

Modified: hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalid_t_alter2.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE alter_test (d STRING)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@alter_test
-FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead

Modified: hive/trunk/ql/src/test/results/clientnegative/invalid_t_create1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalid_t_create1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalid_t_create1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalid_t_create1.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead

Modified: hive/trunk/ql/src/test/results/clientnegative/invalid_t_create2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalid_t_create2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalid_t_create2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalid_t_create2.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead

Modified: hive/trunk/ql/src/test/results/clientnegative/invalid_t_create3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalid_t_create3.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalid_t_create3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalid_t_create3.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead

Modified: hive/trunk/ql/src/test/results/clientnegative/invalid_t_transform.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalid_t_transform.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalid_t_transform.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalid_t_transform.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead

Modified: hive/trunk/ql/src/test/results/clientnegative/invalidate_view1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/invalidate_view1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/invalidate_view1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/invalidate_view1.q.out Tue May 17 01:29:14 2011
@@ -19,18 +19,18 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@xxx10
 PREHOOK: query: CREATE VIEW xxx9 AS SELECT * FROM xxx10
 PREHOOK: type: CREATEVIEW
-PREHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-05_20-52-14_588_3232160525939411769/-mr-10000
+PREHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-11_12-10-09_986_3605866348000607411/-mr-10000
 POSTHOOK: query: CREATE VIEW xxx9 AS SELECT * FROM xxx10
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Output: default@xxx9
-POSTHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-05_20-52-14_588_3232160525939411769/-mr-10000
+POSTHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-11_12-10-09_986_3605866348000607411/-mr-10000
 PREHOOK: query: CREATE VIEW xxx8 AS SELECT * FROM xxx9 xxx
 PREHOOK: type: CREATEVIEW
-PREHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-05_20-52-14_617_9002044296975800480/-mr-10000
+PREHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-11_12-10-10_017_1075686745843891926/-mr-10000
 POSTHOOK: query: CREATE VIEW xxx8 AS SELECT * FROM xxx9 xxx
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Output: default@xxx8
-POSTHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-05_20-52-14_617_9002044296975800480/-mr-10000
+POSTHOOK: Output: file:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-11_12-10-10_017_1075686745843891926/-mr-10000
 PREHOOK: query: ALTER TABLE xxx10 REPLACE COLUMNS (key int)
 PREHOOK: type: ALTERTABLE_REPLACECOLS
 PREHOOK: Input: default@xxx10
@@ -39,7 +39,7 @@ POSTHOOK: query: ALTER TABLE xxx10 REPLA
 POSTHOOK: type: ALTERTABLE_REPLACECOLS
 POSTHOOK: Input: default@xxx10
 POSTHOOK: Output: default@xxx10
-FAILED: Error in semantic analysis: Line 1:30 Invalid column reference `value` in definition of VIEW xxx9 [
+FAILED: Error in semantic analysis: Line 1:30 Invalid column reference '`value`' in definition of VIEW xxx9 [
 SELECT `xxx10`.`key`, `xxx10`.`value` FROM `xxx10`
 ] used as xxx at Line 1:39 in definition of VIEW xxx8 [
 SELECT `xxx`.`key`, `xxx`.`value` FROM `xxx9` `xxx`

Modified: hive/trunk/ql/src/test/results/clientnegative/joinneg.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/joinneg.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/joinneg.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/joinneg.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 5:12 Invalid table alias b
+FAILED: Error in semantic analysis: Line 5:12 Invalid table alias 'b'

Modified: hive/trunk/ql/src/test/results/clientnegative/line_terminator.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/line_terminator.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/line_terminator.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/line_terminator.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: LINES TERMINATED BY only supports newline '\n' right now
+FAILED: Error in semantic analysis: 3:20 LINES TERMINATED BY only supports newline '\n' right now. Error encountered near token '',''

Modified: hive/trunk/ql/src/test/results/clientnegative/load_part_nospec.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/load_part_nospec.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/load_part_nospec.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/load_part_nospec.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@hive_test_src
-FAILED: Error in semantic analysis: Need to specify partition columns because the destination table is partitioned.
+FAILED: Error in semantic analysis: Need to specify partition columns because the destination table is partitioned

Modified: hive/trunk/ql/src/test/results/clientnegative/load_wrong_noof_part.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/load_wrong_noof_part.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/load_wrong_noof_part.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/load_wrong_noof_part.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE loadpart1(a STRING, b STRING) PARTITIONED BY (ds STRING,ds1 STRING)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@loadpart1
-FAILED: Error in semantic analysis: Line 2:79 Partition not found '2009-05-05'
+FAILED: Error in semantic analysis: Line 2:79 Partition not found ''2009-05-05''

Modified: hive/trunk/ql/src/test/results/clientnegative/nopart_insert.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/nopart_insert.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/nopart_insert.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/nopart_insert.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE nopart_insert(a STRING, b STRING) PARTITIONED BY (ds STRING)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@nopart_insert
-FAILED: Error in semantic analysis: Need to specify partition columns because the destination table is partitioned.
+FAILED: Error in semantic analysis: 3:23 Need to specify partition columns because the destination table is partitioned. Error encountered near token 'nopart_insert'

Modified: hive/trunk/ql/src/test/results/clientnegative/nopart_load.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/nopart_load.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/nopart_load.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/nopart_load.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE nopart_load(a STRING, b STRING) PARTITIONED BY (ds STRING)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@nopart_load
-FAILED: Error in semantic analysis: Need to specify partition columns because the destination table is partitioned.
+FAILED: Error in semantic analysis: Need to specify partition columns because the destination table is partitioned

Modified: hive/trunk/ql/src/test/results/clientnegative/notable_alias3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/notable_alias3.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/notable_alias3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/notable_alias3.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@dest1
-FAILED: Error in semantic analysis: Line 4:44 Expression not in GROUP BY key src
+FAILED: Error in semantic analysis: Line 4:44 Expression not in GROUP BY key 'key'

Modified: hive/trunk/ql/src/test/results/clientnegative/orderbysortby.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/orderbysortby.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/orderbysortby.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/orderbysortby.q.out Tue May 17 01:29:14 2011
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@dest1
-FAILED: Error in semantic analysis: Line 8:8 Cannot have both ORDER BY and SORT BY clauses one
+FAILED: Error in semantic analysis: 8:8 Cannot have both ORDER BY and SORT BY clauses. Error encountered near token 'one'

Modified: hive/trunk/ql/src/test/results/clientnegative/regex_col_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/regex_col_1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/regex_col_1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/regex_col_1.q.out Tue May 17 01:29:14 2011
@@ -1,3 +1,3 @@
-FAILED: Error in semantic analysis: Line 2:7 Invalid column reference `+++`: Dangling meta character '+' near index 0
+FAILED: Error in semantic analysis: Line 2:7 Invalid column reference '`+++`': Dangling meta character '+' near index 0
 +++
 ^

Modified: hive/trunk/ql/src/test/results/clientnegative/regex_col_2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/regex_col_2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/regex_col_2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/regex_col_2.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:7 Invalid column reference `.a.`
+FAILED: Error in semantic analysis: Line 2:7 Invalid column reference '`.a.`'

Modified: hive/trunk/ql/src/test/results/clientnegative/regex_col_groupby.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/regex_col_groupby.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/regex_col_groupby.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/regex_col_groupby.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:44 Invalid table alias or column reference `..`
+FAILED: Error in semantic analysis: Line 2:44 Invalid table alias or column reference '`..`'

Modified: hive/trunk/ql/src/test/results/clientnegative/semijoin1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/semijoin1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/semijoin1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/semijoin1.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:7 Invalid table alias or column reference b
+FAILED: Error in semantic analysis: Line 2:7 Invalid table alias or column reference 'b'

Modified: hive/trunk/ql/src/test/results/clientnegative/semijoin2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/semijoin2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/semijoin2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/semijoin2.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:70 Invalid table alias or column reference b
+FAILED: Error in semantic analysis: Line 2:70 Invalid table alias or column reference 'b'

Modified: hive/trunk/ql/src/test/results/clientnegative/semijoin3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/semijoin3.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/semijoin3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/semijoin3.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:67 Invalid table alias or column reference b
+FAILED: Error in semantic analysis: Line 2:67 Invalid table alias or column reference 'b'

Modified: hive/trunk/ql/src/test/results/clientnegative/semijoin4.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/semijoin4.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/semijoin4.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/semijoin4.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:112 Invalid table alias or column reference b
+FAILED: Error in semantic analysis: Line 2:112 Invalid table alias or column reference 'b'

Modified: hive/trunk/ql/src/test/results/clientnegative/split_sample_out_of_range.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/split_sample_out_of_range.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/split_sample_out_of_range.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/split_sample_out_of_range.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Sampling percentage should be between 0 and 100.
+FAILED: Error in semantic analysis: 3:32 Sampling percentage should be between 0 and 100. Error encountered near token '105'

Modified: hive/trunk/ql/src/test/results/clientnegative/split_sample_wrong_format.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/split_sample_wrong_format.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/split_sample_wrong_format.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/split_sample_wrong_format.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Percentage sampling is not supported in org.apache.hadoop.hive.ql.io.HiveInputFormat
+FAILED: Error in semantic analysis: 3:32 Percentage sampling is not supported in org.apache.hadoop.hive.ql.io.HiveInputFormat. Error encountered near token '1'

Modified: hive/trunk/ql/src/test/results/clientnegative/strict_orderby.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/strict_orderby.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/strict_orderby.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/strict_orderby.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 4:44 In strict mode, if ORDER BY is specified, LIMIT must also be specified key
+FAILED: Error in semantic analysis: 4:47 In strict mode, if ORDER BY is specified, LIMIT must also be specified. Error encountered near token 'key'

Modified: hive/trunk/ql/src/test/results/clientnegative/subq_insert.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/subq_insert.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/subq_insert.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/subq_insert.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:38 Cannot insert in a subquery. Inserting to table  src1
+FAILED: Error in semantic analysis: Line 2:38 Cannot insert in a subquery. Inserting to table  'src1'

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong1.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:22 Argument type mismatch 1: "array" expected at function ARRAY_CONTAINS, but "int" is found
+FAILED: Error in semantic analysis: Line 2:22 Argument type mismatch '1': "array" expected at function ARRAY_CONTAINS, but "int" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_array_contains_wrong2.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:38 Argument type mismatch '2': "int" expected at function ARRAY_CONTAINS, but "string" is found
+FAILED: Error in semantic analysis: Line 2:38 Argument type mismatch ''2'': "int" expected at function ARRAY_CONTAINS, but "string" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:13 Argument type mismatch 1: The expressions after WHEN should have the same type with that after CASE: "string" is expected but "int" is found
+FAILED: Error in semantic analysis: Line 2:13 Argument type mismatch '1': The expressions after WHEN should have the same type with that after CASE: "string" is expected but "int" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong2.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 3:20 Argument type mismatch 4: The expressions after THEN should have the same type: "string" is expected but "int" is found
+FAILED: Error in semantic analysis: Line 3:20 Argument type mismatch '4': The expressions after THEN should have the same type: "string" is expected but "int" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong3.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_case_type_wrong3.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 4:13 Argument type mismatch 7: The expression after ELSE should have the same type as those after THEN: "string" is expected but "int" is found
+FAILED: Error in semantic analysis: Line 4:13 Argument type mismatch '7': The expression after ELSE should have the same type as those after THEN: "string" is expected but "int" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_coalesce.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_coalesce.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_coalesce.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_coalesce.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:19 Argument type mismatch 2.0: The expressions after COALESCE should all have the same type: "int" is expected but "double" is found
+FAILED: Error in semantic analysis: Line 1:19 Argument type mismatch '2.0': The expressions after COALESCE should all have the same type: "int" is expected but "double" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_args_len.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_args_len.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_args_len.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_args_len.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch 3: The function ELT(N,str1,str2,str3,...) needs at least two arguments.
+FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch '3': The function ELT(N,str1,str2,str3,...) needs at least two arguments.

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_type.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_type.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_type.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_elt_wrong_type.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:14 Argument type mismatch lintstring: The 2nd argument of function ELT is expected to a primitive type, but list is found
+FAILED: Error in semantic analysis: Line 2:14 Argument type mismatch 'lintstring': The 2nd argument of function ELT is expected to a primitive type, but list is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_args_len.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_args_len.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_args_len.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_args_len.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:7 Wrong arguments 3: The function FIELD(str, str1, str2, ...) needs at least two arguments.
+FAILED: Error in semantic analysis: Line 1:7 Wrong arguments '3': The function FIELD(str, str1, str2, ...) needs at least two arguments.

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_type.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_type.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_type.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_field_wrong_type.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:16 Argument type mismatch lintstring: The 2nd argument of function FIELD is expected to a primitive type, but list is found
+FAILED: Error in semantic analysis: Line 2:16 Argument type mismatch 'lintstring': The 2nd argument of function FIELD is expected to a primitive type, but list is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_if_not_bool.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_if_not_bool.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_if_not_bool.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_if_not_bool.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:10 Argument type mismatch 'STRING': The first argument of function IF should be "boolean", but "string" is found
+FAILED: Error in semantic analysis: Line 1:10 Argument type mismatch ''STRING'': The first argument of function IF should be "boolean", but "string" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_if_wrong_args_len.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_if_wrong_args_len.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_if_wrong_args_len.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_if_wrong_args_len.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch 1: The function IF(expr1,expr2,expr3) accepts exactly 3 arguments.
+FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch '1': The function IF(expr1,expr2,expr3) accepts exactly 3 arguments.

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_in.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_in.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_in.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_in.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:9 Wrong arguments 3: The arguments for IN should be the same type! Types are: {int IN (array<int>)}
+FAILED: Error in semantic analysis: Line 1:9 Wrong arguments '3': The arguments for IN should be the same type! Types are: {int IN (array<int>)}

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_args_len.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_args_len.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_args_len.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_args_len.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch 'abcd': The function INSTR accepts exactly 2 arguments.
+FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch ''abcd'': The function INSTR accepts exactly 2 arguments.

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_type.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_type.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_type.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_instr_wrong_type.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:21 Argument type mismatch lintstring: The 2nd argument of function INSTR is expected to a primitive type, but list is found
+FAILED: Error in semantic analysis: Line 2:21 Argument type mismatch 'lintstring': The 2nd argument of function INSTR is expected to a primitive type, but list is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_args_len.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_args_len.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_args_len.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_args_len.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch 2: The function LOCATE accepts exactly 2 or 3 arguments.
+FAILED: Error in semantic analysis: Line 1:7 Arguments length mismatch '2': The function LOCATE accepts exactly 2 or 3 arguments.

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_type.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_type.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_type.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_locate_wrong_type.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:22 Argument type mismatch lintstring: The 2nd argument of function LOCATE is expected to a primitive type, but list is found
+FAILED: Error in semantic analysis: Line 2:22 Argument type mismatch 'lintstring': The 2nd argument of function LOCATE is expected to a primitive type, but list is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_args_len.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_args_len.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_args_len.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_args_len.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 2:7 Arguments length mismatch lintstring: The function SIZE only accepts 1 argument.
+FAILED: Error in semantic analysis: Line 2:7 Arguments length mismatch 'lintstring': The function SIZE only accepts 1 argument.

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_type.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_type.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_type.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_size_wrong_type.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 1:12 Argument type mismatch 'wrong type: string': "map" or "list" is expected at function SIZE, but "string" is found
+FAILED: Error in semantic analysis: Line 1:12 Argument type mismatch ''wrong type: string'': "map" or "list" is expected at function SIZE, but "string" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 3:13 Argument type mismatch '1': "boolean" is expected after WHEN, but "string" is found
+FAILED: Error in semantic analysis: Line 3:13 Argument type mismatch ''1'': "boolean" is expected after WHEN, but "string" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong2.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong2.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 3:22 Argument type mismatch 4: The expressions after THEN should have the same type: "string" is expected but "int" is found
+FAILED: Error in semantic analysis: Line 3:22 Argument type mismatch '4': The expressions after THEN should have the same type: "string" is expected but "int" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong3.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udf_when_type_wrong3.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Line 4:13 Argument type mismatch 5.3: The expression after ELSE should have the same type as those after THEN: "string" is expected but "double" is found
+FAILED: Error in semantic analysis: Line 4:13 Argument type mismatch '5.3': The expression after ELSE should have the same type as those after THEN: "string" is expected but "double" is found

Modified: hive/trunk/ql/src/test/results/clientnegative/udtf_not_supported1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/udtf_not_supported1.q.out?rev=1103980&r1=1103979&r2=1103980&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/udtf_not_supported1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/udtf_not_supported1.q.out Tue May 17 01:29:14 2011
@@ -1 +1 @@
-FAILED: Error in semantic analysis: Only a single expression in the SELECT clause is supported with UDTF's
+FAILED: Error in semantic analysis: 1:39 Only a single expression in the SELECT clause is supported with UDTF's. Error encountered near token 'key'