You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2022/05/04 09:42:18 UTC

[shardingsphere] branch master updated: Refactor database isSchemaAvailable (#17311)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 84fde6ecbd3 Refactor database isSchemaAvailable (#17311)
84fde6ecbd3 is described below

commit 84fde6ecbd3c1127a2d7018179a4cbbf706d33c0
Author: Hongsheng Zhong <sa...@126.com>
AuthorDate: Wed May 4 17:42:07 2022 +0800

    Refactor database isSchemaAvailable (#17311)
    
    * Refactor database isSchemaAvailable
    
    * Fix unit test
---
 .../shardingsphere/infra/database/type/DatabaseType.java      |  9 +++++++++
 .../infra/database/type/dialect/OpenGaussDatabaseType.java    |  5 +++++
 .../infra/database/type/dialect/PostgreSQLDatabaseType.java   |  5 +++++
 .../core/metadata/loader/PipelineTableMetaDataLoader.java     | 11 ++++-------
 .../pipeline/core/sqlbuilder/AbstractPipelineSQLBuilder.java  |  7 +++++--
 .../pipeline/mysql/sqlbuilder/MySQLPipelineSQLBuilder.java    |  5 -----
 .../pipeline/mysql/fixture/FixturePipelineSQLBuilder.java     |  5 -----
 .../opengauss/sqlbuilder/OpenGaussPipelineSQLBuilder.java     |  5 -----
 .../pipeline/opengauss/fixture/FixturePipelineSQLBuilder.java |  5 -----
 .../postgresql/sqlbuilder/PostgreSQLPipelineSQLBuilder.java   |  5 -----
 .../postgresql/fixture/FixturePipelineSQLBuilder.java         |  5 -----
 .../data/pipeline/core/fixture/FixturePipelineSQLBuilder.java |  5 -----
 .../core/metadata/loader/PipelineTableMetaDataLoaderTest.java | 11 ++++++-----
 13 files changed, 34 insertions(+), 49 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/DatabaseType.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/DatabaseType.java
index 04a46390dac..05c202a4c55 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/DatabaseType.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/DatabaseType.java
@@ -83,6 +83,15 @@ public interface DatabaseType {
      */
     Collection<String> getSystemSchemas();
     
+    /**
+     * Is schema feature available.
+     *
+     * @return true or false
+     */
+    default boolean isSchemaAvailable() {
+        return false;
+    }
+    
     /**
      * Get default schema.
      *
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/OpenGaussDatabaseType.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/OpenGaussDatabaseType.java
index 1088ca19597..7f3800e7cbe 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/OpenGaussDatabaseType.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/OpenGaussDatabaseType.java
@@ -89,6 +89,11 @@ public final class OpenGaussDatabaseType implements DatabaseType {
         return SYSTEM_SCHEMAS;
     }
     
+    @Override
+    public boolean isSchemaAvailable() {
+        return true;
+    }
+    
     @Override
     public String getDefaultSchema(final String databaseName) {
         return "public";
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/PostgreSQLDatabaseType.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/PostgreSQLDatabaseType.java
index f0f8a46866a..28581e47480 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/PostgreSQLDatabaseType.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/PostgreSQLDatabaseType.java
@@ -88,6 +88,11 @@ public final class PostgreSQLDatabaseType implements DatabaseType {
         return SYSTEM_SCHEMAS;
     }
     
+    @Override
+    public boolean isSchemaAvailable() {
+        return true;
+    }
+    
     @Override
     public String getDefaultSchema(final String databaseName) {
         return "public";
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoader.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoader.java
index e0969d10a38..04d0ce486f8 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoader.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoader.java
@@ -23,9 +23,7 @@ import org.apache.shardingsphere.data.pipeline.api.datasource.PipelineDataSource
 import org.apache.shardingsphere.data.pipeline.api.metadata.TableName;
 import org.apache.shardingsphere.data.pipeline.core.metadata.model.PipelineColumnMetaData;
 import org.apache.shardingsphere.data.pipeline.core.metadata.model.PipelineTableMetaData;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 
 import java.sql.Connection;
 import java.sql.ResultSet;
@@ -60,7 +58,7 @@ public final class PipelineTableMetaDataLoader {
     public void loadTableMetaData(final String schemaName, final String tableNamePattern) throws SQLException {
         try (Connection connection = dataSource.getConnection()) {
             long startMillis = System.currentTimeMillis();
-            String schemaNameFinal = isSchemaEnabled() ? schemaName : null;
+            String schemaNameFinal = isSchemaAvailable() ? schemaName : null;
             Map<TableName, PipelineTableMetaData> tableMetaDataMap = loadTableMetaData0(connection, schemaNameFinal, tableNamePattern);
             log.info("loadTableMetaData, schemaNameFinal={}, tableNamePattern={}, result={}, cost time={} ms",
                     schemaNameFinal, tableNamePattern, tableMetaDataMap, System.currentTimeMillis() - startMillis);
@@ -68,9 +66,8 @@ public final class PipelineTableMetaDataLoader {
         }
     }
     
-    private boolean isSchemaEnabled() {
-        DatabaseType databaseType = dataSource.getDatabaseType();
-        return databaseType instanceof PostgreSQLDatabaseType || databaseType instanceof OpenGaussDatabaseType;
+    private boolean isSchemaAvailable() {
+        return DatabaseTypeRegistry.getActualDatabaseType(dataSource.getDatabaseType().getName()).isSchemaAvailable();
     }
     
     private Map<TableName, PipelineTableMetaData> loadTableMetaData0(final Connection connection, final String schemaName, final String tableNamePattern) throws SQLException {
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/sqlbuilder/AbstractPipelineSQLBuilder.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/sqlbuilder/AbstractPipelineSQLBuilder.java
index ea782aabbdb..eeba675b50c 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/sqlbuilder/AbstractPipelineSQLBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/sqlbuilder/AbstractPipelineSQLBuilder.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.data.pipeline.api.ingest.record.DataRecord;
 import org.apache.shardingsphere.data.pipeline.api.metadata.LogicTableName;
 import org.apache.shardingsphere.data.pipeline.core.record.RecordUtil;
 import org.apache.shardingsphere.data.pipeline.spi.sqlbuilder.PipelineSQLBuilder;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -78,14 +79,16 @@ public abstract class AbstractPipelineSQLBuilder implements PipelineSQLBuilder {
     
     protected String decorate(final String schemaName, final String tableName) {
         StringBuilder result = new StringBuilder();
-        if (isSchemaEnabled() && !Strings.isNullOrEmpty(schemaName)) {
+        if (isSchemaAvailable() && !Strings.isNullOrEmpty(schemaName)) {
             result.append(quote(schemaName)).append(".");
         }
         result.append(quote(tableName));
         return result.toString();
     }
     
-    protected abstract boolean isSchemaEnabled();
+    private boolean isSchemaAvailable() {
+        return DatabaseTypeRegistry.getActualDatabaseType(getType()).isSchemaAvailable();
+    }
     
     @Override
     public String buildInsertSQL(final String schemaName, final DataRecord dataRecord, final Map<LogicTableName, Set<String>> shardingColumnsMap) {
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/sqlbuilder/MySQLPipelineSQLBuilder.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/sqlbuilder/MySQLPipelineSQLBuilder.java
index 7d51c66d969..a50a48c28a4 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/sqlbuilder/MySQLPipelineSQLBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/sqlbuilder/MySQLPipelineSQLBuilder.java
@@ -41,11 +41,6 @@ public final class MySQLPipelineSQLBuilder extends AbstractPipelineSQLBuilder {
         return "`";
     }
     
-    @Override
-    protected boolean isSchemaEnabled() {
-        return false;
-    }
-    
     @Override
     public String buildInsertSQL(final String schemaName, final DataRecord dataRecord, final Map<LogicTableName, Set<String>> shardingColumnsMap) {
         return super.buildInsertSQL(schemaName, dataRecord, shardingColumnsMap) + buildDuplicateUpdateSQL(dataRecord, shardingColumnsMap);
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/test/java/org/apache/shardingsphere/data/pipeline/mysql/fixture/FixturePipelineSQLBuilder.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/test/java/org/apache/shardingsphere/data/pipeline/mysql/fixture/FixturePipelineSQLBuilder.java
index 4c0e9a033ff..48f4f2888b2 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/test/java/org/apache/shardingsphere/data/pipeline/mysql/fixture/FixturePipelineSQLBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/test/java/org/apache/shardingsphere/data/pipeline/mysql/fixture/FixturePipelineSQLBuilder.java
@@ -31,11 +31,6 @@ public final class FixturePipelineSQLBuilder extends AbstractPipelineSQLBuilder
         return "`";
     }
     
-    @Override
-    protected boolean isSchemaEnabled() {
-        return false;
-    }
-    
     @Override
     public String getType() {
         return "H2";
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/main/java/org/apache/shardingsphere/data/pipeline/opengauss/sqlbuilder/OpenGaussPipelineSQLBuilder.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/main/java/org/apache/shardingsphere/data/pipeline/opengauss/sqlbuilder/OpenGaussPipelineSQLBuilder.java
index 4c1008c02cf..f9ca2dc5146 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/main/java/org/apache/shardingsphere/data/pipeline/opengauss/sqlbuilder/OpenGaussPipelineSQLBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/main/java/org/apache/shardingsphere/data/pipeline/opengauss/sqlbuilder/OpenGaussPipelineSQLBuilder.java
@@ -42,11 +42,6 @@ public final class OpenGaussPipelineSQLBuilder extends AbstractPipelineSQLBuilde
         return "";
     }
     
-    @Override
-    protected boolean isSchemaEnabled() {
-        return true;
-    }
-    
     @Override
     public String buildInsertSQL(final String schemaName, final DataRecord dataRecord, final Map<LogicTableName, Set<String>> shardingColumnsMap) {
         return super.buildInsertSQL(schemaName, dataRecord, shardingColumnsMap) + buildConflictSQL(shardingColumnsMap);
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/test/java/org/apache/shardingsphere/data/pipeline/opengauss/fixture/FixturePipelineSQLBuilder.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/test/java/org/apache/shardingsphere/data/pipeline/opengauss/fixture/FixturePipelineSQLBuilder.java
index 798a0d60fb4..08b8694eb3c 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/test/java/org/apache/shardingsphere/data/pipeline/opengauss/fixture/FixturePipelineSQLBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-opengauss/src/test/java/org/apache/shardingsphere/data/pipeline/opengauss/fixture/FixturePipelineSQLBuilder.java
@@ -31,11 +31,6 @@ public final class FixturePipelineSQLBuilder extends AbstractPipelineSQLBuilder
         return "`";
     }
     
-    @Override
-    protected boolean isSchemaEnabled() {
-        return false;
-    }
-    
     @Override
     public String getType() {
         return "H2";
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/sqlbuilder/PostgreSQLPipelineSQLBuilder.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/sqlbuilder/PostgreSQLPipelineSQLBuilder.java
index 85d18c9fb47..6238f66beff 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/sqlbuilder/PostgreSQLPipelineSQLBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/sqlbuilder/PostgreSQLPipelineSQLBuilder.java
@@ -41,11 +41,6 @@ public final class PostgreSQLPipelineSQLBuilder extends AbstractPipelineSQLBuild
         return "\"";
     }
     
-    @Override
-    protected boolean isSchemaEnabled() {
-        return true;
-    }
-    
     @Override
     public String buildInsertSQL(final String schemaName, final DataRecord dataRecord, final Map<LogicTableName, Set<String>> shardingColumnsMap) {
         return super.buildInsertSQL(schemaName, dataRecord, shardingColumnsMap) + buildConflictSQL(dataRecord, shardingColumnsMap);
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/test/java/org/apache/shardingsphere/data/pipeline/postgresql/fixture/FixturePipelineSQLBuilder.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/test/java/org/apache/shardingsphere/data/pipeline/postgresql/fixture/FixturePipelineSQLBuilder.java
index 17721880a65..8f8edc4467d 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/test/java/org/apache/shardingsphere/data/pipeline/postgresql/fixture/FixturePipelineSQLBuilder.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/test/java/org/apache/shardingsphere/data/pipeline/postgresql/fixture/FixturePipelineSQLBuilder.java
@@ -31,11 +31,6 @@ public final class FixturePipelineSQLBuilder extends AbstractPipelineSQLBuilder
         return "`";
     }
     
-    @Override
-    protected boolean isSchemaEnabled() {
-        return false;
-    }
-    
     @Override
     public String getType() {
         return "H2";
diff --git a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixturePipelineSQLBuilder.java b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixturePipelineSQLBuilder.java
index 35c4247f621..fb6f5d92440 100644
--- a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixturePipelineSQLBuilder.java
+++ b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixturePipelineSQLBuilder.java
@@ -31,11 +31,6 @@ public final class FixturePipelineSQLBuilder extends AbstractPipelineSQLBuilder
         return "`";
     }
     
-    @Override
-    protected boolean isSchemaEnabled() {
-        return false;
-    }
-    
     @Override
     public String getType() {
         return "H2";
diff --git a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoaderTest.java b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoaderTest.java
index 60bcf0a5992..3d0da54f2e3 100644
--- a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoaderTest.java
+++ b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/metadata/loader/PipelineTableMetaDataLoaderTest.java
@@ -20,13 +20,14 @@ package org.apache.shardingsphere.data.pipeline.core.metadata.loader;
 import org.apache.shardingsphere.data.pipeline.api.datasource.PipelineDataSourceWrapper;
 import org.apache.shardingsphere.data.pipeline.core.metadata.model.PipelineColumnMetaData;
 import org.apache.shardingsphere.data.pipeline.core.metadata.model.PipelineTableMetaData;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
+import javax.sql.DataSource;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.ResultSet;
@@ -36,11 +37,11 @@ import java.util.List;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
+// TODO use H2 to do real test
 public final class PipelineTableMetaDataLoaderTest {
     
     private static final String TEST_CATALOG = "catalog";
@@ -57,7 +58,6 @@ public final class PipelineTableMetaDataLoaderTest {
     
     private static final String TEST_TABLE = "test";
     
-    @Mock
     private PipelineDataSourceWrapper dataSource;
     
     @Mock
@@ -74,7 +74,9 @@ public final class PipelineTableMetaDataLoaderTest {
     
     @Before
     public void setUp() throws SQLException {
-        when(dataSource.getConnection()).thenReturn(connection);
+        DataSource rawDataSource = mock(DataSource.class);
+        dataSource = new PipelineDataSourceWrapper(rawDataSource, new H2DatabaseType());
+        when(rawDataSource.getConnection()).thenReturn(connection);
         when(connection.getCatalog()).thenReturn(TEST_CATALOG);
         when(connection.getMetaData()).thenReturn(databaseMetaData);
         when(databaseMetaData.getColumns(TEST_CATALOG, null, TEST_TABLE, "%")).thenReturn(columnMetaDataResultSet);
@@ -91,7 +93,6 @@ public final class PipelineTableMetaDataLoaderTest {
     @Test
     public void assertGetTableMetaData() {
         PipelineTableMetaDataLoader metaDataLoader = new PipelineTableMetaDataLoader(dataSource);
-        DatabaseType databaseType = mock(DatabaseType.class, RETURNS_DEEP_STUBS);
         assertColumnMetaData(metaDataLoader.getTableMetaData(null, TEST_TABLE));
         assertPrimaryKeys(metaDataLoader.getTableMetaData(null, TEST_TABLE).getPrimaryKeyColumns());
     }