You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ve...@apache.org on 2016/10/12 22:44:17 UTC

[1/3] drill git commit: DRILL-4452: Uses Apache Calcite Avatica driver vs Optiq driver for Drill JDBC

Repository: drill
Updated Branches:
  refs/heads/master a29f1e292 -> 09abcc32c


http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java
deleted file mode 100644
index b1ae12c..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java
+++ /dev/null
@@ -1,588 +0,0 @@
-/**
- * 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.impl;
-
-import java.sql.DatabaseMetaData;
-import java.sql.ResultSet;
-import java.sql.Types;
-import java.util.List;
-
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaResultSet;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.Cursor;
-import net.hydromatic.avatica.Meta;
-
-import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.util.DrillStringUtils;
-
-
-class MetaImpl implements Meta {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MetaImpl.class);
-
-  // TODO:  Use more central version of these constants if available.
-
-  /** JDBC conventional(?) number of fractional decimal digits for REAL. */
-  private static final int DECIMAL_DIGITS_REAL = 7;
-  /** JDBC conventional(?) number of fractional decimal digits for FLOAT. */
-  private static final int DECIMAL_DIGITS_FLOAT = DECIMAL_DIGITS_REAL;
-  /** JDBC conventional(?) number of fractional decimal digits for DOUBLE. */
-  private static final int DECIMAL_DIGITS_DOUBLE = 15;
-
-  /** Radix used to report precisions of "datetime" types. */
-  private static final int RADIX_DATETIME = 10;
-  /** Radix used to report precisions of interval types. */
-  private static final int RADIX_INTERVAL = 10;
-
-
-  final DrillConnectionImpl connection;
-
-  MetaImpl(DrillConnectionImpl connection) {
-    this.connection = connection;
-  }
-
-  @Override
-  public String getSqlKeywords() {
-    return "";
-  }
-
-  @Override
-  public String getNumericFunctions() {
-    return "";
-  }
-
-  @Override
-  public String getStringFunctions() {
-    return "";
-  }
-
-  @Override
-  public String getSystemFunctions() {
-    return "";
-  }
-
-  @Override
-  public String getTimeDateFunctions() {
-    return "";
-  }
-
-  private ResultSet s(String s) {
-    try {
-      logger.debug("Running {}", s);
-      AvaticaStatement statement = connection.createStatement();
-      statement.execute(s);
-      return statement.getResultSet();
-
-    } catch (Exception e) {
-      // Wrap in RuntimeException because Avatica's abstract method declarations
-      // didn't allow for SQLException!
-      throw new DrillRuntimeException("Failure while attempting to get DatabaseMetadata.", e);
-    }
-
-  }
-
-  /**
-   * Returns interim generic empty result set.
-   * <p>
-   *   (Does not return specific columns expected (and visible in metadata) for
-   *   specific get methods.)
-   * </p>
-   */
-  private ResultSet getEmptyResultSet() {
-    return s(
-        "SELECT '' AS `Interim zero-row result set` "  // dummy row type
-        + "FROM INFORMATION_SCHEMA.CATALOGS "          // any table
-        + "LIMIT 0"                                    // zero rows
-        );
-  }
-
-  @Override
-  public ResultSet getTables(String catalog, final Pat schemaPattern, final Pat tableNamePattern,
-      final List<String> typeList) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("select "
-        + "TABLE_CATALOG as TABLE_CAT, "
-        + "TABLE_SCHEMA as TABLE_SCHEM, "
-        + "TABLE_NAME, "
-        + "TABLE_TYPE, "
-        + "'' as REMARKS, "
-        + "'' as TYPE_CAT, "
-        + "'' as TYPE_SCHEM, "
-        + "'' as TYPE_NAME, "
-        + "'' as SELF_REFERENCING_COL_NAME, "
-        + "'' as REF_GENERATION "
-        + "FROM INFORMATION_SCHEMA.`TABLES` WHERE 1=1 ");
-
-    if (catalog != null) {
-      sb.append(" AND TABLE_CATALOG = '" + DrillStringUtils.escapeSql(catalog) + "' ");
-    }
-
-    if (schemaPattern.s != null) {
-      sb.append(" AND TABLE_SCHEMA like '" + DrillStringUtils.escapeSql(schemaPattern.s) + "'");
-    }
-
-    if (tableNamePattern.s != null) {
-      sb.append(" AND TABLE_NAME like '" + DrillStringUtils.escapeSql(tableNamePattern.s) + "'");
-    }
-
-    if (typeList != null && typeList.size() > 0) {
-      sb.append("AND (");
-      for (int t = 0; t < typeList.size(); t++) {
-        if (t != 0) {
-          sb.append(" OR ");
-        }
-        sb.append(" TABLE_TYPE LIKE '" + DrillStringUtils.escapeSql(typeList.get(t)) + "' ");
-      }
-      sb.append(")");
-    }
-
-    sb.append(" ORDER BY TABLE_TYPE, TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME");
-
-    return s(sb.toString());
-  }
-
-  /**
-   * Implements {@link DatabaseMetaData#getColumns}.
-   */
-  @Override
-  public ResultSet getColumns(String catalog, Pat schemaPattern,
-                              Pat tableNamePattern, Pat columnNamePattern) {
-    StringBuilder sb = new StringBuilder();
-    // TODO:  Resolve the various questions noted below.
-    sb.append(
-        "SELECT "
-        // getColumns   INFORMATION_SCHEMA.COLUMNS        getColumns()
-        // column       source column or                  column name
-        // number       expression
-        // -------      ------------------------          -------------
-        + /*  1 */ "\n  TABLE_CATALOG                 as  TABLE_CAT, "
-        + /*  2 */ "\n  TABLE_SCHEMA                  as  TABLE_SCHEM, "
-        + /*  3 */ "\n  TABLE_NAME                    as  TABLE_NAME, "
-        + /*  4 */ "\n  COLUMN_NAME                   as  COLUMN_NAME, "
-
-        /*    5                                           DATA_TYPE */
-        // TODO:  Resolve the various questions noted below for DATA_TYPE.
-        + "\n  CASE DATA_TYPE "
-        // (All values in JDBC 4.0/Java 7 java.sql.Types except for types.NULL:)
-
-        + "\n    WHEN 'ARRAY'                       THEN " + Types.ARRAY
-
-        + "\n    WHEN 'BIGINT'                      THEN " + Types.BIGINT
-        + "\n    WHEN 'BINARY'                      THEN " + Types.BINARY
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'BINARY LARGE OBJECT'         THEN " + Types.BLOB
-        + "\n    WHEN 'BINARY VARYING'              THEN " + Types.VARBINARY
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'BIT'                         THEN " + Types.BIT
-        + "\n    WHEN 'BOOLEAN'                     THEN " + Types.BOOLEAN
-
-        + "\n    WHEN 'CHARACTER'                   THEN " + Types.CHAR
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'CHARACTER LARGE OBJECT'      THEN " + Types.CLOB
-        + "\n    WHEN 'CHARACTER VARYING'           THEN " + Types.VARCHAR
-
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'DATALINK'                    THEN " + Types.DATALINK
-        + "\n    WHEN 'DATE'                        THEN " + Types.DATE
-        + "\n    WHEN 'DECIMAL'                     THEN " + Types.DECIMAL
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'DISTINCT'                    THEN " + Types.DISTINCT
-        + "\n    WHEN 'DOUBLE', 'DOUBLE PRECISION'  THEN " + Types.DOUBLE
-
-        + "\n    WHEN 'FLOAT'                       THEN " + Types.FLOAT
-
-        + "\n    WHEN 'INTEGER'                     THEN " + Types.INTEGER
-        + "\n    WHEN 'INTERVAL'                    THEN " + Types.OTHER
-
-        // Resolve:  Not seen in Drill yet.  Can it ever appear?:
-        + "\n    WHEN 'JAVA_OBJECT'                 THEN " + Types.JAVA_OBJECT
-
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'LONGNVARCHAR'                THEN " + Types.LONGNVARCHAR
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'LONGVARBINARY'               THEN " + Types.LONGVARBINARY
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'LONGVARCHAR'                 THEN " + Types.LONGVARCHAR
-
-        + "\n    WHEN 'MAP'                         THEN " + Types.OTHER
-
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'NATIONAL CHARACTER'          THEN " + Types.NCHAR
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'NATIONAL CHARACTER LARGE OBJECT' "
-        + "\n                                       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.
-        + "\n    WHEN 'NULL'                        THEN " + Types.NULL
-        // (No NUMERIC--Drill seems to map any to DECIMAL currently.)
-        + "\n    WHEN 'NUMERIC'                     THEN " + Types.NUMERIC
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'NATIONAL CHARACTER'          THEN " + Types.NCHAR
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'NATIONAL CHARACTER VARYING'  THEN " + Types.NVARCHAR
-
-        // Resolve:  Unexpectedly, has appeared in Drill.  Should it?
-        + "\n    WHEN 'OTHER'                       THEN " + Types.OTHER
-
-        + "\n    WHEN 'REAL'                        THEN " + Types.REAL
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'REF'                         THEN " + Types.REF
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'ROWID'                       THEN " + Types.ROWID
-
-        + "\n    WHEN 'SMALLINT'                    THEN " + Types.SMALLINT
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'SQLXML'                      THEN " + Types.SQLXML
-        + "\n    WHEN 'STRUCT'                      THEN " + Types.STRUCT
-
-        + "\n    WHEN 'TIME'                        THEN " + Types.TIME
-        + "\n    WHEN 'TIMESTAMP'                   THEN " + Types.TIMESTAMP
-        + "\n    WHEN 'TINYINT'                     THEN " + Types.TINYINT
-
-        + "\n    ELSE                                    " + Types.OTHER
-        + "\n  END                                    as  DATA_TYPE, "
-
-        + /*  6 */ "\n  DATA_TYPE                     as  TYPE_NAME, "
-
-        /*    7                                           COLUMN_SIZE */
-        /* "... COLUMN_SIZE ....
-         * For numeric data, this is the maximum precision.
-         * For character data, this is the length in characters.
-         * For datetime datatypes, this is the length in characters of the String
-         *   representation (assuming the maximum allowed precision of the
-         *   fractional seconds component).
-         * For binary data, this is the length in bytes.
-         * For the ROWID datatype, this is the length in bytes.
-         * Null is returned for data types where the column size is not applicable."
-         *
-         * Note:  "Maximum precision" seems to mean the maximum number of
-         * significant digits that can appear (not the number of decimal digits
-         * that can be counted on, and not the maximum number of (decimal)
-         * characters needed to display a value).
-         */
-        + "\n  CASE DATA_TYPE "
-
-        // 1. "For numeric data, ... the maximum precision":
-        + "\n    WHEN 'TINYINT', 'SMALLINT', 'INTEGER', 'BIGINT', "
-        + "\n         'DECIMAL', 'NUMERIC', "
-        + "\n         'REAL', 'FLOAT', 'DOUBLE' "
-        + "\n                         THEN NUMERIC_PRECISION "
-
-        // 2. "For character data, ... the length in characters":
-        + "\n    WHEN 'CHARACTER', 'CHARACTER VARYING' "
-        + "\n                         THEN CHARACTER_MAXIMUM_LENGTH "
-
-        // 3. "For datetime datatypes ... length ... String representation
-        //    (assuming the maximum ... precision of ... fractional seconds ...)":
-        // SQL datetime types:
-        + "\n    WHEN 'DATE'          THEN 10 "            // YYYY-MM-DD
-        + "\n    WHEN 'TIME'          THEN "
-        + "\n      CASE "
-        + "\n        WHEN DATETIME_PRECISION > 0 "         // HH:MM:SS.sss
-        + "\n                         THEN          8 + 1 + DATETIME_PRECISION"
-        + "\n        ELSE                           8"     // HH:MM:SS
-        + "\n      END "
-        + "\n    WHEN 'TIMESTAMP'     THEN "
-        + "\n      CASE "                                  // date + "T" + time
-        + "\n        WHEN DATETIME_PRECISION > 0 "
-        + "                           THEN 10 + 1 + 8 + 1 + DATETIME_PRECISION"
-        + "\n        ELSE                  10 + 1 + 8"
-        + "\n      END "
-        // SQL interval types:
-        // Note:  Not addressed by JDBC 4.1; providing length of current string
-        // representation (not length of, say, interval literal).
-        + "\n    WHEN 'INTERVAL'      THEN "
-        + "\n      INTERVAL_PRECISION "
-        + "\n      + "
-        + "\n      CASE INTERVAL_TYPE "
-        // a. Single field, not SECOND:
-        + "\n        WHEN 'YEAR', 'MONTH', 'DAY' THEN 2 "  // like P...Y
-        + "\n        WHEN 'HOUR', 'MINUTE'       THEN 3 "  // like PT...M
-        // b. Two adjacent fields, no SECOND:
-        + "\n        WHEN 'YEAR TO MONTH'        THEN 5 "  // P...Y12M
-        + "\n        WHEN 'DAY TO HOUR'          THEN 6 "  // P...DT12H
-        + "\n        WHEN 'HOUR TO MINUTE'       THEN 6 "  // PT...H12M
-        // c. Three contiguous fields, no SECOND:
-        + "\n        WHEN 'DAY TO MINUTE'        THEN 9 "  // P...DT12H12M
-        // d. With SECOND field:
-        + "\n        ELSE "
-        + "\n          CASE INTERVAL_TYPE "
-        + "\n            WHEN 'DAY TO SECOND'    THEN 12 " // P...DT12H12M12...S
-        + "\n            WHEN 'HOUR TO SECOND'   THEN  9 " // PT...H12M12...S
-        + "\n            WHEN 'MINUTE TO SECOND' THEN  6 " // PT...M12...S
-        + "\n            WHEN 'SECOND'           THEN  3 " // PT......S
-        + "\n            ELSE "                  // Make net result be -1:
-        // WORKAROUND:  This "0" is to work around Drill's failure to support
-        // unary minus syntax (negation):
-        + "\n                                    0-INTERVAL_PRECISION - 1 "
-        + "\n          END "
-        + "\n          + "
-        + "\n          DATETIME_PRECISION"
-        + "\n          + "
-        + "\n          CASE " // If frac. digits, also add 1 for decimal point.
-        + "\n            WHEN DATETIME_PRECISION > 0 THEN 1"
-        + "\n            ELSE                             0 "
-        + "\n          END"
-        // - For INTERVAL ... TO SECOND(0): "P...DT12H12M12S"
-        + "\n      END "
-
-        // 4. "For binary data, ... the length in bytes":
-        + "\n    WHEN 'BINARY', 'BINARY VARYING' "
-        + "\n                         THEN CHARACTER_MAXIMUM_LENGTH "
-
-        // 5. "For ... ROWID datatype...": Not in Drill?
-
-        // 6. "Null ... for data types [for which] ... not applicable.":
-        + "\n    ELSE                      NULL "
-        + "\n  END                                    as  COLUMN_SIZE, "
-
-        + /*  8 */ "\n  CHARACTER_MAXIMUM_LENGTH      as  BUFFER_LENGTH, "
-
-        /*    9                                           DECIMAL_DIGITS */
-        + "\n  CASE  DATA_TYPE"
-        + "\n    WHEN 'TINYINT', 'SMALLINT', 'INTEGER', 'BIGINT', "
-        + "\n         'DECIMAL', 'NUMERIC'        THEN NUMERIC_SCALE "
-        + "\n    WHEN 'REAL'                      THEN " + DECIMAL_DIGITS_REAL
-        + "\n    WHEN 'FLOAT'                     THEN " + DECIMAL_DIGITS_FLOAT
-        + "\n    WHEN 'DOUBLE'                    THEN " + DECIMAL_DIGITS_DOUBLE
-        + "\n    WHEN 'DATE', 'TIME', 'TIMESTAMP' THEN DATETIME_PRECISION "
-        + "\n    WHEN 'INTERVAL'                  THEN DATETIME_PRECISION "
-        + "\n  END                                    as  DECIMAL_DIGITS, "
-
-        /*   10                                           NUM_PREC_RADIX */
-        + "\n  CASE DATA_TYPE "
-        + "\n    WHEN 'TINYINT', 'SMALLINT', 'INTEGER', 'BIGINT', "
-        + "\n         'DECIMAL', 'NUMERIC', "
-        + "\n         'REAL', 'FLOAT', 'DOUBLE'   THEN NUMERIC_PRECISION_RADIX "
-        // (NUMERIC_PRECISION_RADIX is NULL for these:)
-        + "\n    WHEN 'INTERVAL'                  THEN " + RADIX_INTERVAL
-        + "\n    WHEN 'DATE', 'TIME', 'TIMESTAMP' THEN " + RADIX_DATETIME
-        + "\n    ELSE                                  NULL"
-        + "\n  END                                    as  NUM_PREC_RADIX, "
-
-        /*   11                                           NULLABLE */
-        + "\n  CASE IS_NULLABLE "
-        + "\n    WHEN 'YES'      THEN " + DatabaseMetaData.columnNullable
-        + "\n    WHEN 'NO'       THEN " + DatabaseMetaData.columnNoNulls
-        + "\n    WHEN ''         THEN " + DatabaseMetaData.columnNullableUnknown
-        + "\n    ELSE                 -1"
-        + "\n  END                                    as  NULLABLE, "
-
-        + /* 12 */ "\n  CAST( NULL as VARCHAR )       as  REMARKS, "
-        + /* 13 */ "\n  COLUMN_DEFAULT                as  COLUMN_DEF, "
-        + /* 14 */ "\n  0                             as  SQL_DATA_TYPE, "
-        + /* 15 */ "\n  0                             as  SQL_DATETIME_SUB, "
-
-        /*   16                                           CHAR_OCTET_LENGTH */
-        + "\n  CASE DATA_TYPE"
-        + "\n    WHEN 'CHARACTER', "
-        + "\n         'CHARACTER VARYING', "
-        + "\n         'NATIONAL CHARACTER', "
-        + "\n         'NATIONAL CHARACTER VARYING' "
-        + "\n                                 THEN CHARACTER_OCTET_LENGTH "
-        + "\n    ELSE                              NULL "
-        + "\n  END                                    as  CHAR_OCTET_LENGTH, "
-
-        + /* 17 */ "\n  ORDINAL_POSITION              as  ORDINAL_POSITION, "
-        + /* 18 */ "\n  IS_NULLABLE                   as  IS_NULLABLE, "
-        + /* 19 */ "\n  CAST( NULL as VARCHAR )       as  SCOPE_CATALOG, "
-        + /* 20 */ "\n  CAST( NULL as VARCHAR )       as  SCOPE_SCHEMA, "
-        + /* 21 */ "\n  CAST( NULL as VARCHAR )       as  SCOPE_TABLE, "
-        // TODO:  Change to SMALLINT when it's implemented (DRILL-2470):
-        + /* 22 */ "\n  CAST( NULL as INTEGER )       as  SOURCE_DATA_TYPE, "
-        + /* 23 */ "\n  ''                            as  IS_AUTOINCREMENT, "
-        + /* 24 */ "\n  ''                            as  IS_GENERATEDCOLUMN "
-
-        + "\n  FROM INFORMATION_SCHEMA.COLUMNS "
-        + "\n  WHERE 1=1 ");
-
-    if (catalog != null) {
-      sb.append("\n  AND TABLE_CATALOG = '" + DrillStringUtils.escapeSql(catalog) + "'");
-    }
-    if (schemaPattern.s != null) {
-      sb.append("\n  AND TABLE_SCHEMA like '" + DrillStringUtils.escapeSql(schemaPattern.s) + "'");
-    }
-    if (tableNamePattern.s != null) {
-      sb.append("\n  AND TABLE_NAME like '" + DrillStringUtils.escapeSql(tableNamePattern.s) + "'");
-    }
-    if (columnNamePattern.s != null) {
-      sb.append("\n  AND COLUMN_NAME like '" + DrillStringUtils.escapeSql(columnNamePattern.s) + "'");
-    }
-
-    sb.append("\n ORDER BY TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME");
-
-    return s(sb.toString());
-  }
-
-  @Override
-  public ResultSet getSchemas(String catalog, Pat schemaPattern) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("select "
-        + "SCHEMA_NAME as TABLE_SCHEM, "
-        + "CATALOG_NAME as TABLE_CAT "
-        + " FROM INFORMATION_SCHEMA.SCHEMATA WHERE 1=1 ");
-
-    if (catalog != null) {
-      sb.append(" AND CATALOG_NAME = '" + DrillStringUtils.escapeSql(catalog) + "' ");
-    }
-    if (schemaPattern.s != null) {
-      sb.append(" AND SCHEMA_NAME like '" + DrillStringUtils.escapeSql(schemaPattern.s) + "'");
-    }
-    sb.append(" ORDER BY CATALOG_NAME, SCHEMA_NAME");
-
-    return s(sb.toString());
-  }
-
-  @Override
-  public ResultSet getCatalogs() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("select "
-        + "CATALOG_NAME as TABLE_CAT "
-        + " FROM INFORMATION_SCHEMA.CATALOGS ");
-
-    sb.append(" ORDER BY CATALOG_NAME");
-
-    return s(sb.toString());
-  }
-
-  @Override
-  public ResultSet getTableTypes() {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getProcedures(String catalog, Pat schemaPattern, Pat procedureNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getProcedureColumns(String catalog, Pat schemaPattern, Pat procedureNamePattern,
-      Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getColumnPrivileges(String catalog, String schema, String table, Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getTablePrivileges(String catalog, Pat schemaPattern, Pat tableNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getBestRowIdentifier(String catalog, String schema, String table, int scope, boolean nullable) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getVersionColumns(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getPrimaryKeys(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getImportedKeys(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getExportedKeys(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTable,
-      String foreignCatalog, String foreignSchema, String foreignTable) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getTypeInfo() {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique, boolean approximate) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getUDTs(String catalog, Pat schemaPattern, Pat typeNamePattern, int[] types) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getSuperTypes(String catalog, Pat schemaPattern, Pat typeNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getSuperTables(String catalog, Pat schemaPattern, Pat tableNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getAttributes(String catalog, Pat schemaPattern, Pat typeNamePattern, Pat attributeNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getClientInfoProperties() {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getFunctions(String catalog, Pat schemaPattern, Pat functionNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getFunctionColumns(String catalog, Pat schemaPattern, Pat functionNamePattern, Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public ResultSet getPseudoColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  @Override
-  public Cursor createCursor(AvaticaResultSet resultSet_) {
-    return ((DrillResultSetImpl) resultSet_).cursor;
-  }
-
-  @Override
-  public AvaticaPrepareResult prepare(AvaticaStatement statement_, String sql) {
-    //DrillStatement statement = (DrillStatement) statement_;
-    return new DrillPrepareResult(sql);
-  }
-
-  interface Named {
-    String getName();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillColumnMetaDataListTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillColumnMetaDataListTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillColumnMetaDataListTest.java
index d82fd61..9bd8502 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillColumnMetaDataListTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillColumnMetaDataListTest.java
@@ -17,10 +17,22 @@
  */
 package org.apache.drill.jdbc;
 
-import net.hydromatic.avatica.ColumnMetaData;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
 
-import org.apache.drill.common.expression.SchemaPath;
+import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.jdbc.impl.DrillColumnMetaDataList;
@@ -31,20 +43,6 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import static org.apache.drill.common.types.TypeProtos.MajorType;
-import static org.apache.drill.common.types.TypeProtos.MinorType;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class DrillColumnMetaDataListTest {
 
   private DrillColumnMetaDataList emptyList;

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0e92098..dcb1b3d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -35,6 +35,7 @@
     <dep.guava.version>18.0</dep.guava.version>
     <forkCount>2</forkCount>
     <parquet.version>1.8.1-drill-r0</parquet.version>
+    <calcite.version>1.4.0-drill-r19</calcite.version>
     <sqlline.version>1.1.9-drill-r7</sqlline.version>
     <jackson.version>2.7.1</jackson.version>
     <mapr.release.version>5.2.0-mapr</mapr.release.version>
@@ -1446,7 +1447,7 @@
           <dependency>
             <groupId>org.apache.calcite</groupId>
             <artifactId>calcite-core</artifactId>
-            <version>1.4.0-drill-r19</version>
+            <version>${calcite.version}</version>
             <exclusions>
               <exclusion>
                 <groupId>org.jgrapht</groupId>


[2/3] drill git commit: DRILL-4452: Uses Apache Calcite Avatica driver vs Optiq driver for Drill JDBC

Posted by ve...@apache.org.
DRILL-4452: Uses Apache Calcite Avatica driver vs Optiq driver for Drill JDBC

Drill JDBC driver uses Optiq Avatica as its basis, but this dependency has
been moved over to Calcite, for quite some time without Drill code being
updated for it.

This patch updates Avatica version to the version from Calcite
(1.4.0-drill-r19). It also refactors Drill JDBC driver to comply with the
packages and API changes in Avatica. Finally it fixes the the SQL types for
lists and structs, since Drill doesn't support java.sql.Array and
java.sql.Struct interfaces.

this closes #395

Change-Id: Ia608adf900e8708d9e6f6f58ed41e104321a9914


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

Branch: refs/heads/master
Commit: a888ce6ec289a5ecfe056d4db5da417dd4cc95f5
Parents: a29f1e2
Author: Laurent Goujon <la...@dremio.com>
Authored: Mon Aug 15 16:57:23 2016 -0700
Committer: vkorukanti <ve...@dremio.com>
Committed: Wed Oct 12 13:50:48 2016 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/types/Types.java    |   6 +-
 exec/jdbc-all/pom.xml                           |   3 +-
 exec/jdbc/pom.xml                               |  11 +-
 .../drill/jdbc/DrillConnectionConfig.java       |   2 +-
 .../jdbc/impl/AvaticaDrillSqlAccessor.java      |   3 +-
 .../drill/jdbc/impl/DrillAccessorList.java      |   4 +-
 .../jdbc/impl/DrillColumnMetaDataList.java      |  11 +-
 .../drill/jdbc/impl/DrillConnectionImpl.java    |  90 +--
 .../org/apache/drill/jdbc/impl/DrillCursor.java |  18 +-
 .../jdbc/impl/DrillDatabaseMetaDataImpl.java    |   5 +-
 .../apache/drill/jdbc/impl/DrillFactory.java    |   6 +-
 .../apache/drill/jdbc/impl/DrillHandler.java    |   6 +-
 .../drill/jdbc/impl/DrillJdbc41Factory.java     | 134 +++--
 .../apache/drill/jdbc/impl/DrillMetaImpl.java   | 477 +++++++++++++++
 .../drill/jdbc/impl/DrillPrepareResult.java     |  66 ---
 .../jdbc/impl/DrillPreparedStatementImpl.java   |  45 +-
 .../drill/jdbc/impl/DrillResultSetImpl.java     |  46 +-
 .../jdbc/impl/DrillResultSetMetaDataImpl.java   |  12 +-
 .../drill/jdbc/impl/DrillStatementImpl.java     |  22 +-
 .../org/apache/drill/jdbc/impl/DriverImpl.java  |  13 +-
 .../org/apache/drill/jdbc/impl/MetaImpl.java    | 588 -------------------
 .../drill/jdbc/DrillColumnMetaDataListTest.java |  30 +-
 pom.xml                                         |   3 +-
 23 files changed, 732 insertions(+), 869 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java
index ed77ccd..116d0d1 100644
--- a/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -19,6 +19,8 @@ package org.apache.drill.common.types;
 
 import static org.apache.drill.common.types.TypeProtos.DataMode.REPEATED;
 
+import java.sql.ResultSetMetaData;
+
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
@@ -169,7 +171,7 @@ public class Types {
 
     switch (sqlTypeName) {
       case "ANY":                           return java.sql.Types.OTHER;
-      case "ARRAY":                         return java.sql.Types.ARRAY;
+      case "ARRAY":                         return java.sql.Types.OTHER; // Drill doesn't support java.sql.Array
       case "BIGINT":                        return java.sql.Types.BIGINT;
       case "BINARY VARYING":                return java.sql.Types.VARBINARY;
       case "BINARY":                        return java.sql.Types.BINARY;
@@ -182,7 +184,7 @@ public class Types {
       case "FLOAT":                         return java.sql.Types.FLOAT;
       case "INTEGER":                       return java.sql.Types.INTEGER;
       case "INTERVAL":                      return java.sql.Types.OTHER;  // JDBC (4.1) has nothing for INTERVAL
-      case "MAP":                           return java.sql.Types.STRUCT;
+      case "MAP":                           return java.sql.Types.OTHER; // Drill doesn't support java.sql.Struct
       case "NATIONAL CHARACTER VARYING":    return java.sql.Types.NVARCHAR;
       case "NATIONAL CHARACTER":            return java.sql.Types.NCHAR;
       case "NULL":                          return java.sql.Types.NULL;

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 931f610..f7dfde2 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -279,7 +279,8 @@
             <excludes>
               <exclude>org.slf4j:jcl-over-slf4j</exclude>
               <exclude>com.dyuproject.protostuff:*</exclude>
-              <exclude>org.apache.calcite:*</exclude>
+              <exclude>org.apache.calcite:calcite-core</exclude>
+              <exclude>org.apache.calcite:calcite-linq4j</exclude>
               <exclude>org.pentaho:*</exclude>
               <exclude>org.msgpack:*</exclude>
               <exclude>xerces:*</exclude>

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc/pom.xml b/exec/jdbc/pom.xml
index fabcaf4..73a61b3 100644
--- a/exec/jdbc/pom.xml
+++ b/exec/jdbc/pom.xml
@@ -21,9 +21,9 @@
 
   <dependencies>
     <dependency>
-      <groupId>net.hydromatic</groupId>
-      <artifactId>optiq-avatica</artifactId>
-      <version>0.9-drill-r20</version>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-avatica</artifactId>
+      <version>${calcite.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.drill</groupId>
@@ -56,6 +56,11 @@
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-annotations</artifactId>
       <version>${jackson.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
index 702c9b9..ca20c01 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
@@ -20,7 +20,7 @@ package org.apache.drill.jdbc;
 import java.util.Properties;
 import java.util.TimeZone;
 
-import net.hydromatic.avatica.ConnectionConfigImpl;
+import org.apache.calcite.avatica.ConnectionConfigImpl;
 
 
 // TODO(DRILL-3730):  Change public DrillConnectionConfig from class to

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/AvaticaDrillSqlAccessor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/AvaticaDrillSqlAccessor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/AvaticaDrillSqlAccessor.java
index bf608fc..5a48e59 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/AvaticaDrillSqlAccessor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/AvaticaDrillSqlAccessor.java
@@ -35,8 +35,7 @@ import java.sql.Timestamp;
 import java.util.Calendar;
 import java.util.Map;
 
-import net.hydromatic.avatica.Cursor.Accessor;
-
+import org.apache.calcite.avatica.util.Cursor.Accessor;
 import org.apache.drill.exec.vector.accessor.SqlAccessor;
 import org.apache.drill.jdbc.InvalidCursorStateSqlException;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillAccessorList.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillAccessorList.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillAccessorList.java
index 6f68415..a41c460 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillAccessorList.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillAccessorList.java
@@ -17,10 +17,10 @@
  */
 package org.apache.drill.jdbc.impl;
 
+import java.sql.ResultSet;
 import java.sql.SQLException;
 
-import net.hydromatic.avatica.Cursor.Accessor;
-
+import org.apache.calcite.avatica.util.Cursor.Accessor;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.vector.ValueVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillColumnMetaDataList.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillColumnMetaDataList.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillColumnMetaDataList.java
index a3e37ae..0530538 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillColumnMetaDataList.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillColumnMetaDataList.java
@@ -24,10 +24,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
 
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.ColumnMetaData.AvaticaType;
-import net.hydromatic.avatica.ColumnMetaData.Rep;
-
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.proto.UserProtos.ColumnSearchability;
@@ -41,7 +40,7 @@ import com.google.common.collect.ImmutableList;
 public class DrillColumnMetaDataList extends BasicList<ColumnMetaData>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillColumnMetaDataList.class);
 
-  private List<ColumnMetaData> columns = new ArrayList<ColumnMetaData>();
+  private List<ColumnMetaData> columns = new ArrayList<>();
 
   @Override
   public int size() {
@@ -116,7 +115,7 @@ public class DrillColumnMetaDataList extends BasicList<ColumnMetaData>{
                                    String tableName, BatchSchema schema,
                                    List<Class<?>> getObjectClasses ) {
     final List<ColumnMetaData> newColumns =
-        new ArrayList<ColumnMetaData>(schema.getFieldCount());
+        new ArrayList<>(schema.getFieldCount());
     for (int colOffset = 0; colOffset < schema.getFieldCount(); colOffset++) {
       final MaterializedField field = schema.getColumn(colOffset);
       Class<?> objectClass = getObjectClasses.get( colOffset );

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
index 855a27e..34b6094 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
@@ -29,7 +29,6 @@ import java.sql.SQLClientInfoException;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLNonTransientConnectionException;
-import java.sql.SQLTimeoutException;
 import java.sql.SQLWarning;
 import java.sql.SQLXML;
 import java.sql.Savepoint;
@@ -40,13 +39,11 @@ import java.util.Properties;
 import java.util.TimeZone;
 import java.util.concurrent.Executor;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaFactory;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.Helper;
-import net.hydromatic.avatica.Meta;
-import net.hydromatic.avatica.UnregisteredDriver;
-
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Meta.ExecuteResult;
+import org.apache.calcite.avatica.UnregisteredDriver;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
@@ -54,9 +51,6 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
-import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementResp;
-import org.apache.drill.exec.proto.UserProtos.RequestStatus;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
@@ -69,6 +63,8 @@ import org.apache.drill.jdbc.InvalidParameterSqlException;
 import org.apache.drill.jdbc.JdbcApiSqlException;
 import org.slf4j.Logger;
 
+import com.google.common.base.Throwables;
+
 /**
  * Drill's implementation of {@link Connection}.
  */
@@ -163,6 +159,16 @@ class DrillConnectionImpl extends AvaticaConnection
     }
   }
 
+  @Override
+  protected ExecuteResult prepareAndExecuteInternal(AvaticaStatement statement, String sql, long maxRowCount)
+      throws SQLException {
+    try {
+      return super.prepareAndExecuteInternal(statement, sql, maxRowCount);
+    } catch(RuntimeException e) {
+      Throwables.propagateIfInstanceOf(e.getCause(), SQLException.class);
+      throw e;
+    }
+  }
   /**
    * Throws AlreadyClosedSqlException <i>iff</i> this Connection is closed.
    *
@@ -179,15 +185,6 @@ class DrillConnectionImpl extends AvaticaConnection
     return config;
   }
 
-  @Override
-  protected Meta createMeta() {
-    return new MetaImpl(this);
-  }
-
-  MetaImpl meta() {
-    return (MetaImpl) meta;
-  }
-
   BufferAllocator getAllocator() {
     return allocator;
   }
@@ -364,53 +361,12 @@ class DrillConnectionImpl extends AvaticaConnection
                                             int resultSetConcurrency,
                                             int resultSetHoldability) throws SQLException {
     throwIfClosed();
-    try {
-      DrillRpcFuture<CreatePreparedStatementResp> respFuture = client.createPreparedStatement(sql);
-
-      CreatePreparedStatementResp resp;
-      try {
-        resp = respFuture.get();
-      } catch (InterruptedException e) {
-        // Preserve evidence that the interruption occurred so that code higher up
-        // on the call stack can learn of the interruption and respond to it if it
-        // wants to.
-        Thread.currentThread().interrupt();
-
-        throw new SQLException( "Interrupted", e );
-      }
-
-      final RequestStatus status = resp.getStatus();
-      if (status != RequestStatus.OK) {
-        final String errMsgFromServer = resp.getError() != null ? resp.getError().getMessage() : "";
-
-        if (status == RequestStatus.TIMEOUT) {
-          logger.error("Request timed out to create prepare statement: {}", errMsgFromServer);
-          throw new SQLTimeoutException("Failed to create prepared statement: " + errMsgFromServer);
-        }
-
-        if (status == RequestStatus.FAILED) {
-          logger.error("Failed to create prepared statement: {}", errMsgFromServer);
-          throw new SQLException("Failed to create prepared statement: " + errMsgFromServer);
-        }
-
-        logger.error("Failed to create prepared statement. Unknown status: {}, Error: {}", status, errMsgFromServer);
-        throw new SQLException(String.format(
-            "Failed to create prepared statement. Unknown status: %s, Error: %s", status, errMsgFromServer));
-      }
-
-      DrillPrepareResult prepareResult = new DrillPrepareResult(sql, resp.getPreparedStatement());
-      DrillPreparedStatementImpl statement =
-          (DrillPreparedStatementImpl) factory.newPreparedStatement(
-              this, prepareResult, resultSetType, resultSetConcurrency,
-              resultSetHoldability);
-      return statement;
-    } catch (SQLException e) {
-      throw e;
-    } catch (RuntimeException e) {
-      throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
-    } catch (Exception e) {
-      throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
-    }
+    DrillPreparedStatementImpl statement =
+        (DrillPreparedStatementImpl) super.prepareStatement(sql,
+                                                            resultSetType,
+                                                            resultSetConcurrency,
+                                                            resultSetHoldability);
+    return statement;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index b36658b..a19f82a 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -17,19 +17,18 @@
  */
 package org.apache.drill.jdbc.impl;
 
-import java.sql.SQLException;
-import java.sql.ResultSet;
+import static org.slf4j.LoggerFactory.getLogger;
 
+import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
 
-import net.hydromatic.avatica.ArrayImpl.Factory;
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.Cursor;
-import net.hydromatic.avatica.AvaticaResultSet;
-
-import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.util.ArrayImpl.Factory;
+import org.apache.calcite.avatica.util.Cursor;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.BatchSchema;
@@ -37,7 +36,6 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.store.ischema.InfoSchemaConstants;
 import org.slf4j.Logger;
-import static org.slf4j.LoggerFactory.getLogger;
 
 
 class DrillCursor implements Cursor {
@@ -134,7 +132,7 @@ class DrillCursor implements Cursor {
     // DrillAccessorList blocks iterator() (throwing exception).)
     for ( int ax = 0; ax < accessors.size(); ax++ ) {
       final AvaticaDrillSqlAccessor accessor =
-          (AvaticaDrillSqlAccessor) accessors.get( ax );
+          accessors.get( ax );
       getObjectClasses.add( accessor.getObjectClass() );
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
index a986749..91d4cae 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
@@ -24,12 +24,11 @@ import java.sql.RowIdLifetime;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaDatabaseMetaData;
 import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.DrillDatabaseMetaData;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaDatabaseMetaData;
-
 
 /**
  * Drill's implementation of {@link DatabaseMetaData}.

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillFactory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillFactory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillFactory.java
index b3223b1..0c3c3e8 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillFactory.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillFactory.java
@@ -21,9 +21,9 @@ package org.apache.drill.jdbc.impl;
 import java.sql.SQLException;
 import java.util.Properties;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaFactory;
-import net.hydromatic.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.UnregisteredDriver;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillHandler.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillHandler.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillHandler.java
index e6f4045..169c3cd 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillHandler.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillHandler.java
@@ -19,9 +19,9 @@ package org.apache.drill.jdbc.impl;
 
 import java.sql.SQLException;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.Handler;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Handler;
 
 class DrillHandler implements Handler {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillHandler.class);

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
index 4a8d3bc..670a5f2 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
@@ -24,15 +24,19 @@ import java.sql.NClob;
 import java.sql.ResultSetMetaData;
 import java.sql.RowId;
 import java.sql.SQLException;
+import java.sql.SQLTimeoutException;
 import java.sql.SQLXML;
-import java.util.List;
 import java.util.Properties;
 import java.util.TimeZone;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Helper;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.Meta.StatementHandle;
+import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementResp;
+import org.apache.drill.exec.proto.UserProtos.RequestStatus;
+import org.apache.drill.exec.rpc.DrillRpcFuture;
 
 
 /**
@@ -42,6 +46,8 @@ import net.hydromatic.avatica.ColumnMetaData;
 // Note:  Must be public so net.hydromatic.avatica.UnregisteredDriver can
 // (reflectively) call no-args constructor.
 public class DrillJdbc41Factory extends DrillFactory {
+  private static final org.slf4j.Logger logger =
+      org.slf4j.LoggerFactory.getLogger(DrillJdbc41Factory.class);
 
   /** Creates a factory for JDBC version 4.1. */
   // Note:  Must be public so net.hydromatic.avatica.UnregisteredDriver can
@@ -66,16 +72,18 @@ public class DrillJdbc41Factory extends DrillFactory {
 
   @Override
   public DrillDatabaseMetaDataImpl newDatabaseMetaData(AvaticaConnection connection) {
-    return new DrillDatabaseMetaDataImpl((DrillConnectionImpl) connection);
+    return new DrillDatabaseMetaDataImpl(connection);
   }
 
 
   @Override
   public DrillStatementImpl newStatement(AvaticaConnection connection,
+                                         StatementHandle h,
                                          int resultSetType,
                                          int resultSetConcurrency,
                                          int resultSetHoldability) {
     return new DrillStatementImpl((DrillConnectionImpl) connection,
+                                  h,
                                   resultSetType,
                                   resultSetConcurrency,
                                   resultSetHoldability);
@@ -83,31 +91,79 @@ public class DrillJdbc41Factory extends DrillFactory {
 
   @Override
   public DrillJdbc41PreparedStatement newPreparedStatement(AvaticaConnection connection,
-                                                       AvaticaPrepareResult prepareResult,
+                                                       StatementHandle h,
+                                                       Meta.Signature signature,
                                                        int resultSetType,
                                                        int resultSetConcurrency,
                                                        int resultSetHoldability)
       throws SQLException {
-    return new DrillJdbc41PreparedStatement((DrillConnectionImpl) connection,
-                                            (DrillPrepareResult) prepareResult,
-                                            resultSetType,
-                                            resultSetConcurrency,
-                                            resultSetHoldability);
+    String sql = signature.sql;
+    DrillConnectionImpl drillConnection = (DrillConnectionImpl) connection;
+    try {
+      DrillRpcFuture<CreatePreparedStatementResp> respFuture = drillConnection.getClient().createPreparedStatement(signature.sql);
+
+      CreatePreparedStatementResp resp;
+      try {
+        resp = respFuture.get();
+      } catch (InterruptedException e) {
+        // Preserve evidence that the interruption occurred so that code higher up
+        // on the call stack can learn of the interruption and respond to it if it
+        // wants to.
+        Thread.currentThread().interrupt();
+
+        throw new SQLException( "Interrupted", e );
+      }
+
+      final RequestStatus status = resp.getStatus();
+      if (status != RequestStatus.OK) {
+        final String errMsgFromServer = resp.getError() != null ? resp.getError().getMessage() : "";
+
+        if (status == RequestStatus.TIMEOUT) {
+          logger.error("Request timed out to create prepare statement: {}", errMsgFromServer);
+          throw new SQLTimeoutException("Failed to create prepared statement: " + errMsgFromServer);
+        }
+
+        if (status == RequestStatus.FAILED) {
+          logger.error("Failed to create prepared statement: {}", errMsgFromServer);
+          throw new SQLException("Failed to create prepared statement: " + errMsgFromServer);
+        }
+
+        logger.error("Failed to create prepared statement. Unknown status: {}, Error: {}", status, errMsgFromServer);
+        throw new SQLException(String.format(
+            "Failed to create prepared statement. Unknown status: %s, Error: %s", status, errMsgFromServer));
+      }
+
+      return new DrillJdbc41PreparedStatement((DrillConnectionImpl) connection,
+          h,
+          signature,
+          resp.getPreparedStatement(),
+          resultSetType,
+          resultSetConcurrency,
+          resultSetHoldability);
+    } catch (SQLException e) {
+      throw e;
+    } catch (RuntimeException e) {
+      throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
+    } catch (Exception e) {
+      throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
+    }
+
   }
 
   @Override
   public DrillResultSetImpl newResultSet(AvaticaStatement statement,
-                                         AvaticaPrepareResult prepareResult,
-                                         TimeZone timeZone) {
+                                         Meta.Signature signature,
+                                         TimeZone timeZone,
+                                         Meta.Frame firstFrame) {
     final ResultSetMetaData metaData =
-        newResultSetMetaData(statement, prepareResult.getColumnList());
-    return new DrillResultSetImpl(statement, prepareResult, metaData, timeZone);
+        newResultSetMetaData(statement, signature);
+    return new DrillResultSetImpl(statement, signature, metaData, timeZone, firstFrame);
   }
 
   @Override
   public ResultSetMetaData newResultSetMetaData(AvaticaStatement statement,
-                                                List<ColumnMetaData> columnMetaDataList) {
-    return new DrillResultSetMetaDataImpl(statement, null, columnMetaDataList);
+                                                Meta.Signature signature) {
+    return new DrillResultSetMetaDataImpl(statement, null, signature);
   }
 
 
@@ -117,11 +173,13 @@ public class DrillJdbc41Factory extends DrillFactory {
   private static class DrillJdbc41PreparedStatement extends DrillPreparedStatementImpl {
 
     DrillJdbc41PreparedStatement(DrillConnectionImpl connection,
-                                 DrillPrepareResult prepareResult,
+                                 StatementHandle h,
+                                 Meta.Signature signature,
+                                 org.apache.drill.exec.proto.UserProtos.PreparedStatement pstmt,
                                  int resultSetType,
                                  int resultSetConcurrency,
                                  int resultSetHoldability) throws SQLException {
-      super(connection, prepareResult,
+      super(connection, h, signature, pstmt,
             resultSetType, resultSetConcurrency, resultSetHoldability);
     }
 
@@ -129,104 +187,104 @@ public class DrillJdbc41Factory extends DrillFactory {
 
     @Override
     public void setRowId(int parameterIndex, RowId x) throws SQLException {
-      getParameter(parameterIndex).setRowId(x);
+      getSite(parameterIndex).setRowId(x);
     }
 
     @Override
     public void setNString(int parameterIndex, String value) throws SQLException {
-      getParameter(parameterIndex).setNString(value);
+      getSite(parameterIndex).setNString(value);
     }
 
     @Override
     public void setNCharacterStream(int parameterIndex, Reader value,
                                     long length) throws SQLException {
-      getParameter(parameterIndex).setNCharacterStream(value, length);
+      getSite(parameterIndex).setNCharacterStream(value, length);
     }
 
     @Override
     public void setNClob(int parameterIndex, NClob value) throws SQLException {
-      getParameter(parameterIndex).setNClob(value);
+      getSite(parameterIndex).setNClob(value);
     }
 
     @Override
     public void setClob(int parameterIndex, Reader reader,
                         long length) throws SQLException {
-      getParameter(parameterIndex).setClob(reader, length);
+      getSite(parameterIndex).setClob(reader, length);
     }
 
     @Override
     public void setBlob(int parameterIndex, InputStream inputStream,
                         long length) throws SQLException {
-      getParameter(parameterIndex).setBlob(inputStream, length);
+      getSite(parameterIndex).setBlob(inputStream, length);
     }
 
     @Override
     public void setNClob(int parameterIndex, Reader reader,
                          long length) throws SQLException {
-      getParameter(parameterIndex).setNClob(reader, length);
+      getSite(parameterIndex).setNClob(reader, length);
     }
 
     @Override
     public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {
-      getParameter(parameterIndex).setSQLXML(xmlObject);
+      getSite(parameterIndex).setSQLXML(xmlObject);
     }
 
     @Override
     public void setAsciiStream(int parameterIndex, InputStream x,
                                long length) throws SQLException {
-      getParameter(parameterIndex).setAsciiStream(x, length);
+      getSite(parameterIndex).setAsciiStream(x, length);
     }
 
     @Override
     public void setBinaryStream(int parameterIndex, InputStream x,
                                 long length) throws SQLException {
-      getParameter(parameterIndex).setBinaryStream(x, length);
+      getSite(parameterIndex).setBinaryStream(x, length);
     }
 
     @Override
     public void setCharacterStream(int parameterIndex, Reader reader,
                                    long length) throws SQLException {
-      getParameter(parameterIndex).setCharacterStream(reader, length);
+      getSite(parameterIndex).setCharacterStream(reader, length);
     }
 
     @Override
     public void setAsciiStream(int parameterIndex,
                                InputStream x) throws SQLException {
-      getParameter(parameterIndex).setAsciiStream(x);
+      getSite(parameterIndex).setAsciiStream(x);
     }
 
     @Override
     public void setBinaryStream(int parameterIndex,
                                 InputStream x) throws SQLException {
-      getParameter(parameterIndex).setBinaryStream(x);
+      getSite(parameterIndex).setBinaryStream(x);
     }
 
     @Override
     public void setCharacterStream(int parameterIndex,
                                    Reader reader) throws SQLException {
-      getParameter(parameterIndex).setCharacterStream(reader);
+      getSite(parameterIndex).setCharacterStream(reader);
     }
 
     @Override
     public void setNCharacterStream(int parameterIndex,
                                     Reader value) throws SQLException {
-      getParameter(parameterIndex).setNCharacterStream(value);
+      getSite(parameterIndex).setNCharacterStream(value);
     }
 
     @Override
     public void setClob(int parameterIndex, Reader reader) throws SQLException {
-      getParameter(parameterIndex).setClob(reader);
+      getSite(parameterIndex).setClob(reader);
     }
 
     @Override
     public void setBlob(int parameterIndex,
                         InputStream inputStream) throws SQLException {
-      getParameter(parameterIndex).setBlob(inputStream);
+      getSite(parameterIndex).setBlob(inputStream);
     }
 
     @Override
     public void setNClob(int parameterIndex, Reader reader) throws SQLException {
-      getParameter(parameterIndex).setNClob(reader);
+      getSite(parameterIndex).setNClob(reader);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
new file mode 100644
index 0000000..096b4f0
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
@@ -0,0 +1,477 @@
+/**
+ * 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.impl;
+
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.MetaImpl;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.util.DrillStringUtils;
+
+
+class DrillMetaImpl extends MetaImpl {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillMetaImpl.class);
+
+  // TODO:  Use more central version of these constants if available.
+
+  /** JDBC conventional(?) number of fractional decimal digits for REAL. */
+  private static final int DECIMAL_DIGITS_REAL = 7;
+  /** JDBC conventional(?) number of fractional decimal digits for FLOAT. */
+  private static final int DECIMAL_DIGITS_FLOAT = DECIMAL_DIGITS_REAL;
+  /** JDBC conventional(?) number of fractional decimal digits for DOUBLE. */
+  private static final int DECIMAL_DIGITS_DOUBLE = 15;
+
+  /** Radix used to report precisions of "datetime" types. */
+  private static final int RADIX_DATETIME = 10;
+  /** Radix used to report precisions of interval types. */
+  private static final int RADIX_INTERVAL = 10;
+
+
+  final DrillConnectionImpl connection;
+
+  DrillMetaImpl(DrillConnectionImpl connection) {
+    super(connection);
+    this.connection = connection;
+  }
+
+  private static Signature newSignature(String sql) {
+    return new Signature(
+        new DrillColumnMetaDataList(),
+        sql,
+        Collections.<AvaticaParameter> emptyList(),
+        Collections.<String, Object>emptyMap(),
+        CursorFactory.OBJECT);
+  }
+
+  private MetaResultSet s(String s) {
+    try {
+      logger.debug("Running {}", s);
+
+      AvaticaStatement statement = connection.createStatement();
+      return MetaResultSet.create(connection.id, statement.getId(), true,
+          newSignature(s), null);
+    } catch (Exception e) {
+      // Wrap in RuntimeException because Avatica's abstract method declarations
+      // didn't allow for SQLException!
+      throw new DrillRuntimeException("Failure while attempting to get DatabaseMetadata.", e);
+    }
+  }
+
+
+
+  @Override
+  protected <E> MetaResultSet createEmptyResultSet(Class<E> clazz) {
+    return s(
+        "SELECT '' AS `Interim zero-row result set` "  // dummy row type
+        + "FROM INFORMATION_SCHEMA.CATALOGS "          // any table
+        + "LIMIT 0"                                    // zero rows
+        );
+  }
+
+  @Override
+  public MetaResultSet getTables(String catalog, final Pat schemaPattern, final Pat tableNamePattern,
+      final List<String> typeList) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("select "
+        + "TABLE_CATALOG as TABLE_CAT, "
+        + "TABLE_SCHEMA as TABLE_SCHEM, "
+        + "TABLE_NAME, "
+        + "TABLE_TYPE, "
+        + "'' as REMARKS, "
+        + "'' as TYPE_CAT, "
+        + "'' as TYPE_SCHEM, "
+        + "'' as TYPE_NAME, "
+        + "'' as SELF_REFERENCING_COL_NAME, "
+        + "'' as REF_GENERATION "
+        + "FROM INFORMATION_SCHEMA.`TABLES` WHERE 1=1 ");
+
+    if (catalog != null) {
+      sb.append(" AND TABLE_CATALOG = '" + DrillStringUtils.escapeSql(catalog) + "' ");
+    }
+
+    if (schemaPattern.s != null) {
+      sb.append(" AND TABLE_SCHEMA like '" + DrillStringUtils.escapeSql(schemaPattern.s) + "'");
+    }
+
+    if (tableNamePattern.s != null) {
+      sb.append(" AND TABLE_NAME like '" + DrillStringUtils.escapeSql(tableNamePattern.s) + "'");
+    }
+
+    if (typeList != null && typeList.size() > 0) {
+      sb.append("AND (");
+      for (int t = 0; t < typeList.size(); t++) {
+        if (t != 0) {
+          sb.append(" OR ");
+        }
+        sb.append(" TABLE_TYPE LIKE '" + DrillStringUtils.escapeSql(typeList.get(t)) + "' ");
+      }
+      sb.append(")");
+    }
+
+    sb.append(" ORDER BY TABLE_TYPE, TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME");
+
+    return s(sb.toString());
+  }
+
+  /**
+   * Implements {@link DatabaseMetaData#getColumns}.
+   */
+  @Override
+  public MetaResultSet getColumns(String catalog, Pat schemaPattern,
+                              Pat tableNamePattern, Pat columnNamePattern) {
+    StringBuilder sb = new StringBuilder();
+    // TODO:  Resolve the various questions noted below.
+    sb.append(
+        "SELECT "
+        // getColumns   INFORMATION_SCHEMA.COLUMNS        getColumns()
+        // column       source column or                  column name
+        // number       expression
+        // -------      ------------------------          -------------
+        + /*  1 */ "\n  TABLE_CATALOG                 as  TABLE_CAT, "
+        + /*  2 */ "\n  TABLE_SCHEMA                  as  TABLE_SCHEM, "
+        + /*  3 */ "\n  TABLE_NAME                    as  TABLE_NAME, "
+        + /*  4 */ "\n  COLUMN_NAME                   as  COLUMN_NAME, "
+
+        /*    5                                           DATA_TYPE */
+        // TODO:  Resolve the various questions noted below for DATA_TYPE.
+        + "\n  CASE DATA_TYPE "
+        // (All values in JDBC 4.0/Java 7 java.sql.Types except for types.NULL:)
+
+        + "\n    WHEN 'ARRAY'                       THEN " + Types.ARRAY
+
+        + "\n    WHEN 'BIGINT'                      THEN " + Types.BIGINT
+        + "\n    WHEN 'BINARY'                      THEN " + Types.BINARY
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'BINARY LARGE OBJECT'         THEN " + Types.BLOB
+        + "\n    WHEN 'BINARY VARYING'              THEN " + Types.VARBINARY
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'BIT'                         THEN " + Types.BIT
+        + "\n    WHEN 'BOOLEAN'                     THEN " + Types.BOOLEAN
+
+        + "\n    WHEN 'CHARACTER'                   THEN " + Types.CHAR
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'CHARACTER LARGE OBJECT'      THEN " + Types.CLOB
+        + "\n    WHEN 'CHARACTER VARYING'           THEN " + Types.VARCHAR
+
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'DATALINK'                    THEN " + Types.DATALINK
+        + "\n    WHEN 'DATE'                        THEN " + Types.DATE
+        + "\n    WHEN 'DECIMAL'                     THEN " + Types.DECIMAL
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'DISTINCT'                    THEN " + Types.DISTINCT
+        + "\n    WHEN 'DOUBLE', 'DOUBLE PRECISION'  THEN " + Types.DOUBLE
+
+        + "\n    WHEN 'FLOAT'                       THEN " + Types.FLOAT
+
+        + "\n    WHEN 'INTEGER'                     THEN " + Types.INTEGER
+        + "\n    WHEN 'INTERVAL'                    THEN " + Types.OTHER
+
+        // Resolve:  Not seen in Drill yet.  Can it ever appear?:
+        + "\n    WHEN 'JAVA_OBJECT'                 THEN " + Types.JAVA_OBJECT
+
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'LONGNVARCHAR'                THEN " + Types.LONGNVARCHAR
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'LONGVARBINARY'               THEN " + Types.LONGVARBINARY
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'LONGVARCHAR'                 THEN " + Types.LONGVARCHAR
+
+        + "\n    WHEN 'MAP'                         THEN " + Types.OTHER
+
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'NATIONAL CHARACTER'          THEN " + Types.NCHAR
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'NATIONAL CHARACTER LARGE OBJECT' "
+        + "\n                                       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.
+        + "\n    WHEN 'NULL'                        THEN " + Types.NULL
+        // (No NUMERIC--Drill seems to map any to DECIMAL currently.)
+        + "\n    WHEN 'NUMERIC'                     THEN " + Types.NUMERIC
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'NATIONAL CHARACTER'          THEN " + Types.NCHAR
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'NATIONAL CHARACTER VARYING'  THEN " + Types.NVARCHAR
+
+        // Resolve:  Unexpectedly, has appeared in Drill.  Should it?
+        + "\n    WHEN 'OTHER'                       THEN " + Types.OTHER
+
+        + "\n    WHEN 'REAL'                        THEN " + Types.REAL
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'REF'                         THEN " + Types.REF
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'ROWID'                       THEN " + Types.ROWID
+
+        + "\n    WHEN 'SMALLINT'                    THEN " + Types.SMALLINT
+        // Resolve:  Not seen in Drill yet.  Can it appear?:
+        + "\n    WHEN 'SQLXML'                      THEN " + Types.SQLXML
+        + "\n    WHEN 'STRUCT'                      THEN " + Types.STRUCT
+
+        + "\n    WHEN 'TIME'                        THEN " + Types.TIME
+        + "\n    WHEN 'TIMESTAMP'                   THEN " + Types.TIMESTAMP
+        + "\n    WHEN 'TINYINT'                     THEN " + Types.TINYINT
+
+        + "\n    ELSE                                    " + Types.OTHER
+        + "\n  END                                    as  DATA_TYPE, "
+
+        + /*  6 */ "\n  DATA_TYPE                     as  TYPE_NAME, "
+
+        /*    7                                           COLUMN_SIZE */
+        /* "... COLUMN_SIZE ....
+         * For numeric data, this is the maximum precision.
+         * For character data, this is the length in characters.
+         * For datetime datatypes, this is the length in characters of the String
+         *   representation (assuming the maximum allowed precision of the
+         *   fractional seconds component).
+         * For binary data, this is the length in bytes.
+         * For the ROWID datatype, this is the length in bytes.
+         * Null is returned for data types where the column size is not applicable."
+         *
+         * Note:  "Maximum precision" seems to mean the maximum number of
+         * significant digits that can appear (not the number of decimal digits
+         * that can be counted on, and not the maximum number of (decimal)
+         * characters needed to display a value).
+         */
+        + "\n  CASE DATA_TYPE "
+
+        // 1. "For numeric data, ... the maximum precision":
+        + "\n    WHEN 'TINYINT', 'SMALLINT', 'INTEGER', 'BIGINT', "
+        + "\n         'DECIMAL', 'NUMERIC', "
+        + "\n         'REAL', 'FLOAT', 'DOUBLE' "
+        + "\n                         THEN NUMERIC_PRECISION "
+
+        // 2. "For character data, ... the length in characters":
+        + "\n    WHEN 'CHARACTER', 'CHARACTER VARYING' "
+        + "\n                         THEN CHARACTER_MAXIMUM_LENGTH "
+
+        // 3. "For datetime datatypes ... length ... String representation
+        //    (assuming the maximum ... precision of ... fractional seconds ...)":
+        // SQL datetime types:
+        + "\n    WHEN 'DATE'          THEN 10 "            // YYYY-MM-DD
+        + "\n    WHEN 'TIME'          THEN "
+        + "\n      CASE "
+        + "\n        WHEN DATETIME_PRECISION > 0 "         // HH:MM:SS.sss
+        + "\n                         THEN          8 + 1 + DATETIME_PRECISION"
+        + "\n        ELSE                           8"     // HH:MM:SS
+        + "\n      END "
+        + "\n    WHEN 'TIMESTAMP'     THEN "
+        + "\n      CASE "                                  // date + "T" + time
+        + "\n        WHEN DATETIME_PRECISION > 0 "
+        + "                           THEN 10 + 1 + 8 + 1 + DATETIME_PRECISION"
+        + "\n        ELSE                  10 + 1 + 8"
+        + "\n      END "
+        // SQL interval types:
+        // Note:  Not addressed by JDBC 4.1; providing length of current string
+        // representation (not length of, say, interval literal).
+        + "\n    WHEN 'INTERVAL'      THEN "
+        + "\n      INTERVAL_PRECISION "
+        + "\n      + "
+        + "\n      CASE INTERVAL_TYPE "
+        // a. Single field, not SECOND:
+        + "\n        WHEN 'YEAR', 'MONTH', 'DAY' THEN 2 "  // like P...Y
+        + "\n        WHEN 'HOUR', 'MINUTE'       THEN 3 "  // like PT...M
+        // b. Two adjacent fields, no SECOND:
+        + "\n        WHEN 'YEAR TO MONTH'        THEN 5 "  // P...Y12M
+        + "\n        WHEN 'DAY TO HOUR'          THEN 6 "  // P...DT12H
+        + "\n        WHEN 'HOUR TO MINUTE'       THEN 6 "  // PT...H12M
+        // c. Three contiguous fields, no SECOND:
+        + "\n        WHEN 'DAY TO MINUTE'        THEN 9 "  // P...DT12H12M
+        // d. With SECOND field:
+        + "\n        ELSE "
+        + "\n          CASE INTERVAL_TYPE "
+        + "\n            WHEN 'DAY TO SECOND'    THEN 12 " // P...DT12H12M12...S
+        + "\n            WHEN 'HOUR TO SECOND'   THEN  9 " // PT...H12M12...S
+        + "\n            WHEN 'MINUTE TO SECOND' THEN  6 " // PT...M12...S
+        + "\n            WHEN 'SECOND'           THEN  3 " // PT......S
+        + "\n            ELSE "                  // Make net result be -1:
+        // WORKAROUND:  This "0" is to work around Drill's failure to support
+        // unary minus syntax (negation):
+        + "\n                                    0-INTERVAL_PRECISION - 1 "
+        + "\n          END "
+        + "\n          + "
+        + "\n          DATETIME_PRECISION"
+        + "\n          + "
+        + "\n          CASE " // If frac. digits, also add 1 for decimal point.
+        + "\n            WHEN DATETIME_PRECISION > 0 THEN 1"
+        + "\n            ELSE                             0 "
+        + "\n          END"
+        // - For INTERVAL ... TO SECOND(0): "P...DT12H12M12S"
+        + "\n      END "
+
+        // 4. "For binary data, ... the length in bytes":
+        + "\n    WHEN 'BINARY', 'BINARY VARYING' "
+        + "\n                         THEN CHARACTER_MAXIMUM_LENGTH "
+
+        // 5. "For ... ROWID datatype...": Not in Drill?
+
+        // 6. "Null ... for data types [for which] ... not applicable.":
+        + "\n    ELSE                      NULL "
+        + "\n  END                                    as  COLUMN_SIZE, "
+
+        + /*  8 */ "\n  CHARACTER_MAXIMUM_LENGTH      as  BUFFER_LENGTH, "
+
+        /*    9                                           DECIMAL_DIGITS */
+        + "\n  CASE  DATA_TYPE"
+        + "\n    WHEN 'TINYINT', 'SMALLINT', 'INTEGER', 'BIGINT', "
+        + "\n         'DECIMAL', 'NUMERIC'        THEN NUMERIC_SCALE "
+        + "\n    WHEN 'REAL'                      THEN " + DECIMAL_DIGITS_REAL
+        + "\n    WHEN 'FLOAT'                     THEN " + DECIMAL_DIGITS_FLOAT
+        + "\n    WHEN 'DOUBLE'                    THEN " + DECIMAL_DIGITS_DOUBLE
+        + "\n    WHEN 'DATE', 'TIME', 'TIMESTAMP' THEN DATETIME_PRECISION "
+        + "\n    WHEN 'INTERVAL'                  THEN DATETIME_PRECISION "
+        + "\n  END                                    as  DECIMAL_DIGITS, "
+
+        /*   10                                           NUM_PREC_RADIX */
+        + "\n  CASE DATA_TYPE "
+        + "\n    WHEN 'TINYINT', 'SMALLINT', 'INTEGER', 'BIGINT', "
+        + "\n         'DECIMAL', 'NUMERIC', "
+        + "\n         'REAL', 'FLOAT', 'DOUBLE'   THEN NUMERIC_PRECISION_RADIX "
+        // (NUMERIC_PRECISION_RADIX is NULL for these:)
+        + "\n    WHEN 'INTERVAL'                  THEN " + RADIX_INTERVAL
+        + "\n    WHEN 'DATE', 'TIME', 'TIMESTAMP' THEN " + RADIX_DATETIME
+        + "\n    ELSE                                  NULL"
+        + "\n  END                                    as  NUM_PREC_RADIX, "
+
+        /*   11                                           NULLABLE */
+        + "\n  CASE IS_NULLABLE "
+        + "\n    WHEN 'YES'      THEN " + DatabaseMetaData.columnNullable
+        + "\n    WHEN 'NO'       THEN " + DatabaseMetaData.columnNoNulls
+        + "\n    WHEN ''         THEN " + DatabaseMetaData.columnNullableUnknown
+        + "\n    ELSE                 -1"
+        + "\n  END                                    as  NULLABLE, "
+
+        + /* 12 */ "\n  CAST( NULL as VARCHAR )       as  REMARKS, "
+        + /* 13 */ "\n  COLUMN_DEFAULT                as  COLUMN_DEF, "
+        + /* 14 */ "\n  0                             as  SQL_DATA_TYPE, "
+        + /* 15 */ "\n  0                             as  SQL_DATETIME_SUB, "
+
+        /*   16                                           CHAR_OCTET_LENGTH */
+        + "\n  CASE DATA_TYPE"
+        + "\n    WHEN 'CHARACTER', "
+        + "\n         'CHARACTER VARYING', "
+        + "\n         'NATIONAL CHARACTER', "
+        + "\n         'NATIONAL CHARACTER VARYING' "
+        + "\n                                 THEN CHARACTER_OCTET_LENGTH "
+        + "\n    ELSE                              NULL "
+        + "\n  END                                    as  CHAR_OCTET_LENGTH, "
+
+        + /* 17 */ "\n  ORDINAL_POSITION              as  ORDINAL_POSITION, "
+        + /* 18 */ "\n  IS_NULLABLE                   as  IS_NULLABLE, "
+        + /* 19 */ "\n  CAST( NULL as VARCHAR )       as  SCOPE_CATALOG, "
+        + /* 20 */ "\n  CAST( NULL as VARCHAR )       as  SCOPE_SCHEMA, "
+        + /* 21 */ "\n  CAST( NULL as VARCHAR )       as  SCOPE_TABLE, "
+        // TODO:  Change to SMALLINT when it's implemented (DRILL-2470):
+        + /* 22 */ "\n  CAST( NULL as INTEGER )       as  SOURCE_DATA_TYPE, "
+        + /* 23 */ "\n  ''                            as  IS_AUTOINCREMENT, "
+        + /* 24 */ "\n  ''                            as  IS_GENERATEDCOLUMN "
+
+        + "\n  FROM INFORMATION_SCHEMA.COLUMNS "
+        + "\n  WHERE 1=1 ");
+
+    if (catalog != null) {
+      sb.append("\n  AND TABLE_CATALOG = '" + DrillStringUtils.escapeSql(catalog) + "'");
+    }
+    if (schemaPattern.s != null) {
+      sb.append("\n  AND TABLE_SCHEMA like '" + DrillStringUtils.escapeSql(schemaPattern.s) + "'");
+    }
+    if (tableNamePattern.s != null) {
+      sb.append("\n  AND TABLE_NAME like '" + DrillStringUtils.escapeSql(tableNamePattern.s) + "'");
+    }
+    if (columnNamePattern.s != null) {
+      sb.append("\n  AND COLUMN_NAME like '" + DrillStringUtils.escapeSql(columnNamePattern.s) + "'");
+    }
+
+    sb.append("\n ORDER BY TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME");
+
+    return s(sb.toString());
+  }
+
+  @Override
+  public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("select "
+        + "SCHEMA_NAME as TABLE_SCHEM, "
+        + "CATALOG_NAME as TABLE_CAT "
+        + " FROM INFORMATION_SCHEMA.SCHEMATA WHERE 1=1 ");
+
+    if (catalog != null) {
+      sb.append(" AND CATALOG_NAME = '" + DrillStringUtils.escapeSql(catalog) + "' ");
+    }
+    if (schemaPattern.s != null) {
+      sb.append(" AND SCHEMA_NAME like '" + DrillStringUtils.escapeSql(schemaPattern.s) + "'");
+    }
+    sb.append(" ORDER BY CATALOG_NAME, SCHEMA_NAME");
+
+    return s(sb.toString());
+  }
+
+  @Override
+  public MetaResultSet getCatalogs() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("select "
+        + "CATALOG_NAME as TABLE_CAT "
+        + " FROM INFORMATION_SCHEMA.CATALOGS ");
+
+    sb.append(" ORDER BY CATALOG_NAME");
+
+    return s(sb.toString());
+  }
+
+
+  interface Named {
+    String getName();
+  }
+
+  @Override
+  public StatementHandle prepare(ConnectionHandle ch, String sql, long maxRowCount) {
+    StatementHandle result = super.createStatement(ch);
+    result.signature = newSignature(sql);
+
+    return result;
+  }
+
+  @Override
+  public ExecuteResult prepareAndExecute(StatementHandle h, String sql, long maxRowCount, PrepareCallback callback) {
+    final Signature signature = newSignature(sql);
+    try {
+      synchronized (callback.getMonitor()) {
+        callback.clear();
+        callback.assign(signature, null, -1);
+      }
+      callback.execute();
+      final MetaResultSet metaResultSet = MetaResultSet.create(h.connectionId, h.id, false, signature, null);
+      return new ExecuteResult(Collections.singletonList(metaResultSet));
+    } catch(SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void closeStatement(StatementHandle h) {
+    // Nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPrepareResult.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPrepareResult.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPrepareResult.java
deleted file mode 100644
index 4042ead..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPrepareResult.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * 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.impl;
-
-import org.apache.drill.exec.proto.UserProtos.PreparedStatement;
-
-import java.util.Collections;
-import java.util.List;
-
-import net.hydromatic.avatica.AvaticaParameter;
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.ColumnMetaData;
-
-class DrillPrepareResult implements AvaticaPrepareResult{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillPrepareResult.class);
-
-  final String sql;
-  final DrillColumnMetaDataList columns = new DrillColumnMetaDataList();
-  final PreparedStatement preparedStatement;
-
-  DrillPrepareResult(String sql) {
-    this.sql = sql;
-    this.preparedStatement = null;
-  }
-
-  DrillPrepareResult(String sql, PreparedStatement preparedStatement) {
-    this.sql = sql;
-    this.preparedStatement = preparedStatement;
-    columns.updateColumnMetaData(preparedStatement.getColumnsList());
-  }
-
-  @Override
-  public List<ColumnMetaData> getColumnList() {
-    return columns;
-  }
-
-  @Override
-  public String getSql() {
-    return sql;
-  }
-
-
-  public PreparedStatement getPreparedStatement() {
-    return preparedStatement;
-  }
-
-  @Override
-  public List<AvaticaParameter> getParameterList() {
-    return Collections.emptyList();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
index f86edc6..2894f61 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
@@ -17,9 +17,6 @@
  */
 package org.apache.drill.jdbc.impl;
 
-import org.apache.drill.jdbc.AlreadyClosedSqlException;
-import org.apache.drill.jdbc.DrillPreparedStatement;
-
 import java.sql.ParameterMetaData;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
@@ -27,9 +24,13 @@ import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLWarning;
 
-import net.hydromatic.avatica.AvaticaParameter;
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.Meta.StatementHandle;
+import org.apache.drill.exec.proto.UserProtos.PreparedStatement;
+import org.apache.drill.jdbc.AlreadyClosedSqlException;
+import org.apache.drill.jdbc.DrillPreparedStatement;
 
 /**
  * Implementation of {@link java.sql.PreparedStatement} for Drill.
@@ -44,14 +45,20 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
     implements DrillPreparedStatement,
                DrillRemoteStatement {
 
+  private final PreparedStatement preparedStatementHandle;
+
   protected DrillPreparedStatementImpl(DrillConnectionImpl connection,
-                                       AvaticaPrepareResult prepareResult,
+                                       StatementHandle h,
+                                       Meta.Signature signature,
+                                       PreparedStatement preparedStatementHandle,
                                        int resultSetType,
                                        int resultSetConcurrency,
                                        int resultSetHoldability) throws SQLException {
-    super(connection, prepareResult,
+    super(connection, h, signature,
           resultSetType, resultSetConcurrency, resultSetHoldability);
     connection.openStatementsRegistry.addStatement(this);
+    this.preparedStatementHandle = preparedStatementHandle;
+    ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
   }
 
   /**
@@ -85,6 +92,10 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
     return (DrillConnectionImpl) super.getConnection();
   }
 
+  PreparedStatement getPreparedStatementHandle() {
+    return preparedStatementHandle;
+  }
+
   @Override
   protected AvaticaParameter getParameter(int param) throws SQLException {
     throwIfClosed();
@@ -115,10 +126,10 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
   }
 
   @Override
-  public int executeUpdate(String sql) throws SQLException {
+  public long executeLargeUpdate(String sql) throws SQLException {
     throwIfClosed();
     try {
-      return super.executeUpdate(sql);
+      return super.executeLargeUpdate(sql);
     }
     catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
@@ -150,21 +161,21 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
   }
 
   @Override
-  public int getMaxRows() {
+  public long getLargeMaxRows() {
     try {
       throwIfClosed();
     } catch (AlreadyClosedSqlException e) {
       // Can't throw any SQLException because AvaticaConnection's
-      // getMaxRows() is missing "throws SQLException".
+      // getLargeMaxRows() is missing "throws SQLException".
       throw new RuntimeException(e.getMessage(), e);
     }
-    return super.getMaxRows();
+    return super.getLargeMaxRows();
   }
 
   @Override
-  public void setMaxRows(int max) throws SQLException {
+  public void setLargeMaxRows(long max) throws SQLException {
     throwIfClosed();
-    super.setMaxRows(max);
+    super.setLargeMaxRows(max);
   }
 
   @Override
@@ -489,10 +500,10 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
   }
 
   @Override
-  public int executeUpdate() throws SQLException {
+  public long executeLargeUpdate() throws SQLException {
     throwIfClosed();
     try {
-      return super.executeUpdate();
+      return super.executeLargeUpdate();
     }
     catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index 778504e..a2a7699 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -36,6 +36,7 @@ import java.sql.SQLWarning;
 import java.sql.SQLXML;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.sql.Types;
 import java.util.Calendar;
 import java.util.Map;
 import java.util.TimeZone;
@@ -44,10 +45,12 @@ import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaResultSet;
-import net.hydromatic.avatica.AvaticaStatement;
-
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaSite;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.util.Cursor;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.client.DrillClient;
@@ -87,9 +90,10 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   boolean hasPendingCancelationNotification;
 
 
-  DrillResultSetImpl(AvaticaStatement statement, AvaticaPrepareResult prepareResult,
-                     ResultSetMetaData resultSetMetaData, TimeZone timeZone) {
-    super(statement, prepareResult, resultSetMetaData, timeZone);
+  DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
+                     ResultSetMetaData resultSetMetaData, TimeZone timeZone,
+                     Meta.Frame firstFrame) {
+    super(statement, signature, resultSetMetaData, timeZone, firstFrame);
     connection = (DrillConnectionImpl) statement.getConnection();
     client = connection.getClient();
     final int batchQueueThrottlingThreshold =
@@ -417,7 +421,17 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   @Override
   public Object getObject( int columnIndex ) throws SQLException {
     throwIfClosed();
-    return super.getObject( columnIndex );
+
+    final Cursor.Accessor accessor;
+    try {
+      accessor = accessorList.get(columnIndex - 1);
+    } catch (IndexOutOfBoundsException e) {
+      throw new SQLException("invalid column ordinal: " + columnIndex);
+    }
+    final ColumnMetaData metaData = columnMetaDataList.get(columnIndex - 1);
+    // Drill returns a float (4bytes) for a SQL Float whereas Calcite would return a double (8bytes)
+    int typeId = (metaData.type.id != Types.FLOAT) ? metaData.type.id : Types.REAL;
+    return AvaticaSite.get(accessor, typeId, localCalendar);
   }
 
   @Override
@@ -1883,6 +1897,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   ////////////////////////////////////////
   // DrillResultSet methods:
 
+  @Override
   public String getQueryId() throws SQLException {
     throwIfClosed();
     if (resultsListener.getQueryId() != null) {
@@ -1897,20 +1912,15 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
 
   @Override
   protected DrillResultSetImpl execute() throws SQLException{
-    final DrillPrepareResult drillPrepareResult = ((DrillPrepareResult)prepareResult);
-    /**
-     * {@link DrillPrepareResult} is created both for normal queries and prepared queries.
-     * If the prepared statement exists submit the query as prepared statement, otherwise
-     * regular submission.
-     */
-    if (drillPrepareResult.getPreparedStatement() != null) {
-      client.executePreparedStatement(drillPrepareResult.getPreparedStatement().getServerHandle(), resultsListener);
+    if (statement instanceof DrillPreparedStatementImpl) {
+      DrillPreparedStatementImpl drillPreparedStatement = (DrillPreparedStatementImpl) statement;
+      client.executePreparedStatement(drillPreparedStatement.getPreparedStatementHandle().getServerHandle(), resultsListener);
     } else {
-      client.runQuery(QueryType.SQL, this.prepareResult.getSql(), resultsListener);
+      client.runQuery(QueryType.SQL, this.signature.sql, resultsListener);
     }
     connection.getDriver().handler.onStatementExecute(statement, null);
 
-    super.execute();
+    super.execute2(cursor, this.signature.columns);
 
     // don't return with metadata until we've achieved at least one return message.
     try {

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
index 9847555..ee0fdd0 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
@@ -19,15 +19,13 @@
 package org.apache.drill.jdbc.impl;
 
 import java.sql.SQLException;
-import java.util.List;
 
+import org.apache.calcite.avatica.AvaticaResultSetMetaData;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Meta;
 import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.InvalidParameterSqlException;
 
-import net.hydromatic.avatica.AvaticaResultSetMetaData;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.ColumnMetaData;
-
 
 public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
 
@@ -36,8 +34,8 @@ public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
 
   public DrillResultSetMetaDataImpl(AvaticaStatement statement,
                                     Object query,
-                                    List<ColumnMetaData> columnMetaDataList) {
-    super(statement, query, columnMetaDataList);
+                                    Meta.Signature signature) {
+    super(statement, query, signature);
     this.statement = statement;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
index 5bdf5f8..a01bcf3 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
@@ -23,13 +23,13 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
 
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Meta.StatementHandle;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.DrillStatement;
 import org.apache.drill.jdbc.InvalidParameterSqlException;
 
-import net.hydromatic.avatica.AvaticaStatement;
-
 /**
  * Drill's implementation of {@link Statement}.
  */
@@ -41,9 +41,9 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
 
   private final DrillConnectionImpl connection;
 
-  DrillStatementImpl(DrillConnectionImpl connection, int resultSetType,
+  DrillStatementImpl(DrillConnectionImpl connection, StatementHandle h, int resultSetType,
                      int resultSetConcurrency, int resultSetHoldability) {
-    super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
+    super(connection, h, resultSetType, resultSetConcurrency, resultSetHoldability);
     this.connection = connection;
     connection.openStatementsRegistry.addStatement(this);
   }
@@ -117,10 +117,10 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
   }
 
   @Override
-  public int executeUpdate( String sql ) throws SQLException {
+  public long executeLargeUpdate( String sql ) throws SQLException {
     throwIfClosed();
     try {
-      return super.executeUpdate( sql );
+      return super.executeLargeUpdate( sql );
     }
     catch ( final SQLException possiblyExtraWrapperException ) {
       throw unwrapIfExtra( possiblyExtraWrapperException );
@@ -151,7 +151,7 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
 
   @Override
   public void cleanUp() {
-    final DrillConnectionImpl connection1 = (DrillConnectionImpl) connection;
+    final DrillConnectionImpl connection1 = connection;
     connection1.openStatementsRegistry.removeStatement(this);
   }
 
@@ -225,7 +225,7 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
   }
 
   @Override
-  public int getMaxRows() {
+  public long getLargeMaxRows() {
     try {
       throwIfClosed();
     } catch (AlreadyClosedSqlException e) {
@@ -233,13 +233,13 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
       // getMaxRows() is missing "throws SQLException".
       throw new RuntimeException(e.getMessage(), e);
     }
-    return super.getMaxRows();
+    return super.getLargeMaxRows();
   }
 
   @Override
-  public void setMaxRows(int max) throws SQLException {
+  public void setLargeMaxRows(long max) throws SQLException {
     throwIfClosed();
-    super.setMaxRows(max);
+    super.setLargeMaxRows(max);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/a888ce6e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DriverImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DriverImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DriverImpl.java
index 9fa7ce4..3377b6e 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DriverImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DriverImpl.java
@@ -17,10 +17,11 @@
  */
 package org.apache.drill.jdbc.impl;
 
-import net.hydromatic.avatica.DriverVersion;
-import net.hydromatic.avatica.Handler;
-import net.hydromatic.avatica.HandlerImpl;
-import net.hydromatic.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Handler;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
 
 /**
  * Optiq JDBC driver.
@@ -67,6 +68,10 @@ public class DriverImpl extends UnregisteredDriver {
         "<Properties resource " + METADATA_PROPERTIES_RESOURCE_PATH + " not loaded>");
   }
 
+  @Override
+  public Meta createMeta(AvaticaConnection connection) {
+    return new DrillMetaImpl((DrillConnectionImpl) connection);
+  }
 
   @Override
   protected Handler createHandler() {


[3/3] drill git commit: DRILL-4880: Support JDBC driver registration using ServiceLoader

Posted by ve...@apache.org.
DRILL-4880: Support JDBC driver registration using ServiceLoader

Support loading Drill driver using ServiceLoader. From the user perspective,
it means being able to use the driver without registering it first, like by using
Class.forName("org.apache.drill.jdbc.Driver") for example.

this closes #596

Change-Id: Id26922ee42bef5fbce46ac2bcbb83f1859e9bb48


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

Branch: refs/heads/master
Commit: 09abcc32cc9d6e3de23d3daf633d34fb6183d0f3
Parents: a888ce6
Author: Laurent Goujon <la...@dremio.com>
Authored: Mon Sep 26 17:14:59 2016 -0700
Committer: vkorukanti <ve...@dremio.com>
Committed: Wed Oct 12 13:57:44 2016 -0700

----------------------------------------------------------------------
 exec/jdbc-all/pom.xml                           | 11 +++++++++-
 exec/jdbc/pom.xml                               |  1 +
 .../resources/META-INF/services/java.sql.Driver |  1 +
 .../apache/drill/jdbc/test/JdbcDataTest.java    | 22 +++++++++++++++++++-
 4 files changed, 33 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/09abcc32/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index f7dfde2..c8f27d9 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -388,7 +388,16 @@
                <exclude>**/logback.xml</exclude>
                <exclude>**/LICENSE.txt</exclude>
                <exclude>**/*.java</exclude>
-               <exclude>**/META-INF/**</exclude>
+               <exclude>META-INF/ASL2.0</exclude>
+               <exclude>META-INF/NOTICE.txt</exclude>
+               <exclude>META-INF/drill-module-scan/**</exclude>
+               <exclude>META-INF/jboss-beans.xml</exclude>
+               <exclude>META-INF/license/**</exclude>
+               <exclude>META-INF/maven/**</exclude>
+               <exclude>META-INF/native/**</exclude>
+               <exclude>META-INF/services/com.fasterxml.*</exclude>
+               <exclude>META-INF/services/javax.ws.*</exclude>
+               <exclude>META-INF/**/*.properties</exclude>
                <exclude>**/org.codehaus.commons.compiler.properties</exclude>
                <exclude>**/*.SF</exclude>
                <exclude>**/*.RSA</exclude>

http://git-wip-us.apache.org/repos/asf/drill/blob/09abcc32/exec/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc/pom.xml b/exec/jdbc/pom.xml
index 73a61b3..b26ff6a 100644
--- a/exec/jdbc/pom.xml
+++ b/exec/jdbc/pom.xml
@@ -117,6 +117,7 @@
             <exclude>**/donuts-output-data.txt</exclude>
             <exclude>**/*.tbl</exclude>
             <exclude>**/derby.log</exclude>
+            <exclude>src/main/resources/META-INF/services/**</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/drill/blob/09abcc32/exec/jdbc/src/main/resources/META-INF/services/java.sql.Driver
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/resources/META-INF/services/java.sql.Driver b/exec/jdbc/src/main/resources/META-INF/services/java.sql.Driver
new file mode 100644
index 0000000..e7d99b4
--- /dev/null
+++ b/exec/jdbc/src/main/resources/META-INF/services/java.sql.Driver
@@ -0,0 +1 @@
+org.apache.drill.jdbc.Driver

http://git-wip-us.apache.org/repos/asf/drill/blob/09abcc32/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcDataTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcDataTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcDataTest.java
index fd5d4f0..05e34c6 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcDataTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcDataTest.java
@@ -19,10 +19,14 @@ package org.apache.drill.jdbc.test;
 
 import java.io.IOException;
 import java.sql.Connection;
+import java.sql.Driver;
 import java.sql.DriverManager;
 import java.sql.Statement;
+import java.util.Iterator;
 import java.util.Map;
+import java.util.ServiceLoader;
 
+import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.PlanProperties;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -37,7 +41,6 @@ import org.apache.drill.common.logical.data.Store;
 import org.apache.drill.common.logical.data.Union;
 import org.apache.drill.jdbc.JdbcTestBase;
 import org.apache.drill.jdbc.test.JdbcAssert.TestDataConnection;
-import org.apache.calcite.rel.core.JoinRelType;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -80,6 +83,22 @@ public class JdbcDataTest extends JdbcTestBase {
     Class.forName("org.apache.drill.jdbc.Driver");
   }
 
+  /**
+   * Load the driver using ServiceLoader
+   */
+  @Test
+  public void testLoadDriverServiceLoader() {
+    ServiceLoader<Driver> sl = ServiceLoader.load(Driver.class);
+    for(Iterator<Driver> it = sl.iterator(); it.hasNext(); ) {
+      Driver driver = it.next();
+      if (driver instanceof org.apache.drill.jdbc.Driver) {
+        return;
+      }
+    }
+
+    Assert.fail("org.apache.drill.jdbc.Driver not found using ServiceLoader");
+  }
+
   /** Load driver and make a connection. */
   @Test
   public void testConnect() throws Exception {
@@ -92,6 +111,7 @@ public class JdbcDataTest extends JdbcTestBase {
   @Test
   public void testPrepare() throws Exception {
     JdbcAssert.withModel(MODEL, "DONUTS").withConnection(new Function<Connection, Void>() {
+      @Override
       public Void apply(Connection connection) {
         try {
           final Statement statement = connection.prepareStatement("select * from donuts");