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/01/26 20:46:22 UTC

svn commit: r903385 - in /hadoop/hive/trunk: ./ jdbc/src/test/org/apache/hadoop/hive/jdbc/ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/parse/

Author: namit
Date: Tue Jan 26 19:46:21 2010
New Revision: 903385

URL: http://svn.apache.org/viewvc?rev=903385&view=rev
Log:
HIVE-763. getSchema returns invalid column names, getThriftSchema does
not return old style string schemas (John Sichi via namit)


Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.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/SemanticAnalyzer.java

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=903385&r1=903384&r2=903385&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Tue Jan 26 19:46:21 2010
@@ -70,6 +70,9 @@
     HIVE-1097. Increase memory for groupby_bigdata.q for intermittent OOM
     (Paul Yang via namit)
 
+    HIVE-763. getSchema returns invalid column names, getThriftSchema does
+    not return old style string schemas (John Sichi via namit)
+
 Release 0.5.0 -  Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/hive/trunk/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java?rev=903385&r1=903384&r2=903385&view=diff
==============================================================================
--- hadoop/hive/trunk/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java (original)
+++ hadoop/hive/trunk/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java Tue Jan 26 19:46:21 2010
@@ -323,14 +323,23 @@
     // creating a table with tinyint is failing currently so not including
     res = stmt.executeQuery("create table " + tableName
         + " (a string, b boolean, c bigint, d int, f double)");
-    res = stmt.executeQuery("select * from " + tableName + " limit 1");
+    res = stmt.executeQuery(
+      "select a,b,c,d,f as e,f*2 from " + tableName + " limit 1");
 
     ResultSetMetaData meta = res.getMetaData();
+    assertEquals("Unexpected column count", 6, meta.getColumnCount());
+    assertEquals("Unexpected column name", "a", meta.getColumnName(1));
+    assertEquals("Unexpected column name", "b", meta.getColumnName(2));
+    assertEquals("Unexpected column name", "c", meta.getColumnName(3));
+    assertEquals("Unexpected column name", "d", meta.getColumnName(4));
+    assertEquals("Unexpected column name", "e", meta.getColumnName(5));
+    assertEquals("Unexpected column name", "_c5", meta.getColumnName(6));
     assertEquals("Unexpected column type", Types.VARCHAR, meta.getColumnType(1));
     assertEquals("Unexpected column type", Types.BOOLEAN, meta.getColumnType(2));
     assertEquals("Unexpected column type", Types.BIGINT, meta.getColumnType(3));
     assertEquals("Unexpected column type", Types.INTEGER, meta.getColumnType(4));
     assertEquals("Unexpected column type", Types.DOUBLE, meta.getColumnType(5));
+    assertEquals("Unexpected column type", Types.DOUBLE, meta.getColumnType(6));
     assertEquals("Unexpected column type name", "string", meta
         .getColumnTypeName(1));
     assertEquals("Unexpected column type name", "boolean", meta
@@ -341,6 +350,8 @@
         .getColumnTypeName(4));
     assertEquals("Unexpected column type name", "double", meta
         .getColumnTypeName(5));
+    assertEquals("Unexpected column type name", "double", meta
+        .getColumnTypeName(6));
     assertEquals("Unexpected column display size", 32, meta
         .getColumnDisplaySize(1));
     assertEquals("Unexpected column display size", 8, meta
@@ -351,14 +362,16 @@
         .getColumnDisplaySize(4));
     assertEquals("Unexpected column display size", 16, meta
         .getColumnDisplaySize(5));
+    assertEquals("Unexpected column display size", 16, meta
+        .getColumnDisplaySize(6));
 
