Return-Path: X-Original-To: apmail-drill-commits-archive@www.apache.org Delivered-To: apmail-drill-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id BD22718CFD for ; Wed, 17 Jun 2015 20:30:31 +0000 (UTC) Received: (qmail 91493 invoked by uid 500); 17 Jun 2015 20:30:31 -0000 Delivered-To: apmail-drill-commits-archive@drill.apache.org Received: (qmail 91431 invoked by uid 500); 17 Jun 2015 20:30:31 -0000 Mailing-List: contact commits-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: commits@drill.apache.org Delivered-To: mailing list commits@drill.apache.org Received: (qmail 91197 invoked by uid 99); 17 Jun 2015 20:30:31 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Jun 2015 20:30:31 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 475ABE3C3C; Wed, 17 Jun 2015 20:30:31 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: parthc@apache.org To: commits@drill.apache.org Date: Wed, 17 Jun 2015 20:30:40 -0000 Message-Id: <97dc618d4a364199927e8718a787ed64@git.apache.org> In-Reply-To: <67830bb41ea344e79863dec87b9eb585@git.apache.org> References: <67830bb41ea344e79863dec87b9eb585@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [10/10] drill git commit: DRILL-3198: Fix: DatabaseMetaData.getTypeInfo(...), etc., return null. DRILL-3198: Fix: DatabaseMetaData.getTypeInfo(...), etc., return null. Fixed DatabaseMetaData.getTypeInfo(...) and similar methods enough to not return null. (Each method now returns a ResultSet representing a result set with zero rows, though not with JDBC-specified columns.) - Changed utility method getEmptyResult(). - Added basic test methods. - Added interface DrillDatabaseMetaData (for place to document current Drill-specific behavior). - Added Javadoc for current behavior. Project: http://git-wip-us.apache.org/repos/asf/drill/repo Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/4e2df00e Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/4e2df00e Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/4e2df00e Branch: refs/heads/master Commit: 4e2df00e78cb5b23a82a454d1da3eac1bf354001 Parents: f2231a8 Author: dbarclay Authored: Mon Jun 8 14:03:04 2015 -0700 Committer: Parth Chandra Committed: Wed Jun 17 11:46:15 2015 -0700 ---------------------------------------------------------------------- .../drill/jdbc/DrillDatabaseMetaData.java | 414 +++++++++++++++++++ .../org/apache/drill/jdbc/DrillResultSet.java | 3 +- .../jdbc/impl/DrillDatabaseMetaDataImpl.java | 4 +- .../org/apache/drill/jdbc/impl/MetaImpl.java | 20 +- .../apache/drill/jdbc/DatabaseMetaDataTest.java | 342 ++++++++++++++- 5 files changed, 766 insertions(+), 17 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/drill/blob/4e2df00e/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java new file mode 100644 index 0000000..4391607 --- /dev/null +++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java @@ -0,0 +1,414 @@ +/** + * Licensed to the Apache Software Foundation (ASF); under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.jdbc; + +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; + + +/** + * Drill-specific {@link DatabaseMetaData}. + */ +public interface DrillDatabaseMetaData extends DatabaseMetaData { + + // For matching order of java.sql.DatabaseMetaData: + // + // allProceduresAreCallable() + // allTablesAreSelectable() + // getURL() + // getUserName() + // isReadOnly() + // nullsAreSortedHigh() + // nullsAreSortedLow() + // nullsAreSortedAtStart() + // nullsAreSortedAtEnd() + // getDatabaseProductName() + // getDatabaseProductVersion() + // getDriverName() + // getDriverVersion() + // getDriverMajorVersion() + // getDriverMinorVersion() + // usesLocalFiles() + // usesLocalFilePerTable() + // supportsMixedCaseIdentifiers() + // storesUpperCaseIdentifiers() + // storesLowerCaseIdentifiers() + // storesMixedCaseIdentifiers() + // supportsMixedCaseQuotedIdentifiers() + // storesUpperCaseQuotedIdentifiers() + // storesLowerCaseQuotedIdentifiers() + // storesMixedCaseQuotedIdentifiers() + // getIdentifierQuoteString() + // getSQLKeywords() + // getNumericFunctions() + // getStringFunctions() + // getSystemFunctions() + // getTimeDateFunctions() + // getSearchStringEscape() + // getExtraNameCharacters() + // supportsAlterTableWithAddColumn() + // supportsAlterTableWithDropColumn() + // supportsColumnAliasing() + // nullPlusNonNullIsNull() + // supportsConvert() + // supportsConvert( int, int ) + // supportsTableCorrelationNames() + // supportsDifferentTableCorrelationNames() + // supportsExpressionsInOrderBy() + // supportsOrderByUnrelated() + // supportsGroupBy() + // supportsGroupByUnrelated() + // supportsGroupByBeyondSelect() + // supportsLikeEscapeClause() + // supportsMultipleResultSets() + // supportsMultipleTransactions() + // supportsNonNullableColumns() + // supportsMinimumSQLGrammar() + // supportsCoreSQLGrammar() + // supportsExtendedSQLGrammar() + // supportsANSI92EntryLevelSQL() + // supportsANSI92IntermediateSQL() + // supportsANSI92FullSQL() + // supportsIntegrityEnhancementFacility() + // supportsOuterJoins() + // supportsFullOuterJoins() + // supportsLimitedOuterJoins() + // getSchemaTerm() + // getProcedureTerm() + // getCatalogTerm() + // isCatalogAtStart() + // getCatalogSeparator() + // supportsSchemasInDataManipulation() + // supportsSchemasInProcedureCalls() + // supportsSchemasInTableDefinitions() + // supportsSchemasInIndexDefinitions() + // supportsSchemasInPrivilegeDefinitions() + // supportsCatalogsInDataManipulation() + // supportsCatalogsInProcedureCalls() + // supportsCatalogsInTableDefinitions() + // supportsCatalogsInIndexDefinitions() + // supportsCatalogsInPrivilegeDefinitions() + // supportsPositionedDelete() + // supportsPositionedUpdate() + // supportsSelectForUpdate() + // supportsStoredProcedures() + // supportsSubqueriesInComparisons() + // supportsSubqueriesInExists() + // supportsSubqueriesInIns() + // supportsSubqueriesInQuantifieds() + // supportsCorrelatedSubqueries() + // supportsUnion() + // supportsUnionAll() + // supportsOpenCursorsAcrossCommit() + // supportsOpenCursorsAcrossRollback() + // supportsOpenStatementsAcrossCommit() + // supportsOpenStatementsAcrossRollback() + // getMaxBinaryLiteralLength() + // getMaxCharLiteralLength() + // getMaxColumnNameLength() + // getMaxColumnsInGroupBy() + // getMaxColumnsInIndex() + // getMaxColumnsInOrderBy() + // getMaxColumnsInSelect() + // getMaxColumnsInTable() + // getMaxConnections() + // getMaxCursorNameLength() + // getMaxIndexLength() + // getMaxSchemaNameLength() + // getMaxProcedureNameLength() + // getMaxCatalogNameLength() + // getMaxRowSize() + // doesMaxRowSizeIncludeBlobs() + // getMaxStatementLength() + // getMaxStatements() + // getMaxTableNameLength() + // getMaxTablesInSelect() + // getMaxUserNameLength() + // getDefaultTransactionIsolation() + // supportsTransactions() + // supportsTransactionIsolationLevel( int ) + // supportsDataDefinitionAndDataManipulationTransactions() + // supportsDataManipulationTransactionsOnly() + // dataDefinitionCausesTransactionCommit() + // dataDefinitionIgnoredInTransactions() + + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getProcedures( String catalog, + String schemaPattern, + String procedureNamePattern) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getProcedureColumns( String catalog, + String schemaPattern, + String procedureNamePattern, + String columnNamePattern ) throws SQLException; + + + // For matching order of java.sql.DatabaseMetaData: + // + // getTables( String, String, String, String[] ) + // getSchemas() + // getCatalogs() + + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getTableTypes() throws SQLException; + + + // For matching order of java.sql.DatabaseMetaData: + // + // getColumns( String, String, String, String ) + + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getColumnPrivileges( String catalog, + String schema, + String table, + String columnNamePattern ) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getTablePrivileges( String catalog, + String schemaPattern, + String tableNamePattern ) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getBestRowIdentifier( String catalog, + String schema, + String table, + int scope, + boolean nullable ) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getVersionColumns( String catalog, String schema, String table ) + throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getPrimaryKeys( String catalog, String schema, String table ) + throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getImportedKeys( String catalog, String schema, String table ) + throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getExportedKeys( String catalog, String schema, String table ) + throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getCrossReference( String parentCatalog, + String parentSchema, + String parentTable, + String foreignCatalog, + String foreignSchema, + String foreignTable ) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getTypeInfo() throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getIndexInfo( String catalog, + String schema, + String table, + boolean unique, + boolean approximate ) throws SQLException; + + + // For matching order of java.sql.DatabaseMetaData: + // + // --------------------------JDBC 2.0----------------------------- + // supportsResultSetType( int ) + // supportsResultSetConcurrency( int, int ) + // ownUpdatesAreVisible( int ) + // ownDeletesAreVisible( int ) + // ownInsertsAreVisible( int ) + // othersUpdatesAreVisible( int ) + // othersDeletesAreVisible( int ) + // othersInsertsAreVisible( int ) + // updatesAreDetected( int ) + // deletesAreDetected( int ) + // insertsAreDetected( int ) + // supportsBatchUpdates() + + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getUDTs( String catalog, + String schemaPattern, + String typeNamePattern, + int[] types ) throws SQLException; + + + // For matching order of java.sql.DatabaseMetaData: + // + // getConnection() + // ------------------- JDBC 3.0 ------------------------- + // supportsSavepoints() + // supportsNamedParameters() + // supportsMultipleOpenResults() + // supportsGetGeneratedKeys() + + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getSuperTypes( String catalog, + String schemaPattern, + String typeNamePattern ) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getSuperTables( String catalog, + String schemaPattern, + String tableNamePattern ) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getAttributes( String catalog, + String schemaPattern, + String typeNamePattern, + String attributeNamePattern ) throws SQLException; + + + // For matching order of java.sql.DatabaseMetaData: + // + // supportsResultSetHoldability( int ) + // getResultSetHoldability() + // getDatabaseMajorVersion() + // getDatabaseMinorVersion() + // getJDBCMajorVersion() + // getJDBCMinorVersion() + // getSQLStateType() + // locatorsUpdateCopy() + // supportsStatementPooling() + // ------------------------- JDBC 4.0 ----------------------------------- + // getRowIdLifetime() + // getSchemas( String, String ) + // supportsStoredFunctionsUsingCallSyntax() + // autoCommitFailureClosesAllResultSets() + + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getClientInfoProperties() throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getFunctions( String catalog, + String schemaPattern, + String functionNamePattern ) throws SQLException; + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getFunctionColumns( String catalog, + String schemaPattern, + String functionNamePattern, + String columnNamePattern ) throws SQLException; + + + // --------------------------JDBC 4.1 ----------------------------- + + + /** + * Drill: Currently, returns an empty (zero-row) result set. + * (Note: Currently, result set might not have the expected columns.) + */ + @Override + ResultSet getPseudoColumns( String catalog, + String schemaPattern, + String tableNamePattern, + String columnNamePattern ) throws SQLException; + + + // For matching order of java.sql.DatabaseMetaData: + // + // generatedKeyAlwaysReturned(); + +} http://git-wip-us.apache.org/repos/asf/drill/blob/4e2df00e/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java index 1e1b14f..e0a7763 100644 --- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java +++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java @@ -47,7 +47,8 @@ import java.util.Map; public interface DrillResultSet extends ResultSet { /** - * Gets the ID the associated query (the query results this ResultSet presents). + * Gets the ID of the associated query (the query whose results this ResultSet + * presents). */ String getQueryId(); http://git-wip-us.apache.org/repos/asf/drill/blob/4e2df00e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java index 71c38b9..9d0c132 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 @@ -21,6 +21,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import org.apache.drill.jdbc.AlreadyClosedSqlException; +import org.apache.drill.jdbc.DrillDatabaseMetaData; import net.hydromatic.avatica.AvaticaConnection; import net.hydromatic.avatica.AvaticaDatabaseMetaData; @@ -29,7 +30,8 @@ import net.hydromatic.avatica.AvaticaDatabaseMetaData; /** * Drill's implementation of {@link DatabaseMetaData}. */ -class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData { +class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData + implements DrillDatabaseMetaData { protected DrillDatabaseMetaDataImpl( AvaticaConnection connection ) { super( connection ); http://git-wip-us.apache.org/repos/asf/drill/blob/4e2df00e/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java index bb8be97..8ff7a6c 100644 --- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java +++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/MetaImpl.java @@ -81,11 +81,6 @@ class MetaImpl implements Meta { return ""; } - // TODO: BUG: null is not a proper empty result set. - private static ResultSet getEmptyResultSet() { - return null; - } - private ResultSet s(String s) { try { logger.debug("Running {}", s); @@ -99,6 +94,21 @@ class MetaImpl implements Meta { } + /** + * Returns interim generic empty result set. + *

