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 2020/11/10 10:10:40 UTC

[shardingsphere] branch master updated: Add SchemaBuilderMaterials to simplify refresher (#8117)

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 8c151b2  Add SchemaBuilderMaterials to simplify refresher (#8117)
8c151b2 is described below

commit 8c151b27e25c051dd4e6384200f1ac977743dcb3
Author: Liang Zhang <te...@163.com>
AuthorDate: Tue Nov 10 18:10:20 2020 +0800

    Add SchemaBuilderMaterials to simplify refresher (#8117)
    
    * Add SchemaBuilderMaterials
    
    * Use SchemaBuilderMaterials to refactor SchemaRefresher
    
    * Remove TableMetaDataLoaderCallback
---
 .../metadata/schema/builder/SchemaBuilder.java     | 28 ++++++------------
 .../SchemaBuilderMaterials.java}                   | 34 +++++++++++++---------
 .../builder/TableAddressingMapperBuilder.java      | 10 +++----
 .../schema/builder/TableMetaDataBuilder.java       | 25 +++++-----------
 .../metadata/schema/refresher/SchemaRefresher.java |  5 ++--
 .../type/AlterTableStatementSchemaRefresher.java   |  9 +++---
 .../type/CreateIndexStatementSchemaRefresher.java  |  6 ++--
 .../type/CreateTableStatementSchemaRefresher.java  | 16 ++++------
 .../type/CreateViewStatementSchemaRefresher.java   |  6 ++--
 .../type/DropIndexStatementSchemaRefresher.java    |  6 ++--
 .../type/DropTableStatementSchemaRefresher.java    |  7 ++---
 .../type/DropViewStatementSchemaRefresher.java     |  7 ++---
 .../metadata/schema/builder/SchemaBuilderTest.java |  2 +-
 .../schema/builder/TableMetaDataBuilderTest.java   |  8 ++---
 .../ColumnMetaDataLoaderTest.java                  |  3 +-
 .../IndexMetaDataLoaderTest.java                   |  3 +-
 .../SchemaMetaDataLoaderTest.java                  |  3 +-
 .../TableMetaDataLoaderTest.java                   |  3 +-
 .../MetaDataLoaderConnectionAdapterTest.java       |  3 +-
 ...AbstractDatabaseMetaDataDialectHandlerTest.java |  4 +--
 .../DatabaseMetaDataDialectHandlerFactoryTest.java |  3 +-
 .../DatabaseMetaDataDialectHandlerTest.java        |  4 +--
 .../impl/H2DatabaseMetaDataDialectHandlerTest.java |  4 +--
 .../MariaDBDatabaseMetaDataDialectHandlerTest.java |  4 +--
 .../MySQLDatabaseMetaDataDialectHandlerTest.java   |  4 +--
 .../OracleDatabaseMetaDataDialectHandlerTest.java  |  4 +--
 ...stgreSQLDatabaseMetaDataDialectHandlerTest.java |  4 +--
 .../SQL92DatabaseMetaDataDialectHandlerTest.java   |  4 +--
 ...QLServerDatabaseMetaDataDialectHandlerTest.java |  4 +--
 .../AlterTableStatementSchemaRefresherTest.java    | 23 ++++++---------
 .../CreateIndexStatementSchemaRefresherTest.java   |  7 +++--
 .../CreateTableStatementSchemaRefresherTest.java   | 13 ++++-----
 .../CreateViewStatementSchemaRefresherTest.java    |  6 ++--
 .../DropIndexStatementSchemaRefresherTest.java     |  7 +++--
 .../DropTableStatementSchemaRefresherTest.java     |  7 +++--
 .../type/DropViewStatementSchemaRefresherTest.java |  6 ++--
 .../context/metadata/MetaDataContextsBuilder.java  |  3 +-
 .../driver/executor/AbstractStatementExecutor.java |  8 ++---
 .../jdbc/JDBCDatabaseCommunicationEngine.java      | 12 +++-----
 39 files changed, 141 insertions(+), 174 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilder.java
index f47bb79..1890036 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilder.java
@@ -19,17 +19,13 @@ package org.apache.shardingsphere.infra.metadata.schema.builder;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 
-import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Map.Entry;
 
 /**
@@ -41,28 +37,23 @@ public final class SchemaBuilder {
     /**
      * Build ShardingSphere schema.
      * 
-     * @param databaseType database type
-     * @param dataSourceMap data source map
-     * @param rules ShardingSphere rules
-     * @param props configuration properties
+     * @param materials schema builder materials
      * @return ShardingSphere schema
      * @throws SQLException SQL exception
      */
-    public static ShardingSphereSchema build(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                             final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
-        ShardingSphereSchema result = loadSchema(databaseType, dataSourceMap, rules, props);
-        setTableAddressingMapper(databaseType, dataSourceMap, rules, result);
+    public static ShardingSphereSchema build(final SchemaBuilderMaterials materials) throws SQLException {
+        ShardingSphereSchema result = loadSchema(materials);
+        setTableAddressingMapper(materials, result);
         return result;
     }
     
-    private static ShardingSphereSchema loadSchema(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                                   final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
+    private static ShardingSphereSchema loadSchema(final SchemaBuilderMaterials materials) throws SQLException {
         ShardingSphereSchema result = new ShardingSphereSchema();
-        for (ShardingSphereRule rule : rules) {
+        for (ShardingSphereRule rule : materials.getRules()) {
             if (rule instanceof TableContainedRule) {
                 for (String table : ((TableContainedRule) rule).getTables()) {
                     if (!result.containsTable(table)) {
-                        TableMetaDataBuilder.build(table, databaseType, dataSourceMap, rules, props).ifPresent(optional -> result.put(table, optional));
+                        TableMetaDataBuilder.build(table, materials).ifPresent(optional -> result.put(table, optional));
                     }
                 }
             }
@@ -70,9 +61,8 @@ public final class SchemaBuilder {
         return result;
     }
     
-    private static void setTableAddressingMapper(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                                 final Collection<ShardingSphereRule> rules, final ShardingSphereSchema schema) throws SQLException {
-        for (Entry<String, Collection<String>> entry : TableAddressingMapperBuilder.build(databaseType, dataSourceMap, rules).entrySet()) {
+    private static void setTableAddressingMapper(final SchemaBuilderMaterials materials, final ShardingSphereSchema schema) throws SQLException {
+        for (Entry<String, Collection<String>> entry : TableAddressingMapperBuilder.build(materials).entrySet()) {
             String tableName = entry.getKey();
             if (!schema.containsTable(tableName)) {
                 schema.put(tableName, new TableMetaData());
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/TableMetaDataLoaderCallback.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilderMaterials.java
similarity index 51%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/TableMetaDataLoaderCallback.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilderMaterials.java
index 88cb82c..8fbee43 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/TableMetaDataLoaderCallback.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilderMaterials.java
@@ -15,24 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.refresher;
+package org.apache.shardingsphere.infra.metadata.schema.builder;
 
-import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
-import java.sql.SQLException;
-import java.util.Optional;
+import javax.sql.DataSource;
+import java.util.Collection;
+import java.util.Map;
 
 /**
- * Table meta data loader callback.
+ * ShardingSphere schema builder materials.
  */
-public interface TableMetaDataLoaderCallback {
+@RequiredArgsConstructor
+@Getter
+public final class SchemaBuilderMaterials {
     
-    /**
-     * Load table meta data.
-     *
-     * @param tableName table name
-     * @return table meta data
-     * @throws SQLException SQL exception
-     */
-    Optional<TableMetaData> load(String tableName) throws SQLException;
+    private final DatabaseType databaseType;
+    
+    private final Map<String, DataSource> dataSourceMap;
+    
+    private final Collection<ShardingSphereRule> rules;
+    
+    private final ConfigurationProperties props;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableAddressingMapperBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableAddressingMapperBuilder.java
index 1c75fec..281568e 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableAddressingMapperBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableAddressingMapperBuilder.java
@@ -47,15 +47,13 @@ public final class TableAddressingMapperBuilder {
     /**
      * Build table addressing mapper with related data sources.
      *
-     * @param databaseType database type
-     * @param dataSourceMap data source map
-     * @param rules ShardingSphere rules
+     * @param materials schema builder materials
      * @return table addressing mapper with related data sources
      * @throws SQLException SQL exception
      */
-    public static Map<String, Collection<String>> build(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> rules) throws SQLException {
-        Map<String, Collection<String>> result = load(databaseType, dataSourceMap);
-        decorate(rules, result);
+    public static Map<String, Collection<String>> build(final SchemaBuilderMaterials materials) throws SQLException {
+        Map<String, Collection<String>> result = load(materials.getDatabaseType(), materials.getDataSourceMap());
+        decorate(materials.getRules(), result);
         return result;
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java
index 318a765..ee7c4ab 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilder.java
@@ -19,8 +19,6 @@ package org.apache.shardingsphere.infra.metadata.schema.builder;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.builder.spi.RuleBasedTableMetaDataBuilder;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
@@ -29,10 +27,8 @@ import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 
-import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
 
@@ -50,28 +46,23 @@ public final class TableMetaDataBuilder {
      * Build table meta data.
      *
      * @param tableName table name
-     * @param databaseType database type
-     * @param dataSourceMap data source map
-     * @param rules ShardingSphere rules
-     * @param props configuration properties
+     * @param materials schema builder materials
      * @return table meta data
      * @throws SQLException SQL exception
      */
-    public static Optional<TableMetaData> build(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                                final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
-        Optional<TableMetaData> tableMetaData = load(tableName, databaseType, dataSourceMap, rules, props);
-        return tableMetaData.map(optional -> decorate(tableName, optional, rules));
+    public static Optional<TableMetaData> build(final String tableName, final SchemaBuilderMaterials materials) throws SQLException {
+        Optional<TableMetaData> tableMetaData = load(tableName, materials);
+        return tableMetaData.map(optional -> decorate(tableName, optional, materials.getRules()));
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private static Optional<TableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                                final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
-        DataNodes dataNodes = new DataNodes(rules);
-        for (Entry<ShardingSphereRule, RuleBasedTableMetaDataBuilder> entry : OrderedSPIRegistry.getRegisteredServices(rules, RuleBasedTableMetaDataBuilder.class).entrySet()) {
+    private static Optional<TableMetaData> load(final String tableName, final SchemaBuilderMaterials materials) throws SQLException {
+        DataNodes dataNodes = new DataNodes(materials.getRules());
+        for (Entry<ShardingSphereRule, RuleBasedTableMetaDataBuilder> entry : OrderedSPIRegistry.getRegisteredServices(materials.getRules(), RuleBasedTableMetaDataBuilder.class).entrySet()) {
             if (entry.getKey() instanceof TableContainedRule) {
                 TableContainedRule rule = (TableContainedRule) entry.getKey();
                 RuleBasedTableMetaDataBuilder loader = entry.getValue();
-                Optional<TableMetaData> tableMetaData = loader.load(tableName, databaseType, dataSourceMap, dataNodes, rule, props);
+                Optional<TableMetaData> tableMetaData = loader.load(tableName, materials.getDatabaseType(), materials.getDataSourceMap(), dataNodes, rule, materials.getProps());
                 if (tableMetaData.isPresent()) {
                     return tableMetaData;
                 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/SchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/SchemaRefresher.java
index bd44bd9..22925da 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/SchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/SchemaRefresher.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.SQLException;
@@ -36,8 +37,8 @@ public interface SchemaRefresher<T extends SQLStatement> {
      * @param schema ShardingSphere schema to be refreshed
      * @param routeDataSourceNames route dataSource names
      * @param sqlStatement SQL statement
-     * @param callback callback
+     * @param materials schema builder materials
      * @throws SQLException SQL exception
      */
-    void refresh(ShardingSphereSchema schema, Collection<String> routeDataSourceNames, T sqlStatement, TableMetaDataLoaderCallback callback) throws SQLException;
+    void refresh(ShardingSphereSchema schema, Collection<String> routeDataSourceNames, T sqlStatement, SchemaBuilderMaterials materials) throws SQLException;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresher.java
index abe1ad8..72a3365 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresher.java
@@ -18,8 +18,9 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.schema.builder.TableMetaDataBuilder;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
-import org.apache.shardingsphere.infra.metadata.schema.refresher.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
 
 import java.sql.SQLException;
@@ -31,11 +32,11 @@ import java.util.Collection;
 public final class AlterTableStatementSchemaRefresher implements SchemaRefresher<AlterTableStatement> {
     
     @Override
-    public void refresh(final ShardingSphereSchema schema, final Collection<String> routeDataSourceNames,
-                        final AlterTableStatement sqlStatement, final TableMetaDataLoaderCallback callback) throws SQLException {
+    public void refresh(final ShardingSphereSchema schema, 
+                        final Collection<String> routeDataSourceNames, final AlterTableStatement sqlStatement, final SchemaBuilderMaterials materials) throws SQLException {
         String tableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
         if (null != schema && schema.containsTable(tableName)) {
-            callback.load(tableName).ifPresent(tableMetaData -> schema.put(tableName, tableMetaData));
+            TableMetaDataBuilder.build(tableName, materials).ifPresent(tableMetaData -> schema.put(tableName, tableMetaData));
         }
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresher.java
index c188455..87b2ea2 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresher.java
@@ -18,9 +18,9 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
-import org.apache.shardingsphere.infra.metadata.schema.refresher.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateIndexStatement;
 
 import java.util.Collection;
@@ -31,8 +31,8 @@ import java.util.Collection;
 public final class CreateIndexStatementSchemaRefresher implements SchemaRefresher<CreateIndexStatement> {
     
     @Override
-    public void refresh(final ShardingSphereSchema schema, final Collection<String> routeDataSourceNames,
-                        final CreateIndexStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
+    public void refresh(final ShardingSphereSchema schema, 
+                        final Collection<String> routeDataSourceNames, final CreateIndexStatement sqlStatement, final SchemaBuilderMaterials materials) {
         if (null == sqlStatement.getIndex()) {
             return;
         }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresher.java
index 438a637..bdf0130 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresher.java
@@ -18,14 +18,14 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.schema.builder.TableMetaDataBuilder;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
-import org.apache.shardingsphere.infra.metadata.schema.refresher.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
 
 import java.sql.SQLException;
 import java.util.Collection;
-import java.util.Optional;
 
 /**
  * ShardingSphere schema refresher for create table statement.
@@ -33,15 +33,11 @@ import java.util.Optional;
 public final class CreateTableStatementSchemaRefresher implements SchemaRefresher<CreateTableStatement> {
     
     @Override
-    public void refresh(final ShardingSphereSchema schema, final Collection<String> routeDataSourceNames,
-                        final CreateTableStatement sqlStatement, final TableMetaDataLoaderCallback callback) throws SQLException {
+    public void refresh(final ShardingSphereSchema schema, 
+                        final Collection<String> routeDataSourceNames, final CreateTableStatement sqlStatement, final SchemaBuilderMaterials materials) throws SQLException {
         String tableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
-        Optional<TableMetaData> tableMetaData = callback.load(tableName);
-        if (tableMetaData.isPresent()) {
-            schema.put(tableName, tableMetaData.get());
-        } else {
-            schema.put(tableName, new TableMetaData());
-        }
+        TableMetaData tableMetaData = TableMetaDataBuilder.build(tableName, materials).orElse(new TableMetaData());
+        schema.put(tableName, tableMetaData);
         schema.get(tableName).getAddressingDataSources().addAll(routeDataSourceNames);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresher.java
index f1d5d58..7a464d5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresher.java
@@ -18,9 +18,9 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
-import org.apache.shardingsphere.infra.metadata.schema.refresher.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
 
 import java.util.Collection;
@@ -31,8 +31,8 @@ import java.util.Collection;
 public final class CreateViewStatementSchemaRefresher implements SchemaRefresher<CreateViewStatement> {
     
     @Override
-    public void refresh(final ShardingSphereSchema schema, final Collection<String> routeDataSourceNames,
-                        final CreateViewStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
+    public void refresh(final ShardingSphereSchema schema, 
+                        final Collection<String> routeDataSourceNames, final CreateViewStatement sqlStatement, final SchemaBuilderMaterials materials) {
         String viewName = sqlStatement.getView().getTableName().getIdentifier().getValue();
         schema.put(viewName, new TableMetaData());
         schema.get(viewName).getAddressingDataSources().addAll(routeDataSourceNames);
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresher.java
index 05dd433..eece67d 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresher.java
@@ -18,9 +18,9 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
-import org.apache.shardingsphere.infra.metadata.schema.refresher.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.DropIndexStatementHandler;
@@ -36,8 +36,8 @@ import java.util.stream.Collectors;
 public final class DropIndexStatementSchemaRefresher implements SchemaRefresher<DropIndexStatement> {
     
     @Override
-    public void refresh(final ShardingSphereSchema schema, final Collection<String> routeDataSourceNames,
-                        final DropIndexStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
+    public void refresh(final ShardingSphereSchema schema, 
+                        final Collection<String> routeDataSourceNames, final DropIndexStatement sqlStatement, final SchemaBuilderMaterials materials) {
         Collection<String> indexNames = getIndexNames(sqlStatement);
         Optional<SimpleTableSegment> simpleTableSegment = DropIndexStatementHandler.getSimpleTableSegment(sqlStatement);
         String tableName = simpleTableSegment.map(tableSegment -> tableSegment.getTableName().getIdentifier().getValue()).orElse("");
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresher.java
index ae29f71..7eeac07 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresher.java
@@ -18,8 +18,8 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
-import org.apache.shardingsphere.infra.metadata.schema.refresher.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
 
 import java.util.Collection;
@@ -30,9 +30,8 @@ import java.util.Collection;
 public final class DropTableStatementSchemaRefresher implements SchemaRefresher<DropTableStatement> {
     
     @Override
-    public void refresh(final ShardingSphereSchema schema, final Collection<String> routeDataSourceNames,
-                        final DropTableStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
+    public void refresh(final ShardingSphereSchema schema, 
+                        final Collection<String> routeDataSourceNames, final DropTableStatement sqlStatement, final SchemaBuilderMaterials materials) {
         sqlStatement.getTables().forEach(each -> schema.remove(each.getTableName().getIdentifier().getValue()));
     }
-
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresher.java
index ae140ec..8aeda2b 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresher.java
@@ -18,8 +18,8 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
-import org.apache.shardingsphere.infra.metadata.schema.refresher.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropViewStatement;
 
 import java.util.Collection;
@@ -30,9 +30,8 @@ import java.util.Collection;
 public final class DropViewStatementSchemaRefresher implements SchemaRefresher<DropViewStatement> {
     
     @Override
-    public void refresh(final ShardingSphereSchema schema, final Collection<String> routeDataSourceNames,
-                        final DropViewStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
+    public void refresh(final ShardingSphereSchema schema, 
+                        final Collection<String> routeDataSourceNames, final DropViewStatement sqlStatement, final SchemaBuilderMaterials materials) {
         sqlStatement.getViews().forEach(each -> schema.remove(each.getTableName().getIdentifier().getValue()));
     }
-
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilderTest.java
index 895df7a..3bde21a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/SchemaBuilderTest.java
@@ -52,7 +52,7 @@ public final class SchemaBuilderTest {
     @Test
     public void assertBuild() throws SQLException {
         ShardingSphereSchema actual = SchemaBuilder.build(
-                databaseType, Collections.singletonMap("logic_db", dataSource), Arrays.asList(new CommonFixtureRule(), new DataNodeContainedFixtureRule()), props);
+                new SchemaBuilderMaterials(databaseType, Collections.singletonMap("logic_db", dataSource), Arrays.asList(new CommonFixtureRule(), new DataNodeContainedFixtureRule()), props));
         assertSchema(actual);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilderTest.java
index 6d6c380..8558187 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/TableMetaDataBuilderTest.java
@@ -49,13 +49,13 @@ public final class TableMetaDataBuilderTest {
     
     @Test
     public void assertBuildWithExistedTableName() throws SQLException {
-        assertTrue(TableMetaDataBuilder.build("data_node_routed_table_0", 
-                databaseType, Collections.singletonMap("logic_db", dataSource), Arrays.asList(new CommonFixtureRule(), new DataNodeContainedFixtureRule()), props).isPresent());
+        assertTrue(TableMetaDataBuilder.build("data_node_routed_table_0", new SchemaBuilderMaterials(
+                databaseType, Collections.singletonMap("logic_db", dataSource), Arrays.asList(new CommonFixtureRule(), new DataNodeContainedFixtureRule()), props)).isPresent());
     }
     
     @Test
     public void assertBuildWithNotExistedTableName() throws SQLException {
-        assertFalse(TableMetaDataBuilder.build("invalid_table", 
-                databaseType, Collections.singletonMap("logic_db", dataSource), Arrays.asList(new CommonFixtureRule(), new DataNodeContainedFixtureRule()), props).isPresent());
+        assertFalse(TableMetaDataBuilder.build("invalid_table", new SchemaBuilderMaterials(
+                databaseType, Collections.singletonMap("logic_db", dataSource), Arrays.asList(new CommonFixtureRule(), new DataNodeContainedFixtureRule()), props)).isPresent());
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/ColumnMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/ColumnMetaDataLoaderTest.java
similarity index 97%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/ColumnMetaDataLoaderTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/ColumnMetaDataLoaderTest.java
index f8a4218..0e780bb 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/ColumnMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/ColumnMetaDataLoaderTest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.ColumnMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.junit.Before;
 import org.junit.Test;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/IndexMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/IndexMetaDataLoaderTest.java
similarity index 96%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/IndexMetaDataLoaderTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/IndexMetaDataLoaderTest.java
index 199cc00..d2c34ee 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/IndexMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/IndexMetaDataLoaderTest.java
@@ -15,9 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader;
 
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.IndexMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/SchemaMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/SchemaMetaDataLoaderTest.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/SchemaMetaDataLoaderTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/SchemaMetaDataLoaderTest.java
index 8779608..59b5e68 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/SchemaMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/SchemaMetaDataLoaderTest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.SchemaMetaDataLoader;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/TableMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/TableMetaDataLoaderTest.java
similarity index 98%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/TableMetaDataLoaderTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/TableMetaDataLoaderTest.java
index f563586..811f4c5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/TableMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/TableMetaDataLoaderTest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.TableMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/adapter/MetaDataLoaderConnectionAdapterTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/adapter/MetaDataLoaderConnectionAdapterTest.java
similarity index 96%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/adapter/MetaDataLoaderConnectionAdapterTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/adapter/MetaDataLoaderConnectionAdapterTest.java
index 1253754..e81559a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/adapter/MetaDataLoaderConnectionAdapterTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/adapter/MetaDataLoaderConnectionAdapterTest.java
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.adapter;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.adapter;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.adapter.MetaDataLoaderConnectionAdapter;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/AbstractDatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/AbstractDatabaseMetaDataDialectHandlerTest.java
similarity index 91%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/AbstractDatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/AbstractDatabaseMetaDataDialectHandlerTest.java
index f112d59..1282978 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/AbstractDatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/AbstractDatabaseMetaDataDialectHandlerTest.java
@@ -15,12 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect;
 
 import lombok.Getter;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.DatabaseMetaDataDialectHandler;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.DatabaseMetaDataDialectHandlerFactory;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/DatabaseMetaDataDialectHandlerFactoryTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/DatabaseMetaDataDialectHandlerFactoryTest.java
similarity index 91%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/DatabaseMetaDataDialectHandlerFactoryTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/DatabaseMetaDataDialectHandlerFactoryTest.java
index 436fe3e..257975c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/DatabaseMetaDataDialectHandlerFactoryTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/DatabaseMetaDataDialectHandlerFactoryTest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.DatabaseMetaDataDialectHandlerFactory;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.junit.MockitoJUnitRunner;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/DatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/DatabaseMetaDataDialectHandlerTest.java
similarity index 96%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/DatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/DatabaseMetaDataDialectHandlerTest.java
index b596344..6cee0ea 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/DatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/DatabaseMetaDataDialectHandlerTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
@@ -25,8 +25,6 @@ import org.apache.shardingsphere.infra.database.type.dialect.OracleDatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.SQL92DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.SQLServerDatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.DatabaseMetaDataDialectHandler;
-import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.DatabaseMetaDataDialectHandlerFactory;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/H2DatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/H2DatabaseMetaDataDialectHandlerTest.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/H2DatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/H2DatabaseMetaDataDialectHandlerTest.java
index 73649b5..a2fbbd5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/H2DatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/H2DatabaseMetaDataDialectHandlerTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.impl;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.impl;
 
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Test;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/MariaDBDatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/MariaDBDatabaseMetaDataDialectHandlerTest.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/MariaDBDatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/MariaDBDatabaseMetaDataDialectHandlerTest.java
index 5fa1974..08f3f11 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/MariaDBDatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/MariaDBDatabaseMetaDataDialectHandlerTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.impl;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.impl;
 
 import org.apache.shardingsphere.infra.database.type.dialect.MariaDBDatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Test;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/MySQLDatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/MySQLDatabaseMetaDataDialectHandlerTest.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/MySQLDatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/MySQLDatabaseMetaDataDialectHandlerTest.java
index e797061..e21276a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/MySQLDatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/MySQLDatabaseMetaDataDialectHandlerTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.impl;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.impl;
 
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Test;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/OracleDatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/OracleDatabaseMetaDataDialectHandlerTest.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/OracleDatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/OracleDatabaseMetaDataDialectHandlerTest.java
index 28eca84..71f8179 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/OracleDatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/OracleDatabaseMetaDataDialectHandlerTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.impl;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.impl;
 
 import org.apache.shardingsphere.infra.database.type.dialect.OracleDatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Test;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/PostgreSQLDatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/PostgreSQLDatabaseMetaDataDialectHandlerTest.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/PostgreSQLDatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/PostgreSQLDatabaseMetaDataDialectHandlerTest.java
index 0b7f144..d179237 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/PostgreSQLDatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/PostgreSQLDatabaseMetaDataDialectHandlerTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.impl;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.impl;
 
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Test;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/SQL92DatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/SQL92DatabaseMetaDataDialectHandlerTest.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/SQL92DatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/SQL92DatabaseMetaDataDialectHandlerTest.java
index cd06d03..c86213d 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/SQL92DatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/SQL92DatabaseMetaDataDialectHandlerTest.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.impl;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.impl;
 
 import org.apache.shardingsphere.infra.database.type.dialect.SQL92DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Test;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/SQLServerDatabaseMetaDataDialectHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/SQLServerDatabaseMetaDataDialectHandlerTest.java
similarity index 94%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/SQLServerDatabaseMetaDataDialectHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/SQLServerDatabaseMetaDataDialectHandlerTest.java
index 716aeb5..3befc7f 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/physical/dialect/impl/SQLServerDatabaseMetaDataDialectHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/impl/SQLServerDatabaseMetaDataDialectHandlerTest.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.impl;
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.SQLServerDatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.builder.physical.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.AbstractDatabaseMetaDataDialectHandlerTest;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.junit.Before;
 import org.junit.Test;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresherTest.java
index c65a34e..14007f1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/AlterTableStatementSchemaRefresherTest.java
@@ -18,9 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
@@ -35,9 +33,8 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 
-import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public final class AlterTableStatementSchemaRefresherTest {
     
@@ -45,34 +42,32 @@ public final class AlterTableStatementSchemaRefresherTest {
     public void refreshForMySQL() throws SQLException {
         refresh(new MySQLAlterTableStatement());
     }
-
+    
     @Test
     public void refreshForOracle() throws SQLException {
         refresh(new OracleAlterTableStatement());
     }
-
+    
     @Test
     public void refreshForPostgreSQL() throws SQLException {
         refresh(new PostgreSQLAlterTableStatement());
     }
-
+    
     @Test
     public void refreshForSQL92() throws SQLException {
         refresh(new SQL92AlterTableStatement());
     }
-
+    
     @Test
     public void refreshForSQLServer() throws SQLException {
         refresh(new SQLServerAlterTableStatement());
     }
-
+    
     private void refresh(final AlterTableStatement alterTableStatement) throws SQLException {
         ShardingSphereSchema schema = ShardingSphereSchemaBuildUtil.buildSchema();
         SchemaRefresher<AlterTableStatement> schemaRefresher = new AlterTableStatementSchemaRefresher();
         alterTableStatement.setTable(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order"))));
-        schemaRefresher.refresh(schema, Collections.emptyList(), alterTableStatement, tableName -> Optional.of(new TableMetaData(
-                Collections.singletonList(new ColumnMetaData("order_id", 1, "String", true, false, false)),
-                Collections.singletonList(new IndexMetaData("index_alter")))));
-        assertTrue(schema.get("t_order").getIndexes().containsKey("index_alter"));
+        schemaRefresher.refresh(schema, Collections.emptyList(), alterTableStatement, mock(SchemaBuilderMaterials.class));
+        // TODO mock result of TableMetaDataBuilder.build and assert alter
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresherTest.java
index bca4a51..ea51c65 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateIndexStatementSchemaRefresherTest.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -32,10 +33,10 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public final class CreateIndexStatementSchemaRefresherTest {
     
@@ -64,7 +65,7 @@ public final class CreateIndexStatementSchemaRefresherTest {
         SchemaRefresher<CreateIndexStatement> schemaRefresher = new CreateIndexStatementSchemaRefresher();
         createIndexStatement.setIndex(new IndexSegment(1, 2, new IdentifierValue("t_order_index")));
         createIndexStatement.setTable(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order"))));
-        schemaRefresher.refresh(schema, Collections.emptyList(), createIndexStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(schema, Collections.emptyList(), createIndexStatement, mock(SchemaBuilderMaterials.class));
         assertTrue(schema.get("t_order").getIndexes().containsKey("t_order_index"));
     }
     
@@ -92,7 +93,7 @@ public final class CreateIndexStatementSchemaRefresherTest {
         ShardingSphereSchema schema = ShardingSphereSchemaBuildUtil.buildSchema();
         SchemaRefresher<CreateIndexStatement> schemaRefresher = new CreateIndexStatementSchemaRefresher();
         createIndexStatement.setTable(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order"))));
-        schemaRefresher.refresh(schema, Collections.emptyList(), createIndexStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(schema, Collections.emptyList(), createIndexStatement, mock(SchemaBuilderMaterials.class));
         assertFalse(schema.get("t_order").getIndexes().containsKey("t_order_index"));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresherTest.java
index 7100373..24645dd 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateTableStatementSchemaRefresherTest.java
@@ -18,9 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
@@ -35,9 +33,9 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public final class CreateTableStatementSchemaRefresherTest {
     
@@ -77,9 +75,8 @@ public final class CreateTableStatementSchemaRefresherTest {
         ShardingSphereSchema schema = ShardingSphereSchemaBuildUtil.buildSchema();
         createTableStatement.setTable(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order_0"))));
         SchemaRefresher<CreateTableStatement> schemaRefresher = new CreateTableStatementSchemaRefresher();
-        schemaRefresher.refresh(schema, Collections.emptyList(), createTableStatement, tableName -> Optional.of(new TableMetaData(
-                Collections.singletonList(new ColumnMetaData("order_id", 1, "String", true, false, false)),
-                Collections.singletonList(new IndexMetaData("index")))));
+        SchemaBuilderMaterials materials = mock(SchemaBuilderMaterials.class);
+        schemaRefresher.refresh(schema, Collections.emptyList(), createTableStatement, materials);
         assertTrue(schema.containsTable("t_order_0"));
     }
     
@@ -118,6 +115,6 @@ public final class CreateTableStatementSchemaRefresherTest {
     private void refreshWithUnConfigured(final CreateTableStatement createTableStatement) throws SQLException {
         createTableStatement.setTable(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order_item_0"))));
         SchemaRefresher<CreateTableStatement> schemaRefresher = new CreateTableStatementSchemaRefresher();
-        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), createTableStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), createTableStatement, mock(SchemaBuilderMaterials.class));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresherTest.java
index 953fab2..72f0248 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/CreateViewStatementSchemaRefresherTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
@@ -28,7 +29,8 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
+
+import static org.mockito.Mockito.mock;
 
 public final class CreateViewStatementSchemaRefresherTest {
     
@@ -45,6 +47,6 @@ public final class CreateViewStatementSchemaRefresherTest {
     private void refreshWithUnConfigured(final CreateViewStatement createViewStatement) throws SQLException {
         createViewStatement.setView(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order_item_0"))));
         SchemaRefresher<CreateViewStatement> schemaRefresher = new CreateViewStatementSchemaRefresher();
-        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), createViewStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), createViewStatement, mock(SchemaBuilderMaterials.class));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresherTest.java
index 04d1b19..5713e06 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropIndexStatementSchemaRefresherTest.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
@@ -32,9 +33,9 @@ import org.junit.Test;
 import java.sql.SQLException;
 import java.util.Collections;
 import java.util.Map;
-import java.util.Optional;
 
 import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.mock;
 
 public final class DropIndexStatementSchemaRefresherTest {
     
@@ -56,7 +57,7 @@ public final class DropIndexStatementSchemaRefresherTest {
         ShardingSphereSchema schema = ShardingSphereSchemaBuildUtil.buildSchema();
         dropIndexStatement.getIndexes().add(new IndexSegment(1, 2, new IdentifierValue("index")));
         SchemaRefresher<DropIndexStatement> metaDataRefreshStrategy = new DropIndexStatementSchemaRefresher();
-        metaDataRefreshStrategy.refresh(schema, Collections.emptyList(), dropIndexStatement, tableName -> Optional.empty());
+        metaDataRefreshStrategy.refresh(schema, Collections.emptyList(), dropIndexStatement, mock(SchemaBuilderMaterials.class));
         assertFalse(schema.get("t_order").getIndexes().containsKey("index"));
     }
     
@@ -83,7 +84,7 @@ public final class DropIndexStatementSchemaRefresherTest {
         actualIndex.put("t_order_index", new IndexMetaData("t_order_index"));
         actualIndex.put("order_id_index", new IndexMetaData("order_id_index"));
         SchemaRefresher<DropIndexStatement> schemaRefresher = new DropIndexStatementSchemaRefresher();
-        schemaRefresher.refresh(schema, Collections.emptyList(), dropIndexStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(schema, Collections.emptyList(), dropIndexStatement, mock(SchemaBuilderMaterials.class));
         assertFalse(schema.get("t_order").getIndexes().containsKey("index"));
         assertFalse(schema.get("t_order").getIndexes().containsKey("t_order_index"));
         assertFalse(schema.get("t_order").getIndexes().containsKey("order_id_index"));
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresherTest.java
index f23ed4f..28f60ff 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropTableStatementSchemaRefresherTest.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
@@ -32,9 +33,9 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 
 import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.mock;
 
 public final class DropTableStatementSchemaRefresherTest {
     
@@ -67,7 +68,7 @@ public final class DropTableStatementSchemaRefresherTest {
         ShardingSphereSchema schema = ShardingSphereSchemaBuildUtil.buildSchema();
         SchemaRefresher<DropTableStatement> schemaRefresher = new DropTableStatementSchemaRefresher();
         dropTableStatement.getTables().add(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order"))));
-        schemaRefresher.refresh(schema, Collections.emptyList(), dropTableStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(schema, Collections.emptyList(), dropTableStatement, mock(SchemaBuilderMaterials.class));
         assertFalse(schema.containsTable("t_order"));
     }
     
@@ -99,6 +100,6 @@ public final class DropTableStatementSchemaRefresherTest {
     private void refreshWithUnConfigured(final DropTableStatement dropTableStatement) throws SQLException {
         SchemaRefresher<DropTableStatement> schemaRefresher = new DropTableStatementSchemaRefresher();
         dropTableStatement.getTables().add(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order_item"))));
-        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), dropTableStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), dropTableStatement, mock(SchemaBuilderMaterials.class));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresherTest.java
index 6322a64..5989f90 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresher/type/DropViewStatementSchemaRefresherTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.refresher.type;
 
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
@@ -28,7 +29,8 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
+
+import static org.mockito.Mockito.mock;
 
 public final class DropViewStatementSchemaRefresherTest {
     
@@ -45,6 +47,6 @@ public final class DropViewStatementSchemaRefresherTest {
     private void refreshWithUnConfigured(final DropViewStatement dropViewStatement) throws SQLException {
         SchemaRefresher<DropViewStatement> schemaRefresher = new DropViewStatementSchemaRefresher();
         dropViewStatement.getViews().add(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order_item"))));
-        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), dropViewStatement, tableName -> Optional.empty());
+        schemaRefresher.refresh(ShardingSphereSchemaBuildUtil.buildSchema(), Collections.singletonList("t_order_item"), dropViewStatement, mock(SchemaBuilderMaterials.class));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
index a3316cf..ba1eb96 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
@@ -33,6 +33,7 @@ import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRulesBuilder;
 
@@ -131,7 +132,7 @@ public final class MetaDataContextsBuilder {
     
     private ShardingSphereSchema buildSchema(final String schemaName, final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> rules) throws SQLException {
         long start = System.currentTimeMillis();
-        ShardingSphereSchema result = SchemaBuilder.build(databaseType, dataSourceMap, rules, props);
+        ShardingSphereSchema result = SchemaBuilder.build(new SchemaBuilderMaterials(databaseType, dataSourceMap, rules, props));
         log.info("Load meta data for schema {} finished, cost {} milliseconds.", schemaName, System.currentTimeMillis() - start);
         return result;
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
index caef5a0..01d24a2 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
@@ -28,15 +28,15 @@ import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.StatementExec
 import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.executor.SQLExecutor;
 import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.executor.SQLExecutorCallback;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.builder.TableMetaDataBuilder;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresherFactory;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.spi.SchemaChangedNotifier;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
@@ -84,12 +84,12 @@ public abstract class AbstractStatementExecutor {
         if (schemaRefresher.isPresent()) {
             Collection<String> routeDataSourceNames = routeUnits.stream().map(RouteUnit::getDataSourceMapper).map(RouteMapper::getLogicName).collect(Collectors.toList());
             schemaRefresher.get().refresh(metaData.getSchema(), routeDataSourceNames, sqlStatement, 
-                tableName -> TableMetaDataBuilder.build(tableName, metaDataContexts.getDatabaseType(), dataSourceMap, metaData.getRuleMetaData().getRules(), metaDataContexts.getProps()));
+                    new SchemaBuilderMaterials(metaDataContexts.getDatabaseType(), dataSourceMap, metaData.getRuleMetaData().getRules(), metaDataContexts.getProps()));
             notifyPersistSchema(DefaultSchema.LOGIC_NAME, metaData.getSchema());
         }
     }
     
-    protected boolean executeAndRefreshMetaData(final Collection<InputGroup<StatementExecuteUnit>> inputGroups, final SQLStatement sqlStatement,
+    protected final boolean executeAndRefreshMetaData(final Collection<InputGroup<StatementExecuteUnit>> inputGroups, final SQLStatement sqlStatement,
                                                 final Collection<RouteUnit> routeUnits, final SQLExecutorCallback<Boolean> sqlExecutorCallback) throws SQLException {
         List<Boolean> result = sqlExecutor.execute(inputGroups, sqlExecutorCallback);
         refreshTableMetaData(metaDataContexts.getDefaultMetaData(), sqlStatement, routeUnits);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
index 50ed521..abf6c69 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
@@ -31,8 +31,7 @@ import org.apache.shardingsphere.infra.executor.sql.raw.execute.result.query.Que
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.builder.TableMetaDataBuilder;
-import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresher;
 import org.apache.shardingsphere.infra.metadata.schema.refresher.SchemaRefresherFactory;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
@@ -104,16 +103,13 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
         }
         Optional<SchemaRefresher> schemaRefresher = SchemaRefresherFactory.newInstance(sqlStatement);
         if (schemaRefresher.isPresent()) {
-            schemaRefresher.get().refresh(metaData.getSchema(), routeDataSourceNames, sqlStatement, this::buildTableMetaData);
+            SchemaBuilderMaterials materials = new SchemaBuilderMaterials(ProxyContext.getInstance().getMetaDataContexts().getDatabaseType(), 
+                    metaData.getResource().getDataSources(), metaData.getRuleMetaData().getRules(), ProxyContext.getInstance().getMetaDataContexts().getProps());
+            schemaRefresher.get().refresh(metaData.getSchema(), routeDataSourceNames, sqlStatement, materials);
             GovernanceEventBus.getInstance().post(new SchemaPersistEvent(metaData.getName(), metaData.getSchema()));
         }
     }
     
-    private Optional<TableMetaData> buildTableMetaData(final String tableName) throws SQLException {
-        return TableMetaDataBuilder.build(tableName, ProxyContext.getInstance().getMetaDataContexts().getDatabaseType(), 
-                metaData.getResource().getDataSources(), metaData.getRuleMetaData().getRules(), ProxyContext.getInstance().getMetaDataContexts().getProps());
-    }
-    
     private BackendResponse merge(final SQLStatementContext<?> sqlStatementContext) throws SQLException {
         if (response instanceof UpdateResponse) {
             mergeUpdateCount(sqlStatementContext);