-    for (int i = 1; i <= 5; i++) {
+    for (int i = 1; i <= 6; i++) {
       assertFalse(meta.isAutoIncrement(i));
       assertFalse(meta.isCurrency(i));
       assertEquals(ResultSetMetaData.columnNullable, meta.isNullable(i));
 
-      int expectedPrecision = i == 5 ? -1 : 0;
-      int expectedScale = i == 5 ? -1 : 0;
+      int expectedPrecision = i >= 5 ? -1 : 0;
+      int expectedScale = i >= 5 ? -1 : 0;
       assertEquals("Unexpected precision", expectedPrecision, meta
           .getPrecision(i));
       assertEquals("Unexpected scale", expectedScale, meta.getScale(i));

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=903385&r1=903384&r2=903385&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Tue Jan 26 19:46:21 2010
@@ -141,9 +141,17 @@
    * Get a Schema with fields represented with native Hive types
    */
   public Schema getSchema() throws Exception {
-    Schema schema;
+    Schema schema = null;
     try {
-      if (plan != null && plan.getPlan().getFetchTask() != null) {
+      // If we have a plan, prefer its logical result schema if it's
+      // available; otherwise, try digging out a fetch task; failing that,
+      // give up.
+      if (plan == null) {
+        // can't get any info without a plan
+      } else if (plan.getPlan().getResultSchema() != null) {
+        List<FieldSchema> lst = plan.getPlan().getResultSchema();
+        schema = new Schema(lst, null);
+      } else if (plan.getPlan().getFetchTask() != null) {
         BaseSemanticAnalyzer sem = plan.getPlan();
 
         if (!sem.getFetchTaskInit()) {
@@ -174,7 +182,8 @@
         List<FieldSchema> lst = MetaStoreUtils.getFieldsFromDeserializer(
             tableName, td.getDeserializer());
         schema = new Schema(lst, null);
-      } else {
+      }
+      if (schema == null) {
         schema = new Schema();
       }
     } catch (Exception e) {

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=903385&r1=903384&r2=903385&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 Tue Jan 26 19:46:21 2010
@@ -302,6 +302,14 @@
   }
 
   /**
+   * @return the schema for the fields which will be produced
+   * when the statement is executed, or null if not known
+   */
+  public List<FieldSchema> getResultSchema() {
+    return null;
+  }
+
+  /**
    * Get the list of FieldSchema out of the ASTNode.
    */
   protected List<FieldSchema> getColumns(ASTNode ast) throws SemanticException {

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=903385&r1=903384&r2=903385&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 Tue Jan 26 19:46:21 2010
@@ -171,6 +171,7 @@
   private HashMap<TableScanOperator, sampleDesc> opToSamplePruner;
   Map<GroupByOperator, Set<String>> groupOpToInputTables;
   Map<String, PrunedPartitionList> prunedPartitions;
+  private List<FieldSchema> resultSchema;
   private CreateViewDesc createVwDesc;
   private ASTNode viewSelect;
   private final UnparseTranslator unparseTranslator;
@@ -5580,10 +5581,16 @@
     getMetaData(qb);
     LOG.info("Completed getting MetaData in Semantic Analysis");
 
+    // Save the result schema derived from the sink operator produced
+    // by genPlan.  This has the correct column names, which clients
+    // such as JDBC would prefer instead of the c0, c1 we'll end
+    // up with later.
     Operator sinkOp = genPlan(qb);
+    resultSchema =
+      convertRowSchemaToViewSchema(opParseCtx.get(sinkOp).getRR());
 
     if (createVwDesc != null) {
-      saveViewDefinition(sinkOp);
+      saveViewDefinition();
       // Since we're only creating a view (not executing it), we
       // don't need to optimize or translate the plan (and in fact, those
       // procedures can interfere with the view creation). So
@@ -5615,12 +5622,17 @@
     return;
   }
 
-  private void saveViewDefinition(Operator sinkOp) throws SemanticException {
+  @Override
+  public List<FieldSchema> getResultSchema() {
+    return resultSchema;
+  }
+
+  private void saveViewDefinition() throws SemanticException {
 
-    // Save the view schema derived from the sink operator produced
-    // by genPlan.
-    List<FieldSchema> derivedSchema = convertRowSchemaToViewSchema(opParseCtx
-        .get(sinkOp).getRR());
+    // Make a copy of the statement's result schema, since we may
+    // modify it below as part of imposing view column names.
+    List<FieldSchema> derivedSchema =
+      new ArrayList<FieldSchema>(resultSchema);
     validateColumnNameUniqueness(derivedSchema);
 
     List<FieldSchema> imposedSchema = createVwDesc.getSchema();