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

[shardingsphere] branch master updated: Split actual metadata and logic metadata (#18002)

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 f3b8fd404b0 Split actual metadata and logic metadata (#18002)
f3b8fd404b0 is described below

commit f3b8fd404b0a4bb4f1c877fb7ca6cec24249d37f
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Fri May 27 16:39:48 2022 +0800

    Split actual metadata and logic metadata (#18002)
    
    * Split actual metadata and logic metadata
    
    * Split actual metadata and logic metadata
    
    * Split actual metadata and logic metadata
    
    * fix build exception
---
 .../merge/dal/show/LogicTablesMergedResult.java    |  4 +-
 .../dal/show/ShowCreateTableMergedResult.java      |  8 +-
 .../dql/groupby/GroupByMemoryMergedResult.java     |  8 +-
 .../sharding/merge/dql/orderby/OrderByValue.java   | 12 +--
 .../dal/show/ShowCreateTableMergedResultTest.java  | 10 +--
 .../dal/show/ShowTableStatusMergedResultTest.java  |  4 +-
 .../merge/dal/show/ShowTablesMergedResultTest.java |  4 +-
 .../merge/dql/ShardingDQLResultMergerTest.java     | 12 +--
 .../dql/groupby/GroupByMemoryMergedResultTest.java |  4 +-
 .../dql/groupby/GroupByStreamMergedResultTest.java | 12 +--
 .../dql/orderby/OrderByStreamMergedResultTest.java | 10 +--
 .../merge/dql/orderby/OrderByValueTest.java        |  8 +-
 .../engine/type/standard/AbstractSQLRouteTest.java | 28 +++----
 .../ShardingAlterIndexStatementValidatorTest.java  | 16 ++--
 .../ShardingCreateIndexStatementValidatorTest.java | 20 ++---
 .../ShardingDropIndexStatementValidatorTest.java   | 28 +++----
 .../keygen/engine/GeneratedKeyContextEngine.java   |  4 +-
 .../engine/GeneratedKeyContextEngineTest.java      |  8 +-
 .../binder/segment/table/TablesContextTest.java    | 15 ++--
 .../schema/builder/GenericSchemaBuilder.java       | 69 ++++++++++++----
 .../schema/builder/SystemSchemaBuilder.java        |  4 +-
 .../decorator/model/ShardingSphereColumn.java      | 22 +++---
 .../decorator/model/ShardingSphereConstraint.java  | 20 +++--
 .../decorator/model/ShardingSphereIndex.java       | 20 ++---
 .../decorator/model/ShardingSphereSchema.java      | 13 ++--
 .../decorator/model/ShardingSphereTable.java       | 91 ++++++++++++++++++++++
 .../database/schema/event/SchemaAlteredEvent.java  |  4 +-
 .../yaml/schema/swapper/SchemaYamlSwapper.java     |  6 +-
 .../schema/swapper/TableMetaDataYamlSwapper.java   | 46 +++++------
 .../database/schema/ShardingSphereSchemaTest.java  | 23 +++---
 .../schema/builder/GenericSchemaBuilderTest.java   |  3 +-
 .../schema/util/IndexMetaDataUtilTest.java         |  8 +-
 .../type/AlterIndexStatementSchemaRefresher.java   |  8 +-
 .../type/AlterTableStatementSchemaRefresher.java   |  6 +-
 .../type/AlterViewStatementSchemaRefresher.java    |  6 +-
 .../type/CreateIndexStatementSchemaRefresher.java  |  4 +-
 .../type/CreateTableStatementSchemaRefresher.java  |  6 +-
 .../type/CreateViewStatementSchemaRefresher.java   |  6 +-
 .../type/DropIndexStatementSchemaRefresher.java    |  6 +-
 .../type/RenameTableStatementSchemaRefresher.java  |  6 +-
 .../sql/context/ExecutionContextBuilderTest.java   | 38 ++++-----
 .../CustomizedFilterableExecutorTest.java          | 24 +++---
 .../metadata/FederationDatabaseMetaData.java       |  4 +-
 .../metadata/FederationSchemaMetaData.java         |  8 +-
 .../metadata/FederationTableMetaData.java          | 14 ++--
 .../optimizer/ShardingSphereOptimizerTest.java     | 24 +++---
 .../check/consistency/DataConsistencyChecker.java  |  6 +-
 .../metadata/SingleTableSchemaBuilderTest.java     |  8 +-
 .../mode/manager/ContextManager.java               | 12 +--
 .../service/SchemaMetaDataPersistService.java      | 14 ++--
 .../mode/manager/ContextManagerTest.java           |  4 +-
 .../service/SchemaMetaDataPersistServiceTest.java  |  6 +-
 .../subscriber/ScalingRegistrySubscriber.java      |  4 +-
 .../config/event/schema/SchemaChangedEvent.java    |  4 +-
 .../ClusterContextManagerCoordinatorTest.java      |  4 +-
 .../SchemaMetaDataRegistrySubscriberTest.java      |  6 +-
 .../header/query/impl/MySQLQueryHeaderBuilder.java |  8 +-
 .../admin/mysql/executor/ShowTablesExecutor.java   |  4 +-
 .../common/queryable/ShowTableMetadataHandler.java |  4 +-
 .../jdbc/JDBCDatabaseCommunicationEngineTest.java  | 10 +--
 .../query/impl/MySQLQueryHeaderBuilderTest.java    | 12 +--
 .../mysql/executor/ShowTablesExecutorTest.java     | 12 +--
 .../executor/SelectTableExecutorTest.java          |  4 +-
 .../ExportDatabaseConfigurationHandlerTest.java    | 18 ++---
 .../queryable/ShowTableMetadataHandlerTest.java    | 17 ++--
 .../ImportDatabaseConfigurationHandlerTest.java    | 14 ++--
 .../describe/PostgreSQLComDescribeExecutor.java    | 16 ++--
 .../PostgreSQLComDescribeExecutorTest.java         | 16 ++--
 .../pipeline/core/util/PipelineContextUtil.java    | 10 +--
 ...SchemaMetaDataAwareEncryptAlgorithmFixture.java |  4 +-
 .../EncryptSQLRewriterParameterizedTest.java       |  4 +-
 .../scenario/MixSQLRewriterParameterizedTest.java  | 26 +++----
 .../ShardingSQLRewriterParameterizedTest.java      | 22 +++---
 73 files changed, 555 insertions(+), 418 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/LogicTablesMergedResult.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/LogicTablesMergedResult.java
index 7f9d0ae57bc..f843b31fc09 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/LogicTablesMergedResult.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/LogicTablesMergedResult.java
@@ -22,7 +22,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryMergedResult;
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryQueryResultRow;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
 
@@ -70,6 +70,6 @@ public class LogicTablesMergedResult extends MemoryMergedResult<ShardingRule> {
     }
     
     protected void setCellValue(final MemoryQueryResultRow memoryResultSetRow, final String logicTableName, final String actualTableName,
-                                final TableMetaData tableMetaData, final ShardingRule shardingRule) {
+                                final ShardingSphereTable tableMetaData, final ShardingRule shardingRule) {
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResult.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResult.java
index 6e50c5c07c8..8dae22c54bd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResult.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResult.java
@@ -21,10 +21,10 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryQueryResultRow;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereConstraint;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.IndexMetaDataUtil;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ConstraintMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
 
@@ -45,13 +45,13 @@ public final class ShowCreateTableMergedResult extends LogicTablesMergedResult {
     
     @Override
     protected void setCellValue(final MemoryQueryResultRow memoryResultSetRow, final String logicTableName, final String actualTableName,
-                                final TableMetaData tableMetaData, final ShardingRule shardingRule) {
+                                final ShardingSphereTable tableMetaData, final ShardingRule shardingRule) {
         memoryResultSetRow.setCell(2, memoryResultSetRow.getCell(2).toString().replaceFirst(actualTableName, logicTableName));
         for (String each : tableMetaData.getIndexes().keySet()) {
             String actualIndexName = IndexMetaDataUtil.getActualIndexName(each, actualTableName);
             memoryResultSetRow.setCell(2, memoryResultSetRow.getCell(2).toString().replace(actualIndexName, each));
         }
-        for (Entry<String, ConstraintMetaData> each : tableMetaData.getConstrains().entrySet()) {
+        for (Entry<String, ShardingSphereConstraint> each : tableMetaData.getConstrains().entrySet()) {
             String actualIndexName = IndexMetaDataUtil.getActualIndexName(each.getKey(), actualTableName);
             memoryResultSetRow.setCell(2, memoryResultSetRow.getCell(2).toString().replace(actualIndexName, each.getKey()));
             Optional<TableRule> tableRule = shardingRule.findTableRule(each.getValue().getReferencedTableName());
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
index 4751b1329ef..e3df9dbc915 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
@@ -27,9 +27,9 @@ import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementConte
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryMergedResult;
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryQueryResultRow;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.merge.dql.groupby.aggregation.AggregationUnit;
 import org.apache.shardingsphere.sharding.merge.dql.groupby.aggregation.AggregationUnitFactory;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -131,8 +131,8 @@ public final class GroupByMemoryMergedResult extends MemoryMergedResult<Sharding
                                                     final SelectStatementContext selectStatementContext, final ShardingSphereSchema schema, final int columnIndex) throws SQLException {
         for (SimpleTableSegment each : selectStatementContext.getAllTables()) {
             String tableName = each.getTableName().getIdentifier().getValue();
-            TableMetaData tableMetaData = schema.get(tableName);
-            Map<String, ColumnMetaData> columns = tableMetaData.getColumns();
+            ShardingSphereTable tableMetaData = schema.get(tableName);
+            Map<String, ShardingSphereColumn> columns = tableMetaData.getColumns();
             String columnName = queryResult.getMetaData().getColumnName(columnIndex);
             if (columns.containsKey(columnName)) {
                 return columns.get(columnName).isCaseSensitive();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java
index 7b6724d021c..458acb4f600 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java
@@ -19,16 +19,16 @@ package org.apache.shardingsphere.sharding.merge.dql.orderby;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByItem;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
-import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -70,8 +70,8 @@ public final class OrderByValue implements Comparable<OrderByValue> {
                                                           final ShardingSphereSchema schema, final OrderByItem eachOrderByItem) throws SQLException {
         for (SimpleTableSegment eachSimpleTableSegment : selectStatementContext.getAllTables()) {
             String tableName = eachSimpleTableSegment.getTableName().getIdentifier().getValue();
-            TableMetaData tableMetaData = schema.get(tableName);
-            Map<String, ColumnMetaData> columns = tableMetaData.getColumns();
+            ShardingSphereTable tableMetaData = schema.get(tableName);
+            Map<String, ShardingSphereColumn> columns = tableMetaData.getColumns();
             OrderByItemSegment orderByItemSegment = eachOrderByItem.getSegment();
             if (orderByItemSegment instanceof ColumnOrderByItemSegment) {
                 String columnName = ((ColumnOrderByItemSegment) orderByItemSegment).getColumn().getIdentifier().getValue();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java
index e384c041216..6b8a86dd95d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.sharding.merge.dal.show;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereConstraint;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ConstraintMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -61,10 +61,10 @@ public final class ShowCreateTableMergedResultTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(2, 1);
+        Map<String, ShardingSphereTable> tableMetaDataMap = new HashMap<>(2, 1);
         tableMetaDataMap.put("t_order",
-                new TableMetaData("t_order", Collections.emptyList(), Collections.emptyList(), Collections.singleton(new ConstraintMetaData("t_order_foreign_key", "t_user"))));
-        tableMetaDataMap.put("t_user", new TableMetaData("t_user", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+                new ShardingSphereTable("t_order", Collections.emptyList(), Collections.emptyList(), Collections.singleton(new ShardingSphereConstraint("t_order_foreign_key", "t_user"))));
+        tableMetaDataMap.put("t_user", new ShardingSphereTable("t_user", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
         return new ShardingSphereSchema(tableMetaDataMap);
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTableStatusMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTableStatusMergedResultTest.java
index 9c2ee3ddb21..f5af6c13863 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTableStatusMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTableStatusMergedResultTest.java
@@ -21,7 +21,7 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -54,7 +54,7 @@ public final class ShowTableStatusMergedResultTest {
     @Before
     public void setUp() {
         shardingRule = buildShardingRule();
-        schema = new ShardingSphereSchema(Collections.singletonMap("table", new TableMetaData("table", Collections.emptyList(), Collections.emptyList(), Collections.emptyList())));
+        schema = new ShardingSphereSchema(Collections.singletonMap("table", new ShardingSphereTable("table", Collections.emptyList(), Collections.emptyList(), Collections.emptyList())));
     }
     
     private ShardingRule buildShardingRule() {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java
index 0570cc4f91f..0e1a39c2493 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sharding.merge.dal.show;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -45,7 +45,7 @@ public final class ShowTablesMergedResultTest {
     @Before
     public void setUp() {
         shardingRule = createShardingRule();
-        schema = new ShardingSphereSchema(Collections.singletonMap("table", new TableMetaData("table", Collections.emptyList(), Collections.emptyList(), Collections.emptyList())));
+        schema = new ShardingSphereSchema(Collections.singletonMap("table", new ShardingSphereTable("table", Collections.emptyList(), Collections.emptyList(), Collections.emptyList())));
     }
     
     private ShardingRule createShardingRule() {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java
index 1a147b7e1b4..06b9ce908b3 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java
@@ -26,9 +26,9 @@ import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.merge.dql.groupby.GroupByMemoryMergedResult;
 import org.apache.shardingsphere.sharding.merge.dql.groupby.GroupByStreamMergedResult;
 import org.apache.shardingsphere.sharding.merge.dql.iterator.IteratorStreamMergedResult;
@@ -501,10 +501,10 @@ public final class ShardingDQLResultMergerTest {
     }
     
     private ShardingSphereDatabase createDatabase() {
-        ColumnMetaData columnMetaData1 = new ColumnMetaData("col1", 0, false, false, false);
-        ColumnMetaData columnMetaData2 = new ColumnMetaData("col2", 0, false, false, false);
-        ColumnMetaData columnMetaData3 = new ColumnMetaData("col3", 0, false, false, false);
-        TableMetaData tableMetaData = new TableMetaData("tbl", Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereColumn columnMetaData1 = new ShardingSphereColumn("col1", 0, false, false, false);
+        ShardingSphereColumn columnMetaData2 = new ShardingSphereColumn("col2", 0, false, false, false);
+        ShardingSphereColumn columnMetaData3 = new ShardingSphereColumn("col3", 0, false, false, false);
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("tbl", Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList(), Collections.emptyList());
         ShardingSphereSchema schema = new ShardingSphereSchema(Collections.singletonMap("tbl", tableMetaData));
         return new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, DatabaseTypeFactory.getInstance("MySQL"), mock(ShardingSphereResource.class),
                 mock(ShardingSphereRuleMetaData.class), Collections.singletonMap(DefaultDatabase.LOGIC_NAME, schema));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java
index b7dbc565b26..d689d919ea9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.merge.dql.ShardingDQLResultMerger;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.AggregationType;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.OrderDirection;
@@ -189,7 +189,7 @@ public final class GroupByMemoryMergedResultTest {
         when(queryResult.getMetaData().getColumnLabel(1)).thenReturn("order_id");
         when(queryResult.getMetaData().getColumnLabel(2)).thenReturn("content");
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
-        TableMetaData tableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
         when(schema.get("t_order")).thenReturn(tableMetaData);
         when(tableMetaData.getColumns()).thenReturn(Collections.emptyMap());
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java
index 29c8fdadc3d..cf7b3e76161 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java
@@ -25,9 +25,9 @@ import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.merge.dql.ShardingDQLResultMerger;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.AggregationType;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.OrderDirection;
@@ -172,10 +172,10 @@ public final class GroupByStreamMergedResultTest {
     }
     
     private ShardingSphereDatabase createDatabaaseMetaData() {
-        ColumnMetaData columnMetaData1 = new ColumnMetaData("col1", 0, false, false, false);
-        ColumnMetaData columnMetaData2 = new ColumnMetaData("col2", 0, false, false, false);
-        ColumnMetaData columnMetaData3 = new ColumnMetaData("col3", 0, false, false, false);
-        TableMetaData tableMetaData = new TableMetaData("tbl", Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereColumn columnMetaData1 = new ShardingSphereColumn("col1", 0, false, false, false);
+        ShardingSphereColumn columnMetaData2 = new ShardingSphereColumn("col2", 0, false, false, false);
+        ShardingSphereColumn columnMetaData3 = new ShardingSphereColumn("col3", 0, false, false, false);
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("tbl", Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList(), Collections.emptyList());
         ShardingSphereSchema schema = new ShardingSphereSchema(Collections.singletonMap("tbl", tableMetaData));
         return new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, DatabaseTypeFactory.getInstance("MySQL"),
                 mock(ShardingSphereResource.class), mock(ShardingSphereRuleMetaData.class), Collections.singletonMap(DefaultDatabase.LOGIC_NAME, schema));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java
index 95b1e2b3ad0..ba719605d49 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java
@@ -26,9 +26,9 @@ import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.merge.dql.ShardingDQLResultMerger;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.OrderDirection;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
@@ -195,9 +195,9 @@ public final class OrderByStreamMergedResultTest {
     }
     
     private ShardingSphereDatabase createDatabase() {
-        ColumnMetaData columnMetaData1 = new ColumnMetaData("col1", 0, false, false, true);
-        ColumnMetaData columnMetaData2 = new ColumnMetaData("col2", 0, false, false, false);
-        TableMetaData tableMetaData = new TableMetaData("tbl", Arrays.asList(columnMetaData1, columnMetaData2), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereColumn columnMetaData1 = new ShardingSphereColumn("col1", 0, false, false, true);
+        ShardingSphereColumn columnMetaData2 = new ShardingSphereColumn("col2", 0, false, false, false);
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("tbl", Arrays.asList(columnMetaData1, columnMetaData2), Collections.emptyList(), Collections.emptyList());
         ShardingSphereSchema schema = new ShardingSphereSchema(Collections.singletonMap("tbl", tableMetaData));
         return new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 DatabaseTypeFactory.getInstance("MySQL"), mock(ShardingSphereResource.class), mock(ShardingSphereRuleMetaData.class), Collections.singletonMap(DefaultDatabase.LOGIC_NAME, schema));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValueTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValueTest.java
index b62a8654479..27380ffab13 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValueTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValueTest.java
@@ -17,13 +17,13 @@
 
 package org.apache.shardingsphere.sharding.merge.dql.orderby;
 
+import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByItem;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByItem;
-import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.OrderDirection;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
@@ -141,7 +141,7 @@ public final class OrderByValueTest {
         SelectStatementContext selectStatementContext = new SelectStatementContext(Collections.singletonMap(DefaultDatabase.LOGIC_NAME, database),
                 Collections.emptyList(), selectStatement, DefaultDatabase.LOGIC_NAME);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
-        when(schema.get("table")).thenReturn(new TableMetaData());
+        when(schema.get("table")).thenReturn(new ShardingSphereTable());
         QueryResult queryResult1 = createQueryResult("1", "2");
         OrderByValue orderByValue1 = new OrderByValue(queryResult1, Arrays.asList(
                 createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.DESC, OrderDirection.ASC)),
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
index dcd2b46b2e9..b58479f8b7d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
@@ -27,9 +27,9 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
@@ -74,18 +74,18 @@ public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
     }
     
     private Map<String, ShardingSphereSchema> buildSchemas() {
-        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
-        tableMetaDataMap.put("t_order", new TableMetaData("t_order", Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, true, false, false),
-                new ColumnMetaData("user_id", Types.INTEGER, false, false, false),
-                new ColumnMetaData("status", Types.INTEGER, false, false, false)), Collections.emptyList(), Collections.emptyList()));
-        tableMetaDataMap.put("t_order_item", new TableMetaData("t_order_item", Arrays.asList(new ColumnMetaData("item_id", Types.INTEGER, true, false, false),
-                new ColumnMetaData("order_id", Types.INTEGER, false, false, false),
-                new ColumnMetaData("user_id", Types.INTEGER, false, false, false),
-                new ColumnMetaData("status", Types.VARCHAR, false, false, false),
-                new ColumnMetaData("c_date", Types.TIMESTAMP, false, false, false)), Collections.emptyList(), Collections.emptyList()));
-        tableMetaDataMap.put("t_other", new TableMetaData("t_other", Collections.singletonList(
-                new ColumnMetaData("order_id", Types.INTEGER, true, false, false)), Collections.emptyList(), Collections.emptyList()));
-        tableMetaDataMap.put("t_category", new TableMetaData("t_category", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        Map<String, ShardingSphereTable> tableMetaDataMap = new HashMap<>(3, 1);
+        tableMetaDataMap.put("t_order", new ShardingSphereTable("t_order", Arrays.asList(new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("user_id", Types.INTEGER, false, false, false),
+                new ShardingSphereColumn("status", Types.INTEGER, false, false, false)), Collections.emptyList(), Collections.emptyList()));
+        tableMetaDataMap.put("t_order_item", new ShardingSphereTable("t_order_item", Arrays.asList(new ShardingSphereColumn("item_id", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("order_id", Types.INTEGER, false, false, false),
+                new ShardingSphereColumn("user_id", Types.INTEGER, false, false, false),
+                new ShardingSphereColumn("status", Types.VARCHAR, false, false, false),
+                new ShardingSphereColumn("c_date", Types.TIMESTAMP, false, false, false)), Collections.emptyList(), Collections.emptyList()));
+        tableMetaDataMap.put("t_other", new ShardingSphereTable("t_other", Collections.singletonList(
+                new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false)), Collections.emptyList(), Collections.emptyList()));
+        tableMetaDataMap.put("t_category", new ShardingSphereTable("t_category", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
         return Collections.singletonMap(DefaultDatabase.LOGIC_NAME, new ShardingSphereSchema(tableMetaDataMap));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterIndexStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterIndexStatementValidatorTest.java
index bd51648f16c..59a57627305 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterIndexStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterIndexStatementValidatorTest.java
@@ -20,8 +20,8 @@ package org.apache.shardingsphere.sharding.route.engine.validator.ddl;
 import org.apache.shardingsphere.infra.binder.statement.ddl.AlterIndexStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterIndexStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexNameSegment;
@@ -55,8 +55,8 @@ public final class ShardingAlterIndexStatementValidatorTest {
         PostgreSQLAlterIndexStatement sqlStatement = new PostgreSQLAlterIndexStatement();
         sqlStatement.setIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         sqlStatement.setRenameIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
@@ -70,8 +70,8 @@ public final class ShardingAlterIndexStatementValidatorTest {
         PostgreSQLAlterIndexStatement sqlStatement = new PostgreSQLAlterIndexStatement();
         sqlStatement.setIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         sqlStatement.setRenameIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
@@ -84,8 +84,8 @@ public final class ShardingAlterIndexStatementValidatorTest {
         PostgreSQLAlterIndexStatement sqlStatement = new PostgreSQLAlterIndexStatement();
         sqlStatement.setIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         sqlStatement.setRenameIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateIndexStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateIndexStatementValidatorTest.java
index 252484e0e15..09e6d3fab20 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateIndexStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateIndexStatementValidatorTest.java
@@ -20,8 +20,8 @@ package org.apache.shardingsphere.sharding.route.engine.validator.ddl;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateIndexStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateIndexStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -60,9 +60,9 @@ public final class ShardingCreateIndexStatementValidatorTest {
         sqlStatement.setTable(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("t_order"))));
         sqlStatement.setIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         when(database.getSchemas().get("public").containsTable("t_order")).thenReturn(true);
-        TableMetaData tableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(indexes.containsKey("t_order_index")).thenReturn(false);
         new ShardingCreateIndexStatementValidator().preValidate(shardingRule, new CreateIndexStatementContext(sqlStatement), Collections.emptyList(), database);
@@ -83,9 +83,9 @@ public final class ShardingCreateIndexStatementValidatorTest {
         sqlStatement.setTable(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("t_order"))));
         sqlStatement.setIndex(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         when(database.getSchemas().get("public").containsTable("t_order")).thenReturn(true);
-        TableMetaData tableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(indexes.containsKey("t_order_index")).thenReturn(true);
         new ShardingCreateIndexStatementValidator().preValidate(shardingRule, new CreateIndexStatementContext(sqlStatement), Collections.emptyList(), database);
@@ -98,9 +98,9 @@ public final class ShardingCreateIndexStatementValidatorTest {
         sqlStatement.setColumns(Collections.singletonList(new ColumnSegment(0, 0, new IdentifierValue("content"))));
         sqlStatement.setGeneratedIndexStartIndex(10);
         when(database.getSchemas().get("public").containsTable("t_order")).thenReturn(true);
-        TableMetaData tableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         new ShardingCreateIndexStatementValidator().preValidate(shardingRule, new CreateIndexStatementContext(sqlStatement), Collections.emptyList(), database);
     }
@@ -122,9 +122,9 @@ public final class ShardingCreateIndexStatementValidatorTest {
         sqlStatement.setColumns(Collections.singletonList(new ColumnSegment(0, 0, new IdentifierValue("content"))));
         sqlStatement.setGeneratedIndexStartIndex(10);
         when(database.getSchemas().get("public").containsTable("t_order")).thenReturn(true);
-        TableMetaData tableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(indexes.containsKey("content_idx")).thenReturn(true);
         new ShardingCreateIndexStatementValidator().preValidate(shardingRule, new CreateIndexStatementContext(sqlStatement), Collections.emptyList(), database);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDropIndexStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDropIndexStatementValidatorTest.java
index 959575f1279..5ad7ee81dd5 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDropIndexStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDropIndexStatementValidatorTest.java
@@ -21,8 +21,8 @@ import org.apache.shardingsphere.infra.binder.statement.ddl.DropIndexStatementCo
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
@@ -66,8 +66,8 @@ public final class ShardingDropIndexStatementValidatorTest {
         PostgreSQLDropIndexStatement sqlStatement = new PostgreSQLDropIndexStatement();
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
@@ -81,8 +81,8 @@ public final class ShardingDropIndexStatementValidatorTest {
         PostgreSQLDropIndexStatement sqlStatement = new PostgreSQLDropIndexStatement();
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
@@ -95,8 +95,8 @@ public final class ShardingDropIndexStatementValidatorTest {
         PostgreSQLDropIndexStatement sqlStatement = new PostgreSQLDropIndexStatement();
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
@@ -116,8 +116,8 @@ public final class ShardingDropIndexStatementValidatorTest {
         PostgreSQLDropIndexStatement sqlStatement = new PostgreSQLDropIndexStatement();
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index"))));
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_order_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(database.getSchemas().get("public").get("t_order")).thenReturn(tableMetaData);
@@ -136,8 +136,8 @@ public final class ShardingDropIndexStatementValidatorTest {
         PostgreSQLDropIndexStatement sqlStatement = new PostgreSQLDropIndexStatement();
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_config_index"))));
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_config_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_config"));
         when(database.getSchemas().get("public").get("t_config")).thenReturn(tableMetaData);
@@ -157,8 +157,8 @@ public final class ShardingDropIndexStatementValidatorTest {
         PostgreSQLDropIndexStatement sqlStatement = new PostgreSQLDropIndexStatement();
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_config_index"))));
         sqlStatement.getIndexes().add(new IndexSegment(0, 0, new IndexNameSegment(0, 0, new IdentifierValue("t_config_index_new"))));
-        TableMetaData tableMetaData = mock(TableMetaData.class);
-        Map<String, IndexMetaData> indexes = mock(HashMap.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
+        Map<String, ShardingSphereIndex> indexes = mock(HashMap.class);
         when(tableMetaData.getIndexes()).thenReturn(indexes);
         when(database.getSchemas().get("public").getAllTableNames()).thenReturn(Collections.singletonList("t_config"));
         when(database.getSchemas().get("public").get("t_config")).thenReturn(tableMetaData);
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngine.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngine.java
index 409e4a91056..1e3b0aa8f4c 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngine.java
@@ -19,8 +19,8 @@ package org.apache.shardingsphere.infra.binder.segment.insert.keygen.engine;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.SetAssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
@@ -63,7 +63,7 @@ public final class GeneratedKeyContextEngine {
         if (!schema.containsTable(tableName)) {
             return Optional.empty();
         }
-        for (Entry<String, ColumnMetaData> entry : schema.get(tableName).getColumns().entrySet()) {
+        for (Entry<String, ShardingSphereColumn> entry : schema.get(tableName).getColumns().entrySet()) {
             if (entry.getValue().isGenerated()) {
                 return Optional.of(entry.getKey());
             }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngineTest.java b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngineTest.java
index 18211e62d40..b3f6806a72f 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/insert/keygen/engine/GeneratedKeyContextEngineTest.java
@@ -18,9 +18,9 @@
 package org.apache.shardingsphere.infra.binder.segment.insert.keygen.engine;
 
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.InsertValuesSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.InsertColumnsSegment;
@@ -61,8 +61,8 @@ public final class GeneratedKeyContextEngineTest {
     
     @Before
     public void setUp() {
-        TableMetaData tableMetaData = new TableMetaData(
-                "tbl", Collections.singletonList(new ColumnMetaData("id", Types.INTEGER, true, true, false)), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereTable tableMetaData = new ShardingSphereTable(
+                "tbl", Collections.singletonList(new ShardingSphereColumn("id", Types.INTEGER, true, true, false)), Collections.emptyList(), Collections.emptyList());
         schema = new ShardingSphereSchema(Collections.singletonMap("tbl", tableMetaData));
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContextTest.java b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContextTest.java
index 72140bdcf0b..efaf2eca761 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContextTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContextTest.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.infra.binder.segment.table;
 
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ColumnProjection;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.AliasSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.OwnerSegment;
@@ -109,9 +109,9 @@ public final class TablesContextTest {
     public void assertFindTableNameWhenColumnSegmentOwnerAbsentAndSchemaMetaDataContainsColumnInUpperCase() {
         SimpleTableSegment tableSegment1 = createTableSegment("TABLE_1", "TBL_1");
         SimpleTableSegment tableSegment2 = createTableSegment("TABLE_2", "TBL_2");
-        TableMetaData tableMetaData = new TableMetaData("TABLE_1",
-                Collections.singletonList(new ColumnMetaData("COL", 0, false, false, true)), Collections.emptyList(), Collections.emptyList());
-        ShardingSphereSchema schema = new ShardingSphereSchema(Stream.of(tableMetaData).collect(Collectors.toMap(TableMetaData::getName, value -> value)));
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("TABLE_1",
+                Collections.singletonList(new ShardingSphereColumn("COL", 0, false, false, true)), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereSchema schema = new ShardingSphereSchema(Stream.of(tableMetaData).collect(Collectors.toMap(ShardingSphereTable::getName, value -> value)));
         ColumnSegment columnSegment = createColumnSegment(null, "COL");
         Map<String, String> actual = new TablesContext(Arrays.asList(tableSegment1, tableSegment2),
                 DatabaseTypeEngine.getDatabaseType("MySQL")).findTableNamesByColumnSegment(Collections.singletonList(columnSegment), schema);
@@ -167,8 +167,9 @@ public final class TablesContextTest {
     public void assertFindTableNameWhenColumnProjectionOwnerAbsentAndSchemaMetaDataContainsColumnInUpperCase() {
         SimpleTableSegment tableSegment1 = createTableSegment("TABLE_1", "TBL_1");
         SimpleTableSegment tableSegment2 = createTableSegment("TABLE_2", "TBL_2");
-        TableMetaData tableMetaData = new TableMetaData("TABLE_1", Collections.singletonList(new ColumnMetaData("COL", 0, false, false, true)), Collections.emptyList(), Collections.emptyList());
-        ShardingSphereSchema schema = new ShardingSphereSchema(Stream.of(tableMetaData).collect(Collectors.toMap(TableMetaData::getName, value -> value)));
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("TABLE_1", Collections.singletonList(
+                new ShardingSphereColumn("COL", 0, false, false, true)), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereSchema schema = new ShardingSphereSchema(Stream.of(tableMetaData).collect(Collectors.toMap(ShardingSphereTable::getName, value -> value)));
         ColumnProjection columnProjection = new ColumnProjection(null, "COL", "CL");
         Map<String, String> actual = new TablesContext(Arrays.asList(tableSegment1, tableSegment2), DatabaseTypeEngine.getDatabaseType("MySQL"))
                 .findTableNamesByColumnProjection(Collections.singletonList(columnProjection), schema);
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java
index bca2aa5ff06..2e62c4fe2af 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java
@@ -19,11 +19,18 @@ package org.apache.shardingsphere.infra.metadata.database.schema.builder;
 
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereConstraint;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.spi.RuleBasedSchemaMetaDataDecorator;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.spi.RuleBasedSchemaMetaDataDecoratorFactory;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.SchemaMetaDataLoaderEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.SchemaMetaDataLoaderMaterials;
+import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ConstraintMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.SchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.SchemaMetaDataUtil;
@@ -34,6 +41,7 @@ import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
@@ -65,18 +73,6 @@ public final class GenericSchemaBuilder {
      * @throws SQLException SQL exception
      */
     public static Map<String, ShardingSphereSchema> build(final Collection<String> tableNames, final GenericSchemaBuilderMaterials materials) throws SQLException {
-        Map<String, SchemaMetaData> schemaMetaDataMap = loadSchemaMetaDataMap(tableNames, materials);
-        if (schemaMetaDataMap.isEmpty()) {
-            return Collections.singletonMap(materials.getDefaultSchemaName(), new ShardingSphereSchema());
-        }
-        Map<String, ShardingSphereSchema> result = new ConcurrentHashMap<>();
-        for (Entry<String, SchemaMetaData> entry : schemaMetaDataMap.entrySet()) {
-            result.put(entry.getKey().toLowerCase(), new ShardingSphereSchema(entry.getValue().getTables()));
-        }
-        return result;
-    }
-    
-    private static Map<String, SchemaMetaData> loadSchemaMetaDataMap(final Collection<String> tableNames, final GenericSchemaBuilderMaterials materials) throws SQLException {
         Map<String, SchemaMetaData> result = loadSchemas(tableNames, materials);
         if (!materials.getProtocolType().equals(materials.getStorageType())) {
             result = translate(result, materials);
@@ -107,7 +103,7 @@ public final class GenericSchemaBuilder {
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private static Map<String, SchemaMetaData> decorate(final Map<String, SchemaMetaData> schemaMetaDataMap, final GenericSchemaBuilderMaterials materials) {
+    private static Map<String, ShardingSphereSchema> decorate(final Map<String, SchemaMetaData> schemaMetaDataMap, final GenericSchemaBuilderMaterials materials) {
         Map<String, SchemaMetaData> result = new LinkedHashMap<>(schemaMetaDataMap);
         for (Entry<ShardingSphereRule, RuleBasedSchemaMetaDataDecorator> entry : RuleBasedSchemaMetaDataDecoratorFactory.getInstances(materials.getRules()).entrySet()) {
             if (!(entry.getKey() instanceof TableContainedRule)) {
@@ -115,6 +111,53 @@ public final class GenericSchemaBuilder {
             }
             result.putAll(entry.getValue().decorate(result, (TableContainedRule) entry.getKey(), materials));
         }
+        return convertToSchemaMap(result, materials);
+    }
+    
+    private static Map<String, ShardingSphereSchema> convertToSchemaMap(final Map<String, SchemaMetaData> schemaMetaDataMap, final GenericSchemaBuilderMaterials materials) {
+        if (schemaMetaDataMap.isEmpty()) {
+            return Collections.singletonMap(materials.getDefaultSchemaName(), new ShardingSphereSchema());
+        }
+        Map<String, ShardingSphereSchema> result = new ConcurrentHashMap<>(schemaMetaDataMap.size(), 1);
+        for (Entry<String, SchemaMetaData> entry : schemaMetaDataMap.entrySet()) {
+            Map<String, ShardingSphereTable> tables = convertToTableMap(entry.getValue().getTables());
+            result.put(entry.getKey().toLowerCase(), new ShardingSphereSchema(tables));
+        }
+        return result;
+    }
+    
+    private static Map<String, ShardingSphereTable> convertToTableMap(final Map<String, TableMetaData> tableMetaDataMap) {
+        Map<String, ShardingSphereTable> result = new LinkedHashMap<>(tableMetaDataMap.size(), 1);
+        for (Entry<String, TableMetaData> entry : tableMetaDataMap.entrySet()) {
+            Collection<ShardingSphereColumn> columns = convertToColumns(entry.getValue().getColumns().values());
+            Collection<ShardingSphereIndex> indexes = convertToIndexes(entry.getValue().getIndexes().values());
+            Collection<ShardingSphereConstraint> constraints = convertToConstraints(entry.getValue().getConstrains().values());
+            result.put(entry.getKey(), new ShardingSphereTable(entry.getValue().getName(), columns, indexes, constraints));
+        }
+        return result;
+    }
+    
+    private static Collection<ShardingSphereColumn> convertToColumns(final Collection<ColumnMetaData> columnMetaDataList) {
+        Collection<ShardingSphereColumn> result = new LinkedList<>();
+        for (ColumnMetaData each : columnMetaDataList) {
+            result.add(new ShardingSphereColumn(each.getName(), each.getDataType(), each.isPrimaryKey(), each.isGenerated(), each.isCaseSensitive()));
+        }
+        return result;
+    }
+    
+    private static Collection<ShardingSphereIndex> convertToIndexes(final Collection<IndexMetaData> indexMetaDataList) {
+        Collection<ShardingSphereIndex> result = new LinkedList<>();
+        for (IndexMetaData each : indexMetaDataList) {
+            result.add(new ShardingSphereIndex(each.getName()));
+        }
+        return result;
+    }
+    
+    private static Collection<ShardingSphereConstraint> convertToConstraints(final Collection<ConstraintMetaData> constraintMetaDataList) {
+        Collection<ShardingSphereConstraint> result = new LinkedList<>();
+        for (ConstraintMetaData each : constraintMetaDataList) {
+            result.add(new ShardingSphereConstraint(each.getName(), each.getReferencedTableName()));
+        }
         return result;
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/SystemSchemaBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/SystemSchemaBuilder.java
index 55fededf773..9c1df34e825 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/SystemSchemaBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/SystemSchemaBuilder.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlTableMetaData;
 import org.apache.shardingsphere.infra.yaml.schema.swapper.TableMetaDataYamlSwapper;
 import org.yaml.snakeyaml.Yaml;
@@ -72,7 +72,7 @@ public final class SystemSchemaBuilder {
     }
     
     private static ShardingSphereSchema createSchema(final Collection<InputStream> schemaStreams, final TableMetaDataYamlSwapper swapper) {
-        Map<String, TableMetaData> tables = new LinkedHashMap<>(schemaStreams.size(), 1);
+        Map<String, ShardingSphereTable> tables = new LinkedHashMap<>(schemaStreams.size(), 1);
         for (InputStream each : schemaStreams) {
             YamlTableMetaData metaData = new Yaml().loadAs(each, YamlTableMetaData.class);
             tables.put(metaData.getName(), swapper.swapToObject(metaData));
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereColumn.java
similarity index 63%
copy from shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
copy to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereColumn.java
index bded86662c0..f369edec575 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereColumn.java
@@ -15,25 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.schema;
+package org.apache.shardingsphere.infra.metadata.database.schema.decorator.model;
 
+import lombok.EqualsAndHashCode;
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
-import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceEvent;
+import lombok.ToString;
 
 /**
- * Schema changed event.
+ * ShardingSphere column.
  */
 @RequiredArgsConstructor
 @Getter
-public final class SchemaChangedEvent implements GovernanceEvent {
+@EqualsAndHashCode
+@ToString
+public final class ShardingSphereColumn {
     
-    private final String databaseName;
+    private final String name;
     
-    private final String schemaName;
+    private final int dataType;
     
-    private final TableMetaData changedTableMetaData;
+    private final boolean primaryKey;
     
-    private final String deletedTable;
+    private final boolean generated;
+    
+    private final boolean caseSensitive;
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereConstraint.java
similarity index 62%
copy from shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
copy to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereConstraint.java
index bded86662c0..ecd6dff7719 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereConstraint.java
@@ -15,25 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.schema;
+package org.apache.shardingsphere.infra.metadata.database.schema.decorator.model;
 
+import lombok.EqualsAndHashCode;
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
-import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceEvent;
+import lombok.ToString;
 
 /**
- * Schema changed event.
+ * ShardingSphere constraint.
  */
 @RequiredArgsConstructor
 @Getter
-public final class SchemaChangedEvent implements GovernanceEvent {
+@EqualsAndHashCode
+@ToString
+public final class ShardingSphereConstraint {
     
-    private final String databaseName;
+    private final String name;
     
-    private final String schemaName;
-    
-    private final TableMetaData changedTableMetaData;
-    
-    private final String deletedTable;
+    private final String referencedTableName;
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereIndex.java
similarity index 62%
copy from shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
copy to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereIndex.java
index bded86662c0..283e5541c60 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereIndex.java
@@ -15,25 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.schema;
+package org.apache.shardingsphere.infra.metadata.database.schema.decorator.model;
 
+import lombok.EqualsAndHashCode;
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
-import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceEvent;
+import lombok.ToString;
 
 /**
- * Schema changed event.
+ * ShardingSphere index.
  */
 @RequiredArgsConstructor
 @Getter
-public final class SchemaChangedEvent implements GovernanceEvent {
+@EqualsAndHashCode
+@ToString
+public final class ShardingSphereIndex {
     
-    private final String databaseName;
-    
-    private final String schemaName;
-    
-    private final TableMetaData changedTableMetaData;
-    
-    private final String deletedTable;
+    private final String name;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereSchema.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereSchema.java
index 6df33149606..49724e52023 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereSchema.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereSchema.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.infra.metadata.database.schema.decorator.model;
 
 import lombok.Getter;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -33,14 +32,14 @@ import java.util.concurrent.ConcurrentHashMap;
 @Getter
 public final class ShardingSphereSchema {
     
-    private final Map<String, TableMetaData> tables;
+    private final Map<String, ShardingSphereTable> tables;
     
     @SuppressWarnings("CollectionWithoutInitialCapacity")
     public ShardingSphereSchema() {
         tables = new ConcurrentHashMap<>();
     }
     
-    public ShardingSphereSchema(final Map<String, TableMetaData> tables) {
+    public ShardingSphereSchema(final Map<String, ShardingSphereTable> tables) {
         this.tables = new ConcurrentHashMap<>(tables.size(), 1);
         tables.forEach((key, value) -> this.tables.put(key.toLowerCase(), value));
     }
@@ -60,7 +59,7 @@ public final class ShardingSphereSchema {
      * @param tableName tableName table name
      * @return table meta data
      */
-    public TableMetaData get(final String tableName) {
+    public ShardingSphereTable get(final String tableName) {
         return tables.get(tableName.toLowerCase());
     }
     
@@ -70,7 +69,7 @@ public final class ShardingSphereSchema {
      * @param tableName table name
      * @param tableMetaData table meta data
      */
-    public void put(final String tableName, final TableMetaData tableMetaData) {
+    public void put(final String tableName, final ShardingSphereTable tableMetaData) {
         tables.put(tableName.toLowerCase(), tableMetaData);
     }
     
@@ -79,8 +78,8 @@ public final class ShardingSphereSchema {
      *
      * @param tableMetaDataMap table meta data map
      */
-    public void putAll(final Map<String, TableMetaData> tableMetaDataMap) {
-        for (Entry<String, TableMetaData> entry : tableMetaDataMap.entrySet()) {
+    public void putAll(final Map<String, ShardingSphereTable> tableMetaDataMap) {
+        for (Entry<String, ShardingSphereTable> entry : tableMetaDataMap.entrySet()) {
             put(entry.getKey(), entry.getValue());
         }
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereTable.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereTable.java
new file mode 100644
index 00000000000..678ef856562
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/model/ShardingSphereTable.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.metadata.database.schema.decorator.model;
+
+import lombok.EqualsAndHashCode;
+import lombok.Getter;
+import lombok.ToString;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * ShardingSphere table.
+ */
+@Getter
+@EqualsAndHashCode
+@ToString
+public final class ShardingSphereTable {
+    
+    private final String name;
+    
+    private final Map<String, ShardingSphereColumn> columns;
+    
+    private final Map<String, ShardingSphereIndex> indexes;
+    
+    private final Map<String, ShardingSphereConstraint> constrains;
+    
+    private final List<String> columnNames = new ArrayList<>();
+    
+    private final List<String> primaryKeyColumns = new ArrayList<>();
+    
+    public ShardingSphereTable() {
+        this("", Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+    }
+    
+    public ShardingSphereTable(final String name, final Collection<ShardingSphereColumn> columnList,
+                               final Collection<ShardingSphereIndex> indexList, final Collection<ShardingSphereConstraint> constraintList) {
+        this.name = name;
+        columns = getColumns(columnList);
+        indexes = getIndexes(indexList);
+        constrains = getConstrains(constraintList);
+    }
+    
+    private Map<String, ShardingSphereColumn> getColumns(final Collection<ShardingSphereColumn> columnList) {
+        Map<String, ShardingSphereColumn> result = new LinkedHashMap<>(columnList.size(), 1);
+        for (ShardingSphereColumn each : columnList) {
+            String lowerColumnName = each.getName().toLowerCase();
+            result.put(lowerColumnName, each);
+            columnNames.add(each.getName());
+            if (each.isPrimaryKey()) {
+                primaryKeyColumns.add(lowerColumnName);
+            }
+        }
+        return result;
+    }
+    
+    private Map<String, ShardingSphereIndex> getIndexes(final Collection<ShardingSphereIndex> indexList) {
+        Map<String, ShardingSphereIndex> result = new LinkedHashMap<>(indexList.size(), 1);
+        for (ShardingSphereIndex each : indexList) {
+            result.put(each.getName().toLowerCase(), each);
+        }
+        return result;
+    }
+    
+    private Map<String, ShardingSphereConstraint> getConstrains(final Collection<ShardingSphereConstraint> constraintList) {
+        Map<String, ShardingSphereConstraint> result = new LinkedHashMap<>(constraintList.size(), 1);
+        for (ShardingSphereConstraint each : constraintList) {
+            result.put(each.getName().toLowerCase(), each);
+        }
+        return result;
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/event/SchemaAlteredEvent.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/event/SchemaAlteredEvent.java
index dca791d994f..215d09ecc5f 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/event/SchemaAlteredEvent.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/event/SchemaAlteredEvent.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.database.schema.event;
 
 import lombok.Getter;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -33,7 +33,7 @@ public final class SchemaAlteredEvent {
     
     private final String schemaName;
     
-    private final Collection<TableMetaData> alteredTables = new ArrayList<>();
+    private final Collection<ShardingSphereTable> alteredTables = new ArrayList<>();
     
     private final Collection<String> droppedTables = new ArrayList<>();
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/SchemaYamlSwapper.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/SchemaYamlSwapper.java
index dcc67598125..670ba9a7b8c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/SchemaYamlSwapper.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/SchemaYamlSwapper.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.yaml.schema.swapper;
 
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.yaml.config.swapper.YamlConfigurationSwapper;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlSchema;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlTableMetaData;
@@ -53,11 +53,11 @@ public final class SchemaYamlSwapper implements YamlConfigurationSwapper<YamlSch
                 : schema.getTables().entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> swapTable(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new)));
     }
     
-    private TableMetaData swapTable(final YamlTableMetaData table) {
+    private ShardingSphereTable swapTable(final YamlTableMetaData table) {
         return new TableMetaDataYamlSwapper().swapToObject(table);
     }
     
-    private YamlTableMetaData swapYamlTable(final TableMetaData table) {
+    private YamlTableMetaData swapYamlTable(final ShardingSphereTable table) {
         return new TableMetaDataYamlSwapper().swapToYamlConfiguration(table);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/TableMetaDataYamlSwapper.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/TableMetaDataYamlSwapper.java
index cf6fa538262..cfe79c1bf89 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/TableMetaDataYamlSwapper.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/schema/swapper/TableMetaDataYamlSwapper.java
@@ -17,10 +17,10 @@
 
 package org.apache.shardingsphere.infra.yaml.schema.swapper;
 
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ConstraintMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereConstraint;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.yaml.config.swapper.YamlConfigurationSwapper;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlColumnMetaData;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlConstraintMetaData;
@@ -36,10 +36,10 @@ import java.util.stream.Collectors;
 /**
  * Table meta data YAML swapper.
  */
-public final class TableMetaDataYamlSwapper implements YamlConfigurationSwapper<YamlTableMetaData, TableMetaData> {
+public final class TableMetaDataYamlSwapper implements YamlConfigurationSwapper<YamlTableMetaData, ShardingSphereTable> {
     
     @Override
-    public YamlTableMetaData swapToYamlConfiguration(final TableMetaData table) {
+    public YamlTableMetaData swapToYamlConfiguration(final ShardingSphereTable table) {
         YamlTableMetaData result = new YamlTableMetaData();
         result.setColumns(swapYamlColumns(table.getColumns()));
         result.setIndexes(swapYamlIndexes(table.getIndexes()));
@@ -49,60 +49,60 @@ public final class TableMetaDataYamlSwapper implements YamlConfigurationSwapper<
     }
     
     @Override
-    public TableMetaData swapToObject(final YamlTableMetaData yamlConfig) {
-        return new TableMetaData(yamlConfig.getName(), swapColumns(yamlConfig.getColumns()), swapIndexes(yamlConfig.getIndexes()), swapConstraints(yamlConfig.getConstraints()));
+    public ShardingSphereTable swapToObject(final YamlTableMetaData yamlConfig) {
+        return new ShardingSphereTable(yamlConfig.getName(), swapColumns(yamlConfig.getColumns()), swapIndexes(yamlConfig.getIndexes()), swapConstraints(yamlConfig.getConstraints()));
     }
     
-    private Collection<ConstraintMetaData> swapConstraints(final Map<String, YamlConstraintMetaData> constraints) {
+    private Collection<ShardingSphereConstraint> swapConstraints(final Map<String, YamlConstraintMetaData> constraints) {
         return null == constraints ? Collections.emptyList() : constraints.values().stream().map(this::swapConstraint).collect(Collectors.toList());
     }
     
-    private ConstraintMetaData swapConstraint(final YamlConstraintMetaData constraint) {
-        return new ConstraintMetaData(constraint.getName(), constraint.getReferencedTableName());
+    private ShardingSphereConstraint swapConstraint(final YamlConstraintMetaData constraint) {
+        return new ShardingSphereConstraint(constraint.getName(), constraint.getReferencedTableName());
     }
     
-    private Collection<IndexMetaData> swapIndexes(final Map<String, YamlIndexMetaData> indexes) {
+    private Collection<ShardingSphereIndex> swapIndexes(final Map<String, YamlIndexMetaData> indexes) {
         return null == indexes ? Collections.emptyList() : indexes.values().stream().map(this::swapIndex).collect(Collectors.toList());
     }
     
-    private IndexMetaData swapIndex(final YamlIndexMetaData index) {
-        return new IndexMetaData(index.getName());
+    private ShardingSphereIndex swapIndex(final YamlIndexMetaData index) {
+        return new ShardingSphereIndex(index.getName());
     }
     
-    private Collection<ColumnMetaData> swapColumns(final Map<String, YamlColumnMetaData> indexes) {
+    private Collection<ShardingSphereColumn> swapColumns(final Map<String, YamlColumnMetaData> indexes) {
         return null == indexes ? Collections.emptyList() : indexes.values().stream().map(this::swapColumn).collect(Collectors.toList());
     }
     
-    private ColumnMetaData swapColumn(final YamlColumnMetaData column) {
-        return new ColumnMetaData(column.getName(), column.getDataType(), column.isPrimaryKey(), column.isGenerated(), column.isCaseSensitive());
+    private ShardingSphereColumn swapColumn(final YamlColumnMetaData column) {
+        return new ShardingSphereColumn(column.getName(), column.getDataType(), column.isPrimaryKey(), column.isGenerated(), column.isCaseSensitive());
     }
     
-    private Map<String, YamlConstraintMetaData> swapYamlConstraints(final Map<String, ConstraintMetaData> constrains) {
+    private Map<String, YamlConstraintMetaData> swapYamlConstraints(final Map<String, ShardingSphereConstraint> constrains) {
         return constrains.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> swapYamlConstraint(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
     
-    private YamlConstraintMetaData swapYamlConstraint(final ConstraintMetaData constraint) {
+    private YamlConstraintMetaData swapYamlConstraint(final ShardingSphereConstraint constraint) {
         YamlConstraintMetaData result = new YamlConstraintMetaData();
         result.setName(constraint.getName());
         result.setReferencedTableName(constraint.getReferencedTableName());
         return result;
     }
     
-    private Map<String, YamlIndexMetaData> swapYamlIndexes(final Map<String, IndexMetaData> indexes) {
+    private Map<String, YamlIndexMetaData> swapYamlIndexes(final Map<String, ShardingSphereIndex> indexes) {
         return indexes.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> swapYamlIndex(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
     
-    private YamlIndexMetaData swapYamlIndex(final IndexMetaData index) {
+    private YamlIndexMetaData swapYamlIndex(final ShardingSphereIndex index) {
         YamlIndexMetaData result = new YamlIndexMetaData();
         result.setName(index.getName());
         return result;
     }
     
-    private Map<String, YamlColumnMetaData> swapYamlColumns(final Map<String, ColumnMetaData> columns) {
+    private Map<String, YamlColumnMetaData> swapYamlColumns(final Map<String, ShardingSphereColumn> columns) {
         return columns.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> swapYamlColumn(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
     
-    private YamlColumnMetaData swapYamlColumn(final ColumnMetaData column) {
+    private YamlColumnMetaData swapYamlColumn(final ShardingSphereColumn column) {
         YamlColumnMetaData result = new YamlColumnMetaData();
         result.setName(column.getName());
         result.setCaseSensitive(column.isCaseSensitive());
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/ShardingSphereSchemaTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/ShardingSphereSchemaTest.java
index a292b13bbfd..4867579be04 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/ShardingSphereSchemaTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/ShardingSphereSchemaTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.shardingsphere.infra.metadata.database.schema;
 
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.junit.Test;
 
 import java.util.Collections;
@@ -35,50 +35,53 @@ public final class ShardingSphereSchemaTest {
     
     @Test
     public void assertGetAllTableNames() {
-        assertThat(new ShardingSphereSchema(Collections.singletonMap("tbl", mock(TableMetaData.class))).getAllTableNames(), is(new HashSet<>(Collections.singleton("tbl"))));
+        assertThat(new ShardingSphereSchema(Collections.singletonMap("tbl", mock(ShardingSphereTable.class))).getAllTableNames(), is(new HashSet<>(Collections.singleton("tbl"))));
     }
     
     @Test
     public void assertGet() {
-        TableMetaData tableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
         assertThat(new ShardingSphereSchema(Collections.singletonMap("tbl", tableMetaData)).get("tbl"), is(tableMetaData));
     }
     
     @Test
     public void assertPut() {
         ShardingSphereSchema actual = new ShardingSphereSchema(Collections.emptyMap());
-        TableMetaData tableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable tableMetaData = mock(ShardingSphereTable.class);
         actual.put("tbl", tableMetaData);
         assertThat(actual.get("tbl"), is(tableMetaData));
     }
     
     @Test
     public void assertRemove() {
-        ShardingSphereSchema actual = new ShardingSphereSchema(Collections.singletonMap("tbl", mock(TableMetaData.class)));
+        ShardingSphereSchema actual = new ShardingSphereSchema(Collections.singletonMap("tbl", mock(ShardingSphereTable.class)));
         actual.remove("tbl");
         assertNull(actual.get("tbl"));
     }
     
     @Test
     public void assertContainsTable() {
-        assertTrue(new ShardingSphereSchema(Collections.singletonMap("tbl", mock(TableMetaData.class))).containsTable("tbl"));
+        assertTrue(new ShardingSphereSchema(Collections.singletonMap("tbl", mock(ShardingSphereTable.class))).containsTable("tbl"));
     }
     
     @Test
     public void assertContainsColumn() {
-        TableMetaData tableMetaData = new TableMetaData("tbl", Collections.singletonList(new ColumnMetaData("col", 0, false, false, false)), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("tbl", Collections.singletonList(
+                new ShardingSphereColumn("col", 0, false, false, false)), Collections.emptyList(), Collections.emptyList());
         assertTrue(new ShardingSphereSchema(Collections.singletonMap("tbl", tableMetaData)).containsColumn("tbl", "col"));
     }
     
     @Test
     public void assertGetAllColumnNamesWhenContainsKey() {
-        TableMetaData tableMetaData = new TableMetaData("tbl", Collections.singletonList(new ColumnMetaData("col", 0, false, false, false)), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("tbl", Collections.singletonList(
+                new ShardingSphereColumn("col", 0, false, false, false)), Collections.emptyList(), Collections.emptyList());
         assertThat(new ShardingSphereSchema(Collections.singletonMap("tbl", tableMetaData)).getAllColumnNames("tbl"), is(Collections.singletonList("col")));
     }
     
     @Test
     public void assertGetAllColumnNamesWhenNotContainsKey() {
-        TableMetaData tableMetaData = new TableMetaData("tbl", Collections.singletonList(new ColumnMetaData("col", 0, false, false, false)), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("tbl", Collections.singletonList(
+                new ShardingSphereColumn("col", 0, false, false, false)), Collections.emptyList(), Collections.emptyList());
         assertThat(new ShardingSphereSchema(Collections.singletonMap("tbl1", tableMetaData)).getAllColumnNames("tbl2"), is(Collections.<String>emptyList()));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilderTest.java
index 32d020584e3..1144e42cebc 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilderTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.fixture.rule.CommonFixtureRule;
 import org.apache.shardingsphere.infra.metadata.database.schema.fixture.rule.DataNodeContainedFixtureRule;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.SchemaMetaDataLoaderEngine;
@@ -98,7 +99,7 @@ public final class GenericSchemaBuilderTest {
         schemaMetaDataLoaderEngine.close();
     }
     
-    private void assertTables(final Map<String, TableMetaData> actual) {
+    private void assertTables(final Map<String, ShardingSphereTable> actual) {
         assertThat(actual.size(), is(2));
         assertTrue(actual.get("data_node_routed_table1").getColumns().isEmpty());
         assertTrue(actual.get("data_node_routed_table2").getColumns().isEmpty());
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/IndexMetaDataUtilTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/IndexMetaDataUtilTest.java
index 9de05827153..c4e19456fe0 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/IndexMetaDataUtilTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/IndexMetaDataUtilTest.java
@@ -24,9 +24,9 @@ import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedTable;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexNameSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
@@ -90,8 +90,8 @@ public final class IndexMetaDataUtilTest {
     }
     
     private ShardingSphereDatabase buildDatabaseMetaData() {
-        TableMetaData tableMetaData = new TableMetaData(TABLE_NAME, Collections.emptyList(), Collections.singleton(new IndexMetaData(INDEX_NAME)), Collections.emptyList());
-        Map<String, TableMetaData> tables = Collections.singletonMap(TABLE_NAME, tableMetaData);
+        ShardingSphereTable tableMetaData = new ShardingSphereTable(TABLE_NAME, Collections.emptyList(), Collections.singleton(new ShardingSphereIndex(INDEX_NAME)), Collections.emptyList());
+        Map<String, ShardingSphereTable> tables = Collections.singletonMap(TABLE_NAME, tableMetaData);
         Map<String, ShardingSphereSchema> schemas = Collections.singletonMap(DefaultDatabase.LOGIC_NAME, new ShardingSphereSchema(tables));
         return new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), mock(ShardingSphereResource.class), mock(ShardingSphereRuleMetaData.class), schemas);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterIndexStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterIndexStatementSchemaRefresher.java
index 0bc7f0ea02c..7f671440913 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterIndexStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterIndexStatementSchemaRefresher.java
@@ -24,10 +24,10 @@ import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContext;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.AlterIndexStatementHandler;
@@ -55,11 +55,11 @@ public final class AlterIndexStatementSchemaRefresher implements MetaDataRefresh
         String indexName = sqlStatement.getIndex().get().getIndexName().getIdentifier().getValue();
         Optional<String> logicTableName = findLogicTableName(database.getSchemas().get(actualSchemaName), indexName);
         if (logicTableName.isPresent()) {
-            TableMetaData tableMetaData = database.getSchemas().get(actualSchemaName).get(logicTableName.get());
+            ShardingSphereTable tableMetaData = database.getSchemas().get(actualSchemaName).get(logicTableName.get());
             Preconditions.checkNotNull(tableMetaData, "Can not get the table '%s' metadata!", logicTableName.get());
             tableMetaData.getIndexes().remove(indexName);
             String renameIndexName = renameIndex.get().getIndexName().getIdentifier().getValue();
-            tableMetaData.getIndexes().put(renameIndexName, new IndexMetaData(renameIndexName));
+            tableMetaData.getIndexes().put(renameIndexName, new ShardingSphereIndex(renameIndexName));
             SchemaAlteredEvent event = new SchemaAlteredEvent(database.getName(), actualSchemaName);
             event.getAlteredTables().add(tableMetaData);
             ShardingSphereEventBus.getInstance().post(event);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterTableStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterTableStatementSchemaRefresher.java
index 0ba3f0d7a09..27a4ab48800 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterTableStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterTableStatementSchemaRefresher.java
@@ -24,11 +24,11 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.planner.Opti
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContextFactory;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
@@ -80,7 +80,7 @@ public final class AlterTableStatementSchemaRefresher implements MetaDataRefresh
         GenericSchemaBuilderMaterials materials = new GenericSchemaBuilderMaterials(database.getProtocolType(),
                 database.getResource().getDatabaseType(), database.getResource().getDataSources(), database.getRuleMetaData().getRules(), props, schemaName);
         Map<String, ShardingSphereSchema> schemaMap = GenericSchemaBuilder.build(Collections.singletonList(tableName), materials);
-        Optional<TableMetaData> actualTableMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(tableName));
+        Optional<ShardingSphereTable> actualTableMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(tableName));
         actualTableMetaData.ifPresent(optional -> {
             database.getSchemas().get(schemaName).put(tableName, optional);
             federationDatabaseMetaData.putTableMetadata(schemaName, optional);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterViewStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterViewStatementSchemaRefresher.java
index 10a2f396bb5..22b723925f1 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterViewStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/AlterViewStatementSchemaRefresher.java
@@ -24,11 +24,11 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.planner.Opti
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContextFactory;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -83,7 +83,7 @@ public final class AlterViewStatementSchemaRefresher implements MetaDataRefreshe
         GenericSchemaBuilderMaterials materials = new GenericSchemaBuilderMaterials(database.getProtocolType(),
                 database.getResource().getDatabaseType(), database.getResource().getDataSources(), database.getRuleMetaData().getRules(), props, schemaName);
         Map<String, ShardingSphereSchema> schemaMap = GenericSchemaBuilder.build(Collections.singletonList(viewName), materials);
-        Optional<TableMetaData> actualViewMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(viewName));
+        Optional<ShardingSphereTable> actualViewMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(viewName));
         actualViewMetaData.ifPresent(optional -> {
             database.getSchemas().get(schemaName).put(viewName, optional);
             federationDatabaseMetaData.putTableMetadata(schemaName, optional);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateIndexStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateIndexStatementSchemaRefresher.java
index bbab9850857..e86ce6efcd8 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateIndexStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateIndexStatementSchemaRefresher.java
@@ -24,8 +24,8 @@ import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContext;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.IndexMetaDataUtil;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateIndexStatement;
 
@@ -49,7 +49,7 @@ public final class CreateIndexStatementSchemaRefresher implements MetaDataRefres
             return;
         }
         String tableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
-        database.getSchemas().get(schemaName).get(tableName).getIndexes().put(indexName, new IndexMetaData(indexName));
+        database.getSchemas().get(schemaName).get(tableName).getIndexes().put(indexName, new ShardingSphereIndex(indexName));
         SchemaAlteredEvent event = new SchemaAlteredEvent(database.getName(), schemaName);
         event.getAlteredTables().add(database.getSchemas().get(schemaName).get(tableName));
         ShardingSphereEventBus.getInstance().post(event);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateTableStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateTableStatementSchemaRefresher.java
index 95095ed449d..c1aa43c2200 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateTableStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateTableStatementSchemaRefresher.java
@@ -24,11 +24,11 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.planner.Opti
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContextFactory;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
@@ -56,7 +56,7 @@ public final class CreateTableStatementSchemaRefresher implements MetaDataRefres
         GenericSchemaBuilderMaterials materials = new GenericSchemaBuilderMaterials(database.getProtocolType(),
                 database.getResource().getDatabaseType(), database.getResource().getDataSources(), database.getRuleMetaData().getRules(), props, schemaName);
         Map<String, ShardingSphereSchema> schemaMap = GenericSchemaBuilder.build(Collections.singletonList(tableName), materials);
-        Optional<TableMetaData> actualTableMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(tableName));
+        Optional<ShardingSphereTable> actualTableMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(tableName));
         actualTableMetaData.ifPresent(optional -> {
             database.getSchemas().get(schemaName).put(tableName, optional);
             federationDatabaseMetaData.putTableMetadata(schemaName, optional);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateViewStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateViewStatementSchemaRefresher.java
index 652a53b8c7d..2aec7956656 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateViewStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/CreateViewStatementSchemaRefresher.java
@@ -24,11 +24,11 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.planner.Opti
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContextFactory;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
@@ -56,7 +56,7 @@ public final class CreateViewStatementSchemaRefresher implements MetaDataRefresh
         GenericSchemaBuilderMaterials materials = new GenericSchemaBuilderMaterials(database.getProtocolType(),
                 database.getResource().getDatabaseType(), database.getResource().getDataSources(), database.getRuleMetaData().getRules(), props, schemaName);
         Map<String, ShardingSphereSchema> schemaMap = GenericSchemaBuilder.build(Collections.singletonList(viewName), materials);
-        Optional<TableMetaData> actualViewMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(viewName));
+        Optional<ShardingSphereTable> actualViewMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(viewName));
         actualViewMetaData.ifPresent(optional -> {
             database.getSchemas().get(schemaName).put(viewName, optional);
             federationDatabaseMetaData.putTableMetadata(schemaName, optional);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java
index 67065892da0..990e5055c50 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java
@@ -24,8 +24,8 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.planner.Opti
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedTable;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.IndexMetaDataUtil;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -54,7 +54,7 @@ public final class DropIndexStatementSchemaRefresher implements MetaDataRefreshe
             if (!logicTableName.isPresent()) {
                 continue;
             }
-            TableMetaData tableMetaData = database.getSchemas().get(actualSchemaName).get(logicTableName.get());
+            ShardingSphereTable tableMetaData = database.getSchemas().get(actualSchemaName).get(logicTableName.get());
             tableMetaData.getIndexes().remove(each.getIndexName().getIdentifier().getValue());
             post(database.getName(), actualSchemaName, tableMetaData);
         }
@@ -69,7 +69,7 @@ public final class DropIndexStatementSchemaRefresher implements MetaDataRefreshe
         return tableNames.isEmpty() ? Optional.empty() : Optional.of(tableNames.iterator().next().getTableName());
     }
     
-    private void post(final String databaseName, final String schemaName, final TableMetaData tableMetaData) {
+    private void post(final String databaseName, final String schemaName, final ShardingSphereTable tableMetaData) {
         SchemaAlteredEvent event = new SchemaAlteredEvent(databaseName, schemaName);
         event.getAlteredTables().add(tableMetaData);
         ShardingSphereEventBus.getInstance().post(event);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresher.java
index ab0b19f5cc2..cb0a0f00014 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresher.java
@@ -24,11 +24,11 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.planner.Opti
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContextFactory;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.table.RenameTableDefinitionSegment;
@@ -78,7 +78,7 @@ public final class RenameTableStatementSchemaRefresher implements MetaDataRefres
         GenericSchemaBuilderMaterials materials = new GenericSchemaBuilderMaterials(database.getProtocolType(),
                 database.getResource().getDatabaseType(), database.getResource().getDataSources(), database.getRuleMetaData().getRules(), props, schemaName);
         Map<String, ShardingSphereSchema> schemaMap = GenericSchemaBuilder.build(Collections.singletonList(tableName), materials);
-        Optional<TableMetaData> actualTableMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(tableName));
+        Optional<ShardingSphereTable> actualTableMetaData = Optional.ofNullable(schemaMap.get(schemaName)).map(optional -> optional.getTables().get(tableName));
         actualTableMetaData.ifPresent(optional -> {
             database.getSchemas().get(schemaName).put(tableName, optional);
             federationDatabaseMetaData.putTableMetadata(schemaName, optional);
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
index 75ce54c2e74..cc45949b472 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
@@ -23,9 +23,9 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.RouteSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.SQLRewriteUnit;
@@ -102,27 +102,27 @@ public final class ExecutionContextBuilderTest {
     }
     
     private Map<String, ShardingSphereSchema> buildDatabaseWithoutPrimaryKey() {
-        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
-        tableMetaDataMap.put("logicName1", new TableMetaData("logicName1", Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, true, false, false),
-                new ColumnMetaData("user_id", Types.INTEGER, false, false, false),
-                new ColumnMetaData("status", Types.INTEGER, false, false, false)), Collections.emptySet(), Collections.emptyList()));
-        tableMetaDataMap.put("t_other", new TableMetaData("t_other", Collections.singletonList(
-                new ColumnMetaData("order_id", Types.INTEGER, true, false, false)), Collections.emptySet(), Collections.emptyList()));
+        Map<String, ShardingSphereTable> tableMetaDataMap = new HashMap<>(3, 1);
+        tableMetaDataMap.put("logicName1", new ShardingSphereTable("logicName1", Arrays.asList(new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("user_id", Types.INTEGER, false, false, false),
+                new ShardingSphereColumn("status", Types.INTEGER, false, false, false)), Collections.emptySet(), Collections.emptyList()));
+        tableMetaDataMap.put("t_other", new ShardingSphereTable("t_other", Collections.singletonList(
+                new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false)), Collections.emptySet(), Collections.emptyList()));
         return Collections.singletonMap("name", new ShardingSphereSchema(tableMetaDataMap));
     }
     
     private Map<String, ShardingSphereSchema> buildDatabase() {
-        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
-        tableMetaDataMap.put("logicName1", new TableMetaData("logicName1", Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, true, false, false),
-                new ColumnMetaData("user_id", Types.INTEGER, false, false, false),
-                new ColumnMetaData("status", Types.INTEGER, false, false, false)), Collections.emptySet(), Collections.emptyList()));
-        tableMetaDataMap.put("logicName2", new TableMetaData("logicName2", Arrays.asList(new ColumnMetaData("item_id", Types.INTEGER, true, false, false),
-                new ColumnMetaData("order_id", Types.INTEGER, false, false, false),
-                new ColumnMetaData("user_id", Types.INTEGER, false, false, false),
-                new ColumnMetaData("status", Types.VARCHAR, false, false, false),
-                new ColumnMetaData("c_date", Types.TIMESTAMP, false, false, false)), Collections.emptySet(), Collections.emptyList()));
-        tableMetaDataMap.put("t_other", new TableMetaData("t_other", Collections.singletonList(
-                new ColumnMetaData("order_id", Types.INTEGER, true, false, false)), Collections.emptySet(), Collections.emptyList()));
+        Map<String, ShardingSphereTable> tableMetaDataMap = new HashMap<>(3, 1);
+        tableMetaDataMap.put("logicName1", new ShardingSphereTable("logicName1", Arrays.asList(new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("user_id", Types.INTEGER, false, false, false),
+                new ShardingSphereColumn("status", Types.INTEGER, false, false, false)), Collections.emptySet(), Collections.emptyList()));
+        tableMetaDataMap.put("logicName2", new ShardingSphereTable("logicName2", Arrays.asList(new ShardingSphereColumn("item_id", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("order_id", Types.INTEGER, false, false, false),
+                new ShardingSphereColumn("user_id", Types.INTEGER, false, false, false),
+                new ShardingSphereColumn("status", Types.VARCHAR, false, false, false),
+                new ShardingSphereColumn("c_date", Types.TIMESTAMP, false, false, false)), Collections.emptySet(), Collections.emptyList()));
+        tableMetaDataMap.put("t_other", new ShardingSphereTable("t_other", Collections.singletonList(
+                new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false)), Collections.emptySet(), Collections.emptyList()));
         return Collections.singletonMap("name", new ShardingSphereSchema(tableMetaDataMap));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/customized/CustomizedFilterableExecutorTest.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/customized/CustomizedFilterableExecutorTest.java
index b7bb8f5ddd5..5f959f65037 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/customized/CustomizedFilterableExecutorTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/customized/CustomizedFilterableExecutorTest.java
@@ -24,9 +24,9 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerCon
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.sql.parser.api.CacheOption;
@@ -48,7 +48,7 @@ public class CustomizedFilterableExecutorTest {
     
     @Before
     public void init() throws Exception {
-        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(2, 1);
+        Map<String, ShardingSphereTable> tableMetaDataMap = new HashMap<>(2, 1);
         tableMetaDataMap.put("t_order_federate", createOrderTableMetaData());
         tableMetaDataMap.put("t_user_info", createUserInfoTableMetaData());
         String schemaName = "federate_jdbc";
@@ -70,17 +70,17 @@ public class CustomizedFilterableExecutorTest {
         return result;
     }
     
-    private TableMetaData createOrderTableMetaData() {
-        ColumnMetaData orderIdColumn = new ColumnMetaData("order_id", Types.VARCHAR, true, false, false);
-        ColumnMetaData userIdColumn = new ColumnMetaData("user_id", Types.VARCHAR, false, false, false);
-        ColumnMetaData statusColumn = new ColumnMetaData("status", Types.VARCHAR, false, false, false);
-        return new TableMetaData("t_order_federate", Arrays.asList(orderIdColumn, userIdColumn, statusColumn), Collections.emptyList(), Collections.emptyList());
+    private ShardingSphereTable createOrderTableMetaData() {
+        ShardingSphereColumn orderIdColumn = new ShardingSphereColumn("order_id", Types.VARCHAR, true, false, false);
+        ShardingSphereColumn userIdColumn = new ShardingSphereColumn("user_id", Types.VARCHAR, false, false, false);
+        ShardingSphereColumn statusColumn = new ShardingSphereColumn("status", Types.VARCHAR, false, false, false);
+        return new ShardingSphereTable("t_order_federate", Arrays.asList(orderIdColumn, userIdColumn, statusColumn), Collections.emptyList(), Collections.emptyList());
     }
     
-    private TableMetaData createUserInfoTableMetaData() {
-        ColumnMetaData userIdColumn = new ColumnMetaData("user_id", Types.VARCHAR, true, false, false);
-        ColumnMetaData informationColumn = new ColumnMetaData("information", Types.VARCHAR, false, false, false);
-        return new TableMetaData("t_user_info", Arrays.asList(userIdColumn, informationColumn), Collections.emptyList(), Collections.emptyList());
+    private ShardingSphereTable createUserInfoTableMetaData() {
+        ShardingSphereColumn userIdColumn = new ShardingSphereColumn("user_id", Types.VARCHAR, true, false, false);
+        ShardingSphereColumn informationColumn = new ShardingSphereColumn("information", Types.VARCHAR, false, false, false);
+        return new ShardingSphereTable("t_user_info", Arrays.asList(userIdColumn, informationColumn), Collections.emptyList(), Collections.emptyList());
     }
     
     @Test
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationDatabaseMetaData.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationDatabaseMetaData.java
index 46aac8e0cb3..00963204b13 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationDatabaseMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationDatabaseMetaData.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.federation.optimizer.metadata;
 
 import lombok.Getter;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -61,7 +61,7 @@ public final class FederationDatabaseMetaData {
      * @param schemaName schema name
      * @param tableMetaData table meta data
      */
-    public void putTableMetadata(final String schemaName, final TableMetaData tableMetaData) {
+    public void putTableMetadata(final String schemaName, final ShardingSphereTable tableMetaData) {
         FederationSchemaMetaData schemaMetaData = schemas.computeIfAbsent(schemaName.toLowerCase(), key -> new FederationSchemaMetaData(schemaName, new LinkedHashMap<>()));
         schemaMetaData.put(tableMetaData);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationSchemaMetaData.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationSchemaMetaData.java
index de712aca704..a80bff3e0e6 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationSchemaMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationSchemaMetaData.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.federation.optimizer.metadata;
 
 import lombok.Getter;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 
 import java.util.Map;
 import java.util.Map.Entry;
@@ -34,10 +34,10 @@ public final class FederationSchemaMetaData {
     
     private final Map<String, FederationTableMetaData> tables;
     
-    public FederationSchemaMetaData(final String name, final Map<String, TableMetaData> metaData) {
+    public FederationSchemaMetaData(final String name, final Map<String, ShardingSphereTable> metaData) {
         this.name = name;
         this.tables = new ConcurrentHashMap<>(metaData.size(), 1);
-        for (Entry<String, TableMetaData> entry : metaData.entrySet()) {
+        for (Entry<String, ShardingSphereTable> entry : metaData.entrySet()) {
             tables.put(entry.getKey().toLowerCase(), new FederationTableMetaData(entry.getValue().getName(), entry.getValue()));
         }
     }
@@ -47,7 +47,7 @@ public final class FederationSchemaMetaData {
      * 
      * @param metaData table meta data to be updated
      */
-    public void put(final TableMetaData metaData) {
+    public void put(final ShardingSphereTable metaData) {
         tables.put(metaData.getName().toLowerCase(), new FederationTableMetaData(metaData.getName(), metaData));
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationTableMetaData.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationTableMetaData.java
index 9b5cd2165ee..9585db6a3e3 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationTableMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/metadata/FederationTableMetaData.java
@@ -26,8 +26,8 @@ import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 
 import java.util.List;
 import java.util.stream.Collectors;
@@ -46,21 +46,21 @@ public final class FederationTableMetaData {
     
     private final List<String> columnNames;
     
-    public FederationTableMetaData(final String name, final TableMetaData tableMetaData) {
+    public FederationTableMetaData(final String name, final ShardingSphereTable tableMetaData) {
         this.name = name;
         relProtoDataType = createRelProtoDataType(tableMetaData);
-        columnNames = tableMetaData.getColumns().values().stream().map(ColumnMetaData::getName).collect(Collectors.toList());
+        columnNames = tableMetaData.getColumns().values().stream().map(ShardingSphereColumn::getName).collect(Collectors.toList());
     }
     
-    private RelProtoDataType createRelProtoDataType(final TableMetaData tableMetaData) {
+    private RelProtoDataType createRelProtoDataType(final ShardingSphereTable tableMetaData) {
         Builder fieldInfo = REL_DATA_TYPE_FACTORY.builder();
-        for (ColumnMetaData each : tableMetaData.getColumns().values()) {
+        for (ShardingSphereColumn each : tableMetaData.getColumns().values()) {
             fieldInfo.add(each.getName(), getRelDataType(each));
         }
         return RelDataTypeImpl.proto(fieldInfo.build());
     }
     
-    private RelDataType getRelDataType(final ColumnMetaData columnMetaData) {
+    private RelDataType getRelDataType(final ShardingSphereColumn columnMetaData) {
         Class<?> sqlTypeClass = SqlType.valueOf(columnMetaData.getDataType()).clazz;
         RelDataType javaType = REL_DATA_TYPE_FACTORY.createJavaType(sqlTypeClass);
         return REL_DATA_TYPE_FACTORY.createTypeWithNullability(javaType, true);
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java
index 6dd659d14b1..a90b0e4aa34 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java
@@ -23,9 +23,9 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerCon
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
@@ -94,7 +94,7 @@ public final class ShardingSphereOptimizerTest {
     
     @Before
     public void init() throws Exception {
-        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(2, 1);
+        Map<String, ShardingSphereTable> tableMetaDataMap = new HashMap<>(2, 1);
         tableMetaDataMap.put("t_order_federate", createOrderTableMetaData());
         tableMetaDataMap.put("t_user_info", createUserInfoTableMetaData());
         ShardingSphereDatabase database = new ShardingSphereDatabase(databaseName,
@@ -115,17 +115,17 @@ public final class ShardingSphereOptimizerTest {
         return result;
     }
     
-    private TableMetaData createOrderTableMetaData() {
-        ColumnMetaData orderIdColumn = new ColumnMetaData("order_id", Types.VARCHAR, true, false, false);
-        ColumnMetaData userIdColumn = new ColumnMetaData("user_id", Types.VARCHAR, false, false, false);
-        ColumnMetaData statusColumn = new ColumnMetaData("status", Types.VARCHAR, false, false, false);
-        return new TableMetaData("t_order_federate", Arrays.asList(orderIdColumn, userIdColumn, statusColumn), Collections.emptyList(), Collections.emptyList());
+    private ShardingSphereTable createOrderTableMetaData() {
+        ShardingSphereColumn orderIdColumn = new ShardingSphereColumn("order_id", Types.VARCHAR, true, false, false);
+        ShardingSphereColumn userIdColumn = new ShardingSphereColumn("user_id", Types.VARCHAR, false, false, false);
+        ShardingSphereColumn statusColumn = new ShardingSphereColumn("status", Types.VARCHAR, false, false, false);
+        return new ShardingSphereTable("t_order_federate", Arrays.asList(orderIdColumn, userIdColumn, statusColumn), Collections.emptyList(), Collections.emptyList());
     }
     
-    private TableMetaData createUserInfoTableMetaData() {
-        ColumnMetaData userIdColumn = new ColumnMetaData("user_id", Types.VARCHAR, true, false, false);
-        ColumnMetaData informationColumn = new ColumnMetaData("information", Types.VARCHAR, false, false, false);
-        return new TableMetaData("t_user_info", Arrays.asList(userIdColumn, informationColumn), Collections.emptyList(), Collections.emptyList());
+    private ShardingSphereTable createUserInfoTableMetaData() {
+        ShardingSphereColumn userIdColumn = new ShardingSphereColumn("user_id", Types.VARCHAR, true, false, false);
+        ShardingSphereColumn informationColumn = new ShardingSphereColumn("information", Types.VARCHAR, false, false, false);
+        return new ShardingSphereTable("t_user_info", Arrays.asList(userIdColumn, informationColumn), Collections.emptyList(), Collections.emptyList());
     }
     
     @Test
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/DataConsistencyChecker.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/DataConsistencyChecker.java
index ee3c569ff5f..662f4075c76 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/DataConsistencyChecker.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/DataConsistencyChecker.java
@@ -40,7 +40,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.thread.ExecutorThreadFactoryBuilder;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 
 import javax.sql.DataSource;
@@ -168,7 +168,7 @@ public final class DataConsistencyChecker {
             String sourceDatabaseType = sourceDataSourceConfig.getDatabaseType().getType();
             String targetDatabaseType = targetDataSourceConfig.getDatabaseType().getType();
             for (String each : logicTableNames) {
-                TableMetaData tableMetaData = getTableMetaData(jobConfig.getDatabaseName(), each);
+                ShardingSphereTable tableMetaData = getTableMetaData(jobConfig.getDatabaseName(), each);
                 if (null == tableMetaData) {
                     throw new PipelineDataConsistencyCheckFailedException("Can not get metadata for table " + each);
                 }
@@ -222,7 +222,7 @@ public final class DataConsistencyChecker {
         }
     }
     
-    private TableMetaData getTableMetaData(final String databaseName, final String logicTableName) {
+    private ShardingSphereTable getTableMetaData(final String databaseName, final String logicTableName) {
         ContextManager contextManager = PipelineContext.getContextManager();
         Preconditions.checkNotNull(contextManager, "ContextManager null");
         ShardingSphereDatabase database = contextManager.getMetaDataContexts().getMetaData().getDatabases().get(databaseName);
diff --git a/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/test/java/org/apache/shardingsphere/singletable/metadata/SingleTableSchemaBuilderTest.java b/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/test/java/org/apache/shardingsphere/singletable/metadata/SingleTableSchemaBuilderTest.java
index a25fda4f517..1d4845fac93 100644
--- a/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/test/java/org/apache/shardingsphere/singletable/metadata/SingleTableSchemaBuilderTest.java
+++ b/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/test/java/org/apache/shardingsphere/singletable/metadata/SingleTableSchemaBuilderTest.java
@@ -21,10 +21,10 @@ import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterials;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.singletable.config.SingleTableRuleConfiguration;
 import org.apache.shardingsphere.singletable.rule.SingleTableRule;
@@ -118,8 +118,8 @@ public final class SingleTableSchemaBuilderTest {
                 Collections.emptyList(), new ConfigurationProperties(new Properties()));
     }
     
-    private void assertActualOfSingleTables(final Collection<TableMetaData> actual) {
-        Map<String, TableMetaData> tableMetaDataMap = actual.stream().collect(Collectors.toMap(TableMetaData::getName, v -> v));
+    private void assertActualOfSingleTables(final Collection<ShardingSphereTable> actual) {
+        Map<String, ShardingSphereTable> tableMetaDataMap = actual.stream().collect(Collectors.toMap(ShardingSphereTable::getName, v -> v));
         assertTrue(tableMetaDataMap.containsKey(singleTableNames[0]));
         assertFalse(tableMetaDataMap.get(singleTableNames[0]).getColumns().isEmpty());
         assertTrue(tableMetaDataMap.containsKey(singleTableNames[1]));
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
index 5f2b13ed69c..6bb870093a8 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
@@ -37,11 +37,11 @@ import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterials;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.SystemSchemaBuilder;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.global.GlobalRulesBuilder;
 import org.apache.shardingsphere.infra.rule.builder.schema.DatabaseRulesBuilder;
@@ -159,7 +159,7 @@ public final class ContextManager implements AutoCloseable {
             return;
         }
         FederationDatabaseMetaData federationDatabaseMetaData = metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName);
-        federationDatabaseMetaData.putTableMetadata(schemaName, new TableMetaData());
+        federationDatabaseMetaData.putTableMetadata(schemaName, new ShardingSphereTable());
         metaDataContexts.getOptimizerContext().getPlannerContexts().put(databaseName, OptimizerPlannerContextFactory.create(federationDatabaseMetaData));
         metaDataContexts.getMetaData().getDatabases().get(databaseName).getSchemas().put(schemaName, new ShardingSphereSchema());
     }
@@ -190,7 +190,7 @@ public final class ContextManager implements AutoCloseable {
      * @param changedTableMetaData changed table meta data
      * @param deletedTable deleted table
      */
-    public void alterSchema(final String databaseName, final String schemaName, final TableMetaData changedTableMetaData, final String deletedTable) {
+    public void alterSchema(final String databaseName, final String schemaName, final ShardingSphereTable changedTableMetaData, final String deletedTable) {
         if (null != metaDataContexts.getMetaData().getDatabases().get(databaseName)) {
             Optional.ofNullable(changedTableMetaData).ifPresent(optional -> alterTableSchema(databaseName, schemaName, optional));
             Optional.ofNullable(deletedTable).ifPresent(optional -> deleteTable(databaseName, schemaName, optional));
@@ -202,7 +202,7 @@ public final class ContextManager implements AutoCloseable {
                 .forEach((schemaName, tables) -> metaDataContexts.getPersistService().ifPresent(optional -> optional.getSchemaMetaDataService().persistMetaData(databaseName, schemaName, tables))));
     }
     
-    private void alterTableSchema(final String databaseName, final String schemaName, final TableMetaData changedTableMetaData) {
+    private void alterTableSchema(final String databaseName, final String schemaName, final ShardingSphereTable changedTableMetaData) {
         ShardingSphereDatabase database = metaDataContexts.getMetaData().getDatabases().get(databaseName);
         alterSingleTableDataNodes(databaseName, database, changedTableMetaData);
         FederationDatabaseMetaData federationDatabaseMetaData = metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName);
@@ -211,7 +211,7 @@ public final class ContextManager implements AutoCloseable {
         metaDataContexts.getOptimizerContext().getPlannerContexts().put(databaseName, OptimizerPlannerContextFactory.create(federationDatabaseMetaData));
     }
     
-    private void alterSingleTableDataNodes(final String databaseName, final ShardingSphereDatabase database, final TableMetaData changedTableMetaData) {
+    private void alterSingleTableDataNodes(final String databaseName, final ShardingSphereDatabase database, final ShardingSphereTable changedTableMetaData) {
         if (!containsInImmutableDataNodeContainedRule(changedTableMetaData.getName(), database)) {
             refreshRules(databaseName, database);
         }
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistService.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistService.java
index 41706602454..91d2a2ebb66 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistService.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistService.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.mode.metadata.persist.service;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlTableMetaData;
 import org.apache.shardingsphere.infra.yaml.schema.swapper.TableMetaDataYamlSwapper;
@@ -56,7 +56,7 @@ public final class SchemaMetaDataPersistService {
         persistMetaData(databaseName, schemaName, schema.getTables());
     }
     
-    private void persistMetaData(final String databaseName, final String schemaName, final Map<String, TableMetaData> tables) {
+    private void persistMetaData(final String databaseName, final String schemaName, final Map<String, ShardingSphereTable> tables) {
         if (tables.isEmpty()) {
             persistSchema(databaseName, schemaName);
             return;
@@ -72,7 +72,7 @@ public final class SchemaMetaDataPersistService {
      * @param schemaName schema name
      * @param tableMetaData table meta data
      */
-    public void persistTable(final String databaseName, final String schemaName, final TableMetaData tableMetaData) {
+    public void persistTable(final String databaseName, final String schemaName, final ShardingSphereTable tableMetaData) {
         repository.persist(DatabaseMetaDataNode.getTableMetaDataPath(databaseName, schemaName, tableMetaData.getName().toLowerCase()),
                 YamlEngine.marshal(new TableMetaDataYamlSwapper().swapToYamlConfiguration(tableMetaData)));
     }
@@ -88,10 +88,10 @@ public final class SchemaMetaDataPersistService {
     }
     
     private void compareAndPersist(final String databaseName, final String schemaName, final ShardingSphereSchema schema, final ShardingSphereSchema originalSchema) {
-        Map<String, TableMetaData> cachedLocalTables = new LinkedHashMap<>(schema.getTables());
-        for (Entry<String, TableMetaData> entry : originalSchema.getTables().entrySet()) {
+        Map<String, ShardingSphereTable> cachedLocalTables = new LinkedHashMap<>(schema.getTables());
+        for (Entry<String, ShardingSphereTable> entry : originalSchema.getTables().entrySet()) {
             String onlineTableName = entry.getKey();
-            TableMetaData localTableMetaData = cachedLocalTables.remove(onlineTableName);
+            ShardingSphereTable localTableMetaData = cachedLocalTables.remove(onlineTableName);
             if (null == localTableMetaData) {
                 deleteTable(databaseName, schemaName, onlineTableName);
                 continue;
@@ -150,7 +150,7 @@ public final class SchemaMetaDataPersistService {
         ShardingSphereSchema schema = new ShardingSphereSchema();
         tables.forEach(each -> {
             String content = repository.get(DatabaseMetaDataNode.getTableMetaDataPath(databaseName, schemaName, each));
-            TableMetaData tableMetaData = new TableMetaDataYamlSwapper().swapToObject(YamlEngine.unmarshal(content, YamlTableMetaData.class));
+            ShardingSphereTable tableMetaData = new TableMetaDataYamlSwapper().swapToObject(YamlEngine.unmarshal(content, YamlTableMetaData.class));
             schema.getTables().put(each, tableMetaData);
         });
         return Optional.of(schema);
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
index 3d8b6d27d43..68b34c6a654 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
@@ -33,7 +33,7 @@ import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.mode.metadata.persist.service.SchemaMetaDataPersistService;
@@ -121,7 +121,7 @@ public final class ContextManagerTest {
     @Test
     public void assertAlterSchemas() {
         contextManager.alterSchemas("foo_db", Collections.singletonMap("foo_db", new ShardingSphereSchema(Collections.singletonMap("foo_table",
-                new TableMetaData("foo_table", Collections.emptyList(), Collections.emptyList(), Collections.emptyList())))));
+                new ShardingSphereTable("foo_table", Collections.emptyList(), Collections.emptyList(), Collections.emptyList())))));
         assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabases().get("foo_db").getSchemas().get("foo_db").containsTable("foo_table"));
         assertTrue(contextManager.getMetaDataContexts().getOptimizerContext().getFederationMetaData().getDatabases().containsKey("foo_db"));
         Map<String, FederationSchemaMetaData> schemas = contextManager.getMetaDataContexts().getOptimizerContext().getFederationMetaData().getDatabases().get("foo_db").getSchemas();
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistServiceTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistServiceTest.java
index 004e1e06308..39b2e576f4a 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistServiceTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/persist/service/SchemaMetaDataPersistServiceTest.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.mode.metadata.persist.service;
 
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlTableMetaData;
 import org.apache.shardingsphere.infra.yaml.schema.swapper.TableMetaDataYamlSwapper;
@@ -55,7 +55,7 @@ public final class SchemaMetaDataPersistServiceTest {
     
     @Test
     public void assertPersist() {
-        TableMetaData tableMetaData = new TableMetaDataYamlSwapper().swapToObject(YamlEngine.unmarshal(readYAML(), YamlTableMetaData.class));
+        ShardingSphereTable tableMetaData = new TableMetaDataYamlSwapper().swapToObject(YamlEngine.unmarshal(readYAML(), YamlTableMetaData.class));
         ShardingSphereSchema schema = new ShardingSphereSchema();
         schema.getTables().put("t_order", tableMetaData);
         new SchemaMetaDataPersistService(repository).persistMetaData("foo_db", "foo_schema", schema);
@@ -93,7 +93,7 @@ public final class SchemaMetaDataPersistServiceTest {
     
     @Test
     public void assertPersistTableMetaData() {
-        TableMetaData tableMetaData = new TableMetaData("FOO_TABLE", Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereTable tableMetaData = new ShardingSphereTable("FOO_TABLE", Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
         new SchemaMetaDataPersistService(repository).persistTable("foo_db", "foo_schema", tableMetaData);
         verify(repository).persist(eq("/metadata/foo_db/schemas/foo_schema/tables/foo_table"), anyString());
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/cache/subscriber/ScalingRegistrySubscriber.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/cache/subscriber/ScalingRegistrySubscriber.java
index 1786b16f6c9..0139159ff5f 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/cache/subscriber/ScalingRegistrySubscriber.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/cache/subscriber/ScalingRegistrySubscriber.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.cach
 import com.google.common.eventbus.Subscribe;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.cache.event.StartScalingEvent;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.rule.ScalingTaskFinishedEvent;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.schema.SchemaChangedEvent;
@@ -92,7 +92,7 @@ public final class ScalingRegistrySubscriber {
      */
     @Subscribe
     public void schemaChanged(final SchemaChangedEvent event) {
-        TableMetaData changedTableMetaData = event.getChangedTableMetaData();
+        ShardingSphereTable changedTableMetaData = event.getChangedTableMetaData();
         String changedTableName = null != changedTableMetaData ? changedTableMetaData.getName() : null;
         log.info("schemaChanged, databaseName={}, schemaName={}, changedTableName={}, deletedTable={}", event.getDatabaseName(), event.getSchemaName(), changedTableName, event.getDeletedTable());
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
index bded86662c0..b99f7f56589 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/config/event/schema/SchemaChangedEvent.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.conf
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceEvent;
 
 /**
@@ -33,7 +33,7 @@ public final class SchemaChangedEvent implements GovernanceEvent {
     
     private final String schemaName;
     
-    private final TableMetaData changedTableMetaData;
+    private final ShardingSphereTable changedTableMetaData;
     
     private final String deletedTable;
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
index 7a44a11fa12..30c4087babf 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
@@ -41,7 +41,7 @@ import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphere
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.StatusContainedRule;
@@ -202,7 +202,7 @@ public final class ClusterContextManagerCoordinatorTest {
     
     @Test
     public void assertSchemaChanged() {
-        TableMetaData changedTableMetaData = new TableMetaData("t_order", Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+        ShardingSphereTable changedTableMetaData = new ShardingSphereTable("t_order", Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
         SchemaChangedEvent event = new SchemaChangedEvent("db", "db", changedTableMetaData, null);
         coordinator.renew(event);
         assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabases().containsKey("db"));
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/metadata/subscriber/SchemaMetaDataRegistrySubscriberTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/metadata/subscriber/SchemaMetaDataRegist [...]
index 695cb3094e3..400d40a883c 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/metadata/subscriber/SchemaMetaDataRegistrySubscriberTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/metadata/subscriber/SchemaMetaDataRegistrySubscriberTest.java
@@ -18,11 +18,11 @@
 package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.metadata.subscriber;
 
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.AddSchemaEvent;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.AlterSchemaEvent;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.DropSchemaEvent;
 import org.apache.shardingsphere.infra.metadata.database.schema.event.SchemaAlteredEvent;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
 import org.apache.shardingsphere.mode.metadata.persist.service.SchemaMetaDataPersistService;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.junit.Before;
@@ -57,7 +57,7 @@ public final class SchemaMetaDataRegistrySubscriberTest {
     @Test
     public void assertUpdateWithMetaDataAlteredEvent() {
         SchemaAlteredEvent event = new SchemaAlteredEvent("foo_db", "foo_schema");
-        TableMetaData tableMetaData = new TableMetaData();
+        ShardingSphereTable tableMetaData = new ShardingSphereTable();
         event.getAlteredTables().add(tableMetaData);
         event.getDroppedTables().add("foo_table");
         schemaMetaDataRegistrySubscriber.update(event);
@@ -81,7 +81,7 @@ public final class SchemaMetaDataRegistrySubscriberTest {
     
     @Test
     public void assertAlterSchemaEventWhenContainsTable() {
-        ShardingSphereSchema schema = new ShardingSphereSchema(Collections.singletonMap("t_order", new TableMetaData()));
+        ShardingSphereSchema schema = new ShardingSphereSchema(Collections.singletonMap("t_order", new ShardingSphereTable()));
         AlterSchemaEvent event = new AlterSchemaEvent("foo_db", "foo_schema", "new_foo_schema", schema);
         schemaMetaDataRegistrySubscriber.alterSchema(event);
         verify(persistService).persistMetaData("foo_db", "new_foo_schema", schema);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilder.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilder.java
index d0b5cf9883a..7247cc742f6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilder.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilder.java
@@ -22,8 +22,8 @@ import org.apache.commons.lang3.concurrent.ConcurrentException;
 import org.apache.commons.lang3.concurrent.LazyInitializer;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResultMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeaderBuilder;
@@ -46,8 +46,8 @@ public final class MySQLQueryHeaderBuilder implements QueryHeaderBuilder {
         boolean primaryKey;
         if (null != actualTableName && null != dataNodeContainedRule.get()) {
             tableName = dataNodeContainedRule.get().findLogicTableByActualTable(actualTableName).orElse("");
-            TableMetaData tableMetaData = database.getSchemas().get(schemaName).get(tableName);
-            primaryKey = null != tableMetaData && Optional.ofNullable(tableMetaData.getColumns().get(columnName.toLowerCase())).map(ColumnMetaData::isPrimaryKey).orElse(false);
+            ShardingSphereTable tableMetaData = database.getSchemas().get(schemaName).get(tableName);
+            primaryKey = null != tableMetaData && Optional.ofNullable(tableMetaData.getColumns().get(columnName.toLowerCase())).map(ShardingSphereColumn::isPrimaryKey).orElse(false);
         } else {
             tableName = actualTableName;
             primaryKey = false;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutor.java
index c5dabf9d118..fe44e30fe3a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutor.java
@@ -28,7 +28,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.query.impl.ra
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.type.memory.row.MemoryQueryResultDataRow;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.merge.result.impl.transparent.TransparentMergedResult;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
 import org.apache.shardingsphere.proxy.backend.text.admin.executor.DatabaseAdminQueryExecutor;
@@ -82,7 +82,7 @@ public final class ShowTablesExecutor implements DatabaseAdminQueryExecutor {
     
     private Collection<String> getAllTableNames(final String databaseName) {
         Collection<String> result = ProxyContext.getInstance()
-                .getDatabase(databaseName).getSchemas().get(databaseName).getTables().values().stream().map(TableMetaData::getName).collect(Collectors.toList());
+                .getDatabase(databaseName).getSchemas().get(databaseName).getTables().values().stream().map(ShardingSphereTable::getName).collect(Collectors.toList());
         if (showTablesStatement.getFilter().isPresent()) {
             Optional<String> pattern = showTablesStatement.getFilter().get().getLike().map(optional -> SQLUtil.convertLikePatternToRegex(optional.getPattern()));
             return pattern.isPresent() ? result.stream().filter(each -> RegularUtil.matchesCaseInsensitive(pattern.get(), each)).collect(Collectors.toList()) : result;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandler.java
index 3a72f666de2..e9e5be06e2b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandler.java
@@ -21,8 +21,8 @@ import com.google.common.base.Strings;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowTableMetadataStatement;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.exception.DatabaseNotExistedException;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.NoDatabaseSelectedException;
@@ -86,7 +86,7 @@ public final class ShowTableMetadataHandler extends QueryableRALBackendHandler<S
         Collection<List<Object>> result = new LinkedList<>();
         Collection<List<Object>> columnRows = schema.getAllColumnNames(tableName).stream().map(each -> buildRow(databaseName, tableName, "COLUMN", each))
                 .collect(Collectors.toList());
-        Collection<List<Object>> indexRows = schema.getTables().get(tableName).getIndexes().values().stream().map(IndexMetaData::getName)
+        Collection<List<Object>> indexRows = schema.getTables().get(tableName).getIndexes().values().stream().map(ShardingSphereIndex::getName)
                 .map(each -> buildRow(databaseName, tableName, "INDEX", each)).collect(Collectors.toList());
         result.addAll(columnRows);
         result.addAll(indexRows);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java
index 06df0196d57..8e424bab0b1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java
@@ -32,10 +32,10 @@ import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryQueryResul
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -145,9 +145,9 @@ public final class JDBCDatabaseCommunicationEngineTest extends ProxyContextResto
     
     private ShardingSphereDatabase createDatabaseMetaData() {
         ShardingSphereDatabase result = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        ColumnMetaData columnMetaData = new ColumnMetaData("order_id", Types.INTEGER, true, false, false);
+        ShardingSphereColumn columnMetaData = new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false);
         when(result.getSchemas().get(DefaultDatabase.LOGIC_NAME).get("t_logic_order")).thenReturn(
-                new TableMetaData("t_logic_order", Collections.singletonList(columnMetaData), Collections.singletonList(new IndexMetaData("order_id")), Collections.emptyList()));
+                new ShardingSphereTable("t_logic_order", Collections.singletonList(columnMetaData), Collections.singletonList(new ShardingSphereIndex("order_id")), Collections.emptyList()));
         ShardingRule shardingRule = mock(ShardingRule.class);
         when(shardingRule.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_logic_order"));
         when(result.getRuleMetaData().getRules()).thenReturn(Collections.singletonList(shardingRule));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilderTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilderTest.java
index 786bdf4bce0..14ee01f2ce0 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilderTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/header/query/impl/MySQLQueryHeaderBuilderTest.java
@@ -23,10 +23,10 @@ import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResultMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeaderBuilder;
@@ -96,10 +96,10 @@ public final class MySQLQueryHeaderBuilderTest {
     
     private ShardingSphereDatabase createDatabase() {
         ShardingSphereDatabase result = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        ColumnMetaData columnMetaData = new ColumnMetaData("order_id", Types.INTEGER, true, false, false);
+        ShardingSphereColumn columnMetaData = new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
-        when(schema.get("t_logic_order")).thenReturn(new TableMetaData("t_logic_order",
-                Collections.singletonList(columnMetaData), Collections.singletonList(new IndexMetaData("order_id")), Collections.emptyList()));
+        when(schema.get("t_logic_order")).thenReturn(new ShardingSphereTable("t_logic_order",
+                Collections.singletonList(columnMetaData), Collections.singletonList(new ShardingSphereIndex("order_id")), Collections.emptyList()));
         when(result.getResource().getDataSourceMetaData("ds_0")).thenReturn(mock(DataSourceMetaData.class));
         when(result.getSchemas().get(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
         ShardingRule shardingRule = mock(ShardingRule.class);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
index 82c3f3023e3..e65d9649462 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
@@ -25,7 +25,7 @@ import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
@@ -68,11 +68,11 @@ public final class ShowTablesExecutorTest extends ProxyContextRestorer {
     }
     
     private Map<String, ShardingSphereDatabase> getDatabases() {
-        Map<String, TableMetaData> tables = new HashMap<>(4, 1);
-        tables.put("t_account", new TableMetaData("t_account", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
-        tables.put("t_account_bak", new TableMetaData("t_account_bak", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
-        tables.put("t_account_detail", new TableMetaData("t_account_detail", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
-        tables.put("t_test", new TableMetaData("T_TEST", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        Map<String, ShardingSphereTable> tables = new HashMap<>(4, 1);
+        tables.put("t_account", new ShardingSphereTable("t_account", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        tables.put("t_account_bak", new ShardingSphereTable("t_account_bak", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        tables.put("t_account_detail", new ShardingSphereTable("t_account_detail", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        tables.put("t_test", new ShardingSphereTable("T_TEST", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
         ShardingSphereSchema schema = new ShardingSphereSchema(tables);
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         when(database.getSchemas().get(String.format(DATABASE_PATTERN, 0))).thenReturn(schema);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java
index c90cf0a7ca8..ebbfe01bb9a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java
@@ -25,7 +25,7 @@ import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -89,7 +89,7 @@ public final class SelectTableExecutorTest extends ProxyContextRestorer {
     
     private ShardingSphereDatabase createDatabase() throws SQLException {
         return new ShardingSphereDatabase("sharding_db", new PostgreSQLDatabaseType(), new ShardingSphereResource(Collections.singletonMap("foo_ds", new MockedDataSource(mockConnection()))),
-                mock(ShardingSphereRuleMetaData.class), Collections.singletonMap("public", new ShardingSphereSchema(Collections.singletonMap("t_order", mock(TableMetaData.class)))));
+                mock(ShardingSphereRuleMetaData.class), Collections.singletonMap("public", new ShardingSphereSchema(Collections.singletonMap("t_order", mock(ShardingSphereTable.class)))));
     }
     
     private Connection mockConnection() throws SQLException {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ExportDatabaseConfigurationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ExportDatabaseConfigurationHandlerTest.java
index b42f66418b7..792bd87e4b8 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ExportDatabaseConfigurationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ExportDatabaseConfigurationHandlerTest.java
@@ -22,10 +22,10 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.E
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
@@ -44,9 +44,9 @@ import org.junit.Test;
 import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
@@ -74,7 +74,7 @@ public final class ExportDatabaseConfigurationHandlerTest extends ProxyContextRe
         ExportDatabaseConfigurationHandler handler = new ExportDatabaseConfigurationHandler().init(createParameter(createSQLStatement(), mock(ConnectionSession.class)));
         handler.execute();
         handler.next();
-        List<Object> data = new ArrayList<>(handler.getRowData());
+        Collection<Object> data = new ArrayList<>(handler.getRowData());
         assertThat(data.size(), is(1));
     }
     
@@ -117,10 +117,10 @@ public final class ExportDatabaseConfigurationHandlerTest extends ProxyContextRe
         return result;
     }
     
-    private Map<String, TableMetaData> createTableMap() {
-        List<ColumnMetaData> columns = Collections.singletonList(new ColumnMetaData("order_id", 0, false, false, false));
-        List<IndexMetaData> indexes = Collections.singletonList(new IndexMetaData("primary"));
-        return Collections.singletonMap("t_order", new TableMetaData("t_order", columns, indexes, Collections.emptyList()));
+    private Map<String, ShardingSphereTable> createTableMap() {
+        Collection<ShardingSphereColumn> columns = Collections.singletonList(new ShardingSphereColumn("order_id", 0, false, false, false));
+        Collection<ShardingSphereIndex> indexes = Collections.singletonList(new ShardingSphereIndex("primary"));
+        return Collections.singletonMap("t_order", new ShardingSphereTable("t_order", columns, indexes, Collections.emptyList()));
     }
     
     private ShardingTableRuleConfiguration createTableRuleConfiguration() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandlerTest.java
index f1eaa762f7c..ec2737d06b0 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTableMetadataHandlerTest.java
@@ -20,10 +20,10 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryabl
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowTableMetadataStatement;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
@@ -35,6 +35,7 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -75,11 +76,11 @@ public final class ShowTableMetadataHandlerTest extends ProxyContextRestorer {
         assertThat(data.get(3), is("primary"));
     }
     
-    private Map<String, TableMetaData> createTableMap() {
-        Map<String, TableMetaData> result = new HashMap<>();
-        List<ColumnMetaData> columns = Collections.singletonList(new ColumnMetaData("order_id", 0, false, false, false));
-        List<IndexMetaData> indexes = Collections.singletonList(new IndexMetaData("primary"));
-        result.put("t_order", new TableMetaData("t_order", columns, indexes, Collections.emptyList()));
+    private Map<String, ShardingSphereTable> createTableMap() {
+        Map<String, ShardingSphereTable> result = new HashMap<>();
+        Collection<ShardingSphereColumn> columns = Collections.singletonList(new ShardingSphereColumn("order_id", 0, false, false, false));
+        Collection<ShardingSphereIndex> indexes = Collections.singletonList(new ShardingSphereIndex("primary"));
+        result.put("t_order", new ShardingSphereTable("t_order", columns, indexes, Collections.emptyList()));
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/ImportDatabaseConfigurationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/ImportDatabaseConfigurationHandlerTest.java
index 28301efd9d6..7fed3ee78fd 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/ImportDatabaseConfigurationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/ImportDatabaseConfigurationHandlerTest.java
@@ -22,10 +22,10 @@ import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.datasource.props.DataSourcePropertiesValidator;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
@@ -149,10 +149,10 @@ public final class ImportDatabaseConfigurationHandlerTest extends ProxyContextRe
         return result;
     }
     
-    private Map<String, TableMetaData> createTableMap() {
-        Collection<ColumnMetaData> columns = Collections.singleton(new ColumnMetaData("order_id", 0, false, false, false));
-        Collection<IndexMetaData> indexes = Collections.singleton(new IndexMetaData("primary"));
-        return Collections.singletonMap("t_order", new TableMetaData("t_order", columns, indexes, Collections.emptyList()));
+    private Map<String, ShardingSphereTable> createTableMap() {
+        Collection<ShardingSphereColumn> columns = Collections.singleton(new ShardingSphereColumn("order_id", 0, false, false, false));
+        Collection<ShardingSphereIndex> indexes = Collections.singleton(new ShardingSphereIndex("primary"));
+        return Collections.singletonMap("t_order", new ShardingSphereTable("t_order", columns, indexes, Collections.emptyList()));
     }
     
     private RALBackendHandler.HandlerParameter<ImportDatabaseConfigurationStatement> getParameter(final String importFilePath, final ConnectionSession connectionSession) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java
index e738bcc1546..63f6a7fd31b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java
@@ -37,8 +37,8 @@ import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -132,9 +132,9 @@ public final class PostgreSQLComDescribeExecutor implements CommandExecutor {
         ShardingSphereDatabase database = ProxyContext.getInstance().getDatabase(databaseName);
         String schemaName = insertStatement.getTable().getOwner().map(optional -> optional.getIdentifier()
                 .getValue()).orElseGet(() -> DatabaseTypeEngine.getDefaultSchemaName(database.getResource().getDatabaseType(), databaseName));
-        TableMetaData tableMetaData = database.getSchemas().get(schemaName).get(logicTableName);
-        Map<String, ColumnMetaData> columnMetaData = tableMetaData.getColumns();
-        Map<String, ColumnMetaData> caseInsensitiveColumnMetaData = null;
+        ShardingSphereTable tableMetaData = database.getSchemas().get(schemaName).get(logicTableName);
+        Map<String, ShardingSphereColumn> columnMetaData = tableMetaData.getColumns();
+        Map<String, ShardingSphereColumn> caseInsensitiveColumnMetaData = null;
         List<String> columnNames;
         if (insertStatement.getColumns().isEmpty()) {
             columnNames = new ArrayList<>(tableMetaData.getColumns().keySet());
@@ -156,7 +156,7 @@ public final class PostgreSQLComDescribeExecutor implements CommandExecutor {
                     continue;
                 }
                 String columnName = columnNames.get(columnIndex);
-                ColumnMetaData column = columnMetaData.get(columnName);
+                ShardingSphereColumn column = columnMetaData.get(columnName);
                 if (null == column) {
                     if (null == caseInsensitiveColumnMetaData) {
                         caseInsensitiveColumnMetaData = convertToCaseInsensitiveColumnMetaDataMap(columnMetaData);
@@ -184,8 +184,8 @@ public final class PostgreSQLComDescribeExecutor implements CommandExecutor {
         return unspecifiedTypeParameterIndexes;
     }
     
-    private Map<String, ColumnMetaData> convertToCaseInsensitiveColumnMetaDataMap(final Map<String, ColumnMetaData> columnMetaDataMap) {
-        Map<String, ColumnMetaData> result = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    private Map<String, ShardingSphereColumn> convertToCaseInsensitiveColumnMetaDataMap(final Map<String, ShardingSphereColumn> columnMetaDataMap) {
+        Map<String, ShardingSphereColumn> result = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
         result.putAll(columnMetaDataMap);
         return result;
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java
index 83463f4a9ae..cbcfbccc609 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java
@@ -32,8 +32,8 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
@@ -116,12 +116,12 @@ public final class PostgreSQLComDescribeExecutorTest extends ProxyContextRestore
     }
     
     private void prepareTableMetaData() {
-        Collection<ColumnMetaData> columnMetaData = Arrays.asList(
-                new ColumnMetaData("id", Types.INTEGER, true, false, false),
-                new ColumnMetaData("k", Types.INTEGER, true, false, false),
-                new ColumnMetaData("c", Types.CHAR, true, false, false),
-                new ColumnMetaData("pad", Types.CHAR, true, false, false));
-        TableMetaData tableMetaData = new TableMetaData(TABLE_NAME, columnMetaData, Collections.emptyList(), Collections.emptyList());
+        Collection<ShardingSphereColumn> columnMetaData = Arrays.asList(
+                new ShardingSphereColumn("id", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("k", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("c", Types.CHAR, true, false, false),
+                new ShardingSphereColumn("pad", Types.CHAR, true, false, false));
+        ShardingSphereTable tableMetaData = new ShardingSphereTable(TABLE_NAME, columnMetaData, Collections.emptyList(), Collections.emptyList());
         when(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(DATABASE_NAME).getSchemas().get("public").get(TABLE_NAME)).thenReturn(tableMetaData);
         when(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(DATABASE_NAME).getResource().getDatabaseType()).thenReturn(new PostgreSQLDatabaseType());
     }
diff --git a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java
index a361c3d2e25..257dc470163 100644
--- a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java
+++ b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java
@@ -30,8 +30,8 @@ import org.apache.shardingsphere.data.pipeline.spi.ingest.channel.PipelineChanne
 import org.apache.shardingsphere.driver.jdbc.core.datasource.ShardingSphereDataSource;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -114,9 +114,9 @@ public final class PipelineContextUtil {
     }
     
     private static MetaDataContexts renewMetaDataContexts(final MetaDataContexts old, final MetaDataPersistService metaDataPersistService) {
-        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
-        tableMetaDataMap.put("t_order", new TableMetaData("t_order", Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, true, false, false),
-                new ColumnMetaData("user_id", Types.VARCHAR, false, false, false)), Collections.emptyList(), Collections.emptyList()));
+        Map<String, ShardingSphereTable> tableMetaDataMap = new HashMap<>(3, 1);
+        tableMetaDataMap.put("t_order", new ShardingSphereTable("t_order", Arrays.asList(new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false),
+                new ShardingSphereColumn("user_id", Types.VARCHAR, false, false, false)), Collections.emptyList(), Collections.emptyList()));
         old.getMetaData().getDatabases().get(DefaultDatabase.LOGIC_NAME).getSchemas().get(DefaultDatabase.LOGIC_NAME).putAll(tableMetaDataMap);
         return new MetaDataContexts(metaDataPersistService, old.getMetaData(), old.getOptimizerContext());
     }
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteSchemaMetaDataAwareEncryptAlgorithmFixture.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteSchemaMetaDataAwareEncryptAlgorithmFixture.java
index 35082371263..4c7388e79ee 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteSchemaMetaDataAwareEncryptAlgorithmFixture.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/fixture/encrypt/RewriteSchemaMetaDataAwareEncryptAlgorithmFixture.java
@@ -22,7 +22,7 @@ import lombok.Setter;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.spi.context.EncryptContext;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.aware.SchemaMetaDataAware;
 
 import java.util.Map;
@@ -51,7 +51,7 @@ public final class RewriteSchemaMetaDataAwareEncryptAlgorithmFixture implements
     
     @Override
     public Object decrypt(final String cipherValue, final EncryptContext encryptContext) {
-        TableMetaData tableMetaData = schemas.get(databaseName).get(encryptContext.getTableName());
+        ShardingSphereTable tableMetaData = schemas.get(databaseName).get(encryptContext.getTableName());
         return cipherValue.replaceAll("encrypt_", "").replaceAll("_" + tableMetaData.getName(), "");
     }
     
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/EncryptSQLRewriterParameterizedTest.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/EncryptSQLRewriterParameterizedTest.java
index d68ad9b0a1a..493263318a8 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/EncryptSQLRewriterParameterizedTest.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/EncryptSQLRewriterParameterizedTest.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.sharding.rewrite.parameterized.scenario;
 
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.yaml.config.pojo.YamlRootConfiguration;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
@@ -76,7 +76,7 @@ public final class EncryptSQLRewriterParameterizedTest extends AbstractSQLRewrit
         when(result.getAllColumnNames("t_account_bak")).thenReturn(Arrays.asList("account_id", "certificate_number", "password", "amount", "status"));
         when(result.getAllColumnNames("t_account_detail")).thenReturn(Arrays.asList("account_id", "certificate_number", "password", "amount", "status"));
         when(result.getAllColumnNames("t_order")).thenReturn(Arrays.asList("ORDER_ID", "USER_ID", "CONTENT"));
-        when(result.get("t_order")).thenReturn(new TableMetaData("t_order", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        when(result.get("t_order")).thenReturn(new ShardingSphereTable("t_order", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
         return Collections.singletonMap(schemaName, result);
     }
     
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/MixSQLRewriterParameterizedTest.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/MixSQLRewriterParameterizedTest.java
index 73a2d6619f4..bf9cc0228d5 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/MixSQLRewriterParameterizedTest.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/MixSQLRewriterParameterizedTest.java
@@ -18,10 +18,10 @@
 package org.apache.shardingsphere.sharding.rewrite.parameterized.scenario;
 
 import com.google.common.base.Preconditions;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.yaml.config.pojo.YamlRootConfiguration;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
@@ -69,16 +69,16 @@ public final class MixSQLRewriterParameterizedTest extends AbstractSQLRewriterPa
     protected Map<String, ShardingSphereSchema> mockSchemas(final String schemaName) {
         ShardingSphereSchema result = mock(ShardingSphereSchema.class);
         when(result.getAllTableNames()).thenReturn(Arrays.asList("t_account", "t_account_bak", "t_account_detail"));
-        TableMetaData accountTableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable accountTableMetaData = mock(ShardingSphereTable.class);
         when(accountTableMetaData.getColumns()).thenReturn(createColumnMetaDataMap());
-        when(accountTableMetaData.getIndexes()).thenReturn(Collections.singletonMap("index_name", new IndexMetaData("index_name")));
+        when(accountTableMetaData.getIndexes()).thenReturn(Collections.singletonMap("index_name", new ShardingSphereIndex("index_name")));
         when(result.containsTable("t_account")).thenReturn(true);
         when(result.get("t_account")).thenReturn(accountTableMetaData);
-        TableMetaData accountBakTableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable accountBakTableMetaData = mock(ShardingSphereTable.class);
         when(accountBakTableMetaData.getColumns()).thenReturn(createColumnMetaDataMap());
         when(result.containsTable("t_account_bak")).thenReturn(true);
         when(result.get("t_account_bak")).thenReturn(accountBakTableMetaData);
-        when(result.get("t_account_detail")).thenReturn(mock(TableMetaData.class));
+        when(result.get("t_account_detail")).thenReturn(mock(ShardingSphereTable.class));
         when(result.getAllColumnNames("t_account")).thenReturn(Arrays.asList("account_id", "password", "amount", "status"));
         when(result.getAllColumnNames("t_account_bak")).thenReturn(Arrays.asList("account_id", "password", "amount", "status"));
         return Collections.singletonMap(schemaName, result);
@@ -92,12 +92,12 @@ public final class MixSQLRewriterParameterizedTest extends AbstractSQLRewriterPa
     protected void mockDataSource(final Map<String, DataSource> dataSources) {
     }
     
-    private Map<String, ColumnMetaData> createColumnMetaDataMap() {
-        Map<String, ColumnMetaData> result = new LinkedHashMap<>(4, 1);
-        result.put("account_id", new ColumnMetaData("account_id", Types.INTEGER, true, true, false));
-        result.put("password", mock(ColumnMetaData.class));
-        result.put("amount", mock(ColumnMetaData.class));
-        result.put("status", mock(ColumnMetaData.class));
+    private Map<String, ShardingSphereColumn> createColumnMetaDataMap() {
+        Map<String, ShardingSphereColumn> result = new LinkedHashMap<>(4, 1);
+        result.put("account_id", new ShardingSphereColumn("account_id", Types.INTEGER, true, true, false));
+        result.put("password", mock(ShardingSphereColumn.class));
+        result.put("amount", mock(ShardingSphereColumn.class));
+        result.put("status", mock(ShardingSphereColumn.class));
         return result;
     }
 }
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/ShardingSQLRewriterParameterizedTest.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/ShardingSQLRewriterParameterizedTest.java
index befd02d4a1b..7f23df5a22c 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/ShardingSQLRewriterParameterizedTest.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/scenario/ShardingSQLRewriterParameterizedTest.java
@@ -18,10 +18,10 @@
 package org.apache.shardingsphere.sharding.rewrite.parameterized.scenario;
 
 import com.google.common.base.Preconditions;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereIndex;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
-import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.yaml.config.pojo.YamlRootConfiguration;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
@@ -81,13 +81,13 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
     protected Map<String, ShardingSphereSchema> mockSchemas(final String schemaName) {
         ShardingSphereSchema result = mock(ShardingSphereSchema.class);
         when(result.getAllTableNames()).thenReturn(Arrays.asList("t_account", "t_account_detail"));
-        TableMetaData accountTableMetaData = mock(TableMetaData.class);
+        ShardingSphereTable accountTableMetaData = mock(ShardingSphereTable.class);
         when(accountTableMetaData.getColumns()).thenReturn(createColumnMetaDataMap());
-        when(accountTableMetaData.getIndexes()).thenReturn(Collections.singletonMap("status_idx_exist", new IndexMetaData("status_idx_exist")));
+        when(accountTableMetaData.getIndexes()).thenReturn(Collections.singletonMap("status_idx_exist", new ShardingSphereIndex("status_idx_exist")));
         when(accountTableMetaData.getPrimaryKeyColumns()).thenReturn(Collections.singletonList("account_id"));
         when(result.containsTable("t_account")).thenReturn(true);
         when(result.get("t_account")).thenReturn(accountTableMetaData);
-        when(result.get("t_account_detail")).thenReturn(mock(TableMetaData.class));
+        when(result.get("t_account_detail")).thenReturn(mock(ShardingSphereTable.class));
         when(result.getAllColumnNames("t_account")).thenReturn(new ArrayList<>(Arrays.asList("account_id", "amount", "status")));
         when(result.getAllColumnNames("t_user")).thenReturn(new ArrayList<>(Arrays.asList("id", "content")));
         when(result.getAllColumnNames("t_user_extend")).thenReturn(new ArrayList<>(Arrays.asList("user_id", "content")));
@@ -95,11 +95,11 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
         return Collections.singletonMap(schemaName, result);
     }
     
-    private Map<String, ColumnMetaData> createColumnMetaDataMap() {
-        Map<String, ColumnMetaData> result = new LinkedHashMap<>(3, 1);
-        result.put("account_id", new ColumnMetaData("account_id", Types.INTEGER, true, true, false));
-        result.put("amount", mock(ColumnMetaData.class));
-        result.put("status", mock(ColumnMetaData.class));
+    private Map<String, ShardingSphereColumn> createColumnMetaDataMap() {
+        Map<String, ShardingSphereColumn> result = new LinkedHashMap<>(3, 1);
+        result.put("account_id", new ShardingSphereColumn("account_id", Types.INTEGER, true, true, false));
+        result.put("amount", mock(ShardingSphereColumn.class));
+        result.put("status", mock(ShardingSphereColumn.class));
         return result;
     }