You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by su...@apache.org on 2022/11/23 05:38:48 UTC

[shardingsphere] branch master updated: Merge DialectSchemaMetaDataLoader#loadSchemaNames to SchemaMetaDataLoader#loadSchemaNames method (#22351)

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

sunnianjun 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 28660fa8272 Merge DialectSchemaMetaDataLoader#loadSchemaNames to SchemaMetaDataLoader#loadSchemaNames method (#22351)
28660fa8272 is described below

commit 28660fa8272b6fc5cc0a39df45bcfa7b00e55321
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Wed Nov 23 13:38:37 2022 +0800

    Merge DialectSchemaMetaDataLoader#loadSchemaNames to SchemaMetaDataLoader#loadSchemaNames method (#22351)
    
    * Merge DialectSchemaMetaDataLoader#loadSchemaNames to SchemaMetaDataLoader#loadSchemaNames method
    
    * rename SchemaTableNamesLoaderTest to SchemaMetaDataLoaderTest
---
 ...eNamesLoader.java => SchemaMetaDataLoader.java} |  55 ++++---
 .../dialect/OpenGaussSchemaMetaDataLoader.java     | 114 +++++++------
 .../dialect/PostgreSQLSchemaMetaDataLoader.java    | 179 ++++++++++-----------
 .../loader/spi/DialectSchemaMetaDataLoader.java    |  25 ---
 .../loader/common/SchemaMetaDataLoaderTest.java    | 113 +++++++++++++
 .../loader/common/SchemaTableNamesLoaderTest.java  |  65 --------
 .../datanode/SingleTableDataNodeLoader.java        |   4 +-
 7 files changed, 284 insertions(+), 271 deletions(-)

diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaTableNamesLoader.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaMetaDataLoader.java
similarity index 77%
rename from infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaTableNamesLoader.java
rename to infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaMetaDataLoader.java
index 3b655d53ab3..f1be915f889 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaTableNamesLoader.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaMetaDataLoader.java
@@ -35,10 +35,10 @@ import java.util.LinkedList;
 import java.util.Map;
 
 /**
- * Schema table names loader.
+ * Schema meta data loader.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class SchemaTableNamesLoader {
+public final class SchemaMetaDataLoader {
     
     private static final String TABLE_TYPE = "TABLE";
     
@@ -60,34 +60,24 @@ public final class SchemaTableNamesLoader {
     public static Map<String, Collection<String>> loadSchemaTableNames(final String databaseName, final DatabaseType databaseType, final DataSource dataSource) throws SQLException {
         try (MetaDataLoaderConnectionAdapter connectionAdapter = new MetaDataLoaderConnectionAdapter(databaseType, dataSource.getConnection())) {
             Collection<String> schemaNames = loadSchemaNames(connectionAdapter, databaseType);
-            return loadSchemaTableNames(connectionAdapter, databaseName, databaseType, schemaNames);
-        }
-    }
-    
-    private static Map<String, Collection<String>> loadSchemaTableNames(final Connection connection, final String databaseName,
-                                                                        final DatabaseType databaseType, final Collection<String> schemaNames) throws SQLException {
-        Map<String, Collection<String>> result = new HashMap<>(schemaNames.size(), 1);
-        for (String each : schemaNames) {
-            String schemaName = databaseType instanceof PostgreSQLDatabaseType || databaseType instanceof OpenGaussDatabaseType ? each : databaseName;
-            result.put(schemaName, loadSchemaTableNames(connection, each));
-        }
-        return result;
-    }
-    
-    private static Collection<String> loadSchemaTableNames(final Connection connection, final String schemaName) throws SQLException {
-        Collection<String> result = new LinkedList<>();
-        try (ResultSet resultSet = connection.getMetaData().getTables(connection.getCatalog(), schemaName, null, new String[]{TABLE_TYPE, VIEW_TYPE})) {
-            while (resultSet.next()) {
-                String table = resultSet.getString(TABLE_NAME);
-                if (!isSystemTable(table)) {
-                    result.add(table);
-                }
+            Map<String, Collection<String>> result = new HashMap<>(schemaNames.size(), 1);
+            for (String each : schemaNames) {
+                String schemaName = databaseType instanceof PostgreSQLDatabaseType || databaseType instanceof OpenGaussDatabaseType ? each : databaseName;
+                result.put(schemaName, loadTableNames(connectionAdapter, each));
             }
+            return result;
         }
-        return result;
     }
     
-    private static Collection<String> loadSchemaNames(final Connection connection, final DatabaseType databaseType) throws SQLException {
+    /**
+     * Load schema names.
+     *
+     * @param connection connection
+     * @param databaseType database type
+     * @return schema names collection
+     * @throws SQLException SQL exception
+     */
+    public static Collection<String> loadSchemaNames(final Connection connection, final DatabaseType databaseType) throws SQLException {
         if (!(databaseType instanceof PostgreSQLDatabaseType) && !(databaseType instanceof OpenGaussDatabaseType)) {
             return Collections.singletonList(connection.getSchema());
         }
@@ -103,6 +93,19 @@ public final class SchemaTableNamesLoader {
         return result.isEmpty() ? Collections.singletonList(connection.getSchema()) : result;
     }
     
+    private static Collection<String> loadTableNames(final Connection connection, final String schemaName) throws SQLException {
+        Collection<String> result = new LinkedList<>();
+        try (ResultSet resultSet = connection.getMetaData().getTables(connection.getCatalog(), schemaName, null, new String[]{TABLE_TYPE, VIEW_TYPE})) {
+            while (resultSet.next()) {
+                String table = resultSet.getString(TABLE_NAME);
+                if (!isSystemTable(table)) {
+                    result.add(table);
+                }
+            }
+        }
+        return result;
+    }
+    
     private static boolean isSystemTable(final String table) {
         return table.contains("$") || table.contains("/") || table.contains("##");
     }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/OpenGaussSchemaMetaDataLoader.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/OpenGaussSchemaMetaDataLoader.java
index 32eeca38650..c1d582fbdeb 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/OpenGaussSchemaMetaDataLoader.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/OpenGaussSchemaMetaDataLoader.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.infra.metadata.database.schema.loader.dialect;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Multimap;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeFactory;
+import org.apache.shardingsphere.infra.metadata.database.schema.loader.common.SchemaMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.SchemaMetaData;
@@ -38,7 +39,6 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -61,56 +61,68 @@ public final class OpenGaussSchemaMetaDataLoader implements DialectSchemaMetaDat
     
     @Override
     public Collection<SchemaMetaData> load(final DataSource dataSource, final Collection<String> tables, final String defaultSchemaName) throws SQLException {
-        Collection<String> schemaNames = loadSchemaNames(dataSource, DatabaseTypeFactory.getInstance(getType()));
-        Map<String, Multimap<String, IndexMetaData>> schemaIndexMetaDataMap = loadIndexMetaDataMap(dataSource, schemaNames);
-        Map<String, Multimap<String, ColumnMetaData>> schemaColumnMetaDataMap = loadColumnMetaDataMap(dataSource, tables, schemaNames);
-        Collection<SchemaMetaData> result = new LinkedList<>();
-        for (String each : schemaNames) {
-            Multimap<String, IndexMetaData> tableIndexMetaDataMap = schemaIndexMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
-            Multimap<String, ColumnMetaData> tableColumnMetaDataMap = schemaColumnMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
-            result.add(new SchemaMetaData(each, createTableMetaDataList(tableIndexMetaDataMap, tableColumnMetaDataMap)));
+        try (Connection connection = dataSource.getConnection()) {
+            Collection<String> schemaNames = SchemaMetaDataLoader.loadSchemaNames(connection, DatabaseTypeFactory.getInstance(getType()));
+            Map<String, Multimap<String, IndexMetaData>> schemaIndexMetaDataMap = loadIndexMetaDataMap(connection, schemaNames);
+            Map<String, Multimap<String, ColumnMetaData>> schemaColumnMetaDataMap = loadColumnMetaDataMap(connection, tables, schemaNames);
+            Collection<SchemaMetaData> result = new LinkedList<>();
+            for (String each : schemaNames) {
+                Multimap<String, IndexMetaData> tableIndexMetaDataMap = schemaIndexMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
+                Multimap<String, ColumnMetaData> tableColumnMetaDataMap = schemaColumnMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
+                result.add(new SchemaMetaData(each, createTableMetaDataList(tableIndexMetaDataMap, tableColumnMetaDataMap)));
+            }
+            return result;
         }
-        return result;
     }
     
-    private Collection<TableMetaData> createTableMetaDataList(final Multimap<String, IndexMetaData> tableIndexMetaDataMap, final Multimap<String, ColumnMetaData> tableColumnMetaDataMap) {
-        Collection<TableMetaData> result = new LinkedList<>();
-        for (String each : tableColumnMetaDataMap.keySet()) {
-            Collection<ColumnMetaData> columnMetaDataList = tableColumnMetaDataMap.get(each);
-            Collection<IndexMetaData> indexMetaDataList = tableIndexMetaDataMap.get(each);
-            result.add(new TableMetaData(each, columnMetaDataList, indexMetaDataList, Collections.emptyList()));
+    private Map<String, Multimap<String, IndexMetaData>> loadIndexMetaDataMap(final Connection connection, final Collection<String> schemaNames) throws SQLException {
+        Map<String, Multimap<String, IndexMetaData>> result = new LinkedHashMap<>();
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getIndexMetaDataSQL(schemaNames)); ResultSet resultSet = preparedStatement.executeQuery()) {
+            while (resultSet.next()) {
+                String schemaName = resultSet.getString("schemaname");
+                String tableName = resultSet.getString("tablename");
+                String indexName = resultSet.getString("indexname");
+                Multimap<String, IndexMetaData> indexMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
+                indexMetaDataMap.put(tableName, new IndexMetaData(indexName));
+            }
         }
         return result;
     }
     
-    private Map<String, Multimap<String, ColumnMetaData>> loadColumnMetaDataMap(final DataSource dataSource, final Collection<String> tables,
+    private String getIndexMetaDataSQL(final Collection<String> schemaNames) {
+        return String.format(BASIC_INDEX_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
+    }
+    
+    private Map<String, Multimap<String, ColumnMetaData>> loadColumnMetaDataMap(final Connection connection, final Collection<String> tables,
                                                                                 final Collection<String> schemaNames) throws SQLException {
         Map<String, Multimap<String, ColumnMetaData>> result = new LinkedHashMap<>();
-        try (Connection connection = dataSource.getConnection(); PreparedStatement preparedStatement = connection.prepareStatement(getColumnMetaDataSQL(schemaNames, tables))) {
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getColumnMetaDataSQL(schemaNames, tables)); ResultSet resultSet = preparedStatement.executeQuery()) {
             Map<String, Integer> dataTypes = DataTypeLoaderFactory.getInstance(DatabaseTypeFactory.getInstance("openGauss")).load(connection.getMetaData());
-            Set<String> primaryKeys = loadPrimaryKeys(connection, schemaNames);
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    String tableName = resultSet.getString("table_name");
-                    String schemaName = resultSet.getString("table_schema");
-                    Multimap<String, ColumnMetaData> columnMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
-                    columnMetaDataMap.put(tableName, loadColumnMetaData(dataTypes, primaryKeys, resultSet));
-                }
+            Collection<String> primaryKeys = loadPrimaryKeys(connection, schemaNames);
+            while (resultSet.next()) {
+                String tableName = resultSet.getString("table_name");
+                String schemaName = resultSet.getString("table_schema");
+                Multimap<String, ColumnMetaData> columnMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
+                columnMetaDataMap.put(tableName, loadColumnMetaData(dataTypes, primaryKeys, resultSet));
             }
         }
         return result;
     }
     
-    private Set<String> loadPrimaryKeys(final Connection connection, final Collection<String> schemaNames) throws SQLException {
-        Set<String> result = new HashSet<>();
-        try (PreparedStatement preparedStatement = connection.prepareStatement(getPrimaryKeyMetaDataSQL(schemaNames))) {
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    String schemaName = resultSet.getString("table_schema");
-                    String tableName = resultSet.getString("table_name");
-                    String columnName = resultSet.getString("column_name");
-                    result.add(schemaName + "," + tableName + "," + columnName);
-                }
+    private String getColumnMetaDataSQL(final Collection<String> schemaNames, final Collection<String> tables) {
+        String schemaNameParam = schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(","));
+        return tables.isEmpty() ? String.format(TABLE_META_DATA_SQL_WITHOUT_TABLES, schemaNameParam)
+                : String.format(TABLE_META_DATA_SQL_WITH_TABLES, schemaNameParam, tables.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
+    }
+    
+    private Collection<String> loadPrimaryKeys(final Connection connection, final Collection<String> schemaNames) throws SQLException {
+        Collection<String> result = new HashSet<>();
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getPrimaryKeyMetaDataSQL(schemaNames)); ResultSet resultSet = preparedStatement.executeQuery()) {
+            while (resultSet.next()) {
+                String schemaName = resultSet.getString("table_schema");
+                String tableName = resultSet.getString("table_name");
+                String columnName = resultSet.getString("column_name");
+                result.add(schemaName + "," + tableName + "," + columnName);
             }
         }
         return result;
@@ -120,7 +132,7 @@ public final class OpenGaussSchemaMetaDataLoader implements DialectSchemaMetaDat
         return String.format(PRIMARY_KEY_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
     }
     
-    private ColumnMetaData loadColumnMetaData(final Map<String, Integer> dataTypeMap, final Set<String> primaryKeys, final ResultSet resultSet) throws SQLException {
+    private ColumnMetaData loadColumnMetaData(final Map<String, Integer> dataTypeMap, final Collection<String> primaryKeys, final ResultSet resultSet) throws SQLException {
         String schemaName = resultSet.getString("table_schema");
         String tableName = resultSet.getString("table_name");
         String columnName = resultSet.getString("column_name");
@@ -133,32 +145,16 @@ public final class OpenGaussSchemaMetaDataLoader implements DialectSchemaMetaDat
         return new ColumnMetaData(columnName, dataTypeMap.get(dataType), isPrimaryKey, generated, caseSensitive, true, false);
     }
     
-    private String getColumnMetaDataSQL(final Collection<String> schemaNames, final Collection<String> tables) {
-        String schemaNameParam = schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(","));
-        return tables.isEmpty() ? String.format(TABLE_META_DATA_SQL_WITHOUT_TABLES, schemaNameParam)
-                : String.format(TABLE_META_DATA_SQL_WITH_TABLES, schemaNameParam, tables.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
-    }
-    
-    private Map<String, Multimap<String, IndexMetaData>> loadIndexMetaDataMap(final DataSource dataSource, final Collection<String> schemaNames) throws SQLException {
-        Map<String, Multimap<String, IndexMetaData>> result = new LinkedHashMap<>();
-        try (Connection connection = dataSource.getConnection(); PreparedStatement preparedStatement = connection.prepareStatement(getIndexMetaDataSQL(schemaNames))) {
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    String schemaName = resultSet.getString("schemaname");
-                    String tableName = resultSet.getString("tablename");
-                    String indexName = resultSet.getString("indexname");
-                    Multimap<String, IndexMetaData> indexMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
-                    indexMetaDataMap.put(tableName, new IndexMetaData(indexName));
-                }
-            }
+    private Collection<TableMetaData> createTableMetaDataList(final Multimap<String, IndexMetaData> tableIndexMetaDataMap, final Multimap<String, ColumnMetaData> tableColumnMetaDataMap) {
+        Collection<TableMetaData> result = new LinkedList<>();
+        for (String each : tableColumnMetaDataMap.keySet()) {
+            Collection<ColumnMetaData> columnMetaDataList = tableColumnMetaDataMap.get(each);
+            Collection<IndexMetaData> indexMetaDataList = tableIndexMetaDataMap.get(each);
+            result.add(new TableMetaData(each, columnMetaDataList, indexMetaDataList, Collections.emptyList()));
         }
         return result;
     }
     
-    private String getIndexMetaDataSQL(final Collection<String> schemaNames) {
-        return String.format(BASIC_INDEX_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
-    }
-    
     @Override
     public String getType() {
         return "openGauss";
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/PostgreSQLSchemaMetaDataLoader.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/PostgreSQLSchemaMetaDataLoader.java
index 1055ade4d71..1b5573e4375 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/PostgreSQLSchemaMetaDataLoader.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/dialect/PostgreSQLSchemaMetaDataLoader.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.infra.metadata.database.schema.loader.dialect;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Multimap;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeFactory;
+import org.apache.shardingsphere.infra.metadata.database.schema.loader.common.SchemaMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ConstraintMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
@@ -66,91 +67,93 @@ public final class PostgreSQLSchemaMetaDataLoader implements DialectSchemaMetaDa
     
     private static final String LOAD_ALL_ROLE_TABLE_GRANTS_SQL = "SELECT table_name FROM information_schema.role_table_grants";
     
-    private static final String LOAD_FILTED_ROLE_TABLE_GRANTS_SQL = LOAD_ALL_ROLE_TABLE_GRANTS_SQL + " WHERE table_name IN (%s)";
+    private static final String LOAD_FILTERED_ROLE_TABLE_GRANTS_SQL = LOAD_ALL_ROLE_TABLE_GRANTS_SQL + " WHERE table_name IN (%s)";
     
     @Override
     public Collection<SchemaMetaData> load(final DataSource dataSource, final Collection<String> tables, final String defaultSchemaName) throws SQLException {
-        Collection<String> schemaNames = loadSchemaNames(dataSource, DatabaseTypeFactory.getInstance(getType()));
-        Map<String, Multimap<String, IndexMetaData>> schemaIndexMetaDataMap = loadIndexMetaDataMap(dataSource, schemaNames);
-        Map<String, Multimap<String, ColumnMetaData>> schemaColumnMetaDataMap = loadColumnMetaDataMap(dataSource, tables, schemaNames);
-        Map<String, Multimap<String, ConstraintMetaData>> schemaConstraintMetaDataMap = loadConstraintMetaDataMap(dataSource, schemaNames);
-        Collection<SchemaMetaData> result = new LinkedList<>();
-        for (String each : schemaNames) {
-            Multimap<String, IndexMetaData> tableIndexMetaDataMap = schemaIndexMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
-            Multimap<String, ColumnMetaData> tableColumnMetaDataMap = schemaColumnMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
-            Multimap<String, ConstraintMetaData> tableConstraintMetaDataMap = schemaConstraintMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
-            result.add(new SchemaMetaData(each, createTableMetaDataList(tableIndexMetaDataMap, tableColumnMetaDataMap, tableConstraintMetaDataMap)));
-        }
-        return result;
-    }
-    
-    private Collection<TableMetaData> createTableMetaDataList(final Multimap<String, IndexMetaData> tableIndexMetaDataMap,
-                                                              final Multimap<String, ColumnMetaData> tableColumnMetaDataMap,
-                                                              final Multimap<String, ConstraintMetaData> tableConstraintMetaDataMap) {
-        Collection<TableMetaData> result = new LinkedList<>();
-        for (String each : tableColumnMetaDataMap.keySet()) {
-            Collection<ColumnMetaData> columnMetaDataList = tableColumnMetaDataMap.get(each);
-            Collection<IndexMetaData> indexMetaDataList = tableIndexMetaDataMap.get(each);
-            Collection<ConstraintMetaData> constraintMetaDataList = tableConstraintMetaDataMap.get(each);
-            result.add(new TableMetaData(each, columnMetaDataList, indexMetaDataList, constraintMetaDataList));
+        try (Connection connection = dataSource.getConnection()) {
+            Collection<String> schemaNames = SchemaMetaDataLoader.loadSchemaNames(connection, DatabaseTypeFactory.getInstance(getType()));
+            Map<String, Multimap<String, IndexMetaData>> schemaIndexMetaDataMap = loadIndexMetaDataMap(connection, schemaNames);
+            Map<String, Multimap<String, ColumnMetaData>> schemaColumnMetaDataMap = loadColumnMetaDataMap(connection, tables, schemaNames);
+            Map<String, Multimap<String, ConstraintMetaData>> schemaConstraintMetaDataMap = loadConstraintMetaDataMap(connection, schemaNames);
+            Collection<SchemaMetaData> result = new LinkedList<>();
+            for (String each : schemaNames) {
+                Multimap<String, IndexMetaData> tableIndexMetaDataMap = schemaIndexMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
+                Multimap<String, ColumnMetaData> tableColumnMetaDataMap = schemaColumnMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
+                Multimap<String, ConstraintMetaData> tableConstraintMetaDataMap = schemaConstraintMetaDataMap.getOrDefault(each, LinkedHashMultimap.create());
+                result.add(new SchemaMetaData(each, createTableMetaDataList(tableIndexMetaDataMap, tableColumnMetaDataMap, tableConstraintMetaDataMap)));
+            }
+            return result;
         }
-        return result;
     }
     
-    private Map<String, Multimap<String, ConstraintMetaData>> loadConstraintMetaDataMap(final DataSource dataSource,
-                                                                                        final Collection<String> schemaNames) throws SQLException {
-        Map<String, Multimap<String, ConstraintMetaData>> result = new LinkedHashMap<>();
-        try (Connection connection = dataSource.getConnection(); PreparedStatement preparedStatement = connection.prepareStatement(getConstraintKeyMetaDataSQL(schemaNames))) {
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    String schemaName = resultSet.getString("table_schema");
-                    Multimap<String, ConstraintMetaData> constraintMetaData = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
-                    String tableName = resultSet.getString("table_name");
-                    String constraintName = resultSet.getString("constraint_name");
-                    String referencedTableName = resultSet.getString("refer_table_name");
-                    constraintMetaData.put(tableName, new ConstraintMetaData(constraintName, referencedTableName));
-                }
+    private Map<String, Multimap<String, IndexMetaData>> loadIndexMetaDataMap(final Connection connection, final Collection<String> schemaNames) throws SQLException {
+        Map<String, Multimap<String, IndexMetaData>> result = new LinkedHashMap<>();
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getIndexMetaDataSQL(schemaNames)); ResultSet resultSet = preparedStatement.executeQuery()) {
+            while (resultSet.next()) {
+                String schemaName = resultSet.getString("schemaname");
+                String tableName = resultSet.getString("tablename");
+                String indexName = resultSet.getString("indexname");
+                Multimap<String, IndexMetaData> indexMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
+                indexMetaDataMap.put(tableName, new IndexMetaData(indexName));
             }
         }
         return result;
     }
     
-    private String getConstraintKeyMetaDataSQL(final Collection<String> schemaNames) {
-        return String.format(FOREIGN_KEY_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
+    private String getIndexMetaDataSQL(final Collection<String> schemaNames) {
+        return String.format(BASIC_INDEX_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
     }
     
-    private Map<String, Multimap<String, ColumnMetaData>> loadColumnMetaDataMap(final DataSource dataSource, final Collection<String> tables,
+    private Map<String, Multimap<String, ColumnMetaData>> loadColumnMetaDataMap(final Connection connection, final Collection<String> tables,
                                                                                 final Collection<String> schemaNames) throws SQLException {
         Map<String, Multimap<String, ColumnMetaData>> result = new LinkedHashMap<>();
-        Collection<String> roleTableGrants = loadRoleTableGrants(dataSource, tables);
-        try (Connection connection = dataSource.getConnection(); PreparedStatement preparedStatement = connection.prepareStatement(getColumnMetaDataSQL(schemaNames, tables))) {
+        Collection<String> roleTableGrants = loadRoleTableGrants(connection, tables);
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getColumnMetaDataSQL(schemaNames, tables)); ResultSet resultSet = preparedStatement.executeQuery()) {
             Map<String, Integer> dataTypes = DataTypeLoaderFactory.getInstance(DatabaseTypeFactory.getInstance("PostgreSQL")).load(connection.getMetaData());
-            Set<String> primaryKeys = loadPrimaryKeys(connection, schemaNames);
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    String tableName = resultSet.getString("table_name");
-                    if (!roleTableGrants.contains(tableName)) {
-                        continue;
-                    }
-                    String schemaName = resultSet.getString("table_schema");
-                    Multimap<String, ColumnMetaData> columnMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
-                    columnMetaDataMap.put(tableName, loadColumnMetaData(dataTypes, primaryKeys, resultSet));
+            Collection<String> primaryKeys = loadPrimaryKeys(connection, schemaNames);
+            while (resultSet.next()) {
+                String tableName = resultSet.getString("table_name");
+                if (!roleTableGrants.contains(tableName)) {
+                    continue;
                 }
+                String schemaName = resultSet.getString("table_schema");
+                Multimap<String, ColumnMetaData> columnMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
+                columnMetaDataMap.put(tableName, loadColumnMetaData(dataTypes, primaryKeys, resultSet));
+            }
+        }
+        return result;
+    }
+    
+    private Collection<String> loadRoleTableGrants(final Connection connection, final Collection<String> tables) throws SQLException {
+        Collection<String> result = new HashSet<>(tables.size(), 1);
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getLoadRoleTableGrantsSQL(tables)); ResultSet resultSet = preparedStatement.executeQuery()) {
+            while (resultSet.next()) {
+                result.add(resultSet.getString("table_name"));
             }
         }
         return result;
     }
     
+    private String getLoadRoleTableGrantsSQL(final Collection<String> tables) {
+        return tables.isEmpty() ? LOAD_ALL_ROLE_TABLE_GRANTS_SQL
+                : String.format(LOAD_FILTERED_ROLE_TABLE_GRANTS_SQL, tables.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
+    }
+    
+    private String getColumnMetaDataSQL(final Collection<String> schemaNames, final Collection<String> tables) {
+        String schemaNameParam = schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(","));
+        return tables.isEmpty() ? String.format(TABLE_META_DATA_SQL_WITHOUT_TABLES, schemaNameParam)
+                : String.format(TABLE_META_DATA_SQL_WITH_TABLES, schemaNameParam, tables.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
+    }
+    
     private Set<String> loadPrimaryKeys(final Connection connection, final Collection<String> schemaNames) throws SQLException {
         Set<String> result = new HashSet<>();
-        try (PreparedStatement preparedStatement = connection.prepareStatement(getPrimaryKeyMetaDataSQL(schemaNames))) {
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    String schemaName = resultSet.getString("table_schema");
-                    String tableName = resultSet.getString("table_name");
-                    String columnName = resultSet.getString("column_name");
-                    result.add(schemaName + "," + tableName + "," + columnName);
-                }
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getPrimaryKeyMetaDataSQL(schemaNames)); ResultSet resultSet = preparedStatement.executeQuery()) {
+            while (resultSet.next()) {
+                String schemaName = resultSet.getString("table_schema");
+                String tableName = resultSet.getString("table_name");
+                String columnName = resultSet.getString("column_name");
+                result.add(schemaName + "," + tableName + "," + columnName);
             }
         }
         return result;
@@ -160,7 +163,7 @@ public final class PostgreSQLSchemaMetaDataLoader implements DialectSchemaMetaDa
         return String.format(PRIMARY_KEY_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
     }
     
-    private ColumnMetaData loadColumnMetaData(final Map<String, Integer> dataTypeMap, final Set<String> primaryKeys, final ResultSet resultSet) throws SQLException {
+    private ColumnMetaData loadColumnMetaData(final Map<String, Integer> dataTypeMap, final Collection<String> primaryKeys, final ResultSet resultSet) throws SQLException {
         String schemaName = resultSet.getString("table_schema");
         String tableName = resultSet.getString("table_name");
         String columnName = resultSet.getString("column_name");
@@ -173,49 +176,37 @@ public final class PostgreSQLSchemaMetaDataLoader implements DialectSchemaMetaDa
         return new ColumnMetaData(columnName, dataTypeMap.get(dataType), isPrimaryKey, generated, caseSensitive, true, false);
     }
     
-    private String getColumnMetaDataSQL(final Collection<String> schemaNames, final Collection<String> tables) {
-        String schemaNameParam = schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(","));
-        return tables.isEmpty() ? String.format(TABLE_META_DATA_SQL_WITHOUT_TABLES, schemaNameParam)
-                : String.format(TABLE_META_DATA_SQL_WITH_TABLES, schemaNameParam, tables.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
-    }
-    
-    private Map<String, Multimap<String, IndexMetaData>> loadIndexMetaDataMap(final DataSource dataSource, final Collection<String> schemaNames) throws SQLException {
-        Map<String, Multimap<String, IndexMetaData>> result = new LinkedHashMap<>();
-        try (Connection connection = dataSource.getConnection(); PreparedStatement preparedStatement = connection.prepareStatement(getIndexMetaDataSQL(schemaNames))) {
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    String schemaName = resultSet.getString("schemaname");
-                    String tableName = resultSet.getString("tablename");
-                    String indexName = resultSet.getString("indexname");
-                    Multimap<String, IndexMetaData> indexMetaDataMap = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
-                    indexMetaDataMap.put(tableName, new IndexMetaData(indexName));
-                }
+    private Map<String, Multimap<String, ConstraintMetaData>> loadConstraintMetaDataMap(final Connection connection, final Collection<String> schemaNames) throws SQLException {
+        Map<String, Multimap<String, ConstraintMetaData>> result = new LinkedHashMap<>();
+        try (PreparedStatement preparedStatement = connection.prepareStatement(getConstraintKeyMetaDataSQL(schemaNames)); ResultSet resultSet = preparedStatement.executeQuery()) {
+            while (resultSet.next()) {
+                String schemaName = resultSet.getString("table_schema");
+                Multimap<String, ConstraintMetaData> constraintMetaData = result.computeIfAbsent(schemaName, key -> LinkedHashMultimap.create());
+                String tableName = resultSet.getString("table_name");
+                String constraintName = resultSet.getString("constraint_name");
+                String referencedTableName = resultSet.getString("refer_table_name");
+                constraintMetaData.put(tableName, new ConstraintMetaData(constraintName, referencedTableName));
             }
         }
         return result;
     }
     
-    private String getIndexMetaDataSQL(final Collection<String> schemaNames) {
-        return String.format(BASIC_INDEX_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
+    private String getConstraintKeyMetaDataSQL(final Collection<String> schemaNames) {
+        return String.format(FOREIGN_KEY_META_DATA_SQL, schemaNames.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
     }
     
-    private Collection<String> loadRoleTableGrants(final DataSource dataSource, final Collection<String> tables) throws SQLException {
-        Collection<String> result = new HashSet<>(tables.size(), 1);
-        try (Connection connection = dataSource.getConnection(); PreparedStatement preparedStatement = connection.prepareStatement(getLoadRoleTableGrantsSQL(tables))) {
-            try (ResultSet resultSet = preparedStatement.executeQuery()) {
-                while (resultSet.next()) {
-                    result.add(resultSet.getString("table_name"));
-                }
-            }
+    private Collection<TableMetaData> createTableMetaDataList(final Multimap<String, IndexMetaData> tableIndexMetaDataMap, final Multimap<String, ColumnMetaData> tableColumnMetaDataMap,
+                                                              final Multimap<String, ConstraintMetaData> tableConstraintMetaDataMap) {
+        Collection<TableMetaData> result = new LinkedList<>();
+        for (String each : tableColumnMetaDataMap.keySet()) {
+            Collection<ColumnMetaData> columnMetaDataList = tableColumnMetaDataMap.get(each);
+            Collection<IndexMetaData> indexMetaDataList = tableIndexMetaDataMap.get(each);
+            Collection<ConstraintMetaData> constraintMetaDataList = tableConstraintMetaDataMap.get(each);
+            result.add(new TableMetaData(each, columnMetaDataList, indexMetaDataList, constraintMetaDataList));
         }
         return result;
     }
     
-    private String getLoadRoleTableGrantsSQL(final Collection<String> tables) {
-        return tables.isEmpty() ? LOAD_ALL_ROLE_TABLE_GRANTS_SQL
-                : String.format(LOAD_FILTED_ROLE_TABLE_GRANTS_SQL, tables.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
-    }
-    
     @Override
     public String getType() {
         return "PostgreSQL";
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/spi/DialectSchemaMetaDataLoader.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/spi/DialectSchemaMetaDataLoader.java
index 7a531bbcfc5..aa20e7b38df 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/spi/DialectSchemaMetaDataLoader.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/spi/DialectSchemaMetaDataLoader.java
@@ -17,17 +17,13 @@
 
 package org.apache.shardingsphere.infra.metadata.database.schema.loader.spi;
 
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.SchemaMetaData;
 import org.apache.shardingsphere.infra.util.spi.annotation.SingletonSPI;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPI;
 
 import javax.sql.DataSource;
-import java.sql.Connection;
-import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Collection;
-import java.util.LinkedList;
 
 /**
  * Dialect schema meta data loader.
@@ -45,25 +41,4 @@ public interface DialectSchemaMetaDataLoader extends TypedSPI {
      * @throws SQLException SQL exception
      */
     Collection<SchemaMetaData> load(DataSource dataSource, Collection<String> tables, String defaultSchemaName) throws SQLException;
-    
-    /**
-     * Load schema names.
-     *
-     * @param dataSource dataSource
-     * @param databaseType database type
-     * @return schema names collection
-     * @throws SQLException SQL exception
-     */
-    default Collection<String> loadSchemaNames(final DataSource dataSource, final DatabaseType databaseType) throws SQLException {
-        Collection<String> result = new LinkedList<>();
-        try (Connection connection = dataSource.getConnection(); ResultSet resultSet = connection.getMetaData().getSchemas()) {
-            while (resultSet.next()) {
-                String schema = resultSet.getString("TABLE_SCHEM");
-                if (!databaseType.getSystemSchemas().contains(schema)) {
-                    result.add(schema);
-                }
-            }
-        }
-        return result;
-    }
 }
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaMetaDataLoaderTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaMetaDataLoaderTest.java
new file mode 100644
index 00000000000..0025683623f
--- /dev/null
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaMetaDataLoaderTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.shardingsphere.infra.metadata.database.schema.loader.common;
+
+import org.apache.shardingsphere.infra.database.DefaultDatabase;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import javax.sql.DataSource;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class SchemaMetaDataLoaderTest {
+    
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private DataSource dataSource;
+    
+    @Before
+    public void setUp() throws SQLException {
+        ResultSet tableResultSet = mockTableResultSet();
+        when(dataSource.getConnection().getMetaData().getTables("catalog", "public", null, new String[]{"TABLE", "VIEW"})).thenReturn(tableResultSet);
+        when(dataSource.getConnection().getCatalog()).thenReturn("catalog");
+        when(dataSource.getConnection().getSchema()).thenReturn("public");
+        ResultSet schemaResultSet = mockSchemaResultSet();
+        when(dataSource.getConnection().getMetaData().getSchemas()).thenReturn(schemaResultSet);
+    }
+    
+    private ResultSet mockTableResultSet() throws SQLException {
+        ResultSet result = mock(ResultSet.class);
+        when(result.next()).thenReturn(true, true, true, true, false);
+        when(result.getString("TABLE_NAME")).thenReturn("tbl", "$tbl", "/tbl", "##tbl");
+        return result;
+    }
+    
+    private ResultSet mockSchemaResultSet() throws SQLException {
+        ResultSet result = mock(ResultSet.class);
+        when(result.next()).thenReturn(true, true, true, true, false);
+        when(result.getString("TABLE_SCHEM")).thenReturn("information_schema", "public", "schema_1", "schema_2");
+        return result;
+    }
+    
+    @Test
+    public void assertLoadSchemaTableNamesForPostgreSQL() throws SQLException {
+        assertThat(SchemaMetaDataLoader.loadSchemaTableNames(DefaultDatabase.LOGIC_NAME, new PostgreSQLDatabaseType(), dataSource), is(createSchemaTableNames()));
+    }
+    
+    @Test
+    public void assertLoadSchemaTableNamesForOpenGauss() throws SQLException {
+        assertThat(SchemaMetaDataLoader.loadSchemaTableNames(DefaultDatabase.LOGIC_NAME, new OpenGaussDatabaseType(), dataSource), is(createSchemaTableNames()));
+    }
+    
+    @Test
+    public void assertLoadSchemaTableNamesForMySQL() throws SQLException {
+        Map<String, Collection<String>> schemaTableNames = Collections.singletonMap(DefaultDatabase.LOGIC_NAME, Collections.singletonList("tbl"));
+        assertThat(SchemaMetaDataLoader.loadSchemaTableNames(DefaultDatabase.LOGIC_NAME, new MySQLDatabaseType(), dataSource), is(schemaTableNames));
+    }
+    
+    private static Map<String, Collection<String>> createSchemaTableNames() {
+        Map<String, Collection<String>> result = new LinkedHashMap<>();
+        result.put("public", Collections.singletonList("tbl"));
+        result.put("schema_1", Collections.emptyList());
+        result.put("schema_2", Collections.emptyList());
+        return result;
+    }
+    
+    @Test
+    public void assertLoadSchemaNamesForPostgreSQL() throws SQLException {
+        assertThat(SchemaMetaDataLoader.loadSchemaNames(dataSource.getConnection(), new PostgreSQLDatabaseType()), is(Arrays.asList("public", "schema_1", "schema_2")));
+    }
+    
+    @Test
+    public void assertLoadSchemaNamesForOpenGauss() throws SQLException {
+        assertThat(SchemaMetaDataLoader.loadSchemaNames(dataSource.getConnection(), new OpenGaussDatabaseType()), is(Arrays.asList("public", "schema_1", "schema_2")));
+    }
+    
+    @Test
+    public void assertLoadSchemaNamesForMySQL() throws SQLException {
+        assertThat(SchemaMetaDataLoader.loadSchemaNames(dataSource.getConnection(), new MySQLDatabaseType()), is(Collections.singletonList("public")));
+    }
+}
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaTableNamesLoaderTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaTableNamesLoaderTest.java
deleted file mode 100644
index 6ed5d38b598..00000000000
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/loader/common/SchemaTableNamesLoaderTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.metadata.database.schema.loader.common;
-
-import org.apache.shardingsphere.infra.database.DefaultDatabase;
-import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Answers;
-import org.mockito.Mock;
-import org.mockito.junit.MockitoJUnitRunner;
-
-import javax.sql.DataSource;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.Collections;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-@RunWith(MockitoJUnitRunner.class)
-public final class SchemaTableNamesLoaderTest {
-    
-    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
-    private DataSource dataSource;
-    
-    @Before
-    public void setUp() throws SQLException {
-        ResultSet resultSet = mockResultSet();
-        when(dataSource.getConnection().getMetaData().getTables("catalog", "public", null, new String[]{"TABLE", "VIEW"})).thenReturn(resultSet);
-        when(dataSource.getConnection().getCatalog()).thenReturn("catalog");
-        when(dataSource.getConnection().getSchema()).thenReturn("public");
-    }
-    
-    private ResultSet mockResultSet() throws SQLException {
-        ResultSet result = mock(ResultSet.class);
-        when(result.next()).thenReturn(true, true, true, false);
-        when(result.getString("TABLE_NAME")).thenReturn("tbl", "$tbl", "/tbl", "##tbl");
-        return result;
-    }
-    
-    @Test
-    public void assertLoadSchemaTableNames() throws SQLException {
-        assertThat(SchemaTableNamesLoader.loadSchemaTableNames(DefaultDatabase.LOGIC_NAME,
-                new PostgreSQLDatabaseType(), dataSource), is(Collections.singletonMap("public", Collections.singletonList("tbl"))));
-    }
-}
diff --git a/kernel/single-table/core/src/main/java/org/apache/shardingsphere/singletable/datanode/SingleTableDataNodeLoader.java b/kernel/single-table/core/src/main/java/org/apache/shardingsphere/singletable/datanode/SingleTableDataNodeLoader.java
index 9bcd09cc4ba..a023e877a82 100644
--- a/kernel/single-table/core/src/main/java/org/apache/shardingsphere/singletable/datanode/SingleTableDataNodeLoader.java
+++ b/kernel/single-table/core/src/main/java/org/apache/shardingsphere/singletable/datanode/SingleTableDataNodeLoader.java
@@ -21,7 +21,7 @@ import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.common.SchemaTableNamesLoader;
+import org.apache.shardingsphere.infra.metadata.database.schema.loader.common.SchemaMetaDataLoader;
 import org.apache.shardingsphere.singletable.exception.SingleTablesLoadingException;
 
 import javax.sql.DataSource;
@@ -85,7 +85,7 @@ public final class SingleTableDataNodeLoader {
     
     private static Map<String, Collection<String>> loadSchemaTableNames(final String databaseName, final DatabaseType databaseType, final DataSource dataSource, final String dataSourceName) {
         try {
-            return SchemaTableNamesLoader.loadSchemaTableNames(databaseName, databaseType, dataSource);
+            return SchemaMetaDataLoader.loadSchemaTableNames(databaseName, databaseType, dataSource);
         } catch (final SQLException ex) {
             throw new SingleTablesLoadingException(databaseName, dataSourceName, ex);
         }