You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2015/03/13 22:10:58 UTC

drill git commit: DRILL-2128.2: Fixed DatabaseMetaData.getColumns's DATA_TYPE, TYPE_NAME.

Repository: drill
Updated Branches:
  refs/heads/master 710656a19 -> 7b4c887e8


DRILL-2128.2: Fixed DatabaseMetaData.getColumns's DATA_TYPE, TYPE_NAME.

- Created basic test for DATA_TYPE and TYPE_NAME.
- Fixed DATA_TYPE:  Added mapping from type name/descriptor strings from
  INFORMATION_SCHEMA.COLUMNS.DATA_TYPE to java.sql.Types.* integer type codes
  for DatabaseMetaData.getColumns's DATA_TYPE.
- Fixed TYPE_NAME:  Added TYPE_NAME returning type name/descriptor strings from
  INFORMATION_SCHEMA.COLUMNS.DATA_TYPE
- Added FIXMEs for some missing/misnamed/wrong fields.  (See DRILL-2420.)


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/7b4c887e
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/7b4c887e
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/7b4c887e

Branch: refs/heads/master
Commit: 7b4c887e852ed1fd526953e55df485a5aaae6e22
Parents: 710656a
Author: dbarclay <db...@maprtech.com>
Authored: Tue Mar 10 19:30:36 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Fri Mar 13 14:09:14 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/drill/jdbc/MetaImpl.java    | 114 ++++++++++++-
 .../jdbc/test/Drill2128GetColumnsBugsTest.java  | 169 +++++++++++++++++++
 2 files changed, 279 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/7b4c887e/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java
index 127751d..99e0d22 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java
@@ -19,6 +19,8 @@ package org.apache.drill.jdbc;
 
 import java.sql.DatabaseMetaData;
 import java.sql.ResultSet;
+import java.sql.Types;
+
 import java.util.List;
 
 import net.hydromatic.avatica.AvaticaPrepareResult;
