You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2018/08/27 08:25:33 UTC

[drill] 02/06: DRILL-6647: Update Calcite version to 1.17.0

This is an automated email from the ASF dual-hosted git repository.

arina pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 989618287559de5e87349c637cb91150ea6a5135
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Mon Jul 2 17:59:42 2018 +0300

    DRILL-6647: Update Calcite version to 1.17.0
    
    closes #1425
---
 .../drill/exec/store/jdbc/JdbcStoragePlugin.java   |    2 +-
 exec/java-exec/src/main/codegen/data/Parser.tdd    |    8 +-
 .../logical/DirPrunedEnumerableTableScan.java      |   12 +-
 .../exec/planner/logical/DrillConditions.java      |   20 +-
 .../exec/planner/sql/DrillConvertletTable.java     |   35 +-
 .../drill/exec/planner/sql/SqlConverter.java       |    2 +-
 .../apache/drill/exec/store/AbstractSchema.java    |   27 +
 .../drill/jdbc/impl/DrillConnectionImpl.java       |  400 ++----
 .../drill/jdbc/impl/DrillDatabaseMetaDataImpl.java |  354 +++--
 .../apache/drill/jdbc/impl/DrillJdbc41Factory.java |  120 +-
 .../jdbc/impl/DrillPreparedStatementImpl.java      |  463 +-----
 .../apache/drill/jdbc/impl/DrillResultSetImpl.java | 1494 ++++++--------------
 .../jdbc/impl/DrillResultSetMetaDataImpl.java      |   10 +-
 .../apache/drill/jdbc/impl/DrillStatementImpl.java |  327 +----
 .../java/org/apache/drill/jdbc/JdbcTestBase.java   |  129 +-
 ...rill2489CallsAfterCloseThrowExceptionsTest.java |  107 +-
 pom.xml                                            |    4 +-
 17 files changed, 917 insertions(+), 2597 deletions(-)

diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
index efb5a5c..4de3cf8 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
@@ -46,7 +46,7 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlDialectFactoryImpl;
-import org.apache.commons.dbcp.BasicDataSource;
+import org.apache.commons.dbcp2.BasicDataSource;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index 507d692..5891fb7 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -85,13 +85,17 @@
 
   # List of methods for parsing extensions to "DROP" calls.
   # Each must accept arguments "(SqlParserPos pos)".