+ * (Does not return specific columns expected (and visible in metadata) for + * specific get methods.) + *

+ */ + private ResultSet getEmptyResultSet() { + return s( + "SELECT '' AS `Interim zero-row result set` " // dummy row type + + "FROM INFORMATION_SCHEMA.CATALOGS " // any table + + "LIMIT 0" // zero rows + ); + } + @Override public ResultSet getTables(String catalog, final Pat schemaPattern, final Pat tableNamePattern, final List typeList) { http://git-wip-us.apache.org/repos/asf/drill/blob/4e2df00e/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java index 886f974..f4bd703 100644 --- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java +++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java @@ -44,6 +44,7 @@ import java.sql.SQLException; public class DatabaseMetaDataTest { private static Connection connection; + private static DatabaseMetaData dbmd; @BeforeClass public static void setUpConnection() throws SQLException { @@ -51,6 +52,7 @@ public class DatabaseMetaDataTest { // Connection--and other JDBC objects--on test method failure, but this test // class uses some objects across methods.) connection = new Driver().connect( "jdbc:drill:zk=local", null ); + dbmd = connection.getMetaData(); } @AfterClass @@ -58,32 +60,352 @@ public class DatabaseMetaDataTest { connection.close(); } + + // For matching order of java.sql.DatabaseMetaData: + // + // allProceduresAreCallable() + // allTablesAreSelectable() + // getURL() + // getUserName() + // isReadOnly() + // nullsAreSortedHigh() + // nullsAreSortedLow() + // nullsAreSortedAtStart() + // nullsAreSortedAtEnd() + // getDatabaseProductName() + // getDatabaseProductVersion() + // getDriverName() + // getDriverVersion() + // getDriverMajorVersion(); + // getDriverMinorVersion(); + // usesLocalFiles() + // usesLocalFilePerTable() + // supportsMixedCaseIdentifiers() + // storesUpperCaseIdentifiers() + // storesLowerCaseIdentifiers() + // storesMixedCaseIdentifiers() + // supportsMixedCaseQuotedIdentifiers() + // storesUpperCaseQuotedIdentifiers() + // storesLowerCaseQuotedIdentifiers() + // storesMixedCaseQuotedIdentifiers() + // getIdentifierQuoteString() + // getSQLKeywords() + // getNumericFunctions() + // getStringFunctions() + // getSystemFunctions() + // getTimeDateFunctions() + // getSearchStringEscape() + // getExtraNameCharacters() + // supportsAlterTableWithAddColumn() + // supportsAlterTableWithDropColumn() + // supportsColumnAliasing() + // nullPlusNonNullIsNull() + // supportsConvert() + // supportsConvert(int fromType, int toType) + // supportsTableCorrelationNames() + // supportsDifferentTableCorrelationNames() + // supportsExpressionsInOrderBy() + // supportsOrderByUnrelated() + // supportsGroupBy() + // supportsGroupByUnrelated() + // supportsGroupByBeyondSelect() + // supportsLikeEscapeClause() + // supportsMultipleResultSets() + // supportsMultipleTransactions() + // supportsNonNullableColumns() + // supportsMinimumSQLGrammar() + // supportsCoreSQLGrammar() + // supportsExtendedSQLGrammar() + // supportsANSI92EntryLevelSQL() + // supportsANSI92IntermediateSQL() + // supportsANSI92FullSQL() + // supportsIntegrityEnhancementFacility() + // supportsOuterJoins() + // supportsFullOuterJoins() + // supportsLimitedOuterJoins() + // getSchemaTerm() + // getProcedureTerm() + // getCatalogTerm() + // isCatalogAtStart() + // getCatalogSeparator() + // supportsSchemasInDataManipulation() + // supportsSchemasInProcedureCalls() + // supportsSchemasInTableDefinitions() + // supportsSchemasInIndexDefinitions() + // supportsSchemasInPrivilegeDefinitions() + // supportsCatalogsInDataManipulation() + // supportsCatalogsInProcedureCalls() + // supportsCatalogsInTableDefinitions() + // supportsCatalogsInIndexDefinitions() + // supportsCatalogsInPrivilegeDefinitions() + // supportsPositionedDelete() + // supportsPositionedUpdate() + // supportsSelectForUpdate() + // supportsStoredProcedures() + // supportsSubqueriesInComparisons() + // supportsSubqueriesInExists() + // supportsSubqueriesInIns() + // supportsSubqueriesInQuantifieds() + // supportsCorrelatedSubqueries() + // supportsUnion() + // supportsUnionAll() + // supportsOpenCursorsAcrossCommit() + // supportsOpenCursorsAcrossRollback() + // supportsOpenStatementsAcrossCommit() + // supportsOpenStatementsAcrossRollback() + // getMaxBinaryLiteralLength() + // getMaxCharLiteralLength() + // getMaxColumnNameLength() + // getMaxColumnsInGroupBy() + // getMaxColumnsInIndex() + // getMaxColumnsInOrderBy() + // getMaxColumnsInSelect() + // getMaxColumnsInTable() + // getMaxConnections() + // getMaxCursorNameLength() + // getMaxIndexLength() + // getMaxSchemaNameLength() + // getMaxProcedureNameLength() + // getMaxCatalogNameLength() + // getMaxRowSize() + // doesMaxRowSizeIncludeBlobs() + // getMaxStatementLength() + // getMaxStatements() + // getMaxTableNameLength() + // getMaxTablesInSelect() + // getMaxUserNameLength() + // getDefaultTransactionIsolation() + // supportsTransactions() + // supportsTransactionIsolationLevel(int level) + // supportsDataDefinitionAndDataManipulationTransactions() + // supportsDataManipulationTransactionsOnly() + // dataDefinitionCausesTransactionCommit() + // dataDefinitionIgnoredInTransactions() + + @Test public void testGetDefaultTransactionIsolationSaysNone() throws SQLException { - final DatabaseMetaData md = connection.getMetaData(); - assertThat( md.getDefaultTransactionIsolation(), equalTo( TRANSACTION_NONE ) ); + assertThat( dbmd.getDefaultTransactionIsolation(), equalTo( TRANSACTION_NONE ) ); } @Test public void testSupportsTransactionsSaysNo() throws SQLException { - assertThat( connection.getMetaData().supportsTransactions(), equalTo( false ) ); + assertThat( dbmd.supportsTransactions(), equalTo( false ) ); } @Test public void testSupportsTransactionIsolationLevelNoneSaysYes() throws SQLException { - final DatabaseMetaData md = connection.getMetaData(); - assertTrue( md.supportsTransactionIsolationLevel( TRANSACTION_NONE ) ); + assertTrue( dbmd.supportsTransactionIsolationLevel( TRANSACTION_NONE ) ); } @Test public void testSupportsTransactionIsolationLevelOthersSayNo() throws SQLException { - final DatabaseMetaData md = connection.getMetaData(); - assertFalse( md.supportsTransactionIsolationLevel( TRANSACTION_READ_UNCOMMITTED ) ); - assertFalse( md.supportsTransactionIsolationLevel( TRANSACTION_READ_COMMITTED ) ); - assertFalse( md.supportsTransactionIsolationLevel( TRANSACTION_REPEATABLE_READ ) ); - assertFalse( md.supportsTransactionIsolationLevel( TRANSACTION_SERIALIZABLE ) ); + assertFalse( dbmd.supportsTransactionIsolationLevel( TRANSACTION_READ_UNCOMMITTED ) ); + assertFalse( dbmd.supportsTransactionIsolationLevel( TRANSACTION_READ_COMMITTED ) ); + assertFalse( dbmd.supportsTransactionIsolationLevel( TRANSACTION_REPEATABLE_READ ) ); + assertFalse( dbmd.supportsTransactionIsolationLevel( TRANSACTION_SERIALIZABLE ) ); + } + + @Test + public void testGetProceduresReturnsNonNull() throws SQLException { + assertThat( dbmd.getProcedures( null, null, "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetProcedureColumnsReturnsNonNull() throws SQLException { + assertThat( dbmd.getProcedureColumns( null, null, "%", "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + + // For matching order of java.sql.DatabaseMetaData: + // + // getTables(String catalog, String schemaPattern, String tableNamePattern, String types[]) + // getSchemas() + // getCatalogs() + + + @Test + public void testGetTableTypesReturnsNonNull() throws SQLException { + assertThat( dbmd.getTableTypes(), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + + // For matching order of java.sql.DatabaseMetaData: + // + // getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern) + + + @Test + public void testGetColumnPrivilegesReturnsNonNull() throws SQLException { + assertThat( dbmd.getColumnPrivileges( null, null, "%", "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetTablePrivilegesReturnsNonNull() throws SQLException { + assertThat( dbmd.getTablePrivileges( null, null, "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetBestRowIdentifierReturnsNonNull() throws SQLException { + assertThat( dbmd.getBestRowIdentifier( null, null, "%", DatabaseMetaData.bestRowTemporary, true ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetVersionColumnsReturnsNonNull() throws SQLException { + assertThat( dbmd.getVersionColumns( null, null, "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetPrimaryKeysReturnsNonNull() throws SQLException { + assertThat( dbmd.getPrimaryKeys( null, null, "%" ), notNullValue() ); } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetImportedKeysReturnsNonNull() throws SQLException { + assertThat( dbmd.getImportedKeys( null, null, "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetExportedKeysReturnsNonNull() throws SQLException { + assertThat( dbmd.getExportedKeys( null, null, "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetCrossReferenceReturnsNonNull() throws SQLException { + assertThat( dbmd.getCrossReference( null, null, "%", null, null, "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetTypeInfoReturnsNonNull() throws SQLException { + assertThat( dbmd.getTypeInfo(), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetIndexInfoReturnsNonNull() throws SQLException { + assertThat( dbmd.getIndexInfo( null, null, "%", false, true ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + + // For matching order of java.sql.DatabaseMetaData: + // + // --------------------------JDBC 2.0----------------------------- + // supportsResultSetType(int type) + // supportsResultSetConcurrency(int type, int concurrency) + // ownUpdatesAreVisible(int type) + // ownDeletesAreVisible(int type) + // ownInsertsAreVisible(int type) + // othersUpdatesAreVisible(int type) + // othersDeletesAreVisible(int type) + // othersInsertsAreVisible(int type) + // updatesAreDetected(int type) + // deletesAreDetected(int type) + // insertsAreDetected(int type) + // supportsBatchUpdates() + + + @Test + public void testGetUDTsReturnsNonNull() throws SQLException { + assertThat( dbmd.getUDTs( null, null, "%", null ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + + // For matching order of java.sql.DatabaseMetaData: + // getConnection() + // ------------------- JDBC 3.0 ------------------------- + // supportsSavepoints() + // supportsNamedParameters() + // supportsMultipleOpenResults() + // supportsGetGeneratedKeys() + + + @Test + public void testGetSuperTypesReturnsNonNull() throws SQLException { + assertThat( dbmd.getSuperTypes( null, "%", "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetSuperTablesReturnsNonNull() throws SQLException { + assertThat( dbmd.getSuperTables( null, "%", "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetAttributesReturnsNonNull() throws SQLException { + assertThat( dbmd.getAttributes( null, null, "%", "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + + // For matching order of java.sql.DatabaseMetaData: + // + // supportsResultSetHoldability(int holdability) + // getResultSetHoldability() + // getDatabaseMajorVersion() + // getDatabaseMinorVersion() + // getJDBCMajorVersion() + // getJDBCMinorVersion() + // getSQLStateType() + // locatorsUpdateCopy() + // supportsStatementPooling() + //- ------------------------ JDBC 4.0 ----------------------------------- + // getRowIdLifetime() + // getSchemas(String catalog, String schemaPattern) + // getSchemas(String, String) + + + @Test + public void testGetClientInfoPropertiesReturnsNonNull() throws SQLException { + assertThat( dbmd.getClientInfoProperties(), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + + @Test + public void testGetFunctionsReturnsNonNull() throws SQLException { + assertThat( dbmd.getFunctions( null, "%", "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + @Test + public void testGetFunctionColumnsReturnsNonNull() throws SQLException { + assertThat( dbmd.getFunctionColumns( null, null, "%", null ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + // For matching order of java.sql.DatabaseMetaData: + // + // supportsStoredFunctionsUsingCallSyntax() + // autoCommitFailureClosesAllResultSets() + //??--------------------------JDBC 4.1 ----------------------------- + + + @Test + public void testGetPseudoColumnsReturnsNonNull() throws SQLException { + assertThat( dbmd.getPseudoColumns( null, null, "%", "%" ), notNullValue() ); + } + // TODO: Later, test more (e.g., right columns (even if/though zero rows)). + + // For matching order of java.sql.DatabaseMetaData: + // + // generatedKeyAlwaysReturned() + + }