@@ -126,6 +128,8 @@ public class MetaImpl implements Meta {
 
   public ResultSet getColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
     StringBuilder sb = new StringBuilder();
+    // TODO:  Fix the various remaining bugs and resolve the various questions
+    // noted below.
     sb.append(
         "SELECT \n"
         // getColumns INFORMATION_SCHEMA.COLUMNS   getColumns()
@@ -136,11 +140,113 @@ public class MetaImpl implements Meta {
         + /*  2 */ "  TABLE_SCHEMA             as  TABLE_SCHEM, \n"
         + /*  3 */ "  TABLE_NAME               as  TABLE_NAME, \n"
         + /*  4 */ "  COLUMN_NAME              as  COLUMN_NAME, \n"
-        + /*  5 */ "  DATA_TYPE                as  DATA_TYPE, \n"
-        ///*  6 */                             #6: TYPE_NAME
-        ///*  7 */                             #7: COLUMN_SIZE
+
+        // TODO:  Resolve the various questions noted below for DATA_TYPE.
+        /*    5  (DATA_TYPE) */
+        + "  CASE \n"
+        // (All values in JDBC 4.0/Java 7 java.sql.Types except for types.NULL:)
+
+        // TODO:  RESOLVE:  How does ARRAY appear in COLUMNS.DATA_TYPE?
+        // - Only at end (with no maximum size, as "VARCHAR(65535) ARRAY")?
+        // - Possibly with maximum size (as "... ARRAY[10]")?
+        // (SQL source syntax:
+        //   <array type> ::=
+        //     <data type> ARRAY
+        //       [ <left bracket or trigraph> <maximum cardinality> <right bracket or trigraph> ]
+        + "    WHEN DATA_TYPE LIKE '% ARRAY'    THEN " + Types.ARRAY
+
+        + "    WHEN DATA_TYPE = 'BIGINT'        THEN " + Types.BIGINT
+        + "    WHEN DATA_TYPE = 'BINARY'        THEN " + Types.BINARY
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'BIT'           THEN " + Types.BIT
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'BLOB'          THEN " + Types.BLOB
+        + "    WHEN DATA_TYPE = 'BOOLEAN'       THEN " + Types.BOOLEAN
+
+        + "    WHEN DATA_TYPE = 'CHAR'          THEN " + Types.CHAR
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'CLOB'          THEN " + Types.CLOB
+
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'DATALINK'      THEN " + Types.DATALINK
+        + "    WHEN DATA_TYPE = 'DATE'          THEN " + Types.DATE
+        + "    WHEN DATA_TYPE = 'DECIMAL'       THEN " + Types.DECIMAL
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'DISTINCT'      THEN " + Types.DISTINCT
+        + "    WHEN DATA_TYPE = 'DOUBLE'        THEN " + Types.DOUBLE
+
+        + "    WHEN DATA_TYPE = 'FLOAT'         THEN " + Types.FLOAT
+
+        + "    WHEN DATA_TYPE = 'INTEGER'       THEN " + Types.INTEGER
+
+        // Resolve:  Not seen in Drill yet.  Can it ever appear?:
+        + "    WHEN DATA_TYPE = 'JAVA_OBJECT'   THEN " + Types.JAVA_OBJECT
+
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'LONGNVARCHAR'  THEN " + Types.LONGNVARCHAR
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'LONGVARBINARY' THEN " + Types.LONGVARBINARY
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'LONGVARCHAR'   THEN " + Types.LONGVARCHAR
+
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'NCHAR'         THEN " + Types.NCHAR
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'NCLOB'         THEN " + Types.NCLOB
+        // TODO:  Resolve following about NULL (and then update comment and code):
+        // It is not clear whether Types.NULL can represent a type (perhaps the
+        // type of the literal NULL when no further type information is known?) or
+        // whether 'NULL' can appear in INFORMATION_SCHEMA.COLUMNS.DATA_TYPE.
+        // For now, since it shouldn't hurt, include 'NULL'/Types.NULL in mapping.
+        + "    WHEN DATA_TYPE = 'NULL'          THEN " + Types.NULL
+        // (No NUMERIC--Drill seems to map any to DECIMAL currently.)
+        + "    WHEN DATA_TYPE = 'NUMERIC'       THEN " + Types.NUMERIC
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'NVARCHAR'      THEN " + Types.NVARCHAR
+
+        // Resolve:  Unexpectedly, has appeared in Drill.  Should it?
+        + "    WHEN DATA_TYPE = 'OTHER'         THEN " + Types.OTHER
+
+        + "    WHEN DATA_TYPE = 'REAL'          THEN " + Types.REAL
+        // SQL source syntax:
+        //   <reference type> ::=
+        //     REF <left paren> <referenced type> <right paren> [ <scope clause> ]
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'REF'           THEN " + Types.REF
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'ROWID'         THEN " + Types.ROWID
+
+        + "    WHEN DATA_TYPE = 'SMALLINT'      THEN " + Types.SMALLINT
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "    WHEN DATA_TYPE = 'SQLXML'        THEN " + Types.SQLXML
+
+        // TODO:  RESOLVE:  How does "STRUCT" appear?
+        // - Only at beginning (as "STRUCT(INTEGER sint, BOOLEAN sboolean")?
+        // - Otherwise too?
+        + "    WHEN DATA_TYPE LIKE 'STRUCT(%'   THEN " + Types.STRUCT
+
+        + "    WHEN DATA_TYPE = 'TIME'          THEN " + Types.TIME
+        + "    WHEN DATA_TYPE = 'TIMESTAMP'     THEN " + Types.TIMESTAMP
+        + "    WHEN DATA_TYPE = 'TINYINT'       THEN " + Types.TINYINT
+
+        + "    WHEN DATA_TYPE = 'VARBINARY'     THEN " + Types.VARBINARY
+        + "    WHEN DATA_TYPE = 'VARCHAR'       THEN " + Types.VARCHAR
+
+        // TODO:  RESOLVE:  How does MAP appear in COLUMNS.DATA_TYPE?
+        // - Only at end?
+        // - Otherwise?
+        // TODO:  RESOLVE:  Should it map to Types.OTHER or something else?
+        // Has appeared in Drill.  Should it?
+        + "    WHEN DATA_TYPE LIKE '% MAP'      THEN " + Types.OTHER
+
+        + "    ELSE                                  " + Types.OTHER
+        + "  END                               as  DATA_TYPE, \n"
+
+        + /*  6 */ "  DATA_TYPE                as  TYPE_NAME, \n"
+        ///*  7 */  FIXME:  BUG:  There should be: COLUMN_SIZE
         + /*  8 */ "  CHARACTER_MAXIMUM_LENGTH as  BUFFER_LENGTH, \n"
-        + /*  9 */ "  NUMERIC_PRECISION        as  DECIMAL_PRECISION, \n" // #9: DECIMAL_DIGITS
+        //  FIXME:  BUG:  Many of the following are wrong.
+        + /*  9 */ "  NUMERIC_PRECISION        as  DECIMAL_PRECISION, \n" // FIXME:  BUG:  Should be "DECIMAL_DIGITS"
         + /* 10 */ "  NUMERIC_PRECISION_RADIX  as  NUM_PREC_RADIX, \n"
         + /* 11 */ "  " + DatabaseMetaData.columnNullableUnknown
         +             "                        as  NULLABLE, \n"

http://git-wip-us.apache.org/repos/asf/drill/blob/7b4c887e/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2128GetColumnsBugsTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2128GetColumnsBugsTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2128GetColumnsBugsTest.java
new file mode 100644
index 0000000..0a9104b
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2128GetColumnsBugsTest.java
@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.jdbc.test;
+
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.hamcrest.CoreMatchers.*;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+
+import org.apache.drill.jdbc.JdbcTest;
+
+
+/**
+ * Basic (spot-check/incomplete) tests for DRILL-2128 bugs (many
+ * DatabaseMetaData.getColumns(...) result table problems).
+ */
+public class Drill2128GetColumnsBugsTest extends JdbcTest {
+
+  private static Connection connection;
+  private static DatabaseMetaData dbMetadata;
+
+  @BeforeClass
+  public static void setUpConnection() throws Exception {
+    connection = connect( "jdbc:drill:zk=local" );
+    dbMetadata = connection.getMetaData();
+  }
+
+  @AfterClass
+  public static void tearDownConnection() throws SQLException {
+    connection.close();
+  }
+
+
+  /**
+   * Basic test that column DATA_TYPE is integer type codes (not strings such
+   * as "VARCHAR" or "INTEGER").
+   */
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testColumn_DATA_TYPE_isInteger() throws Exception {
+    // Get metadata for some column(s).
+    final ResultSet columns = dbMetadata.getColumns( null, null, null, null );
+    final boolean hasRow = columns.next();
+    assert hasRow : "DatabaseMetaData.getColumns(...) returned no rows";
+
+    do {
+      // DATA_TYPE should be INTEGER, so getInt( "DATA_TYPE" ) should succeed:
+      final int typeCode1 = columns.getInt( "DATA_TYPE" );
+
+      // DATA_TYPE should be at ordinal position 5 (seemingly):
+      assertThat( "Column 5's label",
+                  columns.getMetaData().getColumnLabel( 5 ), equalTo( "DATA_TYPE" ) );
+
+      // Also, getInt( 5 ) should succeed and return the same type code as above:
+      final int typeCode2 = columns.getInt( 5 );
+      assertThat( "getInt( 5 ) (expected to be same as getInt( \"DATA_TYPE\" ))",
+                  typeCode2, equalTo( typeCode1 ) );
+
+      // Type code should be one of java.sql.Types.*:
+      assertThat(
+          typeCode1,
+          anyOf( // List is from java.sql.Types
+                 equalTo( Types.ARRAY ),
+                 equalTo( Types.BIGINT ),
+                 equalTo( Types.BINARY ),
+                 equalTo( Types.BIT ),
+                 equalTo( Types.BLOB ),
+                 equalTo( Types.BOOLEAN ),
+                 equalTo( Types.CHAR ),
+                 equalTo( Types.CLOB ),
+                 equalTo( Types.DATALINK ),
+                 equalTo( Types.DATE ),
+                 equalTo( Types.DECIMAL ),
+                 equalTo( Types.DISTINCT ),
+                 equalTo( Types.DOUBLE ),
+                 equalTo( Types.FLOAT ),
+                 equalTo( Types.INTEGER ),
+                 equalTo( Types.JAVA_OBJECT ),
+                 equalTo( Types.LONGNVARCHAR ),
+                 equalTo( Types.LONGVARBINARY ),
+                 equalTo( Types.LONGVARCHAR ),
+                 equalTo( Types.NCHAR ),
+                 equalTo( Types.NCLOB ),
+                 // TODO:  Resolve:  Is it not clear whether Types.NULL can re-
+                 // present a type (e.g., the type of NULL), or whether a column
+                 // can ever have that type, and therefore whether Types.NULL
+                 // can appear.  Currently, exclude NULL so we'll notice if it
+                 // does appear:
+                 // No equalTo( Types.NULL ).
+                 equalTo( Types.NUMERIC ),
+                 equalTo( Types.NVARCHAR ),
+                 equalTo( Types.OTHER ),
+                 equalTo( Types.REAL ),
+                 equalTo( Types.REF ),
+                 equalTo( Types.ROWID ),
+                 equalTo( Types.SMALLINT ),
+                 equalTo( Types.SQLXML ),
+                 equalTo( Types.STRUCT ),
+                 equalTo( Types.TIME ),
+                 equalTo( Types.TIMESTAMP ),
+                 equalTo( Types.TINYINT ),
+                 equalTo( Types.VARBINARY ),
+                 equalTo( Types.VARCHAR )
+              ) );
+    } while ( columns.next() );
+  }
+
+  /**
+   * Basic test that column TYPE_NAME exists and is strings (such "INTEGER").
+   */
+  @Test
+  public void testColumn_TYPE_NAME_isString() throws Exception {
+    // Get metadata for some INTEGER column.
+    final ResultSet columns =
+        dbMetadata.getColumns( null, "INFORMATION_SCHEMA", "COLUMNS",
+                               "ORDINAL_POSITION" );
+    final boolean hasRow = columns.next();
+    assert hasRow : "DatabaseMetaData.getColumns(...) returned no rows";
+
+    // TYPE_NAME should be character string for type name "INTEGER", so
+    // getString( "TYPE_NAME" ) should succeed and getInt( "TYPE_NAME" ) should
+    // fail:
+    final String typeName1 = columns.getString( "TYPE_NAME" );
+    assertThat( "getString( \"TYPE_NAME\" )", typeName1, equalTo( "INTEGER" ) );
+
+    try {
+      final int unexpected = columns.getInt( "TYPE_NAME"  );
+      fail( "getInt( \"TYPE_NAME\" ) didn't throw exception (and returned "
+            + unexpected + ")" );
+    }
+    catch ( SQLException e ) {
+      // Expected.
+    }
+
+    // TYPE_NAME should be at ordinal position 6 (seemingly):
+    assertThat( "Column 6's label",
+                columns.getMetaData().getColumnLabel( 6 ), equalTo( "TYPE_NAME" ) );
+
+    // Also, getString( 6 ) should succeed and return the same type name as above:
+    final String typeName2 = columns.getString( 6 );
+    assertThat( "getString( 6 ) (expected to be same as getString( \"TYPE_NAME\" ))",
+                  typeName2, equalTo( typeName1 ) );
+  }
+
+}