-    dropStatementParserMethods: [
+  dropStatementParserMethods: [
   ]
 
   # List of keywords from "keywords" section that are not reserved.
-    nonReservedKeywords: [
+  nonReservedKeywords: [
   ]
 
+  # List of additional join types. Each is a method with no arguments.
+  # Example: LeftSemiJoin()
+  joinTypes: [
+  ]
   
   includeCompoundIdentifier: false,
   includeBraces: true,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
index 2202986..7fd348f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
@@ -17,14 +17,12 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
@@ -62,13 +60,11 @@ public class DirPrunedEnumerableTableScan extends EnumerableTableScan {
     final RelTraitSet traitSet =
         cluster.traitSetOf(EnumerableConvention.INSTANCE)
             .replaceIfs(RelCollationTraitDef.INSTANCE,
-                new Supplier<List<RelCollation>>() {
-                  public List<RelCollation> get() {
-                    if (table != null) {
-                      return table.getStatistic().getCollations();
-                    }
-                    return ImmutableList.of();
+                () -> {
+                  if (table != null) {
+                    return table.getStatistic().getCollations();
                   }
+                  return ImmutableList.of();
                 });
     return new DirPrunedEnumerableTableScan(cluster, traitSet, relOptTable, elementType, digestFromSelection);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
index 266ce64..1341be1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
@@ -19,24 +19,14 @@ package org.apache.drill.exec.planner.logical;
 
 import org.apache.calcite.rel.rules.PushProjector;
 import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexNode;
 
 public final class DrillConditions {
 
-  public static PushProjector.ExprCondition PRESERVE_ITEM = new PushProjector.ExprCondition() {
-
-    @Override
-    public boolean apply(RexNode rexNode) {
-      return false;
-    }
-
-    @Override
-    public boolean test(RexNode expr) {
-      if (expr instanceof RexCall) {
-        RexCall call = (RexCall)expr;
-        return "item".equals(call.getOperator().getName().toLowerCase());
-      }
-      return false;
+  public static final PushProjector.ExprCondition PRESERVE_ITEM = expr -> {
+    if (expr instanceof RexCall) {
+      RexCall call = (RexCall) expr;
+      return "item".equals(call.getOperator().getName().toLowerCase());
     }
+    return false;
   };
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
index 34141e5..5f3b95e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.planner.sql;
 
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlBasicCall;
@@ -25,7 +27,6 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql2rel.SqlRexContext;
 import org.apache.calcite.sql2rel.SqlRexConvertlet;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 import org.apache.calcite.sql2rel.StandardConvertletTable;
@@ -36,19 +37,37 @@ public class DrillConvertletTable implements SqlRexConvertletTable{
   public static HashMap<SqlOperator, SqlRexConvertlet> map = new HashMap<>();
 
   public static SqlRexConvertletTable INSTANCE = new DrillConvertletTable();
-  private static SqlRexConvertlet sqrtConvertlet = new SqlRexConvertlet() {
-    public RexNode convertCall(SqlRexContext cx, SqlCall call) {
-      RexNode operand = cx.convertExpression(call.operand(0));
-      return cx.getRexBuilder().makeCall(SqlStdOperatorTable.SQRT, operand);
+
+  private static final SqlRexConvertlet SQRT_CONVERTLET = (cx, call) -> {
+    RexNode operand = cx.convertExpression(call.operand(0));
+    return cx.getRexBuilder().makeCall(SqlStdOperatorTable.SQRT, operand);
+  };
+
+  // Rewrites COALESCE function into CASE WHEN IS NOT NULL operand1 THEN operand1...
+  private static final SqlRexConvertlet COALESCE_CONVERTLET = (cx, call) -> {
+    int operandsCount = call.operandCount();
+    if (operandsCount == 1) {
+      return cx.convertExpression(call.operand(0));
+    } else {
+      List<RexNode> caseOperands = new ArrayList<>();
+      for (int i = 0; i < operandsCount - 1; i++) {
+        RexNode caseOperand = cx.convertExpression(call.operand(i));
+        caseOperands.add(cx.getRexBuilder().makeCall(
+            SqlStdOperatorTable.IS_NOT_NULL, caseOperand));
+        caseOperands.add(caseOperand);
+      }
+      caseOperands.add(cx.convertExpression(call.operand(operandsCount - 1)));
+      return cx.getRexBuilder().makeCall(SqlStdOperatorTable.CASE, caseOperands);
     }
   };
 
   static {
-    // Use custom convertlet for extract function
+    // Use custom convertlet for EXTRACT function
     map.put(SqlStdOperatorTable.EXTRACT, DrillExtractConvertlet.INSTANCE);
-    // sqrt needs it's own convertlet because calcite overrides it to power(x,0.5)
+    // SQRT needs it's own convertlet because calcite overrides it to POWER(x, 0.5)
     // which is not suitable for Infinity value case
-    map.put(SqlStdOperatorTable.SQRT, sqrtConvertlet);
+    map.put(SqlStdOperatorTable.SQRT, SQRT_CONVERTLET);
+    map.put(SqlStdOperatorTable.COALESCE, COALESCE_CONVERTLET);
     map.put(SqlStdOperatorTable.AVG, new DrillAvgVarianceConvertlet(SqlKind.AVG));
     map.put(SqlStdOperatorTable.STDDEV_POP, new DrillAvgVarianceConvertlet(SqlKind.STDDEV_POP));
     map.put(SqlStdOperatorTable.STDDEV_SAMP, new DrillAvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index d4da23f..3ab92ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -377,7 +377,7 @@ public class SqlConverter {
      * during creating new projects since it may cause changing data mode
      * which causes to assertion errors during type validation
      */
-    Hook.REL_BUILDER_SIMPLIFY.add(Hook.property(false));
+    Hook.REL_BUILDER_SIMPLIFY.add(Hook.propertyJ(false));
 
     //To avoid unexpected column errors set a value of top to false
     final RelRoot rel = sqlToRelConverter.convertQuery(validatedNode, false, false);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index 2b70c3b..18e3963 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -22,10 +22,13 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.calcite.linq4j.tree.DefaultExpression;
 import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
@@ -163,6 +166,30 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
     return Collections.emptyList();
   }
 
+  /**
+   * Returns a map of types in this schema by name.
+   *
+   * <p>The implementations of {@link #getTypeNames()}
+   * and {@link #getType(String)} depend on this map.
+   * The default implementation of this method returns the empty map.
+   * Override this method to change their behavior.</p>
+   *
+   * @return Map of types in this schema by name
+   */
+  protected Map<String, RelProtoDataType> getTypeMap() {
+    return ImmutableMap.of();
+  }
+
+  @Override
+  public Set<String> getTypeNames() {
+    return getTypeMap().keySet();
+  }
+
+  @Override
+  public RelProtoDataType getType(String name) {
+    return getTypeMap().get(name);
+  }
+
   @Override
   public Set<String> getFunctionNames() {
     return Collections.emptySet();
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
index 2bac4ad..e005175 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
@@ -18,11 +18,9 @@
 package org.apache.drill.jdbc.impl;
 
 import java.io.File;
-import java.sql.Array;
 import java.sql.Blob;
 import java.sql.CallableStatement;
 import java.sql.Clob;
-import java.sql.DatabaseMetaData;
 import java.sql.NClob;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -30,10 +28,8 @@ import java.sql.SQLClientInfoException;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLNonTransientConnectionException;
-import java.sql.SQLWarning;
 import java.sql.SQLXML;
 import java.sql.Savepoint;
-import java.sql.Statement;
 import java.sql.Struct;
 import java.util.Map;
 import java.util.Properties;
@@ -80,7 +76,7 @@ import static org.apache.drill.exec.util.StoragePluginTestUtils.UNIT_TEST_PROP_P
 import static org.apache.drill.exec.util.StoragePluginTestUtils.updateSchemaLocation;
 
 /**
- * Drill's implementation of {@link Connection}.
+ * Drill's implementation of {@link java.sql.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
@@ -208,9 +204,10 @@ class DrillConnectionImpl extends AvaticaConnection
    *
    * @throws  AlreadyClosedSqlException  if Connection is closed
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException {
-    if ( isClosed() ) {
-      throw new AlreadyClosedSqlException( "Connection is already closed." );
+  @Override
+  protected void checkOpen() throws AlreadyClosedSqlException {
+    if (isClosed()) {
+      throw new AlreadyClosedSqlException("Connection is already closed.");
     }
   }
 
@@ -229,9 +226,9 @@ class DrillConnectionImpl extends AvaticaConnection
   }
 
   @Override
-  public void setAutoCommit( boolean autoCommit ) throws SQLException {
-    throwIfClosed();
-    if ( ! autoCommit ) {
+  public void setAutoCommit(boolean autoCommit) throws SQLException {
+    checkOpen();
+    if (!autoCommit) {
       throw new SQLFeatureNotSupportedException(
           "Can't turn off auto-committing; transactions are not supported.  "
           + "(Drill is not transactional.)" );
@@ -241,27 +238,25 @@ class DrillConnectionImpl extends AvaticaConnection
 
   @Override
   public void commit() throws SQLException {
-    throwIfClosed();
-    if ( getAutoCommit() ) {
-      throw new JdbcApiSqlException( "Can't call commit() in auto-commit mode." );
-    }
-    else {
+    checkOpen();
+    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.)" );
+          "Connection.commit() is not supported.  (Drill is not transactional.)");
     }
   }
 
   @Override
   public void rollback() throws SQLException {
-    throwIfClosed();
-    if ( getAutoCommit()  ) {
-      throw new JdbcApiSqlException( "Can't call rollback() in auto-commit mode." );
-    }
-    else {
+    checkOpen();
+    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.)" );
+          "Connection.rollback() is not supported.  (Drill is not transactional.)");
     }
   }
 
@@ -270,13 +265,12 @@ class DrillConnectionImpl extends AvaticaConnection
   public boolean isClosed() {
     try {
       return super.isClosed();
-    }
-    catch ( SQLException e ) {
+    } catch (SQLException e) {
       // Currently can't happen, since AvaticaConnection.isClosed() never throws
       // SQLException.
       throw new DrillRuntimeException(
           "Unexpected exception from " + getClass().getSuperclass()
-          + ".isClosed(): " + e,
+            + ".isClosed(): " + e,
           e );
     }
   }
@@ -284,40 +278,45 @@ class DrillConnectionImpl extends AvaticaConnection
 
   @Override
   public Savepoint setSavepoint() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "Savepoints are not supported. (Drill is not transactional.)" );
   }
 
   @Override
   public Savepoint setSavepoint(String name) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "Savepoints are not supported. (Drill is not transactional.)" );
   }
 
   @Override
     public void rollback(Savepoint savepoint) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "Savepoints are not supported. (Drill is not transactional.)" );
   }
 
   @Override
   public void releaseSavepoint(Savepoint savepoint) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "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";
+  private String isolationValueToString(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 + ">";
     }
@@ -325,8 +324,8 @@ class DrillConnectionImpl extends AvaticaConnection
 
   @Override
   public void setTransactionIsolation(int level) throws SQLException {
-    throwIfClosed();
-    switch ( level ) {
+    checkOpen();
+    switch (level) {
       case TRANSACTION_NONE:
         // No-op.  (Is already set in constructor, and we disallow changing it.)
         break;
@@ -334,73 +333,50 @@ class DrillConnectionImpl extends AvaticaConnection
       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.)" );
+        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 );
+            "Invalid transaction isolation level value " + level);
         //break;
     }
   }
 
   @Override
-  public void setNetworkTimeout( Executor executor, int milliseconds )
-      throws AlreadyClosedSqlException,
-             JdbcApiSqlException,
-             SQLFeatureNotSupportedException {
-    throwIfClosed();
-    if ( null == executor ) {
+  public void setNetworkTimeout(Executor executor, int milliseconds)
+      throws JdbcApiSqlException, SQLFeatureNotSupportedException {
+    checkOpen();
+    if (null == executor) {
       throw new InvalidParameterSqlException(
-          "Invalid (null) \"executor\" parameter to setNetworkTimeout(...)" );
-    }
-    else if ( milliseconds < 0 ) {
+          "Invalid (null) \"executor\" parameter to setNetworkTimeout(...)");
+    } else if (milliseconds < 0) {
       throw new InvalidParameterSqlException(
           "Invalid (negative) \"milliseconds\" parameter to"
-          + " setNetworkTimeout(...) (" + milliseconds + ")" );
-    }
-    else {
-      if ( 0 != milliseconds ) {
+          + " setNetworkTimeout(...) (" + milliseconds + ")");
+    } else {
+      if (0 != milliseconds) {
         throw new SQLFeatureNotSupportedException(
-            "Setting network timeout is not supported." );
+            "Setting network timeout is not supported.");
       }
     }
   }
 
   @Override
-  public int getNetworkTimeout() throws AlreadyClosedSqlException
-  {
-    throwIfClosed();
+  public int getNetworkTimeout() throws AlreadyClosedSqlException {
+    checkOpen();
     return 0;  // (No timeout.)
   }
 
-
   @Override
   public DrillStatementImpl createStatement(int resultSetType,
                                             int resultSetConcurrency,
                                             int resultSetHoldability) throws SQLException {
-    throwIfClosed();
-    DrillStatementImpl statement =
-        (DrillStatementImpl) super.createStatement(resultSetType,
-                                                   resultSetConcurrency,
-                                                   resultSetHoldability);
-    return statement;
-  }
-
-  @Override
-  public PreparedStatement prepareStatement(String sql, int resultSetType,
-                                            int resultSetConcurrency,
-                                            int resultSetHoldability) throws SQLException {
-    throwIfClosed();
-    DrillPreparedStatementImpl statement =
-        (DrillPreparedStatementImpl) super.prepareStatement(sql,
-                                                            resultSetType,
-                                                            resultSetConcurrency,
-                                                            resultSetHoldability);
-    return statement;
+    return (DrillStatementImpl) super.createStatement(resultSetType,
+        resultSetConcurrency, resultSetHoldability);
   }
 
   @Override
@@ -420,172 +396,67 @@ class DrillConnectionImpl extends AvaticaConnection
   // No unwrap(Class<T>) (it doesn't throw SQLException if already closed).
 
   @Override
-  public AvaticaStatement createStatement() throws SQLException {
-    throwIfClosed();
-    return super.createStatement();
-  }
-
-  @Override
-  public PreparedStatement prepareStatement(String sql) throws SQLException {
-    throwIfClosed();
-    return super.prepareStatement(sql);
-  }
-
-  @Override
   public CallableStatement prepareCall(String sql) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareCall(sql);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public String nativeSQL(String sql) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.nativeSQL(sql);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-
-  @Override
-  public boolean getAutoCommit() throws SQLException {
-    throwIfClosed();
-    return super.getAutoCommit();
-  }
-
   // No close() (it doesn't throw SQLException if already closed).
 
   @Override
-  public DatabaseMetaData getMetaData() throws SQLException {
-    throwIfClosed();
-    return super.getMetaData();
-  }
-
-  @Override
-  public void setReadOnly(boolean readOnly) throws SQLException {
-    throwIfClosed();
-    super.setReadOnly(readOnly);
-  }
-
-  @Override
-  public boolean isReadOnly() throws SQLException {
-    throwIfClosed();
-    return super.isReadOnly();
-  }
-
-  @Override
-  public void setCatalog(String catalog) throws SQLException {
-    throwIfClosed();
-    super.setCatalog(catalog);
-  }
-
-  @Override
-  public String getCatalog() {
-    // Can't throw any SQLException because AvaticaConnection's getCatalog() is
-    // missing "throws SQLException".
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getCatalog();
-  }
-
-  @Override
-  public int getTransactionIsolation() throws SQLException {
-    throwIfClosed();
-    return super.getTransactionIsolation();
-  }
-
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
-  public Statement createStatement(int resultSetType,
-                                   int resultSetConcurrency) throws SQLException {
-    throwIfClosed();
-    return super.createStatement(resultSetType, resultSetConcurrency);
-  }
-
-  @Override
-  public PreparedStatement prepareStatement(String sql, int resultSetType,
-                                            int resultSetConcurrency) throws SQLException {
-    throwIfClosed();
-    return super.prepareStatement(sql, resultSetType, resultSetConcurrency);
-  }
-
-  @Override
   public CallableStatement prepareCall(String sql, int resultSetType,
                                        int resultSetConcurrency) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareCall(sql, resultSetType, resultSetConcurrency);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public Map<String,Class<?>> getTypeMap() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getTypeMap();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void setTypeMap(Map<String,Class<?>> map) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setTypeMap(map);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void setHoldability(int holdability) throws SQLException {
-    throwIfClosed();
-    super.setHoldability(holdability);
-  }
-
-  @Override
-  public int getHoldability() throws SQLException {
-    throwIfClosed();
-    return super.getHoldability();
-  }
-
-  @Override
   public CallableStatement prepareCall(String sql, int resultSetType,
                                        int resultSetConcurrency,
                                        int resultSetHoldability) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareCall(sql, resultSetType, resultSetConcurrency,
-                               resultSetHoldability);
-    }
-    catch (UnsupportedOperationException e) {
+        resultSetHoldability);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -593,11 +464,10 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public PreparedStatement prepareStatement(String sql,
                                             int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareStatement(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -605,11 +475,10 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public PreparedStatement prepareStatement(String sql,
                                             int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareStatement(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -617,81 +486,70 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public PreparedStatement prepareStatement(String sql,
                                             String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareStatement(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public Clob createClob() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createClob();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public Blob createBlob() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createBlob();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public NClob createNClob() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createNClob();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public SQLXML createSQLXML() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createSQLXML();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean isValid(int timeout) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.isValid(timeout);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+    checkOpen();
+    return super.isValid(timeout);
   }
 
   @Override
   public void setClientInfo(String name, String value) throws SQLClientInfoException {
     try {
-      throwIfClosed();
+      checkOpen();
     } catch (AlreadyClosedSqlException e) {
       throw new SQLClientInfoException(e.getMessage(), null, e);
     }
     try {
-      super.setClientInfo(name,  value);
-    }
-    catch (UnsupportedOperationException e) {
+      super.setClientInfo(name, value);
+    } catch (UnsupportedOperationException e) {
       SQLFeatureNotSupportedException intended =
           new SQLFeatureNotSupportedException(e.getMessage(), e);
       throw new SQLClientInfoException(e.getMessage(), null, intended);
@@ -701,14 +559,13 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public void setClientInfo(Properties properties) throws SQLClientInfoException {
     try {
-      throwIfClosed();
+      checkOpen();
     } catch (AlreadyClosedSqlException e) {
       throw new SQLClientInfoException(e.getMessage(), null, e);
     }
     try {
       super.setClientInfo(properties);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       SQLFeatureNotSupportedException intended =
           new SQLFeatureNotSupportedException(e.getMessage(), e);
       throw new SQLClientInfoException(e.getMessage(), null, intended);
@@ -716,80 +573,25 @@ class DrillConnectionImpl extends AvaticaConnection
   }
 
   @Override
-  public String getClientInfo(String name) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getClientInfo(name);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public Properties getClientInfo() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getClientInfo();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.createArrayOf(typeName, elements);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
   public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createStruct(typeName, attributes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void setSchema(String schema) throws SQLException {
-    throwIfClosed();
-    super.setSchema(schema);
-  }
-
-  @Override
-  public String getSchema() {
-    // Can't throw any SQLException because AvaticaConnection's getCatalog() is
-    // missing "throws SQLException".
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getSchema();
-  }
-
-  @Override
   public void abort(Executor executor) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.abort(executor);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-
-
   // do not make public
   UnregisteredDriver getDriver() {
     return driver;
@@ -800,7 +602,7 @@ class DrillConnectionImpl extends AvaticaConnection
     return factory;
   }
 
-  private static void closeOrWarn(final AutoCloseable autoCloseable, final String message, final Logger logger) {
+  private static void closeOrWarn(AutoCloseable autoCloseable, String message, Logger logger) {
     if (autoCloseable == null) {
       return;
     }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
index ccdc124..b7617b2 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
@@ -53,9 +53,8 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableSet;
 
-
 /**
- * Drill's implementation of {@link DatabaseMetaData}.
+ * Drill's implementation of {@link java.sql.DatabaseMetaData}.
  */
 class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
                                 implements DrillDatabaseMetaData {
@@ -126,11 +125,10 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
    * @throws AlreadyClosedSqlException if Connection is closed
    * @throws SQLException if error in calling {@link Connection#isClosed()}
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException,
-                                      SQLException {
-    if ( getConnection().isClosed() ) {
+  protected void checkOpen() throws AlreadyClosedSqlException, SQLException {
+    if (getConnection().isClosed()) {
       throw new AlreadyClosedSqlException(
-          "DatabaseMetaData's Connection is already closed." );
+          "DatabaseMetaData's Connection is already closed.");
     }
   }
 
@@ -196,13 +194,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean allProceduresAreCallable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.allProceduresAreCallable();
   }
 
   @Override
   public boolean allTablesAreSelectable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.allTablesAreSelectable();
     }
@@ -211,19 +209,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getURL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getURL();
   }
 
   @Override
   public String getUserName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getUserName();
   }
 
   @Override
   public boolean isReadOnly() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.isReadOnly();
     }
@@ -235,7 +233,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedHigh() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return true;
     }
@@ -244,7 +242,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedLow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return false;
     }
@@ -253,7 +251,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedAtStart() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return false;
     }
@@ -262,7 +260,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedAtEnd() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return false;
     }
@@ -271,7 +269,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getDatabaseProductName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     String name = getServerName();
     if (name == null) {
       return super.getDatabaseProductName();
@@ -281,7 +279,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getDatabaseProductVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     Version version = getServerVersion();
     if (version == null) {
       return super.getDatabaseProductVersion();
@@ -291,13 +289,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getDriverName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getDriverName();
   }
 
   @Override
   public String getDriverVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getDriverVersion();
   }
 
@@ -315,19 +313,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean usesLocalFiles() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.usesLocalFiles();
   }
 
   @Override
   public boolean usesLocalFilePerTable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.usesLocalFilePerTable();
   }
 
   @Override
   public boolean supportsMixedCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsMixedCaseIdentifiers();
     }
@@ -336,7 +334,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesUpperCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesUpperCaseIdentifiers();
     }
@@ -345,7 +343,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesLowerCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesLowerCaseIdentifiers();
     }
@@ -354,7 +352,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesMixedCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesMixedCaseIdentifiers();
     }
@@ -363,7 +361,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsMixedCaseQuotedIdentifiers();
     }
@@ -372,7 +370,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesUpperCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesUpperCaseQuotedIdentifiers();
     }
@@ -381,7 +379,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesLowerCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesLowerCaseQuotedIdentifiers();
     }
@@ -390,7 +388,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesMixedCaseQuotedIdentifiers();
     }
@@ -399,7 +397,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getIdentifierQuoteString() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return Quoting.BACK_TICK.string;
     }
@@ -408,7 +406,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSQLKeywords() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSQLKeywords();
     }
