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:33 UTC

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

DRILL-2089: Split JDBC implementation out of org.apache.drill.jdbc. (Combined patch.)

**********
DRILL-2089: Part 1--Pre-move preparation and code hygiene.

Preparation for moving classes:
- Added missed "public" to JdbcApiSqlException and InvalidCursorStateSqlException.

Code hygiene:
- Fixed commented-out constructors.  [AlreadyClosedSqlException]
- Fixed missed re-alignment.  [DrillConnectionImpl]
- Fixed basic whitespace.  [Driver]
- Fixed/augmented/added various doc. comments.
- Added @Override annotations.  [MetaImpl, DrillJdbc41Factory]
- Purged an unused import.  [DrillJdbc41Factory]

**********
DRILL-2089: Part 2--Split Driver into published Driver and internal DriverImpl.

Split original Driver class into new Driver and DriverImpl classes (to remove
implementation details from JDBC's published interface (i.e., extension of
Avatica UnregisteredDriver)):

- Moved most of original Driver to new DriverImpl.  (Excluded registration code.)

- Created new Driver class:
  - implementing java.sql.Driver (not extending Avatica UnregisteredDriver)
  - providing java.sql.Driver methods via delegation to a private DriverImpl
    instance (not via extension)
  - having driver registration code

- Updated internal references to Driver to DriverImpl.  [DrillConnectionImpl,
  DrillFactory, and DrillJdbc41Factory]

2089:  MOVED static initialization up.  [Driver]

**********
DRILL-2089: Part 3--Move files.  (Move _only_--no edits (e.g., package renaming).)

Moved files from exec/jdbc/src/main/java/org/apache/drill/jdbc/
              to exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/:
- AvaticaDrillSqlAccessor.java
- BasicList.java
- DrillAccessorList.java
- DrillColumnMetaDataList.java
- DrillConnectionImpl.java
- DrillCursor.java
- DrillDatabaseMetaData.java

- DrillFactory.java
- DrillHandler.java
- DrillJdbc40Factory.java
- DrillJdbc41Factory.java
- DrillPrepareResult.java
- DrillPreparedStatement.java
- DrillRemoteStatement.java
- DrillStatementRegistry.java
- DriverImpl.java
- GlobalServiceSetReference.java
- MetaImpl.java

**********
DRILL-2089: Part 4--Update package names of moved classes.

Updated package names of moved classes:
- In own package declaration.
- In references in imports.
- In references in string literals.  [DriverImpl]
- In ProGuard configuration for JDBC-all Jar file.
  - Also added DrillConnection and DrillStatement to parallel recent addition
    of DrillResultSet.

**********
DRILL-2089: Part 5--Reduce excess visibility (remove (now-)unneeded "public").

Purged interim "public" from earlier partial DRILL-2089 fixes.
Purged other "public" qualifiers no longer needed.

**********
DRILL-2089: Part 6--Rename DrillPreparedStatement to ...Impl. [various]

**********
DRILL-2089: Part 7--Post-move hygiene (simplification, renaming, whitespace).

Fixed "cleanup()" to "cleanUp()".  [DrillRemoteStatement,
DrillPreparedStatement, DrillStatementImpl, DrillHandler]

Renamed DrillConnection.config() to more conventional getConfig().
[DrillConnection, DrillConnectionImpl]

Removed now-unneeded "abstract" and eliminated now-unneeded subclasses.
[DrillConnectionImpl, DrillStatementImpl, DrillJdbc41Factory]

Simplified boolean Driver.load() to void.  [Driver]
- Switched from "new Driver()" to "Driver.load()" in one case.  [ConnectionTest]

Fixed "Optiq" -> "Drill", cleaned up createDriverVersion.  [DriverImpl]

Adjusted unknown-case handling, logging.  [DrillColumnMetaDataList]

Wrapped lines; adjusted other whitespace.


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

Branch: refs/heads/master
Commit: 71082e6cddec098d44db498ccb7ee41b6632eb6f
Parents: 4baf9e7
Author: dbarclay <db...@maprtech.com>
Authored: Wed Apr 22 11:41:18 2015 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Wed Jun 17 11:45:46 2015 -0700

