You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by el...@apache.org on 2016/03/29 00:39:30 UTC

[2/2] calcite git commit: [CALCITE-836] Expose more server-side database properties

[CALCITE-836] Expose more server-side database properties

A number of properties from the Avatica server can be
exposed to clients, including the Avatica server version
and version information about the wrapped JDBC driver and database.

To support this, DatabaseProperty's no longer need to refer to methods in
DatabaseMetaData. Thus, we can support a generic collection
of Avatica-based properties.

This also allows DatabaseMetaData to be unwrap'ed into Properties.

Some error handling in protobuf code is also improved
to give some better error messages (ran into this while
debugging this feature).

Closes apache/calcite#216


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

Branch: refs/heads/master
Commit: ba9783f30398c52e514db06191d5faf631e57490
Parents: 87b0eb5
Author: Josh Elser <el...@apache.org>
Authored: Thu Mar 10 14:36:27 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Mar 28 18:34:10 2016 -0400

----------------------------------------------------------------------
 avatica/core/pom.xml                            |   39 +
 .../calcite/avatica/util/FilteredConstants.java |   26 +
 .../calcite/avatica/AvaticaConnection.java      |    7 +-
 .../avatica/AvaticaDatabaseMetaData.java        | 1421 +----------------
 .../avatica/AvaticaDatabaseMetaDataImpl.java    | 1460 ++++++++++++++++++
 .../calcite/avatica/AvaticaJdbc41Factory.java   |    2 +-
 .../calcite/avatica/InternalProperty.java       |    2 +-
 .../java/org/apache/calcite/avatica/Meta.java   |   60 +-
 .../avatica/remote/ProtobufTranslationImpl.java |   10 +-
 .../apache/calcite/avatica/remote/Service.java  |   27 +-
 avatica/pom.xml                                 |    1 +
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |   18 +-
 .../calcite/avatica/RemoteDriverTest.java       |   33 +
 .../calcite/avatica/remote/RemoteMetaTest.java  |   24 +
 14 files changed, 1721 insertions(+), 1409 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/ba9783f3/avatica/core/pom.xml
----------------------------------------------------------------------
diff --git a/avatica/core/pom.xml b/avatica/core/pom.xml
index 73e97ce..d30865f 100644
--- a/avatica/core/pom.xml
+++ b/avatica/core/pom.xml
@@ -120,6 +120,24 @@ limitations under the License.
     </pluginManagement>
     <plugins>
       <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-filtered-java-source</id>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <phase>generate-sources</phase>
+            <configuration>
+              <sources>
+                <source>${project.build.directory}/filtered-java-src</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <artifactId>maven-dependency-plugin</artifactId>
         <executions>
           <execution>
@@ -170,6 +188,27 @@ limitations under the License.
         </executions>
       </plugin>
       <plugin>
+        <artifactId>maven-resources-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>prepare-filtered-java-source</id>
+            <goals>
+              <goal>copy-resources</goal>
+            </goals>
+            <phase>generate-sources</phase>
+            <configuration>
+              <outputDirectory>${project.build.directory}/filtered-java-src</outputDirectory>
+              <resources>
+                <resource>
+                  <directory>src/main/java-filtered</directory>
+                  <filtering>true</filtering>
+                </resource>
+              </resources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-shade-plugin</artifactId>
         <executions>