@@ -417,7 +415,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getNumericFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getNumericFunctions();
     }
@@ -426,7 +424,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getStringFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getStringFunctions();
     }
@@ -435,7 +433,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSystemFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSystemFunctions();
     }
@@ -444,7 +442,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getTimeDateFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getTimeDateFunctions();
     }
@@ -453,7 +451,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSearchStringEscape() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSearchStringEscape();
     }
@@ -462,7 +460,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getExtraNameCharacters() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getExtraNameCharacters();
     }
@@ -471,19 +469,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsAlterTableWithAddColumn() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsAlterTableWithAddColumn();
   }
 
   @Override
   public boolean supportsAlterTableWithDropColumn() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsAlterTableWithDropColumn();
   }
 
   @Override
   public boolean supportsColumnAliasing() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsColumnAliasing();
     }
@@ -492,7 +490,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullPlusNonNullIsNull() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.nullPlusNonNullIsNull();
     }
@@ -501,7 +499,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsConvert() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsConvert();
     }
@@ -512,7 +510,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsConvert(int fromType, int toType) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsConvert(fromType, toType);
     }
@@ -523,7 +521,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsTableCorrelationNames() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsTableCorrelationNames();
     }
@@ -533,7 +531,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsDifferentTableCorrelationNames() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsDifferentTableCorrelationNames();
     }
@@ -542,7 +540,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsExpressionsInOrderBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsExpressionsInOrderBy();
     }
@@ -551,7 +549,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsOrderByUnrelated() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsOrderByUnrelated();
     }
@@ -560,7 +558,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsGroupBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsGroupBy();
     }
@@ -569,7 +567,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsGroupByUnrelated() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsGroupByUnrelated();
     }
@@ -578,7 +576,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsGroupByBeyondSelect() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsGroupByBeyondSelect();
     }
@@ -587,7 +585,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsLikeEscapeClause() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsLikeEscapeClause();
     }
@@ -596,67 +594,67 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsMultipleResultSets() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMultipleResultSets();
   }
 
   @Override
   public boolean supportsMultipleTransactions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMultipleTransactions();
   }
 
   @Override
   public boolean supportsNonNullableColumns() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsNonNullableColumns();
   }
 
   @Override
   public boolean supportsMinimumSQLGrammar() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMinimumSQLGrammar();
   }
 
   @Override
   public boolean supportsCoreSQLGrammar() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCoreSQLGrammar();
   }
 
   @Override
   public boolean supportsExtendedSQLGrammar() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsExtendedSQLGrammar();
   }
 
   @Override
   public boolean supportsANSI92EntryLevelSQL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsANSI92EntryLevelSQL();
   }
 
   @Override
   public boolean supportsANSI92IntermediateSQL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsANSI92IntermediateSQL();
   }
 
   @Override
   public boolean supportsANSI92FullSQL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsANSI92FullSQL();
   }
 
   @Override
   public boolean supportsIntegrityEnhancementFacility() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsIntegrityEnhancementFacility();
   }
 
   @Override
   public boolean supportsOuterJoins() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsOuterJoins();
     }
@@ -665,7 +663,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsFullOuterJoins() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsFullOuterJoins();
     }
@@ -674,7 +672,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsLimitedOuterJoins() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsFullOuterJoins();
     }
@@ -684,7 +682,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSchemaTerm() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSchemaTerm();
     }
@@ -693,13 +691,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getProcedureTerm() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getProcedureTerm();
   }
 
   @Override
   public String getCatalogTerm() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getCatalogTerm();
     }
@@ -708,7 +706,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean isCatalogAtStart() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.isCatalogAtStart();
     }
@@ -717,7 +715,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getCatalogSeparator() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getCatalogSeparator();
     }
@@ -726,79 +724,79 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSchemasInDataManipulation() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInDataManipulation();
   }
 
   @Override
   public boolean supportsSchemasInProcedureCalls() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInProcedureCalls();
   }
 
   @Override
   public boolean supportsSchemasInTableDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInTableDefinitions();
   }
 
   @Override
   public boolean supportsSchemasInIndexDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInIndexDefinitions();
   }
 
   @Override
   public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInPrivilegeDefinitions();
   }
 
   @Override
   public boolean supportsCatalogsInDataManipulation() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInDataManipulation();
   }
 
   @Override
   public boolean supportsCatalogsInProcedureCalls() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInProcedureCalls();
   }
 
   @Override
   public boolean supportsCatalogsInTableDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInTableDefinitions();
   }
 
   @Override
   public boolean supportsCatalogsInIndexDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInIndexDefinitions();
   }
 
   @Override
   public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInPrivilegeDefinitions();
   }
 
   @Override
   public boolean supportsPositionedDelete() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsPositionedDelete();
   }
 
   @Override
   public boolean supportsPositionedUpdate() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsPositionedUpdate();
   }
 
   @Override
   public boolean supportsSelectForUpdate() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSelectForUpdate();
     }
@@ -807,13 +805,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsStoredProcedures() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsStoredProcedures();
   }
 
   @Override
   public boolean supportsSubqueriesInComparisons() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInComparisons();
     }
@@ -822,7 +820,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSubqueriesInExists() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInExists();
     }
@@ -831,7 +829,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSubqueriesInIns() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInIns();
     }
@@ -840,7 +838,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSubqueriesInQuantifieds() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInQuantifieds();
     }
@@ -849,7 +847,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsCorrelatedSubqueries() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsCorrelatedSubqueries();
     }
@@ -858,7 +856,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsUnion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsUnion();
     }
@@ -867,7 +865,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsUnionAll() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsUnionAll();
     }
@@ -876,31 +874,31 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsOpenCursorsAcrossCommit() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenCursorsAcrossCommit();
   }
 
   @Override
   public boolean supportsOpenCursorsAcrossRollback() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenCursorsAcrossRollback();
   }
 
   @Override
   public boolean supportsOpenStatementsAcrossCommit() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenStatementsAcrossCommit();
   }
 
   @Override
   public boolean supportsOpenStatementsAcrossRollback() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenStatementsAcrossRollback();
   }
 
   @Override
   public int getMaxBinaryLiteralLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxBinaryLiteralLength();
     }
@@ -909,7 +907,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxCharLiteralLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxCharLiteralLength();
     }
@@ -918,7 +916,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnNameLength();
     }
@@ -927,7 +925,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInGroupBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnsInGroupBy();
     }
@@ -936,13 +934,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInIndex() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxColumnsInIndex();
   }
 
   @Override
   public int getMaxColumnsInOrderBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnsInOrderBy();
     }
@@ -951,7 +949,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInSelect() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnsInSelect();
     }
@@ -960,19 +958,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInTable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxColumnsInTable();
   }
 
   @Override
   public int getMaxConnections() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxConnections();
   }
 
   @Override
   public int getMaxCursorNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxCursorNameLength();
     }
@@ -981,13 +979,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxIndexLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxIndexLength();
   }
 
   @Override
   public int getMaxSchemaNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxSchemaNameLength();
     }
@@ -996,13 +994,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxProcedureNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxProcedureNameLength();
   }
 
   @Override
   public int getMaxCatalogNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxCatalogNameLength();
     }
@@ -1011,7 +1009,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxRowSize() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxRowSize();
     }
@@ -1020,7 +1018,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean doesMaxRowSizeIncludeBlobs() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.doesMaxRowSizeIncludeBlobs();
     }
@@ -1029,7 +1027,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxStatementLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxStatementLength();
     }
@@ -1038,7 +1036,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxStatements() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxStatements();
     }
@@ -1047,7 +1045,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxTableNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxTableNameLength();
     }
@@ -1056,7 +1054,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxTablesInSelect() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxTablesInSelect();
     }
@@ -1065,7 +1063,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxUserNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxUserNameLength();
     }
@@ -1074,13 +1072,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getDefaultTransactionIsolation() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getDefaultTransactionIsolation();
   }
 
   @Override
   public boolean supportsTransactions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsTransactions();
     }
@@ -1089,39 +1087,39 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsTransactionIsolationLevel(int level) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsTransactionIsolationLevel(level);
   }
 
   @Override
   public boolean supportsDataDefinitionAndDataManipulationTransactions()
       throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsDataDefinitionAndDataManipulationTransactions();
   }
 
   @Override
   public boolean supportsDataManipulationTransactionsOnly() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsDataManipulationTransactionsOnly();
   }
 
   @Override
   public boolean dataDefinitionCausesTransactionCommit() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.dataDefinitionCausesTransactionCommit();
   }
 
   @Override
   public boolean dataDefinitionIgnoredInTransactions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.dataDefinitionIgnoredInTransactions();
   }
 
   @Override
   public ResultSet getProcedures(String catalog, String schemaPattern,
                                  String procedureNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getProcedures(catalog, schemaPattern, procedureNamePattern);
   }
 
@@ -1129,7 +1127,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getProcedureColumns(String catalog, String schemaPattern,
                                        String procedureNamePattern,
                                        String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getProcedureColumns(catalog, schemaPattern,
                                      procedureNamePattern, columnNamePattern);
   }
@@ -1139,7 +1137,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
                              String schemaPattern,
                              String tableNamePattern,
                              String[] types) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getTables(catalog, schemaPattern,tableNamePattern, types);
     } catch(DrillRuntimeException e) {
@@ -1151,7 +1149,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public ResultSet getSchemas() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getSchemas();
     } catch(DrillRuntimeException e) {
@@ -1162,7 +1160,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public ResultSet getCatalogs() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getCatalogs();
     } catch(DrillRuntimeException e) {
@@ -1173,14 +1171,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public ResultSet getTableTypes() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getTableTypes();
   }
 
   @Override
   public ResultSet getColumns(String catalog, String schema, String table,
                               String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getColumns(catalog, schema, table, columnNamePattern);
     } catch(DrillRuntimeException e) {
@@ -1193,14 +1191,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getColumnPrivileges(String catalog, String schema,
                                        String table,
                                        String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getColumnPrivileges(catalog, schema, table, columnNamePattern);
   }
 
   @Override
   public ResultSet getTablePrivileges(String catalog, String schemaPattern,
                                       String tableNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getTablePrivileges(catalog, schemaPattern, tableNamePattern);
   }
 
@@ -1208,35 +1206,35 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getBestRowIdentifier(String catalog, String schema,
                                         String table, int scope,
                                         boolean nullable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getBestRowIdentifier(catalog, schema, table, scope, nullable);
   }
 
   @Override
   public ResultSet getVersionColumns(String catalog, String schema,
                                      String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getVersionColumns(catalog, schema, table);
   }
 
   @Override
   public ResultSet getPrimaryKeys(String catalog, String schema,
                                   String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getPrimaryKeys(catalog, schema, table);
   }
 
   @Override
   public ResultSet getImportedKeys(String catalog, String schema,
                                    String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getImportedKeys(catalog, schema, table);
   }
 
   @Override
   public ResultSet getExportedKeys(String catalog, String schema,
                                    String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getExportedKeys(catalog, schema, table);
   }
 
@@ -1245,14 +1243,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
       String parentCatalog, String parentSchema, String parentTable,
       String foreignCatalog, String foreignSchema,
       String foreignTable ) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getCrossReference(parentCatalog, parentSchema, parentTable,
                                    foreignCatalog, foreignSchema, foreignTable );
   }
 
   @Override
   public ResultSet getTypeInfo() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getTypeInfo();
   }
 
@@ -1260,26 +1258,26 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getIndexInfo(String catalog, String schema, String table,
                                 boolean unique,
                                 boolean approximate) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getIndexInfo(catalog, schema, table, unique, approximate);
   }
 
   @Override
   public boolean supportsResultSetType(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsResultSetType(type);
   }
 
   @Override
   public boolean supportsResultSetConcurrency(int type,
                                               int concurrency) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsResultSetConcurrency(type, concurrency);
   }
 
   @Override
   public boolean ownUpdatesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.ownUpdatesAreVisible(type);
     }
@@ -1296,7 +1294,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean ownDeletesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.ownDeletesAreVisible(type);
     }
@@ -1313,7 +1311,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean ownInsertsAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.ownInsertsAreVisible(type);
     }
@@ -1330,7 +1328,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean othersUpdatesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.othersUpdatesAreVisible(type);
     }
@@ -1347,7 +1345,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean othersDeletesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.othersDeletesAreVisible(type);
     }
@@ -1364,7 +1362,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean othersInsertsAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.othersInsertsAreVisible(type);
     }
@@ -1381,7 +1379,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean updatesAreDetected(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.updatesAreDetected(type);
     }
@@ -1398,7 +1396,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean deletesAreDetected(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.deletesAreDetected(type);
     }
@@ -1415,7 +1413,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean insertsAreDetected(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.insertsAreDetected(type);
     }