----------------------------------------------------------------------
 exec/jdbc-all/pom.xml                           |   8 +-
 .../drill/jdbc/AlreadyClosedSqlException.java   |   4 +-
 .../drill/jdbc/AvaticaDrillSqlAccessor.java     | 248 -------
 .../java/org/apache/drill/jdbc/BasicList.java   | 136 ----
 .../apache/drill/jdbc/DrillAccessorList.java    |  67 --
 .../drill/jdbc/DrillColumnMetaDataList.java     | 217 ------
 .../org/apache/drill/jdbc/DrillConnection.java  |   2 +-
 .../apache/drill/jdbc/DrillConnectionImpl.java  | 413 ------------
 .../java/org/apache/drill/jdbc/DrillCursor.java | 213 ------
 .../drill/jdbc/DrillDatabaseMetaData.java       | 112 ----
 .../org/apache/drill/jdbc/DrillFactory.java     |  60 --
 .../org/apache/drill/jdbc/DrillHandler.java     |  47 --
 .../apache/drill/jdbc/DrillJdbc40Factory.java   |  29 -
 .../apache/drill/jdbc/DrillJdbc41Factory.java   | 201 ------
 .../apache/drill/jdbc/DrillPrepareResult.java   |  52 --
 .../drill/jdbc/DrillPreparedStatement.java      |  52 --
 .../apache/drill/jdbc/DrillRemoteStatement.java |  22 -
 .../org/apache/drill/jdbc/DrillResultSet.java   |   5 +
 .../org/apache/drill/jdbc/DrillStatement.java   |   1 +
 .../drill/jdbc/DrillStatementRegistry.java      |  78 ---
 .../main/java/org/apache/drill/jdbc/Driver.java | 103 ++-
 .../drill/jdbc/GlobalServiceSetReference.java   |  29 -
 .../jdbc/InvalidCursorStateSqlException.java    |   2 +-
 .../apache/drill/jdbc/JdbcApiSqlException.java  |   2 +-
 .../java/org/apache/drill/jdbc/MetaImpl.java    | 634 ------------------
 .../jdbc/impl/AvaticaDrillSqlAccessor.java      | 250 +++++++
 .../org/apache/drill/jdbc/impl/BasicList.java   | 136 ++++
 .../drill/jdbc/impl/DrillAccessorList.java      |  67 ++
 .../jdbc/impl/DrillColumnMetaDataList.java      | 218 ++++++
 .../drill/jdbc/impl/DrillConnectionImpl.java    | 414 ++++++++++++
 .../org/apache/drill/jdbc/impl/DrillCursor.java | 213 ++++++
 .../drill/jdbc/impl/DrillDatabaseMetaData.java  | 118 ++++
 .../apache/drill/jdbc/impl/DrillFactory.java    |  79 +++
 .../apache/drill/jdbc/impl/DrillHandler.java    |  48 ++
 .../drill/jdbc/impl/DrillJdbc40Factory.java     |  34 +
 .../drill/jdbc/impl/DrillJdbc41Factory.java     | 226 +++++++
 .../drill/jdbc/impl/DrillPrepareResult.java     |  52 ++
 .../jdbc/impl/DrillPreparedStatementImpl.java   |  56 ++
 .../drill/jdbc/impl/DrillRemoteStatement.java   |  22 +
 .../drill/jdbc/impl/DrillResultSetImpl.java     |  55 +-
 .../drill/jdbc/impl/DrillStatementImpl.java     |  18 +-
 .../drill/jdbc/impl/DrillStatementRegistry.java |  77 +++
 .../org/apache/drill/jdbc/impl/DriverImpl.java  |  76 +++
 .../jdbc/impl/GlobalServiceSetReference.java    |  27 +
 .../org/apache/drill/jdbc/impl/MetaImpl.java    | 666 +++++++++++++++++++
 .../jdbc/impl/TypeConvertingSqlAccessor.java    |   3 +-
 .../org/apache/drill/jdbc/ConnectionTest.java   |   4 +-
 .../drill/jdbc/DrillColumnMetaDataListTest.java |   2 +
 48 files changed, 2908 insertions(+), 2690 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index f320ef7..d3525f7 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -221,10 +221,12 @@
             <option>-ignorewarnings</option>
             <option>-keep class org.apache.drill.exec.proto.** { *; }</option>
             <option>-keep class org.apache.drill.common.types.** { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.DrillResultSet { *; }</option>
             <option>-keep class org.apache.drill.jdbc.Driver { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.DrillJdbc40Factory { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.DrillJdbc41Factory { *; }</option>
+            <option>-keep class org.apache.drill.jdbc.DrillConnection { *; }</option>
+            <option>-keep class org.apache.drill.jdbc.DrillStatement { *; }</option>
+            <option>-keep class org.apache.drill.jdbc.DrillResultSet { *; }</option>
+            <option>-keep class org.apache.drill.jdbc.impl.DrillJdbc40Factory { *; }</option>
+            <option>-keep class org.apache.drill.jdbc.impl.DrillJdbc41Factory { *; }</option>
             <option>-keep class org.apache.drill.jdbc.proxy.TracingProxyDriver { *; }</option>
             <option>-keep class org.apache.drill.common.config.CommonConstants { *; }</option>
             <option>-keep class org.apache.drill.common.config.ConfigProvider { *; }</option>

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
index a2a06ab..8ad8d2d 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
@@ -28,7 +28,7 @@ public class AlreadyClosedSqlException extends JdbcApiSqlException {
 
   /**
    * See {@link JdbcApiSqlException#JdbcApiSqlException(String, String, int)}.
-   * /
+   */
   public AlreadyClosedSqlException( String reason,
                                     String SQLState,
                                     int vendorCode ) {
@@ -37,7 +37,7 @@ public class AlreadyClosedSqlException extends JdbcApiSqlException {
 
   /**
    * See {@link JdbcApiSqlException#JdbcApiSqlException(String, String)}.
-   * /
+   */
   public AlreadyClosedSqlException( String reason, String SQLState ) {
     super( reason, SQLState );
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/AvaticaDrillSqlAccessor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/AvaticaDrillSqlAccessor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/AvaticaDrillSqlAccessor.java
deleted file mode 100644
index f5aa3b7..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/AvaticaDrillSqlAccessor.java
+++ /dev/null
@@ -1,248 +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.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;
-
-// TODO:  Revisit adding null check for non-primitive types to SqlAccessor's
-// contract and classes generated by SqlAccessor template (DRILL-xxxx).
-
-public 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;
-
-  public 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/BasicList.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/BasicList.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/BasicList.java
deleted file mode 100644
index 290d680..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/BasicList.java
+++ /dev/null
@@ -1,136 +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.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/DrillAccessorList.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java
deleted file mode 100644
index b31efb1..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java
+++ /dev/null
@@ -1,67 +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.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;
-import org.apache.drill.jdbc.impl.TypeConvertingSqlAccessor;
-
-public 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;
-
-  public 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];
-  }
-
-  public 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/DrillColumnMetaDataList.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
deleted file mode 100644
index 40c7ae7..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
+++ /dev/null
@@ -1,217 +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.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";
-    }
-
-    return "?";
-}
-
-  @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/DrillConnection.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java
index d64f980..608bf05 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java
@@ -199,7 +199,7 @@ public interface DrillConnection extends Connection {
   /** Returns a view onto this connection's configuration properties. Code
    * within Optiq should use this view rather than calling
    * {@link java.util.Properties#getProperty(String)}. */
-  DrillConnectionConfig config();
+  DrillConnectionConfig getConfig();
 
   DrillClient getClient();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
deleted file mode 100644
index 5f82054..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
+++ /dev/null
@@ -1,413 +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.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.impl.DrillStatementImpl;
-
-// (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
-/**
- * Implementation of JDBC connection in Drill.
- *
- * <p>
- * Abstract to allow newer versions of JDBC to add methods.
- * </p>
- */
-public abstract class DrillConnectionImpl extends AvaticaConnection
-                                          implements DrillConnection {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConnection.class);
-
-  // (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
-  public final DrillStatementRegistry openStatementsRegistry = new DrillStatementRegistry();
-  final DrillConnectionConfig config;
-
-  private final DrillClient client;
-  private final BufferAllocator allocator;
-  private Drillbit bit;
-  private RemoteServiceSet serviceSet;
-
-  protected DrillConnectionImpl(Driver 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 config() {
-    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);
-      DrillPreparedStatement statement =
-          (DrillPreparedStatement) 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();
-  }
-
-  // (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
-  // do not make public
-  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/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
deleted file mode 100644
index e2dea4c..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
+++ /dev/null
@@ -1,213 +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.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;
-import org.apache.drill.jdbc.impl.DrillResultSetImpl;
-
-public 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
-   */
-  public DrillCursor(final DrillResultSetImpl resultSet) {
-    this.resultSet = resultSet;
-    currentBatch = resultSet.currentBatch;
-    resultsListener = resultSet.resultsListener;
-  }
-
-  public 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/DrillDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java
deleted file mode 100644
index 4b2d694..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java
+++ /dev/null
@@ -1,112 +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.ResultSet;
-import java.sql.SQLException;
-
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaDatabaseMetaData;
-
-public 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 );
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillFactory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillFactory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillFactory.java
deleted file mode 100644
index c674f8e..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillFactory.java
+++ /dev/null
@@ -1,60 +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.SQLException;
-import java.util.Properties;
-
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaFactory;
-import net.hydromatic.avatica.UnregisteredDriver;
-
-/**
- * Extension of {@link net.hydromatic.avatica.AvaticaFactory} for Drill.
- */
-public abstract class DrillFactory implements AvaticaFactory {
-  protected final int major;
-  protected final int minor;
-
-  /** Creates a JDBC factory with given major/minor version number. */
-  protected DrillFactory(int major, int minor) {
-    this.major = major;
-    this.minor = minor;
-  }
-
-  @Override
-  public int getJdbcMajorVersion() {
-    return major;
-  }
-
-  @Override
-  public int getJdbcMinorVersion() {
-    return minor;
-  }
-
-  @Override
-  public final AvaticaConnection newConnection(UnregisteredDriver driver, AvaticaFactory factory, String url,
-      Properties info)  throws SQLException{
-    return newDrillConnection((Driver) driver, (DrillFactory) factory, url, info);
-  }
-
-  /** Creates a connection with a root schema. */
-  public abstract DrillConnectionImpl newDrillConnection(Driver driver, DrillFactory factory, String url,
-      Properties info)  throws SQLException;
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillHandler.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillHandler.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillHandler.java
deleted file mode 100644
index 592bff3..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillHandler.java
+++ /dev/null
@@ -1,47 +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.SQLException;
-
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.Handler;
-
-public class DrillHandler implements Handler{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillHandler.class);
-
-  @Override
-  public void onConnectionInit(AvaticaConnection c) throws SQLException {
-  }
-
-  @Override
-  public void onConnectionClose(AvaticaConnection c) throws RuntimeException {
-    DrillConnectionImpl connection = (DrillConnectionImpl) c;
-    connection.cleanup();
-  }
-
-  @Override
-  public void onStatementExecute(AvaticaStatement statement, ResultSink resultSink) throws RuntimeException {
-  }
-
-  @Override
-  public void onStatementClose(AvaticaStatement statement) throws RuntimeException {
-    ((DrillRemoteStatement) statement).cleanup();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.java
deleted file mode 100644
index b542eb2..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.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;
-
-/**
- * Implementation of {@link net.hydromatic.avatica.AvaticaFactory}
- * for Drill and JDBC 4.0 (corresponds to JDK 1.6).
- */
-public class DrillJdbc40Factory extends DrillJdbc41Factory {
-  /** Creates a factory for JDBC version 4.1. */
-  public DrillJdbc40Factory() {
-    super(4, 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
deleted file mode 100644
index 93fe59d..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
+++ /dev/null
@@ -1,201 +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.io.InputStream;
-import java.io.Reader;
-import java.sql.NClob;
-import java.sql.ResultSetMetaData;
-import java.sql.RowId;
-import java.sql.SQLException;
-import java.sql.SQLXML;
-import java.util.List;
-import java.util.Properties;
-import java.util.TimeZone;
-
-import org.apache.drill.jdbc.impl.DrillResultSetImpl;
-import org.apache.drill.jdbc.impl.DrillStatementImpl;
-
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaDatabaseMetaData;
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaPreparedStatement;
-import net.hydromatic.avatica.AvaticaResultSetMetaData;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.ColumnMetaData;
-
-/**
- * Implementation of {@link net.hydromatic.avatica.AvaticaFactory} for Drill and JDBC 4.1 (corresponds to JDK 1.7).
- */
-@SuppressWarnings("UnusedDeclaration")
-public class DrillJdbc41Factory extends DrillFactory {
-  /** Creates a factory for JDBC version 4.1. */
-  public DrillJdbc41Factory() {
-    this(4, 1);
-  }
-
-  /** Creates a JDBC factory with given major/minor version number. */
-  protected DrillJdbc41Factory(int major, int minor) {
-    super(major, minor);
-  }
-
-  @Override
-  public DrillJdbc41Connection newDrillConnection(Driver driver, DrillFactory factory, String url, Properties info)  throws SQLException{
-    return new DrillJdbc41Connection((Driver) driver, factory, url, info);
-  }
-
-  @Override
-  public DrillJdbc41DatabaseMetaData newDatabaseMetaData(AvaticaConnection connection) {
-    return new DrillJdbc41DatabaseMetaData((DrillConnectionImpl) connection);
-  }
-
-
-  @Override
-  public DrillJdbc41Statement newStatement(AvaticaConnection connection, int resultSetType, int resultSetConcurrency,
-      int resultSetHoldability) {
-    return new DrillJdbc41Statement((DrillConnectionImpl) connection, resultSetType, resultSetConcurrency,
-        resultSetHoldability);
-  }
-
-  @Override
-  public AvaticaPreparedStatement newPreparedStatement(AvaticaConnection connection,
-      AvaticaPrepareResult prepareResult, int resultSetType, int resultSetConcurrency, int resultSetHoldability)
-      throws SQLException {
-    return new DrillJdbc41PreparedStatement((DrillConnectionImpl) connection, (DrillPrepareResult) prepareResult,
-        resultSetType, resultSetConcurrency, resultSetHoldability);
-  }
-
-  @Override
-  public DrillResultSetImpl newResultSet( AvaticaStatement statement,
-                                          AvaticaPrepareResult prepareResult,
-                                          TimeZone timeZone ) {
-    final ResultSetMetaData metaData =
-        newResultSetMetaData(statement, prepareResult.getColumnList());
-    return new DrillResultSetImpl( (DrillStatementImpl) statement,
-                                   (DrillPrepareResult) prepareResult,
-                                   metaData, timeZone);
-  }
-
-  @Override
-  public ResultSetMetaData newResultSetMetaData(AvaticaStatement statement, List<ColumnMetaData> columnMetaDataList) {
-    return new AvaticaResultSetMetaData(statement, null, columnMetaDataList);
-  }
-
-  private static class DrillJdbc41Connection extends DrillConnectionImpl {
-    DrillJdbc41Connection(Driver driver, DrillFactory factory, String url, Properties info) throws SQLException {
-      super(driver, factory, url, info);
-    }
-
-  }
-
-  private static class DrillJdbc41Statement extends DrillStatementImpl {
-    public DrillJdbc41Statement(DrillConnectionImpl connection, int resultSetType, int resultSetConcurrency,
-        int resultSetHoldability) {
-      super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
-    }
-  }
-
-  private static class DrillJdbc41PreparedStatement extends DrillPreparedStatement {
-    DrillJdbc41PreparedStatement(DrillConnectionImpl connection, DrillPrepareResult prepareResult, int resultSetType,
-        int resultSetConcurrency, int resultSetHoldability) throws SQLException {
-      super(connection, prepareResult, resultSetType, resultSetConcurrency, resultSetHoldability);
-    }
-
-    public void setRowId(int parameterIndex, RowId x) throws SQLException {
-      getParameter(parameterIndex).setRowId(x);
-    }
-
-    public void setNString(int parameterIndex, String value) throws SQLException {
-      getParameter(parameterIndex).setNString(value);
-    }
-
-    public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException {
-      getParameter(parameterIndex).setNCharacterStream(value, length);
-    }
-
-    public void setNClob(int parameterIndex, NClob value) throws SQLException {
-      getParameter(parameterIndex).setNClob(value);
-    }
-
-    public void setClob(int parameterIndex, Reader reader, long length) throws SQLException {
-      getParameter(parameterIndex).setClob(reader, length);
-    }
-
-    public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException {
-      getParameter(parameterIndex).setBlob(inputStream, length);
-    }
-
-    public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException {
-      getParameter(parameterIndex).setNClob(reader, length);
-    }
-
-    public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {
-      getParameter(parameterIndex).setSQLXML(xmlObject);
-    }
-
-    public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException {
-      getParameter(parameterIndex).setAsciiStream(x, length);
-    }
-
-    public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException {
-      getParameter(parameterIndex).setBinaryStream(x, length);
-    }
-
-    public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException {
-      getParameter(parameterIndex).setCharacterStream(reader, length);
-    }
-
-    public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException {
-      getParameter(parameterIndex).setAsciiStream(x);
-    }
-
-    public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException {
-      getParameter(parameterIndex).setBinaryStream(x);
-    }
-
-    public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException {
-      getParameter(parameterIndex).setCharacterStream(reader);
-    }
-
-    public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException {
-      getParameter(parameterIndex).setNCharacterStream(value);
-    }
-
-    public void setClob(int parameterIndex, Reader reader) throws SQLException {
-      getParameter(parameterIndex).setClob(reader);
-    }
-
-    public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException {
-      getParameter(parameterIndex).setBlob(inputStream);
-    }
-
-    public void setNClob(int parameterIndex, Reader reader) throws SQLException {
-      getParameter(parameterIndex).setNClob(reader);
-    }
-  }
-
-  private static class DrillJdbc41DatabaseMetaData extends DrillDatabaseMetaData {
-    DrillJdbc41DatabaseMetaData(DrillConnectionImpl connection) {
-      super(connection);
-    }
-  }
-
-}
-
-// End DrillJdbc41Factory.java

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

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java
deleted file mode 100644
index 4397c2f..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java
+++ /dev/null
@@ -1,52 +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.SQLException;
-
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaPreparedStatement;
-
-/**
- * Implementation of {@link java.sql.PreparedStatement} for Drill.
- *
- * <p>
- * This class has sub-classes which implement JDBC 3.0 and JDBC 4.0 APIs; it is instantiated using
- * {@link net.hydromatic.avatica.AvaticaFactory#newPreparedStatement}.
- * </p>
- */
-abstract class DrillPreparedStatement extends AvaticaPreparedStatement
-    implements DrillRemoteStatement {
-
-  protected DrillPreparedStatement(DrillConnectionImpl connection, AvaticaPrepareResult prepareResult,
-      int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
-    super(connection, prepareResult, resultSetType, resultSetConcurrency, resultSetHoldability);
-    connection.openStatementsRegistry.addStatement(this);
-  }
-
-  @Override
-  public DrillConnectionImpl getConnection() {
-    return (DrillConnectionImpl) super.getConnection();
-  }
-
-  @Override
-  public void cleanup() {
-    final DrillConnectionImpl connection1 = (DrillConnectionImpl) connection;
-    connection1.openStatementsRegistry.removeStatement(this);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java
deleted file mode 100644
index e19b49d..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java
+++ /dev/null
@@ -1,22 +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;
-
-public interface DrillRemoteStatement {
-  public void cleanup();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
index 2e3d49f..1e1b14f 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
@@ -39,6 +39,11 @@ import java.sql.Timestamp;
 import java.util.Calendar;
 import java.util.Map;
 
+
+/**
+ * Drill-specific {@link ResultSet}.
+ * @see #unwrap
+ */
 public interface DrillResultSet extends ResultSet  {
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
index 56a8b7f..903b18b 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
@@ -22,6 +22,7 @@ import java.sql.Statement;
 
 /**
  * Drill-specific {@link Statement}.
+ * @see #unwrap
  */
 public interface DrillStatement extends Statement {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/71082e6c/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
deleted file mode 100644
index fb38e31..0000000
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
+++ /dev/null
@@ -1,78 +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.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.IdentityHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.slf4j.Logger;
-
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Registry of open statements (for a connection), for closing them when a
- * connection is closed.
- * <p>
- *   Concurrency:  Not thread-safe.  (Creating statements, closing statements,
- *   and closing connection cannot be concurrent (unless concurrency is
- *   coordinated elsewhere).)
- * </p>
- */
-// (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
-public class DrillStatementRegistry {
-
-  private static final Logger logger = getLogger( DrillStatementRegistry.class );
-
-  /** ... (using as IdentityHash*Set*) */
-  private final Map<Statement, Object> openStatements = new IdentityHashMap<>();
-
-
-  public void addStatement( Statement statement ) {
-    logger.debug( "Adding to open-statements registry: " + statement );
-    openStatements.put( statement, statement );
-  }
-
-  public void removeStatement( Statement statement ) {
-    logger.debug( "Removing from open-statements registry: " + statement );
-    openStatements.remove( statement );
-  }
-
-  public void close() {
-    // Note:  Can't call close() on statement during iteration of map because
-    // close() calls our removeStatement(...), which modifies the map.
-
-    // Copy set of open statements to other collection before closing:
-    final List<Statement> copiedList = new ArrayList<>( openStatements.keySet() );
-
-    for ( final Statement statement : copiedList ) {
-      try {
-        logger.debug( "Auto-closing (via open-statements registry): " + statement );
-        statement.close();
-      }
-      catch ( SQLException e ) {
-        logger.error( "Error auto-closing statement " + statement + ": " + e, e );
-        // Otherwise ignore the error, to close which statements we can close.
-      }
-    }
-    openStatements.clear();
-  }
-}