http://git-wip-us.apache.org/repos/asf/calcite/blob/ba9783f3/avatica/core/src/main/java-filtered/org/apache/calcite/avatica/util/FilteredConstants.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java-filtered/org/apache/calcite/avatica/util/FilteredConstants.java b/avatica/core/src/main/java-filtered/org/apache/calcite/avatica/util/FilteredConstants.java
new file mode 100644
index 0000000..53b76ee
--- /dev/null
+++ b/avatica/core/src/main/java-filtered/org/apache/calcite/avatica/util/FilteredConstants.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.util;
+
+/**
+ * A class which, at build time, will have build-specific variables substituted into it.
+ */
+public class FilteredConstants {
+  public static final String VERSION = "${avatica.release.version}";
+}
+
+// End FilteredConstants.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/ba9783f3/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
index 091fe6b..4cd529f 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -111,7 +111,12 @@ public abstract class AvaticaConnection implements Connection {
     this.info = info;
     this.meta = driver.createMeta(this);
     this.metaData = factory.newDatabaseMetaData(this);
-    this.holdability = metaData.getResultSetHoldability();
+    try {
+      this.holdability = metaData.getResultSetHoldability();
+    } catch (SQLException e) {
+      // We know the impl doesn't throw this.
+      throw new RuntimeException(e);
+    }
     this.maxRetriesPerExecute = getNumStatementRetries(info);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/ba9783f3/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
index b57f36c..10ced87 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
@@ -16,1401 +16,42 @@
  */
 package org.apache.calcite.avatica;
 
-import org.apache.calcite.avatica.AvaticaConnection.CallableWithoutException;
-import org.apache.calcite.avatica.remote.MetaDataOperation;
-import org.apache.calcite.avatica.util.Casing;
-import org.apache.calcite.avatica.util.Quoting;
-
-import java.sql.Connection;
 import java.sql.DatabaseMetaData;
-import java.sql.ResultSet;
-import java.sql.RowIdLifetime;
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.calcite.avatica.InternalProperty.CASE_SENSITIVE;
-import static org.apache.calcite.avatica.InternalProperty.NULL_SORTING;
-import static org.apache.calcite.avatica.InternalProperty.NullSorting;
-import static org.apache.calcite.avatica.InternalProperty.QUOTED_CASING;
-import static org.apache.calcite.avatica.InternalProperty.QUOTING;
-import static org.apache.calcite.avatica.InternalProperty.UNQUOTED_CASING;
+import java.util.Properties;
 
 /**
- * Implementation of {@link java.sql.DatabaseMetaData}
- * for the Avatica engine.
+ * Avatica additions to the JDBC {@link DatabaseMetaData} interface. An instance of this is can be
+ * obtained by using {@link #unwrap(Class)} to cast an instance of {@link DatabaseMetaData} to
+ * {@link AvaticaDatabaseMetaData}. {@link #isWrapperFor(Class)} can be used to ensure that
+ * the generic interface can be cast to the desired class.
+ *
+ * <p>A list of all available server-side properties is enumerated by
+ * {@link org.apache.calcite.avatica.Meta.DatabaseProperty}. The name of the enum value will be
+ * the name of the key in the {@link Properties} returned.
  *
- * <p>This class has sub-classes which implement JDBC 3.0 and JDBC 4.0 APIs;
- * it is instantiated using {@link AvaticaFactory#newDatabaseMetaData}.</p>
+ * <p>Some properties defined in {@link org.apache.calcite.avatica.Meta.DatabaseProperty} do not
+ * correspond to a typical JDBC method/property. Those are enumerated here:
+ * <table>
+ *   <tr><th>Property</th><th>Method</th></tr>
+ *   <tr><td>AVATICA_VERSION</td><td>getAvaticaServerVersion()</td></tr>
+ * </table>
  */
-public class AvaticaDatabaseMetaData implements DatabaseMetaData {
-  private final AvaticaConnection connection;
-
-  protected  AvaticaDatabaseMetaData(AvaticaConnection connection) {
-    this.connection = connection;
-  }
-
-  // Helper methods
-
-  private NullSorting nullSorting() {
-    return NULL_SORTING.getEnum(getProperties(), NullSorting.class);
-  }
-
-  private Quoting quoting() {
-    return QUOTING.getEnum(getProperties(), Quoting.class);
-  }
-
-  private Casing unquotedCasing() {
-    return UNQUOTED_CASING.getEnum(getProperties(), Casing.class);
-  }
-
-  private Casing quotedCasing() {
-    return QUOTED_CASING.getEnum(getProperties(), Casing.class);
-  }
-
-  private boolean caseSensitive() {
-    return CASE_SENSITIVE.getBoolean(getProperties());
-  }
-
-  // JDBC methods
-
-  public boolean allProceduresAreCallable() throws SQLException {
-    return true;
-  }
-
-  public boolean allTablesAreSelectable() throws SQLException {
-    return true;
-  }
-
-  public String getURL() throws SQLException {
-    return connection.url;
-  }
-
-  public String getUserName() throws SQLException {
-    return connection.info.getProperty("user");
-  }
-
-  public boolean isReadOnly() throws SQLException {
-    return true;
-  }
-
-  public boolean nullsAreSortedHigh() throws SQLException {
-    return nullSorting() == NullSorting.HIGH;
-  }
-
-  public boolean nullsAreSortedLow() throws SQLException {
-    return nullSorting() == NullSorting.LOW;
-  }
-
-  public boolean nullsAreSortedAtStart() throws SQLException {
-    return nullSorting() == NullSorting.START;
-  }
-
-  public boolean nullsAreSortedAtEnd() throws SQLException {
-    return nullSorting() == NullSorting.END;
-  }
-
-  public String getDatabaseProductName() throws SQLException {
-    return connection.driver.version.productName;
-  }
-
-  public String getDatabaseProductVersion() throws SQLException {
-    return connection.driver.version.productVersion;
-  }
-
-  public String getDriverName() throws SQLException {
-    return connection.driver.version.name;
-  }
-
-  public String getDriverVersion() throws SQLException {
-    return connection.driver.version.versionString;
-  }
-
-  public int getDriverMajorVersion() {
-    return connection.driver.getMajorVersion();
-  }
-
-  public int getDriverMinorVersion() {
-    return connection.driver.getMinorVersion();
-  }
-
-  public boolean usesLocalFiles() throws SQLException {
-    return false;
-  }
-
-  public boolean usesLocalFilePerTable() throws SQLException {
-    return false;
-  }
-
-  public boolean storesMixedCaseIdentifiers() throws SQLException {
-    return !caseSensitive() && unquotedCasing() == Casing.UNCHANGED;
-  }
-
-  public boolean supportsMixedCaseIdentifiers() throws SQLException {
-    return caseSensitive() && unquotedCasing() == Casing.UNCHANGED;
-  }
-
-  public boolean storesUpperCaseIdentifiers() throws SQLException {
-    return unquotedCasing() == Casing.TO_UPPER;
-  }
-
-  public boolean storesLowerCaseIdentifiers() throws SQLException {
-    return unquotedCasing() == Casing.TO_LOWER;
-  }
-
-  public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
-    return !caseSensitive() && quotedCasing() == Casing.UNCHANGED;
-  }
-
-  public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
-    return caseSensitive() && quotedCasing() == Casing.UNCHANGED;
-  }
-
-  public boolean storesUpperCaseQuotedIdentifiers() throws SQLException {
-    return quotedCasing() == Casing.TO_UPPER;
-  }
-
-  public boolean storesLowerCaseQuotedIdentifiers() throws SQLException {
-    return quotedCasing() == Casing.TO_LOWER;
-  }
-
-  public String getIdentifierQuoteString() throws SQLException {
-    return quoting().string;
-  }
-
-  private Map<InternalProperty, Object> getProperties() {
-    return connection.properties;
-  }
-
-  public String getSQLKeywords() throws SQLException {
-    return connection.invokeWithRetries(
-        new CallableWithoutException<String>() {
-          public String call() {
-            return Meta.DatabaseProperty.GET_S_Q_L_KEYWORDS
-                .getProp(connection.meta, connection.handle, String.class);
-          }
-        });
-  }
-
-  public String getNumericFunctions() throws SQLException {
-    return connection.invokeWithRetries(
-        new CallableWithoutException<String>() {
-          public String call() {
-            return Meta.DatabaseProperty.GET_NUMERIC_FUNCTIONS
-                .getProp(connection.meta, connection.handle, String.class);
-          }
-        });
-  }
-
-  public String getStringFunctions() throws SQLException {
-    return connection.invokeWithRetries(
-        new CallableWithoutException<String>() {
-          public String call() {
-            return Meta.DatabaseProperty.GET_STRING_FUNCTIONS
-                .getProp(connection.meta, connection.handle, String.class);
-          }
-        });
-  }
-
-  public String getSystemFunctions() throws SQLException {
-    return connection.invokeWithRetries(
-        new CallableWithoutException<String>() {
-          public String call() {
-            return Meta.DatabaseProperty.GET_SYSTEM_FUNCTIONS
-                .getProp(connection.meta, connection.handle, String.class);
-          }
-        });
-  }
-
-  public String getTimeDateFunctions() throws SQLException {
-    return connection.invokeWithRetries(
-        new CallableWithoutException<String>() {
-          public String call() {
-            return Meta.DatabaseProperty.GET_TIME_DATE_FUNCTIONS
-                .getProp(connection.meta, connection.handle, String.class);
-          }
-        });
-  }
-
-  public String getSearchStringEscape() throws SQLException {
-    return "\\";
-  }
-
-  public String getExtraNameCharacters() throws SQLException {
-    return "";
-  }
-
-  public boolean supportsAlterTableWithAddColumn() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsAlterTableWithDropColumn() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsColumnAliasing() throws SQLException {
-    return true;
-  }
-
-  public boolean nullPlusNonNullIsNull() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsConvert() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsConvert(int fromType, int toType) throws SQLException {
-    return false; // TODO: more detail
-  }
-
-  public boolean supportsTableCorrelationNames() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsDifferentTableCorrelationNames() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsExpressionsInOrderBy() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsOrderByUnrelated() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsGroupBy() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsGroupByUnrelated() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsGroupByBeyondSelect() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsLikeEscapeClause() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsMultipleResultSets() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsMultipleTransactions() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsNonNullableColumns() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsMinimumSQLGrammar() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsCoreSQLGrammar() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsExtendedSQLGrammar() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsANSI92EntryLevelSQL() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsANSI92IntermediateSQL() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsANSI92FullSQL() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsIntegrityEnhancementFacility() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsOuterJoins() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsFullOuterJoins() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsLimitedOuterJoins() throws SQLException {
-    return true;
-  }
-
-  public String getSchemaTerm() throws SQLException {
-    return "schema";
-  }
-
-  public String getProcedureTerm() throws SQLException {
-    return "procedure";
-  }
-
-  public String getCatalogTerm() throws SQLException {
-    return "catalog";
-  }
-
-  public boolean isCatalogAtStart() throws SQLException {
-    return true;
-  }
-
-  public String getCatalogSeparator() throws SQLException {
-    return ".";
-  }
-
-  public boolean supportsSchemasInDataManipulation() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsSchemasInProcedureCalls() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsSchemasInTableDefinitions() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsSchemasInIndexDefinitions() throws SQLException {
-    return true; // except that we don't support index definitions
-  }
-
-  public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException {
-    return true; // except that we don't support privilege definitions
-  }
-
-  public boolean supportsCatalogsInDataManipulation() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsCatalogsInProcedureCalls() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsCatalogsInTableDefinitions() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsCatalogsInIndexDefinitions() throws SQLException {
-    return true; // except that we don't support index definitions
-  }
-
-  public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException {
-    return true; // except that we don't support privilege definitions
-  }
-
-  public boolean supportsPositionedDelete() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsPositionedUpdate() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsSelectForUpdate() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsStoredProcedures() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsSubqueriesInComparisons() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsSubqueriesInExists() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsSubqueriesInIns() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsSubqueriesInQuantifieds() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsCorrelatedSubqueries() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsUnion() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsUnionAll() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsOpenCursorsAcrossCommit() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsOpenCursorsAcrossRollback() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsOpenStatementsAcrossCommit() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsOpenStatementsAcrossRollback() throws SQLException {
-    return false;
-  }
-
-  public int getMaxBinaryLiteralLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxCharLiteralLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxColumnNameLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxColumnsInGroupBy() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxColumnsInIndex() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxColumnsInOrderBy() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxColumnsInSelect() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxColumnsInTable() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxConnections() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxCursorNameLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxIndexLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxSchemaNameLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxProcedureNameLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxCatalogNameLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxRowSize() throws SQLException {
-    return 0;
-  }
-
-  public boolean doesMaxRowSizeIncludeBlobs() throws SQLException {
-    return false;
-  }
-
-  public int getMaxStatementLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxStatements() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxTableNameLength() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxTablesInSelect() throws SQLException {
-    return 0;
-  }
-
-  public int getMaxUserNameLength() throws SQLException {
-    return 0;
-  }
-
-  public int getDefaultTransactionIsolation() throws SQLException {
-    return connection.invokeWithRetries(
-        new CallableWithoutException<Integer>() {
-          public Integer call() {
-            return Meta.DatabaseProperty.GET_DEFAULT_TRANSACTION_ISOLATION
-                .getProp(connection.meta, connection.handle, Integer.class);
-          }
-        });
-  }
-
-  public boolean supportsTransactions() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsTransactionIsolationLevel(int level)
-      throws SQLException {
-    return level == Connection.TRANSACTION_NONE;
-  }
-
-  public boolean supportsDataDefinitionAndDataManipulationTransactions()
-      throws SQLException {
-    return false;
-  }
-
-  public boolean supportsDataManipulationTransactionsOnly()
-      throws SQLException {
-    return true;
-  }
-
-  public boolean dataDefinitionCausesTransactionCommit() throws SQLException {
-    return true;
-  }
-
-  public boolean dataDefinitionIgnoredInTransactions() throws SQLException {
-    return false;
-  }
-
-  public ResultSet getProcedures(
-      final String catalog,
-      final String schemaPattern,
-      final String procedureNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getProcedures(connection.handle, catalog, pat(schemaPattern),
-                        pat(procedureNamePattern)),
-                    new QueryState(MetaDataOperation.GET_PROCEDURES, catalog, schemaPattern,
-                        procedureNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getProcedureColumns(
-      final String catalog,
-      final String schemaPattern,
-      final String procedureNamePattern,
-      final String columnNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getProcedureColumns(connection.handle, catalog,
-                        pat(schemaPattern), pat(procedureNamePattern), pat(columnNamePattern)),
-                    new QueryState(MetaDataOperation.GET_PROCEDURE_COLUMNS, catalog, schemaPattern,
-                        procedureNamePattern, columnNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getTables(
-      final String catalog,
-      final String schemaPattern,
-      final String tableNamePattern,
-      final String[] types) throws SQLException {
-    final List<String> typeList = types == null ? null : Arrays.asList(types);
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getTables(connection.handle, catalog, pat(schemaPattern),
-                        pat(tableNamePattern), typeList),
-                    new QueryState(MetaDataOperation.GET_TABLES, catalog, schemaPattern,
-                        tableNamePattern, types));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  private static Meta.Pat pat(String schemaPattern) {
-    return Meta.Pat.of(schemaPattern);
-  }
-
-  public ResultSet getSchemas(
-      final String catalog, final String schemaPattern) throws SQLException {
-    // TODO: add a 'catch ... throw new SQLException' logic to this and other
-    // getXxx methods. Right now any error will throw a RuntimeException
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getSchemas(connection.handle, catalog, pat(schemaPattern)),
-                    new QueryState(MetaDataOperation.GET_SCHEMAS_WITH_ARGS, catalog,
-                        schemaPattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getSchemas() throws SQLException {
-    return getSchemas(null, null);
-  }
-
-  public ResultSet getCatalogs() throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(connection.meta.getCatalogs(connection.handle),
-                    new QueryState(MetaDataOperation.GET_CATALOGS));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getTableTypes() throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(connection.meta.getTableTypes(connection.handle),
-                    new QueryState(MetaDataOperation.GET_TABLE_TYPES));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getColumns(
-      final String catalog,
-      final String schemaPattern,
-      final String tableNamePattern,
-      final String columnNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getColumns(connection.handle, catalog, pat(schemaPattern),
-                        pat(tableNamePattern), pat(columnNamePattern)),
-                    new QueryState(MetaDataOperation.GET_COLUMNS, catalog, schemaPattern,
-                        tableNamePattern, columnNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getColumnPrivileges(
-      final String catalog,
-      final String schema,
-      final String table,
-      final String columnNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getColumnPrivileges(connection.handle, catalog, schema, table,
-                        pat(columnNamePattern)),
-                    new QueryState(MetaDataOperation.GET_COLUMN_PRIVILEGES, catalog, schema, table,
-                        columnNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getTablePrivileges(
-      final String catalog,
-      final String schemaPattern,
-      final String tableNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getTablePrivileges(connection.handle, catalog,
-                        pat(schemaPattern), pat(tableNamePattern)),
-                    new QueryState(MetaDataOperation.GET_TABLE_PRIVILEGES, catalog, schemaPattern,
-                        tableNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getBestRowIdentifier(
-      final String catalog,
-      final String schema,
-      final String table,
-      final int scope,
-      final boolean nullable) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getBestRowIdentifier(connection.handle, catalog, schema, table,
-                        scope, nullable),
-                    new QueryState(MetaDataOperation.GET_BEST_ROW_IDENTIFIER, catalog, table, scope,
-                        nullable));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getVersionColumns(
-      final String catalog, final String schema, final String table) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getVersionColumns(connection.handle, catalog, schema, table),
-                    new QueryState(MetaDataOperation.GET_VERSION_COLUMNS, catalog, schema, table));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getPrimaryKeys(
-      final String catalog, final String schema, final String table) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getPrimaryKeys(connection.handle, catalog, schema, table),
-                    new QueryState(MetaDataOperation.GET_PRIMARY_KEYS, catalog, schema, table));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getImportedKeys(
-      final String catalog, final String schema, final String table) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getImportedKeys(connection.handle, catalog, schema, table),
-                    new QueryState(MetaDataOperation.GET_IMPORTED_KEYS, catalog, schema, table));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getExportedKeys(
-      final String catalog, final String schema, final String table) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getExportedKeys(connection.handle, catalog, schema, table),
-                    new QueryState(MetaDataOperation.GET_EXPORTED_KEYS, catalog, schema, table));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getCrossReference(
-      final String parentCatalog,
-      final String parentSchema,
-      final String parentTable,
-      final String foreignCatalog,
-      final String foreignSchema,
-      final String foreignTable) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getCrossReference(connection.handle, parentCatalog,
-                        parentSchema, parentTable, foreignCatalog, foreignSchema, foreignTable),
-                    new QueryState(MetaDataOperation.GET_CROSS_REFERENCE, parentCatalog,
-                        parentSchema, parentTable, foreignCatalog, foreignSchema, foreignTable));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getTypeInfo() throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(connection.meta.getTypeInfo(connection.handle),
-                    new QueryState(MetaDataOperation.GET_TYPE_INFO));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getIndexInfo(
-      final String catalog,
-      final String schema,
-      final String table,
-      final boolean unique,
-      final boolean approximate) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getIndexInfo(connection.handle, catalog, schema, table, unique,
-                        approximate),
-                    new QueryState(MetaDataOperation.GET_INDEX_INFO, catalog, schema, table, unique,
-                        approximate));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public boolean supportsResultSetType(int type) throws SQLException {
-    return type == ResultSet.TYPE_FORWARD_ONLY;
-  }
-
-  public boolean supportsResultSetConcurrency(
-      int type, int concurrency) throws SQLException {
-    return type == ResultSet.TYPE_FORWARD_ONLY
-        && concurrency == ResultSet.CONCUR_READ_ONLY;
-  }
-
-  public boolean ownUpdatesAreVisible(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean ownDeletesAreVisible(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean ownInsertsAreVisible(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean othersUpdatesAreVisible(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean othersDeletesAreVisible(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean othersInsertsAreVisible(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean updatesAreDetected(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean deletesAreDetected(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean insertsAreDetected(int type) throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public boolean supportsBatchUpdates() throws SQLException {
-    return false;
-  }
-
-  public ResultSet getUDTs(
-      final String catalog,
-      final String schemaPattern,
-      final String typeNamePattern,
-      final int[] types) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getUDTs(connection.handle, catalog, pat(schemaPattern),
-                        pat(typeNamePattern), types),
-                    new QueryState(MetaDataOperation.GET_UDTS, catalog, schemaPattern,
-                        typeNamePattern, types));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public Connection getConnection() throws SQLException {
-    return connection;
-  }
-
-  public boolean supportsSavepoints() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsNamedParameters() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsMultipleOpenResults() throws SQLException {
-    return false;
-  }
-
-  public boolean supportsGetGeneratedKeys() throws SQLException {
-    return false;
-  }
-
-  public ResultSet getSuperTypes(
-      final String catalog,
-      final String schemaPattern,
-      final String typeNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getSuperTypes(connection.handle, catalog, pat(schemaPattern),
-                        pat(typeNamePattern)),
-                    new QueryState(MetaDataOperation.GET_SUPER_TYPES, catalog, schemaPattern,
-                        typeNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getSuperTables(
-      final String catalog,
-      final String schemaPattern,
-      final String tableNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getSuperTables(connection.handle, catalog, pat(schemaPattern),
-                        pat(tableNamePattern)),
-                    new QueryState(MetaDataOperation.GET_SUPER_TABLES, catalog, schemaPattern,
-                        tableNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getAttributes(
-      final String catalog,
-      final String schemaPattern,
-      final String typeNamePattern,
-      final String attributeNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getAttributes(connection.handle, catalog, pat(schemaPattern),
-                        pat(typeNamePattern), pat(attributeNamePattern)),
-                    new QueryState(MetaDataOperation.GET_ATTRIBUTES, catalog, schemaPattern,
-                        typeNamePattern, attributeNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public boolean supportsResultSetHoldability(int holdability)
-      throws SQLException {
-    throw connection.helper.todo();
-  }
-
-  public int getResultSetHoldability() {
-    return ResultSet.HOLD_CURSORS_OVER_COMMIT;
-  }
-
-  public int getDatabaseMajorVersion() throws SQLException {
-    return connection.driver.version.databaseMajorVersion;
-  }
-
-  public int getDatabaseMinorVersion() throws SQLException {
-    return connection.driver.version.databaseMinorVersion;
-  }
-
-  public int getJDBCMajorVersion() throws SQLException {
-    return connection.factory.getJdbcMajorVersion();
-  }
-
-  public int getJDBCMinorVersion() throws SQLException {
-    return connection.factory.getJdbcMinorVersion();
-  }
-
-  public int getSQLStateType() throws SQLException {
-    return sqlStateSQL;
-  }
-
-  public boolean locatorsUpdateCopy() throws SQLException {
-    return true;
-  }
-
-  public boolean supportsStatementPooling() throws SQLException {
-    return false;
-  }
-
-  public RowIdLifetime getRowIdLifetime() throws SQLException {
-    return RowIdLifetime.ROWID_UNSUPPORTED;
-  }
-
-  public boolean supportsStoredFunctionsUsingCallSyntax()
-      throws SQLException {
-    return true;
-  }
-
-  public boolean autoCommitFailureClosesAllResultSets() throws SQLException {
-    return false;
-  }
-
-  public ResultSet getClientInfoProperties() throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getClientInfoProperties(connection.handle),
-                    new QueryState(MetaDataOperation.GET_CLIENT_INFO_PROPERTIES));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getFunctions(
-      final String catalog,
-      final String schemaPattern,
-      final String functionNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getFunctions(connection.handle, catalog, pat(schemaPattern),
-                        pat(functionNamePattern)),
-                    new QueryState(MetaDataOperation.GET_FUNCTIONS, catalog, schemaPattern,
-                        functionNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getFunctionColumns(
-      final String catalog,
-      final String schemaPattern,
-      final String functionNamePattern,
-      final String columnNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getFunctionColumns(connection.handle, catalog,
-                        pat(schemaPattern), pat(functionNamePattern), pat(columnNamePattern)),
-                    new QueryState(MetaDataOperation.GET_FUNCTION_COLUMNS, catalog,
-                        schemaPattern, functionNamePattern, columnNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public ResultSet getPseudoColumns(
-      final String catalog,
-      final String schemaPattern,
-      final String tableNamePattern,
-      final String columnNamePattern) throws SQLException {
-    try {
-      return connection.invokeWithRetries(
-          new CallableWithoutException<ResultSet>() {
-            public ResultSet call() {
-              try {
-                return connection.createResultSet(
-                    connection.meta.getPseudoColumns(connection.handle, catalog, pat(schemaPattern),
-                        pat(tableNamePattern), pat(columnNamePattern)),
-                    new QueryState(MetaDataOperation.GET_PSEUDO_COLUMNS, catalog, schemaPattern,
-                        tableNamePattern, columnNamePattern));
-              } catch (SQLException e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    } catch (RuntimeException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof SQLException) {
-        throw (SQLException) cause;
-      }
-      throw e;
-    }
-  }
-
-  public boolean generatedKeyAlwaysReturned() throws SQLException {
-    return false;
-  }
-
-  // implement Wrapper
-
-  public <T> T unwrap(Class<T> iface) throws SQLException {
-    if (iface.isInstance(this)) {
-      return iface.cast(this);
-    }
-    throw connection.helper.createException(
-        "does not implement '" + iface + "'");
-  }
-
-  public boolean isWrapperFor(Class<?> iface) throws SQLException {
-    return iface.isInstance(this);
-  }
+public interface AvaticaDatabaseMetaData extends DatabaseMetaData {
+
+  /**
+   * Retrieves all Avatica-centric properties from the server. See
+   * {@link org.apache.calcite.avatica.Meta.DatabaseProperty} for a list of properties that will be
+   * returned.
+   *
+   * @return A {@link Properties} instance containing Avatica properties.
+   */
+  Properties getRemoteAvaticaProperties();
+
+  /**
+   * Retrieves the Avatica version from the server.
+   * @return A string corresponding to the server's version.
+   */
+  String getAvaticaServerVersion();
 }
 
 // End AvaticaDatabaseMetaData.java