@@ -1432,7 +1430,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsBatchUpdates() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsBatchUpdates();
   }
 
@@ -1440,7 +1438,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getUDTs(String catalog, String schemaPattern,
                            String typeNamePattern,
                            int[] types) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getUDTs(catalog, schemaPattern, typeNamePattern, types);
   }
 
@@ -1452,39 +1450,39 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSavepoints() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSavepoints();
   }
 
   @Override
   public boolean supportsNamedParameters() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsNamedParameters();
   }
 
   @Override
   public boolean supportsMultipleOpenResults() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMultipleOpenResults();
   }
 
   @Override
   public boolean supportsGetGeneratedKeys() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsGetGeneratedKeys();
   }
 
   @Override
   public ResultSet getSuperTypes(String catalog, String schemaPattern,
                                  String typeNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSuperTypes(catalog, schemaPattern, typeNamePattern);
   }
 
   @Override
   public ResultSet getSuperTables(String catalog, String schemaPattern,
                                   String tableNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSuperTables(catalog, schemaPattern, tableNamePattern);
   }
 
@@ -1492,14 +1490,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getAttributes(String catalog, String schemaPattern,
                                  String typeNamePattern,
                                  String attributeNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getAttributes(catalog, schemaPattern, typeNamePattern,
                                attributeNamePattern);
   }
 
   @Override
   public boolean supportsResultSetHoldability(int holdability) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.supportsResultSetHoldability(holdability);
     }
@@ -1519,9 +1517,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
     // Can't throw any SQLException because Avatica's getResultSetHoldability()
     // is missing "throws SQLException".
     try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
+      checkOpen();
     } catch (SQLException e) {
       throw new RuntimeException(e.getMessage(), e);
     }
@@ -1530,7 +1526,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getDatabaseMajorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     Version version = getServerVersion();
     if (version == null) {
       return super.getDatabaseMajorVersion();
@@ -1540,7 +1536,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getDatabaseMinorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     Version version = getServerVersion();
     if (version == null) {
       return super.getDatabaseMinorVersion();
@@ -1550,69 +1546,69 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getJDBCMajorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getJDBCMajorVersion();
   }
 
   @Override
   public int getJDBCMinorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getJDBCMinorVersion();
   }
 
   @Override
   public int getSQLStateType() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSQLStateType();
   }
 
   @Override
   public boolean locatorsUpdateCopy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.locatorsUpdateCopy();
   }
 
   @Override
   public boolean supportsStatementPooling() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsStatementPooling();
   }
 
   @Override
   public RowIdLifetime getRowIdLifetime() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getRowIdLifetime();
   }
 
   @Override
   public ResultSet getSchemas(String catalog,
                               String schemaPattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSchemas(catalog, schemaPattern);
   }
 
   @Override
   public boolean supportsStoredFunctionsUsingCallSyntax() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsStoredFunctionsUsingCallSyntax();
   }
 
   @Override
   public boolean autoCommitFailureClosesAllResultSets() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.autoCommitFailureClosesAllResultSets();
   }
 
   @Override
   public ResultSet getClientInfoProperties() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getClientInfoProperties();
   }
 
   @Override
   public ResultSet getFunctions(String catalog, String schemaPattern,
                                 String functionNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getFunctions(catalog, schemaPattern, functionNamePattern);
   }
 
@@ -1620,7 +1616,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getFunctionColumns(String catalog, String schemaPattern,
                                       String functionNamePattern,
                                       String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getFunctionColumns(catalog, schemaPattern, functionNamePattern,
                                     columnNamePattern);
   }
@@ -1629,14 +1625,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getPseudoColumns(String catalog, String schemaPattern,
                                     String tableNamePattern,
                                     String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getPseudoColumns(catalog, schemaPattern, tableNamePattern,
                                   columnNamePattern);
   }
 
   @Override
   public boolean generatedKeyAlwaysReturned() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.generatedKeyAlwaysReturned();
   }
 
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
index 308a34d..7d57c85 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
@@ -17,14 +17,9 @@
  */
 package org.apache.drill.jdbc.impl;
 
-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.SQLTimeoutException;
-import java.sql.SQLXML;
 import java.util.Properties;
 import java.util.TimeZone;
 
@@ -42,17 +37,17 @@ import org.apache.drill.exec.rpc.DrillRpcFuture;
 
 
 /**
- * Implementation of {@link net.hydromatic.avatica.AvaticaFactory} for Drill and
+ * Implementation of {@link org.apache.calcite.avatica.AvaticaFactory} for Drill and
  * JDBC 4.1 (corresponds to JDK 1.7).
  */
-// Note:  Must be public so net.hydromatic.avatica.UnregisteredDriver can
+// Note:  Must be public so org.apache.calcite.avatica.UnregisteredDriver can
 // (reflectively) call no-args constructor.
 public class DrillJdbc41Factory extends DrillFactory {
   private static final org.slf4j.Logger logger =
       org.slf4j.LoggerFactory.getLogger(DrillJdbc41Factory.class);
 
   /** Creates a factory for JDBC version 4.1. */
-  // Note:  Must be public so net.hydromatic.avatica.UnregisteredDriver can
+  // Note:  Must be public so org.apache.calcite.avatica.UnregisteredDriver can
   // (reflectively) call this constructor.
   public DrillJdbc41Factory() {
     this(4, 1);
@@ -161,8 +156,6 @@ public class DrillJdbc41Factory extends DrillFactory {
           resultSetHoldability);
     } catch (SQLException e) {
       throw e;
-    } catch (RuntimeException e) {
-      throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
     } catch (Exception e) {
       throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
     }
@@ -173,7 +166,7 @@ public class DrillJdbc41Factory extends DrillFactory {
                                          QueryState state,
                                          Meta.Signature signature,
                                          TimeZone timeZone,
-                                         Meta.Frame firstFrame) {
+                                         Meta.Frame firstFrame) throws SQLException {
     final ResultSetMetaData metaData =
         newResultSetMetaData(statement, signature);
     return new DrillResultSetImpl(statement, state, signature, metaData, timeZone, firstFrame);
@@ -201,111 +194,6 @@ public class DrillJdbc41Factory extends DrillFactory {
       super(connection, h, signature, pstmt,
             resultSetType, resultSetConcurrency, resultSetHoldability);
     }
-
-    // These don't need throwIfClosed(), since getParameter already calls it.
-
-    @Override
-    public void setRowId(int parameterIndex, RowId x) throws SQLException {
-      getSite(parameterIndex).setRowId(x);
-    }
-
-    @Override
-    public void setNString(int parameterIndex, String value) throws SQLException {
-      getSite(parameterIndex).setNString(value);
-    }
-
-    @Override
-    public void setNCharacterStream(int parameterIndex, Reader value,
-                                    long length) throws SQLException {
-      getSite(parameterIndex).setNCharacterStream(value, length);
-    }
-
-    @Override
-    public void setNClob(int parameterIndex, NClob value) throws SQLException {
-      getSite(parameterIndex).setNClob(value);
-    }
-
-    @Override
-    public void setClob(int parameterIndex, Reader reader,
-                        long length) throws SQLException {
-      getSite(parameterIndex).setClob(reader, length);
-    }
-
-    @Override
-    public void setBlob(int parameterIndex, InputStream inputStream,
-                        long length) throws SQLException {
-      getSite(parameterIndex).setBlob(inputStream, length);
-    }
-
-    @Override
-    public void setNClob(int parameterIndex, Reader reader,
-                         long length) throws SQLException {
-      getSite(parameterIndex).setNClob(reader, length);
-    }
-
-    @Override
-    public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {
-      getSite(parameterIndex).setSQLXML(xmlObject);
-    }
-
-    @Override
-    public void setAsciiStream(int parameterIndex, InputStream x,
-                               long length) throws SQLException {
-      getSite(parameterIndex).setAsciiStream(x, length);
-    }
-
-    @Override
-    public void setBinaryStream(int parameterIndex, InputStream x,
-                                long length) throws SQLException {
-      getSite(parameterIndex).setBinaryStream(x, length);
-    }
-
-    @Override
-    public void setCharacterStream(int parameterIndex, Reader reader,
-                                   long length) throws SQLException {
-      getSite(parameterIndex).setCharacterStream(reader, length);
-    }
-
-    @Override
-    public void setAsciiStream(int parameterIndex,
-                               InputStream x) throws SQLException {
-      getSite(parameterIndex).setAsciiStream(x);
-    }
-
-    @Override
-    public void setBinaryStream(int parameterIndex,
-                                InputStream x) throws SQLException {
-      getSite(parameterIndex).setBinaryStream(x);
-    }
-
-    @Override
-    public void setCharacterStream(int parameterIndex,
-                                   Reader reader) throws SQLException {
-      getSite(parameterIndex).setCharacterStream(reader);
-    }
-
-    @Override
-    public void setNCharacterStream(int parameterIndex,
-                                    Reader value) throws SQLException {
-      getSite(parameterIndex).setNCharacterStream(value);
-    }
-
-    @Override
-    public void setClob(int parameterIndex, Reader reader) throws SQLException {
-      getSite(parameterIndex).setClob(reader);
-    }
-
-    @Override
-    public void setBlob(int parameterIndex,
-                        InputStream inputStream) throws SQLException {
-      getSite(parameterIndex).setBlob(inputStream);
-    }
-
-    @Override
-    public void setNClob(int parameterIndex, Reader reader) throws SQLException {
-      getSite(parameterIndex).setNClob(reader);
-    }
-
   }
 
 }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
index a13f936..6b6e57b 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
@@ -17,12 +17,10 @@
  */
 package org.apache.drill.jdbc.impl;
 
-import java.sql.ParameterMetaData;
 import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.sql.SQLWarning;
+import java.sql.SQLType;
 
 import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.AvaticaPreparedStatement;
@@ -68,7 +66,8 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
    *
    * @throws  AlreadyClosedSqlException  if PreparedStatement is closed
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException {
+  @Override
+  protected void checkOpen() throws SQLException {
     if (isClosed()) {
       throw new AlreadyClosedSqlException("PreparedStatement is already closed.");
     }
@@ -77,22 +76,7 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
 
   // Note:  Using dynamic proxies would reduce the quantity (450?) of method
   // overrides by eliminating those that exist solely to check whether the
-  // object is closed.  It would also eliminate the need to throw non-compliant
-  // RuntimeExceptions when Avatica's method declarations won't let us throw
-  // proper SQLExceptions. (Check performance before applying to frequently
-  // called ResultSet.)
-
-  @Override
-  public DrillConnectionImpl getConnection() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getConnection() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return (DrillConnectionImpl) super.getConnection();
-  }
+  // object is closed.
 
   PreparedStatement getPreparedStatementHandle() {
     return preparedStatementHandle;
@@ -100,7 +84,7 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
 
   @Override
   protected AvaticaParameter getParameter(int param) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
         "Prepared-statement dynamic parameters are not supported.");
   }
@@ -116,341 +100,130 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
   // No isWrapperFor(Class<?>) (it doesn't throw SQLException if already closed).
   // No unwrap(Class<T>) (it doesn't throw SQLException if already closed).
 
-  @Override
-  public ResultSet executeQuery(String sql) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeQuery(sql);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public long executeLargeUpdate(String sql) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeLargeUpdate(sql);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
   // No close() (it doesn't throw SQLException if already closed).
 
   @Override
