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

[02/10] drill git commit: DRILL-2089: Split JDBC implementation out of org.apache.drill.jdbc. (Combined patch.)

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
index 55453e8..1d2a521 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
@@ -17,66 +17,97 @@
  */
 package org.apache.drill.jdbc;
 
-import net.hydromatic.avatica.DriverVersion;
-import net.hydromatic.avatica.Handler;
-import net.hydromatic.avatica.HandlerImpl;
-import net.hydromatic.avatica.UnregisteredDriver;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Properties;
+
+import org.apache.drill.jdbc.impl.DriverImpl;
+
+import org.slf4j.Logger;
+import static org.slf4j.LoggerFactory.getLogger;
+
 
 /**
- * Optiq JDBC driver.
+ * Apache Drill JDBC driver main class.
  */
-public class Driver extends UnregisteredDriver {
-  public static final String CONNECT_STRING_PREFIX = "jdbc:drill:";
+public class Driver implements java.sql.Driver {
+  private static final Logger logger = getLogger( Driver.class );
+
+  /** Delegate for everything except registration with DriverManager. */
+  private final DriverImpl impl;
+
+
+  // The following should be the last static initialization, so that any other
+  // static initialization is completed before we create an instance and let
+  // DriverManager access it:
 
+  static {
+    // Upon loading of class, register an instance with DriverManager.
+
+    try {
+      DriverManager.registerDriver( new Driver() );
+    }
+    catch ( SQLException e ) {
+      logger.error( "Error in registering Drill JDBC driver {}: {}",
+                    Driver.class, e, e );
+    }
+  }
+
+  /**
+   * Ensures that class is loaded.
+   * <p>
+   *   (Avoids extra instance of calling {@code new Driver();}; avoids verbosity
+   *   of {@code Class.forName("org.apache.drill.jdbc.Driver");}
+   * </p>
+   */
+  public static boolean load() {
+    return true;
+  }
 
 
   public Driver() {
-    super();
+    impl = new DriverImpl();
   }
 
 
-  public static boolean load(){
-    return true;
+  @Override
+  public Connection connect( String url, Properties info ) throws SQLException {
+    return impl.connect( url, info );
   }
 
+
   @Override
-  protected String getConnectStringPrefix() {
-    return CONNECT_STRING_PREFIX;
+  public boolean acceptsURL( String url ) throws SQLException {
+    return impl.acceptsURL( url );
   }
 
   @Override
-  protected String getFactoryClassName(JdbcVersion jdbcVersion) {
-    switch (jdbcVersion) {
-    case JDBC_30:
-      return "org.apache.drill.jdbc.DrillJdbc3Factory";
-    case JDBC_40:
-      return "org.apache.drill.jdbc.DrillJdbc40Factory";
-    case JDBC_41:
-    default:
-      return "org.apache.drill.jdbc.DrillJdbc41Factory";
-    }
+  public DriverPropertyInfo[] getPropertyInfo( String url, Properties info )
+     throws SQLException {
+    return impl.getPropertyInfo( url,  info );
   }
 
   @Override
-  protected DriverVersion createDriverVersion() {
-    return DriverVersion.load(
-        Driver.class,
-        "apache-drill-jdbc.properties",
-        "Drill JDBC Driver",
-        "unknown version",
-        "Optiq",
-        "unknown version");
+  public int getMajorVersion() {
+    return impl.getMajorVersion();
   }
 
+  @Override
+  public int getMinorVersion() {
+    return impl.getMinorVersion();
+  }
 
   @Override
-  protected Handler createHandler() {
-    return new DrillHandler();
+  public boolean jdbcCompliant() {
+    return impl.jdbcCompliant();
   }
 
-  // Any reference to class loads class, and loading class instantiates an
-  // instance and has it register itself:
-  static {
-    new Driver().register();
+  @Override
+  public java.util.logging.Logger getParentLogger() throws SQLFeatureNotSupportedException {
+    return impl.getParentLogger();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java
deleted file mode 100644
index 7c9329c..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java
+++ /dev/null
@@ -1,29 +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;
-
-import org.apache.drill.exec.server.RemoteServiceSet;
-
-public class GlobalServiceSetReference {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(GlobalServiceSetReference.class);
-
-  public static final ThreadLocal<RemoteServiceSet> SETS = new ThreadLocal<RemoteServiceSet>();
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
index d71248c..b7540be 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
@@ -23,7 +23,7 @@ package org.apache.drill.jdbc;
  * accessor method before calling {@link ResultSet#next()} or after
  * {@link ResultSet#next()} returns false.
  */
-class InvalidCursorStateSqlException extends JdbcApiSqlException {
+public class InvalidCursorStateSqlException extends JdbcApiSqlException {
 
   private static final long serialVersionUID = 2014_12_09L;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
index ce83e01..5e6f475 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
@@ -87,7 +87,7 @@ import java.sql.SQLNonTransientException;
  *   etc.)
  * </p>
  */
-class JdbcApiSqlException extends SQLNonTransientException {
+public class JdbcApiSqlException extends SQLNonTransientException {
 
   private static final long serialVersionUID = 2014_12_12L;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java
deleted file mode 100644
index 2ac0b91..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/MetaImpl.java
+++ /dev/null
@@ -1,634 +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;
-
-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;
-import org.apache.drill.jdbc.impl.DrillResultSetImpl;
-
-
-public 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 availabe.
-
-  /** Radix used to report precision and scale of integral exact numeric types. */
-  private static final int RADIX_INTEGRAL = 10;
-  /** Radix used to report precision and scale of non-integral exact numeric
-      types (DECIMAL). */
-  private static final int RADIX_DECIMAL = 10;
-  /** Radix used to report precision and scale of approximate numeric types
-      (FLOAT, etc.). */
-  private static final int RADIX_APPROXIMATE = 10;
-  /** Radix used to report precisions of interval types. */
-  private static final int RADIX_INTERVAL = 10;
-
-  /** (Maximum) precision of TINYINT. */
-  private static final int PREC_TINYINT  = 3;
-  /** (Maximum) precision of SMALLINT. */
-  private static final int PREC_SMALLINT = 5;
-  /** (Maximum) precision of INTEGER. */
-  private static final int PREC_INTEGER  = 10;
-  /** (Maximum) precision of BIGINT. */
-  private static final int PREC_BIGINT   = 19;
-
-  /** Precision of FLOAT. */
-  private static final int PREC_FLOAT  =  7;
-  /** Precision of DOUBLE. */
-  private static final int PREC_DOUBLE = 15;
-  /** Precision of REAL. */
-  private static final int PREC_REAL   = PREC_DOUBLE;
-
-  /** Scale of INTEGER types. */
-  private static final int SCALE_INTEGRAL = 0;
-  /** JDBC conventional(?) scale value for FLOAT. */
-  private static final int SCALE_FLOAT = 7;
-  /** JDBC conventional(?) scale value for DOUBLE. */
-  private static final int SCALE_DOUBLE = 15;
-  /** JDBC conventional(?) scale value for REAL. */
-  private static final int SCALE_REAL = SCALE_DOUBLE;
-
-  /** (Apparent) maximum precision for starting unit of INTERVAL type. */
-  private static final int PREC_INTERVAL_LEAD_MAX = 10;
-  /** (Apparent) maximum fractional seconds precision for INTERVAL type. */
-  private static final int PREC_INTERVAL_TRAIL_MAX = 9;
-
-
-  final DrillConnectionImpl connection;
-
-  public MetaImpl(DrillConnectionImpl connection) {
-    this.connection = connection;
-  }
-
-  public String getSqlKeywords() {
-    return "";
-  }
-
-  public String getNumericFunctions() {
-    return "";
-  }
-
-  public String getStringFunctions() {
-    return "";
-  }
-
-  public String getSystemFunctions() {
-    return "";
-  }
-
-  public String getTimeDateFunctions() {
-    return "";
-  }
-
-  public static ResultSet getEmptyResultSet() {
-    return null;
-  }
-
-  private ResultSet s(String s) {
-    try {
-      logger.debug("Running {}", s);
-      AvaticaStatement statement = connection.createStatement();
-      statement.execute(s);
-      return statement.getResultSet();
-
-    } catch (Exception e) {
-      throw new DrillRuntimeException("Failure while attempting to get DatabaseMetadata.", e);
-    }
-
-  }
-
-  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()}.
-   */
-  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:)
-
-        // Exact-match cases:
-        + "\n    WHEN 'BIGINT'                      THEN " + Types.BIGINT
-        + "\n    WHEN 'BINARY'                      THEN " + Types.BINARY
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'BIT'                         THEN " + Types.BIT
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'BLOB', 'BINARY LARGE OBJECT' THEN " + Types.BLOB
-        + "\n    WHEN 'BOOLEAN'                     THEN " + Types.BOOLEAN
-
-        + "\n    WHEN 'CHAR', 'CHARACTER'           THEN " + Types.CHAR
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'CLOB', 'CHARACTER LARGE OBJECT' "
-        + "\n                                       THEN " + Types.CLOB
-
-        // 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
-
-        // Drill's INFORMATION_SCHEMA's COLUMNS currently has
-        // "INTERVAL_YEAR_MONTH" and "INTERVAL_DAY_TIME" instead of SQL standard
-        // 'INTERVAL'.
-        + "\n    WHEN 'INTERVAL', "
-        + "\n         'INTERVAL_YEAR_MONTH', "
-        + "\n         'INTERVAL_DAY_TIME'           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
-
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'NCHAR', 'NATIONAL CHARACTER' THEN " + Types.NCHAR
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\n    WHEN 'NCLOB', '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 'NVARCHAR', 'NATIONAL CHARACTER VARYING' "
-        + "\n                                       THEN " + Types.NVARCHAR
-
-        // Resolve:  Unexpectedly, has appeared in Drill.  Should it?
-        + "\n    WHEN 'OTHER'                       THEN " + Types.OTHER
-
-        + "\n    WHEN 'REAL'                        THEN " + Types.REAL
-        // SQL source syntax:
-        //   <reference type> ::=
-        //     REF <left paren> <referenced type> <right paren> [ <scope clause> ]
-        // Resolve:  Not seen in Drill yet.  Can it appear?:
-        + "\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 'TIME'                        THEN " + Types.TIME
-        + "\n    WHEN 'TIMESTAMP'                   THEN " + Types.TIMESTAMP
-        + "\n    WHEN 'TINYINT'                     THEN " + Types.TINYINT
-
-        + "\n    WHEN 'VARBINARY', 'BINARY VARYING' THEN " + Types.VARBINARY
-        + "\n    WHEN 'VARCHAR', 'CHARACTER VARYING' "
-        + "\n                                       THEN " + Types.VARCHAR
-
-        + "\n    ELSE"
-        // Pattern-match cases:
-        + "\n      CASE "
-
-        // TODO:  RESOLVE:  How does ARRAY appear in COLUMNS.DATA_TYPE?
-        // - Only at end (with no maximum size, as "VARCHAR(65535) ARRAY")?
-        // - Possibly with maximum size (as "... ARRAY[10]")?
-        // - Then, how should it appear in JDBC ("ARRAY"? "... ARRAY"?)
-        // (SQL source syntax:
-        //   <array type> ::=
-        //     <data type> ARRAY
-        //       [ <left bracket or trigraph> <maximum cardinality>
-        //         <right bracket or trigraph> ]
-        + "\n        WHEN DATA_TYPE LIKE '% ARRAY'  THEN " + Types.ARRAY
-
-        // TODO:  RESOLVE:  How does MAP appear in COLUMNS.DATA_TYPE?
-        // - Only at end?
-        // - Otherwise?
-        // TODO:  RESOLVE:  Should it map to Types.OTHER or something else?
-        // Has appeared in Drill.  Should it?
-        + "\n        WHEN DATA_TYPE LIKE '% MAP'    THEN " + Types.OTHER
-
-        // TODO:  RESOLVE:  How does "STRUCT" appear?
-        // - Only at beginning (as "STRUCT(INTEGER sint, BOOLEAN sboolean")?
-        // - Otherwise too?
-        // - Then, how should it appear in JDBC ("STRUCT"? "STRUCT(...)"?)
-        + "\n        WHEN DATA_TYPE LIKE 'STRUCT(%' THEN " + Types.STRUCT
-
-        + "\n        ELSE                                " + Types.OTHER
-        + "\n      END "
-        + "\n  END                                    as  DATA_TYPE, "
-
-        /*    6                                           TYPE_NAME */
-        // Map Drill's current info. schema values to what SQL standard
-        // specifies (for DATA_TYPE)--and assume that that's what JDBC wants.
-        + "\n  CASE DATA_TYPE "
-        + "\n    WHEN 'INTERVAL_YEAR_MONTH', "
-        + "\n         'INTERVAL_DAY_TIME'     THEN 'INTERVAL'"
-        // TODO:  Resolve how non-scalar types should appear in
-        // INFORMATION_SCHEMA.COLUMNS and here in JDBC:
-        // - "ARRAY" or "... ARRAY"?
-        // - "MAP" or "... MAP"?
-        // - "STRUCT" or "STRUCT(...)"?
-        + "\n    ELSE                               DATA_TYPE "
-        + "\n  END                                    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 decimal digits that can appear (not the number of digits
-         * that can be counted on, and not the maximum number of characters
-         * needed to display a value).
-         */
-        + "\n  CASE DATA_TYPE "
-
-        // "For numeric data, ... the maximum precision":
-        //   TODO:  Change literals to references to declared constant fields:
-        // - exact numeric types:
-        //   (in decimal digits, coordinated with NUM_PREC_RADIX = 10)
-        + "\n    WHEN 'TINYINT'                      THEN " + PREC_TINYINT
-        + "\n    WHEN 'SMALLINT'                     THEN " + PREC_SMALLINT
-        + "\n    WHEN 'INTEGER'                      THEN " + PREC_INTEGER
-        + "\n    WHEN 'BIGINT'                       THEN " + PREC_BIGINT
-        + "\n    WHEN 'DECIMAL', 'NUMERIC'           THEN NUMERIC_PRECISION "
-        // - approximate numeric types:
-        //   (in decimal digits, coordinated with NUM_PREC_RADIX = 10)
-        // TODO:  REVISIT:  Should these be in bits or decimal digits (with
-        //   NUM_PREC_RADIX coordinated)?  INFORMATION_SCHEMA.COLUMNS's value
-        //   are supposed to be in bits (per the SQL spec.).  What does JDBC
-        //   require and allow?
-        + "\n    WHEN 'FLOAT'                        THEN " + PREC_FLOAT
-        + "\n    WHEN 'DOUBLE'                       THEN " + PREC_DOUBLE
-        + "\n    WHEN 'REAL'                         THEN " + PREC_REAL
-
-        // "For character data, ... the length in characters":
-        // TODO:  BUG:  DRILL-2459:  For CHARACTER / CHAR, length is not in
-        // CHARACTER_MAXIMUM_LENGTH but in NUMERIC_PRECISION.
-        // Workaround:
-        + "\n    WHEN 'VARCHAR', 'CHARACTER VARYING' "
-        + "\n                                    THEN CHARACTER_MAXIMUM_LENGTH "
-        + "\n    WHEN 'CHAR', 'CHARACTER', "
-        + "\n         'NCHAR', 'NATIONAL CHAR', 'NATIONAL CHARACTER' "
-        + "\n                                        THEN NUMERIC_PRECISION "
-
-        // "For datetime datatypes ... length ... String representation
-        // (assuming the maximum ... precision of ... fractional seconds ...)":
-        + "\n    WHEN 'DATE'            THEN 10 "              // YYYY-MM-DD
-        + "\n    WHEN 'TIME'            THEN "
-        + "\n      CASE "
-        + "\n        WHEN NUMERIC_PRECISION > 0 "              // HH:MM:SS.sss
-        + "\n                           THEN          8 + 1 + NUMERIC_PRECISION"
-        + "\n        ELSE                             8"       // HH:MM:SS
-        + "\n      END "
-        + "\n    WHEN 'TIMESTAMP'       THEN "
-        + "\n      CASE "                          // date + "T" + time (above)
-        + "\n        WHEN NUMERIC_PRECISION > 0 "
-        + "                             THEN 10 + 1 + 8 + 1 + NUMERIC_PRECISION"
-        + "\n        ELSE                    10 + 1 + 8"
-        + "\n      END "
-
-        // TODO:  DRILL-2531:  When DRILL-2519 is fixed, use start and end unit
-        // and start-unit precision to implement maximum width more precisely
-        // (narrowly) than this workaround:
-        // For INTERVAL_YEAR_MONTH, maximum width is from "P1234567890Y12M"
-        // (5 + apparent maximum start unit precision of 10)
-        // unit precision):
-        + "\n    WHEN 'INTERVAL_YEAR_MONTH' "
-        + "\n                                        THEN 5 + "
-                                                          + PREC_INTERVAL_LEAD_MAX
-        // For INTERVAL_DAY_TIME, maximum width is from
-        // "P1234567890D12H12M12.123456789S" (12 + apparent maximum start unit
-        // precision of 10 + apparent maximum seconds fractional precision of 9):
-        + "\n    WHEN 'INTERVAL_DAY_TIME' "
-        + "\n                                        THEN 12 + "
-                                                          + ( PREC_INTERVAL_LEAD_MAX
-                                                             + PREC_INTERVAL_TRAIL_MAX )
-
-        // "For binary data, ... the length in bytes":
-        // BUG:  DRILL-2459:  BINARY and BINARY VARYING / VARBINARY length is
-        // not in CHARACTER_MAXIMUM_LENGTH but in NUMERIC_PRECISION.
-        // Workaround:
-        + "\n    WHEN 'VARBINARY', 'BINARY VARYING', "
-        + "\n         'BINARY'                       THEN NUMERIC_PRECISION "
-
-        // "For ... ROWID datatype...": Not in Drill?
-
-        // "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', "
-        + "\n         'SMALLINT', "
-        + "\n         'INTEGER', "
-        + "\n         'BIGINT'                       THEN " + SCALE_INTEGRAL
-        + "\n    WHEN 'DECIMAL', "
-        + "\n         'NUMERIC'                      THEN NUMERIC_SCALE "
-        + "\n    WHEN 'FLOAT'                        THEN " + SCALE_FLOAT
-        + "\n    WHEN 'DOUBLE'                       THEN " + SCALE_DOUBLE
-        + "\n    WHEN 'REAL'                         THEN " + SCALE_REAL
-        + "\n    WHEN 'INTERVAL'                     THEN NUMERIC_SCALE "
-        + "\n    WHEN 'INTERVAL_YEAR_MONTH'          THEN 0 "
-        + "\n    WHEN 'INTERVAL_DAY_TIME'            THEN NUMERIC_SCALE "
-        + "\n  END                                    as  DECIMAL_DIGITS, "
-
-        /*   10                                           NUM_PREC_RADIX */
-        + "\n  CASE DATA_TYPE "
-        + "\n    WHEN 'TINYINT', "
-        + "\n         'SMALLINT', "
-        + "\n         'INTEGER', "
-        + "\n         'BIGINT'                       THEN " + RADIX_INTEGRAL
-        + "\n    WHEN 'DECIMAL', "
-        + "\n         'NUMERIC'                      THEN " + RADIX_DECIMAL
-        + "\n    WHEN 'FLOAT', "
-        + "\n         'DOUBLE', "
-        + "\n         'REAL'                         THEN " + RADIX_APPROXIMATE
-        + "\n    WHEN 'INTERVAL_YEAR_MONTH', "
-        + "\n         'INTERVAL_DAY_TIME'            THEN " + RADIX_INTERVAL
-        + "\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  CAST( NULL as VARCHAR )       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 'VARCHAR', 'CHARACTER VARYING' "
-        + "\n                                 THEN 4 * CHARACTER_MAXIMUM_LENGTH "
-        + "\n    WHEN 'CHAR', 'CHARACTER', "
-        + "\n         'NCHAR', 'NATIONAL CHAR', 'NATIONAL CHARACTER' "
-        // TODO:  BUG:  DRILL-2459:  For CHARACTER / CHAR, length is not in
-        // CHARACTER_MAXIMUM_LENGTH but in NUMERIC_PRECISION.  Workaround:
-        + "\n                                 THEN 4 * NUMERIC_PRECISION "
-        + "\n    ELSE                              NULL "
-        + "\n  END                                    as  CHAR_OCTET_LENGTH, "
-
-        + /* 17 */ "\n  1 + 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());
-  }
-
-  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());
-  }
-
-  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());
-  }
-
-  public ResultSet getTableTypes() {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getProcedures(String catalog, Pat schemaPattern, Pat procedureNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getProcedureColumns(String catalog, Pat schemaPattern, Pat procedureNamePattern,
-      Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getColumnPrivileges(String catalog, String schema, String table, Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getTablePrivileges(String catalog, Pat schemaPattern, Pat tableNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getBestRowIdentifier(String catalog, String schema, String table, int scope, boolean nullable) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getVersionColumns(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getPrimaryKeys(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getImportedKeys(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getExportedKeys(String catalog, String schema, String table) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTable,
-      String foreignCatalog, String foreignSchema, String foreignTable) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getTypeInfo() {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique, boolean approximate) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getUDTs(String catalog, Pat schemaPattern, Pat typeNamePattern, int[] types) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getSuperTypes(String catalog, Pat schemaPattern, Pat typeNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getSuperTables(String catalog, Pat schemaPattern, Pat tableNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getAttributes(String catalog, Pat schemaPattern, Pat typeNamePattern, Pat attributeNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getClientInfoProperties() {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getFunctions(String catalog, Pat schemaPattern, Pat functionNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getFunctionColumns(String catalog, Pat schemaPattern, Pat functionNamePattern, Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public ResultSet getPseudoColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
-    return getEmptyResultSet();
-  }
-
-  public Cursor createCursor(AvaticaResultSet resultSet_) {
-    return ((DrillResultSetImpl) resultSet_).cursor;
-  }
-
-  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/71082e6c/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
new file mode 100644
index 0000000..64f5b87
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/AvaticaDrillSqlAccessor.java
@@ -0,0 +1,250 @@
+/**
+ * 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.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.Map;
+
+import net.hydromatic.avatica.Cursor.Accessor;
+
+import org.apache.drill.exec.vector.accessor.SqlAccessor;
+import org.apache.drill.jdbc.InvalidCursorStateSqlException;
+
+
+// TODO:  Revisit adding null check for non-primitive types to SqlAccessor's
+// contract and classes generated by SqlAccessor template (DRILL-xxxx).
+
+class AvaticaDrillSqlAccessor implements Accessor {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AvaticaDrillSqlAccessor.class);
+
+  private final static byte PRIMITIVE_NUM_NULL_VALUE = 0;
+  private final static boolean BOOLEAN_NULL_VALUE = false;
+
+  private SqlAccessor underlyingAccessor;
+  private DrillCursor cursor;
+
+  AvaticaDrillSqlAccessor(SqlAccessor drillSqlAccessor, DrillCursor cursor) {
+    super();
+    this.underlyingAccessor = drillSqlAccessor;
+    this.cursor = cursor;
+  }
+
+  private int getCurrentRecordNumber() throws SQLException {
+    // WORKAROUND:  isBeforeFirst can't be called first here because AvaticaResultSet
+    // .next() doesn't increment its row field when cursor.next() returns false,
+    // so in that case row can be left at -1, so isBeforeFirst() returns true
+    // even though we're not longer before the empty set of rows--and it's all
+    // private, so we can't get to it to override any of several candidates.
+    if ( cursor.getResultSet().isAfterLast() ) {
+      throw new InvalidCursorStateSqlException(
+          "Result set cursor is already positioned past all rows." );
+    }
+    else if ( cursor.getResultSet().isBeforeFirst() ) {
+      throw new InvalidCursorStateSqlException(
+          "Result set cursor is positioned before all rows.  Call next() first." );
+    }
+    else {
+      return cursor.getCurrentRecordNumber();
+    }
+  }
+
+  @Override
+  public boolean wasNull() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber());
+  }
+
+  @Override
+  public String getString() throws SQLException {
+    return underlyingAccessor.getString(getCurrentRecordNumber());
+  }
+
+  @Override
+  public boolean getBoolean() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber())
+        ? BOOLEAN_NULL_VALUE
+        : underlyingAccessor.getBoolean(getCurrentRecordNumber());
+  }
+
+  @Override
+  public byte getByte() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber())
+        ? PRIMITIVE_NUM_NULL_VALUE
+        : underlyingAccessor.getByte(getCurrentRecordNumber());
+  }
+
+  @Override
+  public short getShort() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber())
+        ? PRIMITIVE_NUM_NULL_VALUE
+        : underlyingAccessor.getShort(getCurrentRecordNumber());
+  }
+
+  @Override
+  public int getInt() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber())
+        ? PRIMITIVE_NUM_NULL_VALUE
+        : underlyingAccessor.getInt(getCurrentRecordNumber());
+  }
+
+  @Override
+  public long getLong() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber())
+        ? PRIMITIVE_NUM_NULL_VALUE
+        : underlyingAccessor.getLong(getCurrentRecordNumber());
+  }
+
+  @Override
+  public float getFloat() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber())
+        ? PRIMITIVE_NUM_NULL_VALUE
+        : underlyingAccessor.getFloat(getCurrentRecordNumber());
+  }
+
+  @Override
+  public double getDouble() throws SQLException {
+    return underlyingAccessor.isNull(getCurrentRecordNumber())
+        ? PRIMITIVE_NUM_NULL_VALUE
+        : underlyingAccessor.getDouble(getCurrentRecordNumber());
+  }
+
+  @Override
+  public BigDecimal getBigDecimal() throws SQLException {
+    return underlyingAccessor.getBigDecimal(getCurrentRecordNumber());
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int scale) throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public byte[] getBytes() throws SQLException {
+    return underlyingAccessor.getBytes(getCurrentRecordNumber());
+  }
+
+  @Override
+  public InputStream getAsciiStream() throws SQLException {
+    return underlyingAccessor.getStream(getCurrentRecordNumber());
+  }
+
+  @Override
+  public InputStream getUnicodeStream() throws SQLException {
+    return underlyingAccessor.getStream(getCurrentRecordNumber());
+  }
+
+  @Override
+  public InputStream getBinaryStream() throws SQLException {
+    return underlyingAccessor.getStream(getCurrentRecordNumber());
+  }
+
+  @Override
+  public Object getObject() throws SQLException {
+    return underlyingAccessor.getObject(getCurrentRecordNumber());
+  }
+
+  @Override
+  public Reader getCharacterStream() throws SQLException {
+    return underlyingAccessor.getReader(getCurrentRecordNumber());
+  }
+
+  @Override
+  public Object getObject(Map<String, Class<?>> map) throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public Ref getRef() throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public Blob getBlob() throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public Clob getClob() throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public Array getArray() throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public Date getDate(Calendar calendar) throws SQLException {
+    return underlyingAccessor.getDate(getCurrentRecordNumber());
+  }
+
+  @Override
+  public Time getTime(Calendar calendar) throws SQLException {
+    return underlyingAccessor.getTime(getCurrentRecordNumber());
+  }
+
+  @Override
+  public Timestamp getTimestamp(Calendar calendar) throws SQLException {
+    return underlyingAccessor.getTimestamp(getCurrentRecordNumber());
+  }
+
+  @Override
+  public URL getURL() throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public NClob getNClob() throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public SQLXML getSQLXML() throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+  @Override
+  public String getNString() throws SQLException {
+    return underlyingAccessor.getString(getCurrentRecordNumber());
+  }
+
+  @Override
+  public Reader getNCharacterStream() throws SQLException {
+    return underlyingAccessor.getReader(getCurrentRecordNumber());
+  }
+
+  @Override
+  public <T> T getObject(Class<T> type) throws SQLException {
+    throw new SQLFeatureNotSupportedException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/BasicList.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/BasicList.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/BasicList.java
new file mode 100644
index 0000000..999d7e0
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/BasicList.java
@@ -0,0 +1,136 @@
+/**
+ * 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.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+
+/**
+ * Helper class to make simple changing lists for Avatica.  Want to make sure Avatica doesn't try any nonsense other than getting with the list.
+ */
+abstract class BasicList<T> implements List<T>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicList.class);
+
+
+  @Override
+  public boolean isEmpty() {
+    return size() == 0;
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Object[] toArray() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T> T[] toArray(T[] a) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean add(T e) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean remove(Object o) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean addAll(Collection<? extends T> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean addAll(int index, Collection<? extends T> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean removeAll(Collection<?> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean retainAll(Collection<?> c) {
+    throw new UnsupportedOperationException();
+  }
+
+
+  public void clear(){
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public T set(int index, T element) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void add(int index, T element) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public T remove(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int indexOf(Object o) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int lastIndexOf(Object o) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ListIterator<T> listIterator() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ListIterator<T> listIterator(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<T> subList(int fromIndex, int toIndex) {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/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
new file mode 100644
index 0000000..25ca1ba
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillAccessorList.java
@@ -0,0 +1,67 @@
+/**
+ * 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.SQLException;
+
+import net.hydromatic.avatica.Cursor.Accessor;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.BoundCheckingAccessor;
+import org.apache.drill.exec.vector.accessor.SqlAccessor;
+
+
+class DrillAccessorList extends BasicList<Accessor>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillAccessorList.class);
+
+  private Accessor[] accessors = new Accessor[0];
+  // TODO  Rename to lastColumnAccessed and/or document.
+  // TODO  Why 1, rather than, say, -1?
+  private int lastColumn = 1;
+
+  void generateAccessors(DrillCursor cursor, RecordBatchLoader currentBatch){
+    int cnt = currentBatch.getSchema().getFieldCount();
+    accessors = new Accessor[cnt];
+    for(int i =0; i < cnt; i++){
+      final ValueVector vector = currentBatch.getValueAccessorById(null, i).getValueVector();
+      final SqlAccessor acc =
+          new TypeConvertingSqlAccessor(
+              new BoundCheckingAccessor(vector, TypeHelper.getSqlAccessor(vector))
+              );
+      accessors[i] = new AvaticaDrillSqlAccessor(acc, cursor);
+    }
+  }
+
+  @Override
+  public Accessor get(int index) {
+    lastColumn = index;
+    return accessors[index];
+  }
+
+  boolean wasNull() throws SQLException{
+    return accessors[lastColumn].wasNull();
+  }
+
+  @Override
+  public int size() {
+    return accessors.length;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/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
new file mode 100644
index 0000000..d43755e
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillColumnMetaDataList.java
@@ -0,0 +1,218 @@
+/**
+ * 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.ResultSetMetaData;
+import java.util.ArrayList;
+import java.util.Collection;
+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.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+
+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>();
+
+  @Override
+  public int size() {
+    return (columns.size());
+  }
+
+  @Override
+  public ColumnMetaData get(int index) {
+    return columns.get(index);
+  }
+
+  public void updateColumnMetaData(String catalogName, String schemaName, String tableName, BatchSchema schema){
+
+    columns = new ArrayList<ColumnMetaData>(schema.getFieldCount());
+    for(int i = 0; i < schema.getFieldCount(); i++){
+      MaterializedField f = schema.getColumn(i);
+      MajorType t = f.getType();
+      ColumnMetaData col = new ColumnMetaData( //
+          i, // ordinal
+          false, // autoIncrement
+          true, // caseSensitive
+          false, // searchable
+          false, // currency
+          f.getDataMode() == DataMode.OPTIONAL ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls, //nullability
+          !Types.isUnSigned(t), // signed
+          10, // display size.
+          f.getAsSchemaPath().getRootSegment().getPath(), // label
+          f.getAsSchemaPath().getRootSegment().getPath(), // columnname
+          schemaName, // schemaname
+          t.hasPrecision() ? t.getPrecision() : 0, // precision
+          t.hasScale() ? t.getScale() : 0, // scale
+          null, // tablename is null so sqlline doesn't try to retrieve primary keys.
+          catalogName, // catalogname
+          getAvaticaType(t),  // sql type
+          true, // readonly
+          false, // writable
+          false, // definitely writable
+          "none" // column class name
+         );
+      columns.add(col);
+    }
+  }
+
+  private static AvaticaType getAvaticaType(MajorType t){
+    final int jdbcTypeId = Types.getJdbcType(t);
+    return ColumnMetaData.scalar(jdbcTypeId, getJdbcTypeName(jdbcTypeId), Rep.BOOLEAN /* dummy value, unused */);
+  }
+
+  private static String getJdbcTypeName(int type) {
+    switch (type) {
+    case java.sql.Types.BIT:
+        return "BIT";
+    case java.sql.Types.TINYINT:
+        return "TINYINT";
+    case java.sql.Types.SMALLINT:
+        return "SMALLINT";
+    case java.sql.Types.INTEGER:
+        return "INTEGER";
+    case java.sql.Types.BIGINT:
+        return "BIGINT";
+    case java.sql.Types.FLOAT:
+        return "FLOAT";
+    case java.sql.Types.REAL:
+        return "REAL";
+    case java.sql.Types.DOUBLE:
+        return "DOUBLE";
+    case java.sql.Types.NUMERIC:
+        return "NUMERIC";
+    case java.sql.Types.DECIMAL:
+        return "DECIMAL";
+    case java.sql.Types.CHAR:
+        return "CHAR";
+    case java.sql.Types.VARCHAR:
+        return "VARCHAR";
+    case java.sql.Types.LONGVARCHAR:
+        return "LONGVARCHAR";
+    case java.sql.Types.DATE:
+        return "DATE";
+    case java.sql.Types.TIME:
+        return "TIME";
+    case java.sql.Types.TIMESTAMP:
+        return "TIMESTAMP";
+    case java.sql.Types.BINARY:
+        return "BINARY";
+    case java.sql.Types.VARBINARY:
+        return "VARBINARY";
+    case java.sql.Types.LONGVARBINARY:
+        return "LONGVARBINARY";
+    case java.sql.Types.NULL:
+        return "NULL";
+    case java.sql.Types.OTHER:
+        return "OTHER";
+    case java.sql.Types.JAVA_OBJECT:
+        return "JAVA_OBJECT";
+    case java.sql.Types.DISTINCT:
+        return "DISTINCT";
+    case java.sql.Types.STRUCT:
+        return "STRUCT";
+    case java.sql.Types.ARRAY:
+        return "ARRAY";
+    case java.sql.Types.BLOB:
+        return "BLOB";
+    case java.sql.Types.CLOB:
+        return "CLOB";
+    case java.sql.Types.REF:
+        return "REF";
+    case java.sql.Types.DATALINK:
+        return "DATALINK";
+    case java.sql.Types.BOOLEAN:
+        return "BOOLEAN";
+    case java.sql.Types.ROWID:
+        return "ROWID";
+    case java.sql.Types.NCHAR:
+        return "NCHAR";
+    case java.sql.Types.NVARCHAR:
+        return "NVARCHAR";
+    case java.sql.Types.LONGNVARCHAR:
+        return "LONGNVARCHAR";
+    case java.sql.Types.NCLOB:
+        return "NCLOB";
+    case java.sql.Types.SQLXML:
+        return "SQLXML";
+    default:
+        logger.error( "Unexpected java.sql.Types value {}", type );
+        return "unknown java.sql.Types value " + type;
+    }
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    return columns.contains(o);
+  }
+
+  @Override
+  public Iterator<ColumnMetaData> iterator() {
+    return columns.iterator();
+  }
+
+  @Override
+  public Object[] toArray() {
+    return columns.toArray();
+  }
+
+  @Override
+  public <T> T[] toArray(T[] a) {
+    return columns.toArray(a);
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> c) {
+    return columns.containsAll(c);
+  }
+
+  @Override
+  public int indexOf(Object o) {
+    return columns.indexOf(o);
+  }
+
+  @Override
+  public int lastIndexOf(Object o) {
+    return columns.lastIndexOf(o);
+  }
+
+  @Override
+  public ListIterator<ColumnMetaData> listIterator() {
+    return columns.listIterator();
+  }
+
+  @Override
+  public ListIterator<ColumnMetaData> listIterator(int index) {
+    return columns.listIterator(index);
+  }
+
+  @Override
+  public List<ColumnMetaData> subList(int fromIndex, int toIndex) {
+    return columns.subList(fromIndex, toIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/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
new file mode 100644
index 0000000..243e627
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
@@ -0,0 +1,414 @@
+/**
+ * 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.io.IOException;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.Savepoint;
+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.Helper;
+import net.hydromatic.avatica.Meta;
+import net.hydromatic.avatica.UnregisteredDriver;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.util.TestUtilities;
+import org.apache.drill.jdbc.AlreadyClosedSqlException;
+import org.apache.drill.jdbc.DrillConnection;
+import org.apache.drill.jdbc.DrillConnectionConfig;
+import org.apache.drill.jdbc.InvalidParameterSqlException;
+import org.apache.drill.jdbc.JdbcApiSqlException;
+
+
+/**
+ * Drill's implementation of {@link Connection}.
+ */
+// (Was abstract to avoid errors _here_ if newer versions of JDBC added
+// interface methods, but now newer versions would probably use Java 8's default
+// methods for compatibility.)
+class DrillConnectionImpl extends AvaticaConnection
+                                   implements DrillConnection {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConnection.class);
+
+  final DrillStatementRegistry openStatementsRegistry = new DrillStatementRegistry();
+  final DrillConnectionConfig config;
+
+  private final DrillClient client;
+  private final BufferAllocator allocator;
+  private Drillbit bit;
+  private RemoteServiceSet serviceSet;
+
+  protected DrillConnectionImpl(DriverImpl driver, AvaticaFactory factory, String url, Properties info) throws SQLException {
+    super(driver, factory, url, info);
+
+    // Initialize transaction-related settings per Drill behavior.
+    super.setTransactionIsolation( TRANSACTION_NONE );
+    super.setAutoCommit( true );
+
+    this.config = new DrillConnectionConfig(info);
+
+    try {
+      if (config.isLocal()) {
+        try {
+          Class.forName("org.eclipse.jetty.server.Handler");
+        } catch (final ClassNotFoundException e) {
+          throw new SQLNonTransientConnectionException(
+              "Running Drill in embedded mode using Drill's jdbc-all JDBC"
+              + " driver Jar file alone is not supported.",  e);
+        }
+
+        final DrillConfig dConfig = DrillConfig.create(info);
+        this.allocator = new TopLevelAllocator(dConfig);
+        RemoteServiceSet set = GlobalServiceSetReference.SETS.get();
+        if (set == null) {
+          // We're embedded; start a local drill bit.
+          serviceSet = RemoteServiceSet.getLocalServiceSet();
+          set = serviceSet;
+          try {
+            bit = new Drillbit(dConfig, serviceSet);
+            bit.run();
+          } catch (final UserException e) {
+            throw new SQLException(
+                "Failure in starting embedded Drillbit: " + e.getMessage(),
+                e);
+          } catch (Exception e) {
+            // (Include cause exception's text in wrapping exception's text so
+            // it's more likely to get to user (e.g., via SQLLine), and use
+            // toString() since getMessage() text doesn't always mention error:)
+            throw new SQLException("Failure in starting embedded Drillbit: " + e, e);
+          }
+        } else {
+          serviceSet = null;
+          bit = null;
+        }
+
+        makeTmpSchemaLocationsUnique(bit.getContext().getStorage(), info);
+
+        this.client = new DrillClient(dConfig, set.getCoordinator());
+        this.client.connect(null, info);
+      } else if(config.isDirect()) {
+        final DrillConfig dConfig = DrillConfig.forClient();
+        this.allocator = new TopLevelAllocator(dConfig);
+        this.client = new DrillClient(dConfig, true); // Get a direct connection
+        this.client.connect(config.getZookeeperConnectionString(), info);
+      } else {
+        final DrillConfig dConfig = DrillConfig.forClient();
+        this.allocator = new TopLevelAllocator(dConfig);
+        // TODO:  Check:  Why does new DrillClient() create another DrillConfig,
+        // with enableServerConfigs true, and cause scanning for function
+        // implementations (needed by a server, but not by a client-only
+        // process, right?)?  Probably pass dConfig to construction.
+        this.client = new DrillClient();
+        this.client.connect(config.getZookeeperConnectionString(), info);
+      }
+    } catch (RpcException e) {
+      // (Include cause exception's text in wrapping exception's text so
+      // it's more likely to get to user (e.g., via SQLLine), and use
+      // toString() since getMessage() text doesn't always mention error:)
+      throw new SQLException("Failure in connecting to Drill: " + e, e);
+    }
+  }
+
+  /**
+   * Throws AlreadyClosedSqlException <i>iff</i> this Connection is closed.
+   *
+   * @throws  AlreadyClosedSqlException  if Connection is closed   */
+  private void checkNotClosed() throws AlreadyClosedSqlException {
+    if ( isClosed() ) {
+      throw new AlreadyClosedSqlException( "Connection is already closed." );
+    }
+  }
+
+  @Override
+  public DrillConnectionConfig getConfig() {
+    return config;
+  }
+
+  @Override
+  protected Meta createMeta() {
+    return new MetaImpl(this);
+  }
+
+  MetaImpl meta() {
+    return (MetaImpl) meta;
+  }
+
+  BufferAllocator getAllocator() {
+    return allocator;
+  }
+
+  @Override
+  public DrillClient getClient() {
+    return client;
+  }
+
+  @Override
+  public void setAutoCommit( boolean autoCommit ) throws SQLException {
+    checkNotClosed();
+    if ( ! autoCommit ) {
+      throw new SQLFeatureNotSupportedException(
+          "Can't turn off auto-committing; transactions are not supported.  "
+          + "(Drill is not transactional.)" );
+    }
+    assert getAutoCommit() : "getAutoCommit() = " + getAutoCommit();
+  }
+
+  @Override
+  public void commit() throws SQLException {
+    checkNotClosed();
+    if ( getAutoCommit() ) {
+      throw new JdbcApiSqlException( "Can't call commit() in auto-commit mode." );
+    }
+    else {
+      // (Currently not reachable.)
+      throw new SQLFeatureNotSupportedException(
+          "Connection.commit() is not supported.  (Drill is not transactional.)" );
+    }
+  }
+
+  @Override
+  public void rollback() throws SQLException {
+    checkNotClosed();
+    if ( getAutoCommit()  ) {
+      throw new JdbcApiSqlException( "Can't call rollback() in auto-commit mode." );
+    }
+    else {
+      // (Currently not reachable.)
+      throw new SQLFeatureNotSupportedException(
+          "Connection.rollback() is not supported.  (Drill is not transactional.)" );
+    }
+  }
+
+
+  @Override
+  public boolean isClosed() {
+    try {
+      return super.isClosed();
+    }
+    catch ( SQLException e ) {
+      // Currently can't happen, since AvaticaConnection.isClosed() never throws
+      // SQLException.
+      throw new DrillRuntimeException(
+          "Unexpected exception from " + getClass().getSuperclass()
+          + ".isClosed(): " + e,
+          e );
+    }
+  }
+
+
+  @Override
+  public Savepoint setSavepoint() throws SQLException {
+    checkNotClosed();
+    throw new SQLFeatureNotSupportedException(
+        "Savepoints are not supported.  (Drill is not transactional.)" );
+  }
+
+  @Override
+  public Savepoint setSavepoint(String name) throws SQLException {
+    checkNotClosed();
+    throw new SQLFeatureNotSupportedException(
+        "Savepoints are not supported.  (Drill is not transactional.)" );
+  }
+
+  @Override
+    public void rollback(Savepoint savepoint) throws SQLException {
+    checkNotClosed();
+    throw new SQLFeatureNotSupportedException(
+        "Savepoints are not supported.  (Drill is not transactional.)" );
+  }
+
+  @Override
+  public void releaseSavepoint(Savepoint savepoint) throws SQLException {
+    checkNotClosed();
+    throw new SQLFeatureNotSupportedException(
+        "Savepoints are not supported.  (Drill is not transactional.)" );
+  }
+
+
+  private String isolationValueToString( final int level ) {
+    switch ( level ) {
+      case TRANSACTION_NONE:             return "TRANSACTION_NONE";
+      case TRANSACTION_READ_UNCOMMITTED: return "TRANSACTION_READ_UNCOMMITTED";
+      case TRANSACTION_READ_COMMITTED:   return "TRANSACTION_READ_COMMITTED";
+      case TRANSACTION_REPEATABLE_READ:  return "TRANSACTION_REPEATABLE_READ";
+      case TRANSACTION_SERIALIZABLE:     return "TRANSACTION_SERIALIZABLE";
+      default:
+        return "<Unknown transaction isolation level value " + level + ">";
+    }
+  }
+
+  @Override
+  public void setTransactionIsolation(int level) throws SQLException {
+    checkNotClosed();
+    switch ( level ) {
+      case TRANSACTION_NONE:
+        // No-op.  (Is already set in constructor, and we disallow changing it.)
+        break;
+      case TRANSACTION_READ_UNCOMMITTED:
+      case TRANSACTION_READ_COMMITTED:
+      case TRANSACTION_REPEATABLE_READ:
+      case TRANSACTION_SERIALIZABLE:
+          throw new SQLFeatureNotSupportedException(
+              "Can't change transaction isolation level to Connection."
+              + isolationValueToString( level ) + " (from Connection."
+              + isolationValueToString( getTransactionIsolation() ) + ")."
+              + "  (Drill is not transactional.)" );
+      default:
+        // Invalid value (or new one unknown to code).
+        throw new JdbcApiSqlException(
+            "Invalid transaction isolation level value " + level );
+        //break;
+    }
+  }
+
+  @Override
+  public void setNetworkTimeout( Executor executor, int milliseconds )
+      throws AlreadyClosedSqlException,
+             JdbcApiSqlException,
+             SQLFeatureNotSupportedException {
+    checkNotClosed();
+    if ( null == executor ) {
+      throw new InvalidParameterSqlException(
+          "Invalid (null) \"executor\" parameter to setNetworkTimeout(...)" );
+    }
+    else if ( milliseconds < 0 ) {
+      throw new InvalidParameterSqlException(
+          "Invalid (negative) \"milliseconds\" parameter to setNetworkTimeout(...)"
+          + " (" + milliseconds + ")" );
+    }
+    else {
+      if ( 0 != milliseconds ) {
+        throw new SQLFeatureNotSupportedException(
+            "Setting network timeout is not supported." );
+      }
+    }
+  }
+
+
+  @Override
+  public int getNetworkTimeout() throws AlreadyClosedSqlException
+  {
+    checkNotClosed();
+    return 0;  // (No no timeout.)
+  }
+
+
+  @Override
+  public DrillStatementImpl createStatement(int resultSetType, int resultSetConcurrency,
+                                            int resultSetHoldability) throws SQLException {
+    checkNotClosed();
+    DrillStatementImpl statement =
+        (DrillStatementImpl) super.createStatement(resultSetType, resultSetConcurrency,
+                                                   resultSetHoldability);
+    return statement;
+  }
+
+  @Override
+  public PreparedStatement prepareStatement(String sql, int resultSetType,
+                                            int resultSetConcurrency,
+                                            int resultSetHoldability) throws SQLException {
+    checkNotClosed();
+    try {
+      DrillPrepareResult prepareResult = new DrillPrepareResult(sql);
+      DrillPreparedStatementImpl statement =
+          (DrillPreparedStatementImpl) factory.newPreparedStatement(
+              this, prepareResult, resultSetType, resultSetConcurrency,
+              resultSetHoldability);
+      return statement;
+    } 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 TimeZone getTimeZone() {
+    return config.getTimeZone();
+  }
+
+  // do not make public
+  UnregisteredDriver getDriver() {
+    return driver;
+  }
+
+  // do not make public
+  AvaticaFactory getFactory() {
+    return factory;
+  }
+
+  void cleanup() {
+    // First close any open JDBC Statement objects, to close any open ResultSet
+    // objects and release their buffers/vectors.
+    openStatementsRegistry.close();
+
+    client.close();
+    allocator.close();
+    if (bit != null) {
+      bit.close();
+    }
+
+    if (serviceSet != null) {
+      try {
+        serviceSet.close();
+      } catch (IOException e) {
+        logger.warn("Exception while closing service set.", e);
+      }
+    }
+  }
+
+  /**
+   * Test only code to make JDBC tests run concurrently. If the property <i>drillJDBCUnitTests</i> is set to
+   * <i>true</i> in connection properties:
+   *   - Update dfs_test.tmp workspace location with a temp directory. This temp is for exclusive use for test jvm.
+   *   - Update dfs.tmp workspace to immutable, so that test writer don't try to create views in dfs.tmp
+   * @param pluginRegistry
+   */
+  private static void makeTmpSchemaLocationsUnique(StoragePluginRegistry pluginRegistry, Properties props) {
+    try {
+      if (props != null && "true".equalsIgnoreCase(props.getProperty("drillJDBCUnitTests"))) {
+        final String tmpDirPath = TestUtilities.createTempDir();
+        TestUtilities.updateDfsTestTmpSchemaLocation(pluginRegistry, tmpDirPath);
+        TestUtilities.makeDfsTmpSchemaImmutable(pluginRegistry);
+      }
+    } catch(Throwable e) {
+      // Reason for catching Throwable is to capture NoSuchMethodError etc which depend on certain classed to be
+      // present in classpath which may not be available when just using the standalone JDBC. This is unlikely to
+      // happen, but just a safeguard to avoid failing user applications.
+      logger.warn("Failed to update tmp schema locations. This step is purely for testing purpose. " +
+          "Shouldn't be seen in production code.");
+      // Ignore the error and go with defaults
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/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
new file mode 100644
index 0000000..13a1d95
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -0,0 +1,213 @@
+/**
+ * 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.SQLException;
+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 org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
+
+
+class DrillCursor implements Cursor {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillCursor.class);
+
+  private static final String UNKNOWN = "--UNKNOWN--";
+
+  /** The associated java.sql.ResultSet implementation. */
+  private final DrillResultSetImpl resultSet;
+
+  private final RecordBatchLoader currentBatch;
+  private final DrillResultSetImpl.ResultsListener resultsListener;
+
+  // TODO:  Doc.:  Say what's started (set of rows?  just current result batch?)
+  private boolean started = false;
+  private boolean finished = false;
+  // TODO:  Doc.: Say what "readFirstNext" means.
+  private boolean redoFirstNext = false;
+  // TODO:  Doc.: First what? (First batch? record? "next" call/operation?)
+  private boolean first = true;
+
+  private DrillColumnMetaDataList columnMetaDataList;
+  private BatchSchema schema;
+
+  /** Zero-based index of current record in record batch. */
+  private int currentRecordNumber = -1;
+  private long recordBatchCount;
+  private final DrillAccessorList accessors = new DrillAccessorList();
+
+
+  /**
+   *
+   * @param  resultSet  the associated ResultSet implementation
+   */
+  DrillCursor(final DrillResultSetImpl resultSet) {
+    this.resultSet = resultSet;
+    currentBatch = resultSet.currentBatch;
+    resultsListener = resultSet.resultsListener;
+  }
+
+  DrillResultSetImpl getResultSet() {
+    return resultSet;
+  }
+
+  protected int getCurrentRecordNumber() {
+    return currentRecordNumber;
+  }
+
+  @Override
+  public List<Accessor> createAccessors(List<ColumnMetaData> types, Calendar localCalendar, Factory factory) {
+    columnMetaDataList = (DrillColumnMetaDataList) types;
+    return accessors;
+  }
+
+  // TODO:  Doc.:  Specify what the return value actually means.  (The wording
+  // "Moves to the next row" and "Whether moved" from the documentation of the
+  // implemented interface (net.hydromatic.avatica.Cursor) doesn't address
+  // moving past last row or how to evaluate "whether moved" on the first call.
+  // In particular, document what the return value indicates about whether we're
+  // currently at a valid row (or whether next() can be called again, or
+  // whatever it does indicate), especially the first time this next() called
+  // for a new result.
+  @Override
+  public boolean next() throws SQLException {
+    if (!started) {
+      started = true;
+      redoFirstNext = true;
+    } else if (redoFirstNext && !finished) {
+      redoFirstNext = false;
+      return true;
+    }
+
+    if (finished) {
+      return false;
+    }
+
+    if (currentRecordNumber + 1 < currentBatch.getRecordCount()) {
+      // Next index is in within current batch--just increment to that record.
+      currentRecordNumber++;
+      return true;
+    } else {
+      // Next index is not in current batch (including initial empty batch--
+      // (try to) get next batch.
+      try {
+        QueryDataBatch qrb = resultsListener.getNext();
+        recordBatchCount++;
+        while (qrb != null && (qrb.getHeader().getRowCount() == 0 || qrb.getData() == null ) && !first) {
+          qrb.release();
+          qrb = resultsListener.getNext();
+          recordBatchCount++;
+          if(qrb != null && qrb.getData()==null){
+            qrb.release();
+            return false;
+          }
+        }
+
+        first = false;
+
+        if (qrb == null) {
+          currentBatch.clear();
+          finished = true;
+          return false;
+        } else {
+          currentRecordNumber = 0;
+          final boolean changed;
+          try {
+            changed = currentBatch.load(qrb.getHeader().getDef(), qrb.getData());
+          }
+          finally {
+            qrb.release();
+          }
+          schema = currentBatch.getSchema();
+          if (changed) {
+            updateColumns();
+          }
+          if (redoFirstNext && currentBatch.getRecordCount() == 0) {
+            redoFirstNext = false;
+          }
+          return true;
+        }
+      }
+      catch ( UserException e ) {
+        // A normally expected case--for any server-side error (e.g., syntax
+        // error in SQL statement).
+        // Construct SQLException with message text from the UserException.
+        // TODO:  Map UserException error type to SQLException subclass (once
+        // error type is accessible, of course. :-( )
+        throw new SQLException( e.getMessage(), e );
+      }
+      catch ( InterruptedException e ) {
+        // Not normally expected--Drill doesn't interrupt in this area (right?)--
+        // but JDBC client certainly could.
+        throw new SQLException( "Interrupted.", e );
+      }
+      catch ( SchemaChangeException e ) {
+        // TODO:  Clean:  DRILL-2933:  RecordBatchLoader.load(...) no longer
+        // throws SchemaChangeException, so check/clean catch clause.
+        throw new SQLException(
+            "Unexpected SchemaChangeException from RecordBatchLoader.load(...)" );
+      }
+      catch ( RuntimeException e ) {
+        throw new SQLException( "Unexpected RuntimeException: " + e.toString(), e );
+      }
+
+    }
+  }
+
+  void updateColumns() {
+    accessors.generateAccessors(this, currentBatch);
+    columnMetaDataList.updateColumnMetaData(UNKNOWN, UNKNOWN, UNKNOWN, schema);
+    if (getResultSet().changeListener != null) {
+      getResultSet().changeListener.schemaChanged(schema);
+    }
+  }
+
+  public long getRecordBatchCount() {
+    return recordBatchCount;
+  }
+
+  @Override
+  public void close() {
+    // currentBatch is owned by resultSet and cleaned up by
+    // DrillResultSet.cleanup()
+
+    // listener is owned by resultSet and cleaned up by
+    // DrillResultSet.cleanup()
+
+    // Clean up result set (to deallocate any buffers).
+    getResultSet().cleanup();
+    // TODO:  CHECK:  Something might need to set statement.openResultSet to
+    // null.  Also, AvaticaResultSet.close() doesn't check whether already
+    // closed and skip calls to cursor.close(), statement.onResultSetClose()
+  }
+
+  @Override
+  public boolean wasNull() throws SQLException {
+    return accessors.wasNull();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaData.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaData.java
new file mode 100644
index 0000000..c975f73
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaData.java
@@ -0,0 +1,118 @@
+/**
+ * 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.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.drill.jdbc.AlreadyClosedSqlException;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaDatabaseMetaData;
+
+
+/**
+ * Drill's implementation of {@link DatabaseMetaData}.
+ */
+class DrillDatabaseMetaData extends AvaticaDatabaseMetaData {
+
+  protected DrillDatabaseMetaData( AvaticaConnection connection ) {
+    super( connection );
+  }
+
+  /**
+   * Throws AlreadyClosedSqlException if the associated Connection is closed.
+   *
+   * @throws AlreadyClosedSqlException if Connection is closed
+   * @throws SQLException if error in calling {@link Connection#isClosed()}
+   */
+  private void checkNotClosed() throws AlreadyClosedSqlException,
+                                       SQLException {
+    if ( getConnection().isClosed() ) {
+      throw new AlreadyClosedSqlException(
+          "DatabaseMetaData's Connection is already closed." );
+    }
+  }
+
+
+  // For omitted NULLS FIRST/NULLS HIGH, Drill sort NULL sorts as highest value:
+
+  @Override
+  public boolean nullsAreSortedHigh() throws SQLException {
+    checkNotClosed();
+    return true;
+  }
+
+  @Override
+  public boolean nullsAreSortedLow() throws SQLException {
+    checkNotClosed();
+    return false;
+  }
+
+  @Override
+  public boolean nullsAreSortedAtStart() throws SQLException {
+    checkNotClosed();
+    return false;
+  }
+
+  @Override
+  public boolean nullsAreSortedAtEnd() throws SQLException {
+    checkNotClosed();
+    return false;
+  }
+
+
+  // For now, check whether connection is closed for most important methods
+  // (DRILL-2565 (partial fix for DRILL-2489)):
+
+
+  @Override
+  public ResultSet getCatalogs() throws SQLException {
+    checkNotClosed();
+    return super.getCatalogs();
+  }
+
+  @Override
+  public ResultSet getSchemas() throws SQLException {
+    checkNotClosed();
+    return super.getSchemas();
+  }
+
+  @Override
+  public ResultSet getSchemas( String catalog, String schemaPattern ) throws SQLException {
+    checkNotClosed();
+    return super.getSchemas( catalog, schemaPattern );
+  }
+
+  @Override
+  public ResultSet getTables( String catalog,
+                              String schemaPattern,
+                              String tableNamePattern,
+                              String[] types ) throws SQLException {
+    checkNotClosed();
+    return super.getTables( catalog, schemaPattern,tableNamePattern, types );
+  }
+
+  @Override
+  public ResultSet getColumns( String catalog, String schema, String table,
+                               String columnNamePattern ) throws SQLException {
+    checkNotClosed();
+    return super.getColumns( catalog, schema, table, columnNamePattern );
+  }
+
+}