-  public int getMaxFieldSize() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getMaxFieldSize();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void setMaxFieldSize(int max) throws SQLException {
-    throwIfClosed();
-    try {
-      super.setMaxFieldSize(max);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public long getLargeMaxRows() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getLargeMaxRows() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getLargeMaxRows();
-  }
-
-  @Override
-  public void setLargeMaxRows(long max) throws SQLException {
-    throwIfClosed();
-    super.setLargeMaxRows(max);
-  }
-
-  @Override
   public void setEscapeProcessing(boolean enable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setEscapeProcessing(enable);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int getQueryTimeout() throws SQLException {
-    throwIfClosed();
-    return super.getQueryTimeout();
-  }
-
-  @Override
-  public void setQueryTimeout(int seconds) throws SQLException {
-    throwIfClosed();
-    super.setQueryTimeout(seconds);
-  }
-
-  @Override
-  public void cancel() throws SQLException {
-    throwIfClosed();
-    super.cancel();
-  }
-
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
   public void setCursorName(String name) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setCursorName(name);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public boolean execute(String sql) throws SQLException {
-    throwIfClosed();
-    return super.execute(sql);
-  }
-
-  @Override
-  public ResultSet getResultSet() throws SQLException {
-    throwIfClosed();
-    return super.getResultSet();
-  }
-
-  @Override
-  public int getUpdateCount() throws SQLException {
-    throwIfClosed();
-    return super.getUpdateCount();
-  }
-
-  @Override
   public boolean getMoreResults() throws SQLException {
-    throwIfClosed();
     try {
       return super.getMoreResults();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void setFetchDirection(int direction) throws SQLException {
-    throwIfClosed();
-    super.setFetchDirection(direction);
-  }
-
-  @Override
-  public int getFetchDirection(){
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchDirection() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchDirection();
-  }
-
-  @Override
-  public void setFetchSize(int rows) throws SQLException {
-    throwIfClosed();
-    super.setFetchSize(rows);
-  }
-
-  @Override
-  public int getFetchSize() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchSize() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchSize();
-  }
-
-  @Override
-  public int getResultSetConcurrency() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetConcurrency();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetType() throws SQLException {
-    throwIfClosed();
+  public boolean getMoreResults(int current) throws SQLException {
     try {
-      return super.getResultSetType();
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getMoreResults(current);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void addBatch(String sql) throws SQLException {
-    throwIfClosed();
+  public ResultSet getGeneratedKeys() throws SQLException {
+    checkOpen();
     try {
-      super.addBatch(sql);
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getGeneratedKeys();
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void clearBatch() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e);
-    }
-    try {
-      super.clearBatch();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new RuntimeException(new SQLFeatureNotSupportedException(e.getMessage(), e));
-    }
+  public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, autoGeneratedKeys);
   }
 
   @Override
-  public int[] executeBatch() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeBatch();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnIndexes);
   }
 
   @Override
-  public boolean getMoreResults(int current) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getMoreResults(current);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public ResultSet getGeneratedKeys() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getGeneratedKeys();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnNames);
   }
 
   @Override
   public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int executeUpdate(String sql, int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int executeUpdate(String sql, String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetHoldability() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetHoldability();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -461,171 +234,29 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
       return super.isClosed();
     } catch (SQLException e) {
       throw new RuntimeException(
-          "Unexpected " + e + " from AvaticaPreparedStatement.isClosed" );
+          "Unexpected " + e + " from AvaticaPreparedStatement.isClosed");
     }
   }
 
   @Override
   public void setPoolable(boolean poolable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setPoolable(poolable);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public boolean isPoolable() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.isPoolable();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void closeOnCompletion() throws SQLException {
-    throwIfClosed();
-    super.closeOnCompletion();
-  }
-
-  @Override
-  public boolean isCloseOnCompletion() throws SQLException {
-    throwIfClosed();
-    return super.isCloseOnCompletion();
-  }
-
-  @Override
-  public ResultSet executeQuery() throws SQLException {
-    throwIfClosed();
-    return super.executeQuery();
-  }
-
-  @Override
-  public long executeLargeUpdate() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeLargeUpdate();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public void setObject(int parameterIndex, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException {
+    checkOpen();
+    super.setObject(parameterIndex, x, targetSqlType, scaleOrLength);
   }
 
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setNull(int, int)
-  // - setBoolean(int, boolean)
-  // - setByte(int, byte)
-  // - setShort(int, short)
-  // - setInt(int, int)
-  // - setLong(int, long)
-  // - setFloat(int, float)
-  // - setDouble(int, double)
-  // - setBigDecimal(int, BigDecimal)
-  // - setString(int, String)
-  // - setBytes(int, byte[])
-  // - setDate(int, Date)
-  // - setTime(int, Time)
-  // - setTimestamp(int, Timestamp)
-  // - setAsciiStream(int, InputStream, int)
-  // - setUnicodeStream(int, InputStream, int)
-  // - setBinaryStream(int, InputStream, int)
-
   @Override
-  public void clearParameters() throws SQLException {
-    throwIfClosed();
-    try {
-      super.clearParameters();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public void setObject(int parameterIndex, Object x, SQLType targetSqlType) throws SQLException {
+    checkOpen();
+    super.setObject(parameterIndex, x, targetSqlType);
   }
-
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setObject(int, Object, int)
-  // - setObject(int, Object)
-
-  @Override
-  public boolean execute() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.execute();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void addBatch() throws SQLException {
-    throwIfClosed();
-    try {
-      super.addBatch();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setCharacterStream(int, Reader, int)
-  // - setRef(int, Ref)
-  // - setBlob(int, Blob)
-  // - setClob(int, Clob)
-  // - setArray(int, Array)
-
-  @Override
-  public ResultSetMetaData getMetaData() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getMetaData() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getMetaData();
-  }
-
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setDate(int, Date, Calendar)
-  // - setTime(int, Time, Calendar)
-  // - setTimestamp(int, Timestamp, Calendar)
-  // - setNull(int, int, String)
-  // - setURL(int, URL)
-
-  @Override
-  public ParameterMetaData getParameterMetaData() throws SQLException {
-    throwIfClosed();
-    return super.getParameterMetaData();
-  }
-
-  // The following methods are abstract in AvaticaPreparedStatement, and so
-  // cannot be overridden here to add throwIfClosed calls.  They are addressed
-  // via DrillJdbc41Factory (which calls back to getParameter(int) in here,
-  // which calls throwIfClosed()).
-  // - setRowId(int, RowId)
-  // - setNString(int, String)
-  // - setNCharacterStream(int, Reader, long)
-  // - setNClob(int, NClob)
-  // - setClob(int, Reader, long)
-  // - setBlob(int, InputStream, long)
-  // - setNClob(int, Reader, long)
-  // - setSQLXML(int, SQLXML xmlObject)
-  // - setObject(int, Object, int, int)
-  // - setAsciiStream(int, InputStream, long)
-  // - setBinaryStream(int, InputStream, long)
-  // - setCharacterStream(int, Reader, long)
-  // - setAsciiStream(int, InputStream)
-  // - setBinaryStream(int, InputStream)
-  // - setCharacterStream(int, Reader)
-  // - setNCharacterStream(int, Reader)
-  // - setClob(int, Reader)
-  // - setBlob(int, InputStream)
-  // - setNClob(int, Reader)
-
 }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index 40f5733..c0544f1 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -20,7 +20,6 @@ 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;
@@ -31,15 +30,13 @@ import java.sql.ResultSetMetaData;
 import java.sql.RowId;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.sql.SQLWarning;
+import java.sql.SQLType;
 import java.sql.SQLXML;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.List;
-import java.util.Map;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 
@@ -59,7 +56,7 @@ import org.apache.drill.jdbc.SqlTimeoutException;
 import com.google.common.base.Stopwatch;
 
 /**
- * Drill's implementation of {@link ResultSet}.
+ * Drill's implementation of {@link java.sql.ResultSet}.
  */
 class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   @SuppressWarnings("unused")
@@ -75,7 +72,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
 
   DrillResultSetImpl(AvaticaStatement statement, QueryState state, Meta.Signature signature,
                      ResultSetMetaData resultSetMetaData, TimeZone timeZone,
-                     Meta.Frame firstFrame) {
+                     Meta.Frame firstFrame) throws SQLException {
     super(statement, state, signature, resultSetMetaData, timeZone, firstFrame);
     connection = (DrillConnectionImpl) statement.getConnection();
   }
@@ -90,7 +87,8 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
    * @throws  AlreadyClosedSqlException  if ResultSet is closed
    * @throws  SQLException  if error in calling {@link #isClosed()}
    */
-  private void throwIfClosed() throws SQLException {
+  @Override
+  protected void checkOpen() throws SQLException {
     if (isClosed()) {
       if (cursor instanceof DrillCursor && hasPendingCancelationNotification) {
         hasPendingCancelationNotification = false;
@@ -137,11 +135,11 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   // (Not delegated.)
   @Override
   public boolean next() throws SQLException {
-    throwIfClosed();
+    checkOpen();
 
     // TODO:  Resolve following comments (possibly obsolete because of later
-    // addition of preceding call to throwIfClosed.  Also, NOTE that the
-    // following check, and maybe some throwIfClosed() calls, probably must
+    // addition of preceding call to checkOpen.  Also, NOTE that the
+    // following check, and maybe some checkOpen() calls, probably must
     // synchronize on the statement, per the comment on AvaticaStatement's
     // openResultSet:
 
@@ -163,1712 +161,1098 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   }
 
   @Override
-  public boolean wasNull() throws SQLException {
-    throwIfClosed();
-    return super.wasNull();
-  }
-
-  // Methods for accessing results by column index
-  @Override
-  public String getString( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getString( columnIndex );
-  }
-
-  @Override
-  public boolean getBoolean( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBoolean( columnIndex );
-  }
-
-  @Override
-  public byte getByte( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getByte( columnIndex );
-  }
-
-  @Override
-  public short getShort( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getShort( columnIndex );
-  }
-
-  @Override
-  public int getInt( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getInt( columnIndex );
-  }
-
-  @Override
-  public long getLong( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getLong( columnIndex );
-  }
-
-  @Override
-  public float getFloat( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getFloat( columnIndex );
-  }
-
-  @Override
-  public double getDouble( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getDouble( columnIndex );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( int columnIndex,
-                                   int scale ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnIndex, scale );
-  }
-
-  @Override
-  public byte[] getBytes( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBytes( columnIndex );
-  }
-
-  @Override
-  public Date getDate( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnIndex );
-  }
-
-  @Override
-  public Time getTime( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnIndex );
-  }
-
-  @Override
-  public Timestamp getTimestamp( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnIndex );
-  }
-
-  @Override
-  public InputStream getAsciiStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getAsciiStream( columnIndex );
-  }
-
-  @Override
-  public InputStream getUnicodeStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getUnicodeStream( columnIndex );
-  }
-
-  @Override
-  public InputStream getBinaryStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBinaryStream( columnIndex );
-  }
-
-  // Methods for accessing results by column label
-  @Override
-  public String getString( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getString( columnLabel );
-  }
-
-  @Override
-  public boolean getBoolean( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBoolean( columnLabel );
-  }
-
-  @Override
-  public byte getByte( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getByte( columnLabel );
-  }
-
-  @Override
-  public short getShort( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getShort( columnLabel );
-  }
-
-  @Override
-  public int getInt( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getInt( columnLabel );
-  }
-
-  @Override
-  public long getLong( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getLong( columnLabel );
-  }
-
-  @Override
-  public float getFloat( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getFloat( columnLabel );
-  }
-
-  @Override
-  public double getDouble( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getDouble( columnLabel );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( String columnLabel,
-                                   int scale ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnLabel, scale );
-  }
-
-  @Override
-  public byte[] getBytes( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBytes( columnLabel );
-  }
-
-  @Override
-  public Date getDate( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnLabel );
-  }
-
-  @Override
-  public Time getTime( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnLabel );
-  }
-
-  @Override
-  public Timestamp getTimestamp( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnLabel );
-  }
-
-  @Override
-  public InputStream getAsciiStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getAsciiStream( columnLabel );
-  }
-
-  @Override
-  public InputStream getUnicodeStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getUnicodeStream( columnLabel );
-  }
-
-  @Override
-  public InputStream getBinaryStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBinaryStream( columnLabel );
-  }
-
-  // Advanced features:
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
   public String getCursorName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getCursorName();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-  // (Not delegated.)
-  @Override
-  public ResultSetMetaData getMetaData() throws SQLException {
-    throwIfClosed();
-    return super.getMetaData();
-  }
-
   @Override
   public Object getObject( int columnIndex ) throws SQLException {
-    throwIfClosed();
+    checkOpen();
 
-    final Cursor.Accessor accessor;
+    Cursor.Accessor accessor;
     try {
       accessor = accessorList.get(columnIndex - 1);
     } catch (RuntimeException e) {
       throw new SQLException(e);
     }
-    final ColumnMetaData metaData = columnMetaDataList.get(columnIndex - 1);
+    ColumnMetaData metaData = columnMetaDataList.get(columnIndex - 1);
     // Drill returns a float (4bytes) for a SQL Float whereas Calcite would return a double (8bytes)
     int typeId = (metaData.type.id != Types.FLOAT) ? metaData.type.id : Types.REAL;
     return AvaticaSite.get(accessor, typeId, localCalendar);
   }
 
-  @Override
-  public Object getObject( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnLabel );
-  }
-
-  //----------------------------------------------------------------
-  @Override
-  public int findColumn( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.findColumn( columnLabel );
-  }
-
   //--------------------------JDBC 2.0-----------------------------------
-  //---------------------------------------------------------------------
-  // Getters and Setters
-  //---------------------------------------------------------------------
-  @Override
-  public Reader getCharacterStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getCharacterStream( columnIndex );
-  }
-
-  @Override
-  public Reader getCharacterStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getCharacterStream( columnLabel );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnIndex );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnLabel );
-  }
 
   //---------------------------------------------------------------------
   // Traversal/Positioning
   //---------------------------------------------------------------------
   @Override
-  public boolean isBeforeFirst() throws SQLException {
-    throwIfClosed();
-    return super.isBeforeFirst();
-  }
-
-  @Override
-  public boolean isAfterLast() throws SQLException {
-    throwIfClosed();
-    return super.isAfterLast();
-  }
-
-  @Override
-  public boolean isFirst() throws SQLException {
-    throwIfClosed();
-    return super.isFirst();
-  }
-
-  @Override
   public boolean isLast() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.isLast();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void beforeFirst() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.beforeFirst();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void afterLast() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.afterLast();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean first() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.first();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean last() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.last();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int getRow() throws SQLException {
-    throwIfClosed();
-    // Map Avatica's erroneous zero-based row numbers to 1-based, and return 0
-    // after end, per JDBC:
-    return isAfterLast() ? 0 : 1 + super.getRow();
-  }
-
-  @Override
-  public boolean absolute( int row ) throws SQLException {
-    throwIfClosed();
+  public boolean absolute(int row) throws SQLException {
+    checkOpen();
     try {
-      return super.absolute( row );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.absolute(row);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean relative( int rows ) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.relative( rows );
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean previous() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.previous();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   //---------------------------------------------------------------------
-  // Properties
-  //---------------------------------------------------------------------
-
-  @Override
-  public void setFetchDirection( int direction ) throws SQLException {
-    throwIfClosed();
-    super.setFetchDirection( direction );
-  }
-
-  @Override
-  public int getFetchDirection() throws SQLException {
-    throwIfClosed();
-    return super.getFetchDirection();
-  }
-
-  @Override
-  public void setFetchSize( int rows ) throws SQLException {
-    throwIfClosed();
-    super.setFetchSize( rows );
-  }
-
-  @Override
-  public int getFetchSize() throws SQLException {
-    throwIfClosed();
-    return super.getFetchSize();
-  }
-
-  @Override
-  public int getType() throws SQLException {
-    throwIfClosed();
-    return super.getType();
-  }
-
-  @Override
-  public int getConcurrency() throws SQLException {
-    throwIfClosed();
-    return super.getConcurrency();
-  }
-
-  //---------------------------------------------------------------------
   // Updates
   //---------------------------------------------------------------------
   @Override
-  public boolean rowUpdated() throws SQLException {
-    throwIfClosed();
-    return super.rowUpdated();
-  }
-
-  @Override
-  public boolean rowInserted() throws SQLException {
-    throwIfClosed();
-    return super.rowInserted();
-  }
-
-  @Override
-  public boolean rowDeleted() throws SQLException {
-    throwIfClosed();
-    return super.rowDeleted();
-  }
-
-  @Override
-  public void updateNull( int columnIndex ) throws SQLException {
-    throwIfClosed();
+  public void updateNull(int columnIndex) throws SQLException {
+    checkOpen();
     try {
       super.updateNull( columnIndex );
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBoolean( int columnIndex, boolean x ) throws SQLException {
-    throwIfClosed();
+  public void updateBoolean(int columnIndex, boolean x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBoolean( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBoolean(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateByte( int columnIndex, byte x ) throws SQLException {
-    throwIfClosed();
+  public void updateByte(int columnIndex, byte x) throws SQLException {
+    checkOpen();
     try {
       super.updateByte( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateShort( int columnIndex, short x ) throws SQLException {
-    throwIfClosed();
+  public void updateShort(int columnIndex, short x) throws SQLException {
+    checkOpen();
     try {
-      super.updateShort( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateShort(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateInt( int columnIndex, int x ) throws SQLException {
-    throwIfClosed();
+  public void updateInt(int columnIndex, int x) throws SQLException {
+    checkOpen();
     try {
-      super.updateInt( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateInt(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateLong( int columnIndex, long x ) throws SQLException {
-    throwIfClosed();
+  public void updateLong(int columnIndex, long x) throws SQLException {
+    checkOpen();
     try {
-      super.updateLong( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateLong(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateFloat( int columnIndex, float x ) throws SQLException {
-    throwIfClosed();
+  public void updateFloat(int columnIndex, float x) throws SQLException {
+    checkOpen();
     try {
-      super.updateFloat( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateFloat(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDouble( int columnIndex, double x ) throws SQLException {
-    throwIfClosed();
+  public void updateDouble(int columnIndex, double x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDouble( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDouble(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBigDecimal( int columnIndex,
-                                BigDecimal x ) throws SQLException {
-    throwIfClosed();
+  public void updateBigDecimal(int columnIndex, BigDecimal x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBigDecimal( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBigDecimal(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateString( int columnIndex, String x ) throws SQLException {
-    throwIfClosed();
+  public void updateString(int columnIndex, String x) throws SQLException {
+    checkOpen();
     try {
-      super.updateString( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateString(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBytes( int columnIndex, byte[] x ) throws SQLException {
-    throwIfClosed();
+  public void updateBytes(int columnIndex, byte[] x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBytes( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBytes(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDate( int columnIndex, Date x ) throws SQLException {
-    throwIfClosed();
+  public void updateDate(int columnIndex, Date x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDate( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDate(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTime( int columnIndex, Time x ) throws SQLException {
-    throwIfClosed();
+  public void updateTime(int columnIndex, Time x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTime( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTime(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTimestamp( int columnIndex, Timestamp x ) throws SQLException {
-    throwIfClosed();
+  public void updateTimestamp(int columnIndex, Timestamp x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTimestamp( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTimestamp(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( int columnIndex, InputStream x,
-                                 int length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(int columnIndex, InputStream x, int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( int columnIndex, InputStream x,
-                                  int length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(int columnIndex, InputStream x,
+                                 int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( int columnIndex, Reader x,
-                                     int length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(int columnIndex, Reader x,
+                                    int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( int columnIndex, Object x,
-                            int scaleOrLength ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(int columnIndex, Object x,
+                           int scaleOrLength) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnIndex, x, scaleOrLength );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnIndex, x, scaleOrLength);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( int columnIndex, Object x ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(int columnIndex, Object x) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNull( String columnLabel ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(int columnIndex, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException {
+    checkOpen();
+    super.updateObject(columnIndex, x, targetSqlType, scaleOrLength);
+  }
+
+  @Override
+  public void updateObject(String columnLabel, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException {
+    checkOpen();
+    super.updateObject(columnLabel, x, targetSqlType, scaleOrLength);
+  }
+
+  @Override
+  public void updateObject(int columnIndex, Object x, SQLType targetSqlType) throws SQLException {
+    checkOpen();
+    super.updateObject(columnIndex, x, targetSqlType);
+  }
+
+  @Override
+  public void updateObject(String columnLabel, Object x, SQLType targetSqlType) throws SQLException {
+    checkOpen();
+    super.updateObject(columnLabel, x, targetSqlType);
+  }
+
+  @Override
+  public void updateNull(String columnLabel) throws SQLException {
+    checkOpen();
     try {
-      super.updateNull( columnLabel );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNull(columnLabel);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBoolean( String columnLabel, boolean x ) throws SQLException {
-    throwIfClosed();
+  public void updateBoolean(String columnLabel, boolean x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBoolean( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBoolean(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateByte( String columnLabel, byte x ) throws SQLException {
-    throwIfClosed();
+  public void updateByte(String columnLabel, byte x) throws SQLException {
+    checkOpen();
     try {
-      super.updateByte( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateByte(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateShort( String columnLabel, short x ) throws SQLException {
-    throwIfClosed();
+  public void updateShort(String columnLabel, short x) throws SQLException {
+    checkOpen();
     try {
-      super.updateShort( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateShort(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateInt( String columnLabel, int x ) throws SQLException {
-    throwIfClosed();
+  public void updateInt(String columnLabel, int x) throws SQLException {
+    checkOpen();
     try {
-      super.updateInt( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateInt(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateLong( String columnLabel, long x ) throws SQLException {
-    throwIfClosed();
+  public void updateLong(String columnLabel, long x) throws SQLException {
+    checkOpen();
     try {
-      super.updateLong( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateLong(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateFloat( String columnLabel, float x ) throws SQLException {
-    throwIfClosed();
+  public void updateFloat(String columnLabel, float x) throws SQLException {
+    checkOpen();
     try {
-      super.updateFloat( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateFloat(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDouble( String columnLabel, double x ) throws SQLException {
-    throwIfClosed();
+  public void updateDouble(String columnLabel, double x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDouble( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDouble(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBigDecimal( String columnLabel,
-                                BigDecimal x ) throws SQLException {
-    throwIfClosed();
+  public void updateBigDecimal(String columnLabel,
+                               BigDecimal x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBigDecimal( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBigDecimal(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateString( String columnLabel, String x ) throws SQLException {
-    throwIfClosed();
+  public void updateString(String columnLabel, String x) throws SQLException {
+    checkOpen();
     try {
-      super.updateString( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateString(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBytes( String columnLabel, byte[] x ) throws SQLException {
-    throwIfClosed();
+  public void updateBytes(String columnLabel, byte[] x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBytes( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBytes(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDate( String columnLabel, Date x ) throws SQLException {
-    throwIfClosed();
+  public void updateDate(String columnLabel, Date x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDate( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDate(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTime( String columnLabel, Time x ) throws SQLException {
-    throwIfClosed();
+  public void updateTime(String columnLabel, Time x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTime( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTime(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTimestamp( String columnLabel, Timestamp x ) throws SQLException {
-    throwIfClosed();
+  public void updateTimestamp(String columnLabel, Timestamp x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTimestamp( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTimestamp(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( String columnLabel, InputStream x,
-                                 int length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(String columnLabel, InputStream x,
+                                int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( String columnLabel, InputStream x,
-                                  int length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(String columnLabel, InputStream x,
+                                 int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( String columnLabel, Reader reader,
-                                     int length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(String columnLabel, Reader reader,
+                                    int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( String columnLabel, Object x,
-                            int scaleOrLength ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(String columnLabel, Object x,
+                           int scaleOrLength) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnLabel, x, scaleOrLength );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnLabel, x, scaleOrLength);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( String columnLabel, Object x ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(String columnLabel, Object x) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void insertRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.insertRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void updateRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.updateRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void deleteRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.deleteRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void refreshRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.refreshRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void cancelRowUpdates() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.cancelRowUpdates();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void moveToInsertRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.moveToInsertRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void moveToCurrentRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.moveToCurrentRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-  @Override
-  public AvaticaStatement getStatement() throws SQLException {
-    throwIfClosed();
-    return super.getStatement();
-  }
-
-  @Override
-  public Object getObject( int columnIndex,
-                           Map<String, Class<?>> map ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnIndex, map );
-  }
-
-  @Override
-  public Ref getRef( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getRef( columnIndex );
-  }
-
-  @Override
-  public Blob getBlob( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBlob( columnIndex );
-  }
-
-  @Override
-  public Clob getClob( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getClob( columnIndex );
-  }
-
-  @Override
-  public Array getArray( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getArray( columnIndex );
-  }
-
-  @Override
-  public Object getObject( String columnLabel,
-                           Map<String,Class<?>> map ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnLabel, map );
-  }
-
-  @Override
-  public Ref getRef( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getRef( columnLabel );
-  }
-
-  @Override
-  public Blob getBlob( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBlob( columnLabel );
-  }
-
-  @Override
-  public Clob getClob( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getClob( columnLabel );
-  }
-
-  @Override
-  public Array getArray( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getArray( columnLabel );
-  }
-
-  @Override
-  public Date getDate( int columnIndex, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnIndex, cal );
-  }
-
-  @Override
-  public Date getDate( String columnLabel, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnLabel, cal );
-  }
-
-  @Override
-  public Time getTime( int columnIndex, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnIndex, cal );
-  }
-
-  @Override
-  public Time getTime( String columnLabel, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnLabel, cal );
-  }
-
-  @Override
-  public Timestamp getTimestamp( int columnIndex, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnIndex, cal );
-  }
-
-  @Override
-  public Timestamp getTimestamp( String columnLabel,
-                                 Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnLabel, cal );
-  }
-
   //-------------------------- JDBC 3.0 ----------------------------------------
 
   @Override
-  public URL getURL( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getURL( columnIndex );
-  }
-
-  @Override
-  public URL getURL( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getURL( columnLabel );
-  }
-
-  @Override
-  public void updateRef( int columnIndex, Ref x ) throws SQLException {
-    throwIfClosed();
+  public void updateRef(int columnIndex, Ref x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRef( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRef(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateRef( String columnLabel, Ref x ) throws SQLException {
-    throwIfClosed();
+  public void updateRef(String columnLabel, Ref x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRef( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRef(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( int columnIndex, Blob x ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(int columnIndex, Blob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( String columnLabel, Blob x ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(String columnLabel, Blob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( int columnIndex, Clob x ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(int columnIndex, Clob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( String columnLabel, Clob x ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(String columnLabel, Clob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateArray( int columnIndex, Array x ) throws SQLException {
-    throwIfClosed();
+  public void updateArray(int columnIndex, Array x) throws SQLException {
+    checkOpen();
     try {
-      super.updateArray( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateArray(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateArray( String columnLabel, Array x ) throws SQLException {
-    throwIfClosed();
+  public void updateArray(String columnLabel, Array x) throws SQLException {
+    checkOpen();
     try {
-      super.updateArray( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateArray(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   //------------------------- JDBC 4.0 -----------------------------------
   @Override
-  public RowId getRowId( int columnIndex ) throws SQLException {
-    throwIfClosed();
+  public RowId getRowId(int columnIndex) throws SQLException {
+    checkOpen();
     try {
-      return super.getRowId( columnIndex );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getRowId(columnIndex);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public RowId getRowId( String columnLabel ) throws SQLException {
-    throwIfClosed();
+  public RowId getRowId(String columnLabel) throws SQLException {
+    checkOpen();
     try {
-      return super.getRowId( columnLabel );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getRowId(columnLabel);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateRowId( int columnIndex, RowId x ) throws SQLException {
-    throwIfClosed();
+  public void updateRowId(int columnIndex, RowId x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRowId( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRowId(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateRowId( String columnLabel, RowId x ) throws SQLException {
-    throwIfClosed();
+  public void updateRowId(String columnLabel, RowId x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRowId( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRowId(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int getHoldability() throws SQLException {
-    throwIfClosed();
-    return super.getHoldability();
-  }
-
-  @Override
-  public boolean isClosed() throws SQLException {
-    // Note:  No already-closed exception for isClosed().
-    return super.isClosed();
-  }
-
-  @Override
-  public void updateNString( int columnIndex, String nString ) throws SQLException {
-    throwIfClosed();
+  public void updateNString(int columnIndex, String nString) throws SQLException {
+    checkOpen();
     try {
-      super.updateNString( columnIndex, nString );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNString(columnIndex, nString);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNString( String columnLabel,
-                             String nString ) throws SQLException {
-    throwIfClosed();
+  public void updateNString(String columnLabel,
+                            String nString) throws SQLException {
+    checkOpen();
     try {
-      super.updateNString( columnLabel, nString );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNString(columnLabel, nString);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( int columnIndex, NClob nClob ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(int columnIndex, NClob nClob) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnIndex, nClob );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnIndex, nClob);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( String columnLabel, NClob nClob ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(String columnLabel, NClob nClob) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnLabel, nClob );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnLabel, nClob);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public NClob getNClob( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getNClob( columnIndex );
-  }
-
-  @Override
-  public NClob getNClob( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getNClob( columnLabel );
-  }
-
-  @Override
-  public SQLXML getSQLXML( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getSQLXML( columnIndex );
-  }
-
-  @Override
-  public SQLXML getSQLXML( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getSQLXML( columnLabel );
-  }
-
-  @Override
-  public void updateSQLXML( int columnIndex,
-                            SQLXML xmlObject ) throws SQLException {
-    throwIfClosed();
+  public void updateSQLXML(int columnIndex,
+                           SQLXML xmlObject) throws SQLException {
+    checkOpen();
     try {
-      super.updateSQLXML( columnIndex, xmlObject );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateSQLXML(columnIndex, xmlObject);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateSQLXML( String columnLabel,
-                            SQLXML xmlObject ) throws SQLException {
-    throwIfClosed();
+  public void updateSQLXML(String columnLabel,
+                           SQLXML xmlObject) throws SQLException {
+    checkOpen();
     try {
-      super.updateSQLXML( columnLabel, xmlObject );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateSQLXML(columnLabel, xmlObject);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public String getNString( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getNString( columnIndex );
-  }
-
-  @Override
-  public String getNString( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getNString( columnLabel );
-  }
-
-  @Override
-  public Reader getNCharacterStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getNCharacterStream( columnIndex );
-  }
-
-  @Override
-  public Reader getNCharacterStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getNCharacterStream( columnLabel );
-  }
-
-  @Override
-  public void updateNCharacterStream( int columnIndex, Reader x,
-                                      long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(int columnIndex, Reader x,
+                                     long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNCharacterStream( String columnLabel, Reader reader,
-                                      long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(String columnLabel, Reader reader,
+                                     long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( int columnIndex, InputStream x,
-                                 long length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(int columnIndex, InputStream x,
+                                long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( int columnIndex, InputStream x,
-                                  long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(int columnIndex, InputStream x,
+                                 long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( int columnIndex, Reader x,
-                                     long length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(int columnIndex, Reader x,
+                                    long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( String columnLabel, InputStream x,
-                                 long length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(String columnLabel, InputStream x,
+                                long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( String columnLabel, InputStream x,
-                                  long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(String columnLabel, InputStream x,
+                                 long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( String columnLabel, Reader reader,
-                                     long length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(String columnLabel, Reader reader,
+                                    long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( int columnIndex, InputStream inputStream,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(int columnIndex, InputStream inputStream,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnIndex, inputStream, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnIndex, inputStream, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( String columnLabel, InputStream inputStream,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(String columnLabel, InputStream inputStream,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnLabel, inputStream, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnLabel, inputStream, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( int columnIndex,  Reader reader,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(int columnIndex, Reader reader,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnIndex, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnIndex, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( String columnLabel,  Reader reader,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(String columnLabel, Reader reader,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( int columnIndex,  Reader reader,
-                           long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(int columnIndex, Reader reader,
+                          long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnIndex, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnIndex, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( String columnLabel,  Reader reader,
-                           long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(String columnLabel, Reader reader,
+                          long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   //---
   @Override
-  public void updateNCharacterStream( int columnIndex,
-                                      Reader x ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(int columnIndex,
+                                     Reader x) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNCharacterStream( String columnLabel,
-                                      Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(String columnLabel,
+                                     Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( int columnIndex,
-                                 InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(int columnIndex,
+                                InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( int columnIndex,
-                                  InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(int columnIndex,
+                                 InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( int columnIndex,
-                                     Reader x ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(int columnIndex,
+                                    Reader x) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( String columnLabel,
-                                 InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(String columnLabel,
+                                InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( String columnLabel,
-                                  InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(String columnLabel,
+                                 InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( String columnLabel,
-                                     Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(String columnLabel,
+                                    Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( int columnIndex,
-                          InputStream inputStream ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(int columnIndex,
+                         InputStream inputStream) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnIndex, inputStream );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnIndex, inputStream);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( String columnLabel,
-                          InputStream inputStream ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(String columnLabel,
+                         InputStream inputStream) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnLabel, inputStream );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnLabel, inputStream);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( int columnIndex,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(int columnIndex, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnIndex, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnIndex, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( String columnLabel,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(String columnLabel, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( int columnIndex,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(int columnIndex, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnIndex, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnIndex, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( String columnLabel,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(String columnLabel, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-  //------------------------- JDBC 4.1 -----------------------------------
-  @Override
-  public <T> T getObject( int columnIndex, Class<T> type ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnIndex, type );
-  }
-
-  @Override
-  public <T> T getObject( String columnLabel, Class<T> type ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnLabel, type );
-  }
-
-
   ////////////////////////////////////////
   // DrillResultSet methods:
 
   @Override
   public String getQueryId() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (cursor instanceof DrillCursor) {
       return ((DrillCursor) cursor).getQueryId();
     }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
index 643f2aa..901a337 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
@@ -44,8 +44,7 @@ public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
    * @throws  AlreadyClosedSqlException  if ResultSet is closed
    * @throws  SQLException  if error in checking ResultSet's status
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException,
-                                      SQLException {
+  private void checkOpen() throws AlreadyClosedSqlException, SQLException {
     // Statement.isClosed() call is to avoid exception from getResultSet().
     if (statement.isClosed()
         || (statement.getResultSet() != null // result set doesn't exist for prepared statement cases
@@ -56,9 +55,8 @@ public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
   }
 
   private void throwIfClosedOrOutOfBounds(int columnNumber)
-      throws InvalidParameterSqlException,
-             SQLException {
-    throwIfClosed();
+      throws SQLException {
+    checkOpen();
     if (1 > columnNumber || columnNumber > getColumnCount()) {
       throw new InvalidParameterSqlException(
           "Column number " + columnNumber + " out of range of from 1 through "
@@ -81,7 +79,7 @@ public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
 
   @Override
   public int getColumnCount() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getColumnCount();
   }
 
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
index a1b16cb..5386ec6 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
@@ -20,7 +20,6 @@ package org.apache.drill.jdbc.impl;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.sql.SQLWarning;
 
 import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.Meta.StatementHandle;
@@ -29,7 +28,7 @@ import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.DrillStatement;
 
 /**
- * Drill's implementation of {@link Statement}.
+ * Drill's implementation of {@link java.sql.Statement}.
  */
 // (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
@@ -51,8 +50,9 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
    *
    * @throws  AlreadyClosedSqlException  if Statement is closed
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException {
-    if ( isClosed() ) {
+  @Override
+  protected void checkOpen() throws AlreadyClosedSqlException {
+    if (isClosed()) {
       throw new AlreadyClosedSqlException( "Statement is already closed." );
     }
   }
@@ -65,107 +65,52 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
   // called ResultSet.)
 
   @Override
-  public DrillConnectionImpl getConnection() {
-    // Can't throw any SQLException because AvaticaConnection's getConnection() is
-    // missing "throws SQLException".
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
+  public DrillConnectionImpl getConnection() throws SQLException {
+    checkOpen();
     return connection;
   }
 
-  // WORKAROUND:  Work around AvaticaStatement's code that wraps _any_ exception,
-  // even if SQLException, by unwrapping to get cause exception so caller can
-  // throw it directly if it's a SQLException:
-  // TODO:  Any ideas for a better name?
-  private SQLException unwrapIfExtra( final SQLException superMethodException ) {
-    final SQLException result;
-    final Throwable cause = superMethodException.getCause();
-    if ( null != cause && cause instanceof SQLException ) {
-      result = (SQLException) cause;
-    }
-    else {
-      result = superMethodException;
-    }
-    return result;
-  }
-
   @Override
-  public boolean execute( String sql ) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.execute( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
+  public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, autoGeneratedKeys);
   }
 
   @Override
-  public ResultSet executeQuery( String sql ) throws SQLException {
-    try {
-       throwIfClosed();
-       return super.executeQuery( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
+  public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnIndexes);
   }
 
   @Override
-  public long executeLargeUpdate( String sql ) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeLargeUpdate( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
+  public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnNames);
   }
 
   @Override
-  public int executeUpdate( String sql, int[] columnIndexes ) throws SQLException {
-    throwIfClosed();
+  public int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    checkOpen();
     try {
-      return super.executeUpdate( sql, columnIndexes );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.executeUpdate(sql, columnIndexes);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int executeUpdate( String sql, String[] columnNames ) throws SQLException {
-    throwIfClosed();
+  public int executeUpdate(String sql, String[] columnNames) throws SQLException {
+    checkOpen();
     try {
-      return super.executeUpdate( sql, columnNames );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.executeUpdate(sql, columnNames);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void cleanUp() {
-    final DrillConnectionImpl connection1 = connection;
-    connection1.openStatementsRegistry.removeStatement(this);
-  }
-
-  @Override
-  public int getQueryTimeout() throws AlreadyClosedSqlException, SQLException
-  {
-    throwIfClosed();
-    return super.getQueryTimeout();
-  }
-
-  @Override
-  public void setQueryTimeout( int seconds )
-      throws AlreadyClosedSqlException,
-             SQLException {
-    throwIfClosed();
-    super.setQueryTimeout(seconds);
+    connection.openStatementsRegistry.removeStatement(this);
   }
 
   @Override
@@ -190,308 +135,128 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
   // No close() (it doesn't throw SQLException if already closed).
 
   @Override
-  public int getMaxFieldSize() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getMaxFieldSize();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void setMaxFieldSize(int max) throws SQLException {
-    throwIfClosed();
-    try {
-      super.setMaxFieldSize(max);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public long getLargeMaxRows() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getMaxRows() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getLargeMaxRows();
-  }
-
-  @Override
-  public void setLargeMaxRows(long max) throws SQLException {
-    throwIfClosed();
-    super.setLargeMaxRows(max);
-  }
-
-  @Override
   public void setEscapeProcessing(boolean enable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setEscapeProcessing(enable);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void cancel() throws SQLException {
-    throwIfClosed();
-    super.cancel();
-  }
-
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
   public void setCursorName(String name) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setCursorName(name);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public ResultSet getResultSet() throws SQLException {
-    throwIfClosed();
-    return super.getResultSet();
-  }
-
-  @Override
-  public int getUpdateCount() throws SQLException {
-    throwIfClosed();
-    return super.getUpdateCount();
-  }
-
-  @Override
   public boolean getMoreResults() throws SQLException {
-    throwIfClosed();
     try {
       return super.getMoreResults();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void setFetchDirection(int direction) throws SQLException {
-    throwIfClosed();
-    super.setFetchDirection(direction);
-  }
-
-  @Override
-  public int getFetchDirection() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchDirection() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchDirection();
-  }
-
-  @Override
-  public void setFetchSize(int rows) throws SQLException {
-    throwIfClosed();
-    super.setFetchSize(rows);
-  }
-
-  @Override
-  public int getFetchSize() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchSize() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchSize();
-  }
-
-  @Override
-  public int getResultSetConcurrency() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetConcurrency();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetType() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetType();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void addBatch(String sql) throws SQLException {
-    throwIfClosed();
     try {
       super.addBatch(sql);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void clearBatch() throws SQLException {
-    throwIfClosed();
     try {
       super.clearBatch();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int[] executeBatch() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeBatch();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean getMoreResults(int current) throws SQLException {
-    throwIfClosed();
     try {
       return super.getMoreResults(current);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public ResultSet getGeneratedKeys() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getGeneratedKeys();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetHoldability() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetHoldability();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void setPoolable(boolean poolable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setPoolable(poolable);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public boolean isPoolable() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.isPoolable();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
-
-  @Override
-  public void closeOnCompletion() throws SQLException {
-    throwIfClosed();
-    super.closeOnCompletion();
-  }
-
-  @Override
-  public boolean isCloseOnCompletion() throws SQLException {
-    throwIfClosed();
-    return super.isCloseOnCompletion();
-  }
-
 }
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
index aa71043..48efec5 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
@@ -34,12 +34,11 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import com.google.common.base.Function;
-import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
 
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.drill.categories.JdbcTest;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.data.LogicalOperator;
@@ -48,7 +47,6 @@ import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.util.StoragePluginTestUtils;
 import org.apache.drill.jdbc.test.Hook;
-import org.apache.drill.categories.JdbcTest;
 import org.apache.drill.test.BaseDirTestWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -85,18 +83,14 @@ public class JdbcTestBase extends ExecTest {
 
   @BeforeClass
   public static void setUpTestCase() {
-    factory = new SingleConnectionCachingFactory(new ConnectionFactory() {
-      @Override
-      public Connection getConnection(ConnectionInfo info) throws SQLException {
-        return DriverManager.getConnection(info.getUrl(), info.getParamsAsProperties());
-      }
-    });
+    factory = new SingleConnectionCachingFactory(
+        info -> DriverManager.getConnection(info.getUrl(), info.getParamsAsProperties()));
   }
 
   /**
    * Creates a {@link java.sql.Connection connection} using default parameters.
    * @param url connection URL
-   * @throws Exception if connection fails
+   * @throws SQLException if connection fails
    */
   protected static Connection connect(String url) throws SQLException {
     return connect(url, getDefaultProperties());
@@ -110,7 +104,7 @@ public class JdbcTestBase extends ExecTest {
    * Creates a {@link java.sql.Connection connection} using the given parameters.
    * @param url connection URL
    * @param info connection info
-   * @throws Exception if connection fails
+   * @throws SQLException if connection fails
    */
   protected static Connection connect(String url, Properties info) throws SQLException {
     final Connection conn = factory.getConnection(new ConnectionInfo(url, info));
@@ -278,12 +272,8 @@ public class JdbcTestBase extends ExecTest {
 
     public ModelAndSchema(final Properties info, final ConnectionFactory factory) {
       this.info = info;
-      this.adapter = new ConnectionFactoryAdapter() {
-        @Override
-        public Connection createConnection() throws SQLException {
-          return factory.getConnection(new ConnectionInfo("jdbc:drill:zk=local", ModelAndSchema.this.info));
-        }
-      };
+      this.adapter = () -> factory.getConnection(
+          new ConnectionInfo("jdbc:drill:zk=local", ModelAndSchema.this.info));
     }
 
     public TestDataConnection sql(String sql) {
@@ -291,14 +281,8 @@ public class JdbcTestBase extends ExecTest {
     }
 
     public <T> T withConnection(Function<Connection, T> function) throws Exception {
-      Connection connection = null;
-      try {
-        connection = adapter.createConnection();
+      try (Connection connection = adapter.createConnection()) {
         return function.apply(connection);
-      } finally {
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
   }
@@ -316,11 +300,8 @@ public class JdbcTestBase extends ExecTest {
      * Checks that the current SQL statement returns the expected result.
      */
     public TestDataConnection returns(String expected) throws Exception {
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         expected = expected.trim();
         String result = JdbcTestBase.toString(resultSet).trim();
@@ -330,22 +311,12 @@ public class JdbcTestBase extends ExecTest {
           Assert.fail(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected));
         }
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
     public TestDataConnection returnsSet(Set<String> expected) throws Exception {
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         Set<String> result = JdbcTestBase.toStringSet(resultSet);
         resultSet.close();
@@ -354,13 +325,6 @@ public class JdbcTestBase extends ExecTest {
           Assert.fail(String.format("Generated set:\n%s\ndoes not match:\n%s", result, expected));
         }
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
@@ -369,44 +333,24 @@ public class JdbcTestBase extends ExecTest {
      * succeeds if the query returns these lines in any order.
      */
     public TestDataConnection returnsUnordered(String... expecteds) throws Exception {
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         Assert.assertEquals(unsortedList(Arrays.asList(expecteds)), unsortedList(JdbcTestBase.toStrings(resultSet)));
         resultSet.close();
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
     public TestDataConnection displayResults(int recordCount) throws Exception {
       // record count check is done in toString method
 
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         logger.debug(JdbcTestBase.toString(resultSet, recordCount));
         resultSet.close();
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
@@ -420,49 +364,26 @@ public class JdbcTestBase extends ExecTest {
 
     public LogicalPlan logicalPlan() {
       final String[] plan0 = {null};
-      Connection connection = null;
-      Statement statement = null;
-      final Hook.Closeable x = Hook.LOGICAL_PLAN.add(new Function<String, Void>() {
-        @Override
-        public Void apply(String o) {
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.prepareStatement(sql);
+           Hook.Closeable x = Hook.LOGICAL_PLAN.add(
+        (Function<String, Void>) o -> {
           plan0[0] = o;
           return null;
-        }
-      });
-      try {
-        connection = adapter.createConnection();
-        statement = connection.prepareStatement(sql);
+        })) {
         statement.close();
         final String plan = plan0[0].trim();
         return LogicalPlan.parse(PhysicalPlanReaderTestFactory.defaultLogicalPlanPersistence(DrillConfig.create()), plan);
       } catch (Exception e) {
         throw new RuntimeException(e);
-      } finally {
-        if (statement != null) {
-          try {
-            statement.close();
-          } catch (SQLException e) {
-            // ignore
-          }
-        }
-        if (connection != null) {
-          try {
-            connection.close();
-          } catch (SQLException e) {
-            // ignore
-          }
-        }
-        x.close();
       }
     }
 
     public <T extends LogicalOperator> T planContains(final Class<T> operatorClazz) {
-      return (T) Iterables.find(logicalPlan().getSortedOperators(), new Predicate<LogicalOperator>() {
-        @Override
-        public boolean apply(LogicalOperator input) {
-          return input.getClass().equals(operatorClazz);
-        }
-      });
+      return (T) logicalPlan().getSortedOperators().stream()
+          .filter(input -> input.getClass().equals(operatorClazz))
+          .findFirst()
+          .get();
     }
   }
 
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java
index 2ce2766..303565f 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java
@@ -41,7 +41,6 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLClientInfoException;
 import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
@@ -64,8 +63,8 @@ import org.apache.drill.jdbc.AlreadyClosedSqlException;
  *   {@link DatabaseMetaData}.
  * </p>
  * <p>
- *   It does not cover unimplemented {@link CallableStatement} or any relevant
- *   secondary objects such as {@link Array} or {@link Struct}).
+ *   It does not cover unimplemented {@link java.sql.CallableStatement} or any relevant
+ *   secondary objects such as {@link java.sql.Array} or {@link java.sql.Struct}).
  * </p>
  */
 @Category(JdbcTest.class)
@@ -425,15 +424,7 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
                     || method.getName().equals("getClientInfo"))) {
         // Special good case--we had to use SQLClientInfoException from those.
         result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && normalClosedExceptionText.equals(cause.getMessage())
-               && (method.getName().equals("getCatalog")
-                  || method.getName().equals("getSchema"))) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      }
-      else {
+      } else {
         result = false;
       }
       return result;
@@ -462,41 +453,6 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
     ClosedPlainStatementChecker(Class<Statement> intf, Statement jdbcObject) {
       super(intf, jdbcObject, PLAIN_STATEMENT_CLOSED_MESSAGE);
     }
-
-    @Override
-    protected boolean isOkayNonthrowingMethod(Method method) {
-      // TODO: Java 8 method
-      if ("getLargeUpdateCount".equals(method.getName())) {
-        return true; }
-      return super.isOkayNonthrowingMethod(method);
-    }
-
-    @Override
-    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
-      final boolean result;
-      if (super.isOkaySpecialCaseException(method, cause)) {
-        result = true;
-      }
-      else if (method.getName().equals("executeLargeBatch")
-               || method.getName().equals("executeLargeUpdate")) {
-        // TODO: New Java 8 methods not implemented in Avatica.
-        result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && normalClosedExceptionText.equals(cause.getMessage())
-               && (method.getName().equals("getConnection")
-                   || method.getName().equals("getFetchDirection")
-                   || method.getName().equals("getFetchSize")
-                   || method.getName().equals("getMaxRows")
-                   || method.getName().equals("getLargeMaxRows"))) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      }
-      else {
-        result = false;
-      }
-      return result;
-    }
   } // class ClosedPlainStatementChecker
 
   @Test
@@ -521,40 +477,6 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
                                    PreparedStatement jdbcObject) {
       super(intf, jdbcObject, PREPAREDSTATEMENT_CLOSED_MESSAGE);
     }
-
-    @Override
-    protected boolean isOkayNonthrowingMethod(Method method) {
-      // TODO: Java 8 methods not yet supported by Avatica.
-      if (method.getName().equals("getLargeUpdateCount")) {
-        return true;
-      }
-      return super.isOkayNonthrowingMethod(method);
-    }
-
-    @Override
-    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
-      final boolean result;
-      if (super.isOkaySpecialCaseException(method, cause)) {
-        result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && cause.getMessage().contains(normalClosedExceptionText)
-               && (method.getName().equals("getConnection")
-                   || method.getName().equals("getFetchDirection")
-                   || method.getName().equals("getFetchSize")
-                   || method.getName().equals("getMaxRows")
-                   || method.getName().equals("getMetaData")
-                   || method.getName().equals("clearBatch"))) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      } else {
-        result = method.getName().equals("setObject")
-          || method.getName().equals("executeLargeUpdate")
-          || method.getName().equals("executeLargeBatch")
-          || method.getName().equals("getLargeMaxRows");
-      }
-      return result;
-    }
   } // class closedPreparedStmtOfOpenConnChecker
 
   @Test
@@ -579,29 +501,6 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
     ClosedResultSetChecker(Class<ResultSet> intf, ResultSet jdbcObject) {
       super(intf, jdbcObject, RESULTSET_CLOSED_MESSAGE);
     }
-
-    @Override
-    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
-      final boolean result;
-      if (super.isOkaySpecialCaseException(method, cause)) {
-        result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && normalClosedExceptionText.equals(cause.getMessage())
-               && method.getName().equals("getStatement")) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      }
-      else if (SQLFeatureNotSupportedException.class == cause.getClass()
-               && (method.getName().equals("updateObject"))) {
-        // TODO: Java 8 methods not yet supported by Avatica.
-        result = true;
-      }
-      else {
-        result = false;
-      }
-      return result;
-    }
   } // class ClosedResultSetChecker
 
   @Test
diff --git a/pom.xml b/pom.xml
index 165e7e2..bf9cf10 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,8 +45,8 @@
     <dep.guava.version>18.0</dep.guava.version>
     <forkCount>2</forkCount>
     <parquet.version>1.10.0</parquet.version>
-    <calcite.version>1.16.0-drill-r7</calcite.version>
-    <avatica.version>1.11.0</avatica.version>
+    <calcite.version>1.17.0-drill-r0</calcite.version>
+    <avatica.version>1.12.0</avatica.version>
     <janino.version>2.7.6</janino.version>
     <sqlline.version>1.1.9-drill-r7</sqlline.version>
     <jackson.version>2.9.5</jackson.version>