You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by me...@apache.org on 2020/11/09 13:42:02 UTC

[shardingsphere] branch master updated: Refactor schema.model package (#8103)

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

menghaoran 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 555155e  Refactor schema.model package (#8103)
555155e is described below

commit 555155ef11bfc220cbd6065704340adb165017d6
Author: Liang Zhang <te...@163.com>
AuthorDate: Mon Nov 9 21:41:35 2020 +0800

    Refactor schema.model package (#8103)
    
    * Remove model.physical package
    
    * Rename schema.model
---
 .../encrypt/metadata/EncryptColumnMetaData.java    |  4 +-
 .../encrypt/metadata/EncryptMetaDataLoader.java    | 22 ++++-----
 .../metadata/EncryptMetaDataLoaderTest.java        | 18 ++++----
 .../merge/dal/impl/EncryptColumnsMergedResult.java |  6 +--
 .../dal/MergedEncryptColumnsMergedResultTest.java  | 22 ++++-----
 .../dql/impl/EncryptColumnsMergedResultTest.java   | 12 ++---
 .../sharding/metadata/ShardingMetaDataLoader.java  | 52 +++++++++++-----------
 .../dql/groupby/GroupByMemoryMergedResult.java     |  8 ++--
 .../sharding/merge/dql/orderby/OrderByValue.java   |  8 ++--
 .../dal/show/ShowCreateTableMergedResultTest.java  |  6 +--
 .../merge/dal/show/ShowTablesMergedResultTest.java |  6 +--
 .../merge/dql/ShardingDQLResultMergerTest.java     | 12 ++---
 .../dql/groupby/GroupByStreamMergedResultTest.java | 12 ++---
 .../dql/orderby/OrderByStreamMergedResultTest.java | 10 ++---
 .../MixSQLRewriterParameterizedTest.java           | 28 ++++++------
 .../ShardingSQLRewriterParameterizedTest.java      | 24 +++++-----
 .../ShardingTableBroadcastRoutingEngineTest.java   | 10 ++---
 .../type/single/SingleTableRoutingEngineTest.java  |  8 ++--
 .../engine/type/standard/AbstractSQLRouteTest.java | 26 +++++------
 .../swapper/LogicSchemaMetaDataYamlSwapper.java    | 32 ++++++-------
 .../keygen/engine/GeneratedKeyContextEngine.java   |  4 +-
 .../engine/GeneratedKeyContextEngineTest.java      |  8 ++--
 .../metadata/schema/ShardingSphereSchema.java      | 10 ++---
 .../schema/loader/SchemaMetaDataLoader.java        |  4 +-
 .../schema/loader/TableMetaDataLoader.java         | 18 ++++----
 .../physical/PhysicalColumnMetaDataLoader.java     |  8 ++--
 .../physical/PhysicalIndexMetaDataLoader.java      |  8 ++--
 .../physical/PhysicalTableMetaDataLoader.java      |  6 +--
 .../loader/spi/ShardingSphereMetaDataLoader.java   |  8 ++--
 ...icalColumnMetaData.java => ColumnMetaData.java} |  6 +--
 ...ysicalIndexMetaData.java => IndexMetaData.java} |  6 +--
 ...ysicalTableMetaData.java => TableMetaData.java} | 28 ++++++------
 .../refresh/TableMetaDataLoaderCallback.java       |  4 +-
 ...reateIndexStatementMetaDataRefreshStrategy.java |  4 +-
 ...reateTableStatementMetaDataRefreshStrategy.java |  6 +--
 ...CreateViewStatementMetaDataRefreshStrategy.java |  4 +-
 .../DropIndexStatementMetaDataRefreshStrategy.java |  4 +-
 .../metadata/schema/ShardingSphereSchemaTest.java  | 20 ++++-----
 .../loader/CommonFixtureLogicMetaDataLoader.java   |  8 ++--
 ...ataNodeContainedFixtureLogicMetaDataLoader.java | 16 +++----
 .../loader/physical/ColumnMetaDataLoaderTest.java  |  8 ++--
 .../loader/physical/IndexMetaDataLoaderTest.java   |  6 +--
 .../loader/physical/TableMetaDataLoaderTest.java   | 16 +++----
 .../schema/model/physical/TableMetaDataTest.java   | 10 +++--
 .../AbstractMetaDataRefreshStrategyTest.java       | 10 ++---
 ...rTableStatementMetaDataRefreshStrategyTest.java | 12 ++---
 ...eTableStatementMetaDataRefreshStrategyTest.java | 12 ++---
 ...pIndexStatementMetaDataRefreshStrategyTest.java |  8 ++--
 .../sql/context/ExecutionContextBuilder.java       |  4 +-
 .../sql/context/ExecutionContextBuilderTest.java   | 34 +++++++-------
 .../jdbc/JDBCDatabaseCommunicationEngine.java      |  4 +-
 .../backend/response/query/QueryHeaderBuilder.java |  4 +-
 .../response/query/QueryHeaderBuilderTest.java     | 10 ++---
 .../hint/ShardingCTLHintBackendHandlerTest.java    |  4 +-
 .../executor/dumper/AbstractJDBCDumper.java        |  6 +--
 .../splitter/InventoryDataTaskSplitter.java        |  4 +-
 .../scaling/core/metadata/MetaDataManager.java     |  6 +--
 .../scaling/core/metadata/MetaDataManagerTest.java |  8 ++--
 .../scaling/mysql/MySQLBinlogDumper.java           |  8 ++--
 .../scaling/postgresql/wal/WalEventConverter.java  |  4 +-
 60 files changed, 343 insertions(+), 341 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptColumnMetaData.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptColumnMetaData.java
index 4d02dee..ad2fc15 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptColumnMetaData.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptColumnMetaData.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.encrypt.metadata;
 import lombok.EqualsAndHashCode;
 import lombok.Getter;
 import lombok.ToString;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 
 /**
  * Column meta data for encrypt.
@@ -28,7 +28,7 @@ import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalCo
 @Getter
 @EqualsAndHashCode(callSuper = true)
 @ToString(callSuper = true)
-public final class EncryptColumnMetaData extends PhysicalColumnMetaData {
+public final class EncryptColumnMetaData extends ColumnMetaData {
     
     private final String cipherColumnName;
     
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java
index 292d2f4..0c65e82 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java
@@ -24,8 +24,8 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.loader.physical.PhysicalTableMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
@@ -40,21 +40,21 @@ import java.util.Optional;
 public final class EncryptMetaDataLoader implements ShardingSphereMetaDataLoader<EncryptRule> {
     
     @Override
-    public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
-                                                final EncryptRule encryptRule, final ConfigurationProperties props) throws SQLException {
+    public Optional<TableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
+                                        final EncryptRule encryptRule, final ConfigurationProperties props) throws SQLException {
         return encryptRule.findEncryptTable(tableName).isPresent() ? PhysicalTableMetaDataLoader.load(dataSourceMap.values().iterator().next(), tableName, databaseType) : Optional.empty();
     }
     
     @Override
-    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final EncryptRule encryptRule) {
-        return new PhysicalTableMetaData(getEncryptColumnMetaDataList(tableName, tableMetaData.getColumns().values(), encryptRule), tableMetaData.getIndexes().values());
+    public TableMetaData decorate(final String tableName, final TableMetaData tableMetaData, final EncryptRule encryptRule) {
+        return new TableMetaData(getEncryptColumnMetaDataList(tableName, tableMetaData.getColumns().values(), encryptRule), tableMetaData.getIndexes().values());
     }
     
-    private Collection<PhysicalColumnMetaData> getEncryptColumnMetaDataList(final String tableName,
-                                                                            final Collection<PhysicalColumnMetaData> originalColumnMetaDataList, final EncryptRule encryptRule) {
-        Collection<PhysicalColumnMetaData> result = new LinkedList<>();
+    private Collection<ColumnMetaData> getEncryptColumnMetaDataList(final String tableName,
+                                                                    final Collection<ColumnMetaData> originalColumnMetaDataList, final EncryptRule encryptRule) {
+        Collection<ColumnMetaData> result = new LinkedList<>();
         Collection<String> derivedColumns = encryptRule.getAssistedQueryAndPlainColumns(tableName);
-        for (PhysicalColumnMetaData each : originalColumnMetaDataList) {
+        for (ColumnMetaData each : originalColumnMetaDataList) {
             if (!derivedColumns.contains(each.getName())) {
                 result.add(getEncryptColumnMetaData(tableName, each, encryptRule));
             }
@@ -62,7 +62,7 @@ public final class EncryptMetaDataLoader implements ShardingSphereMetaDataLoader
         return result;
     }
     
-    private PhysicalColumnMetaData getEncryptColumnMetaData(final String tableName, final PhysicalColumnMetaData originalColumnMetaData, final EncryptRule encryptRule) {
+    private ColumnMetaData getEncryptColumnMetaData(final String tableName, final ColumnMetaData originalColumnMetaData, final EncryptRule encryptRule) {
         if (!encryptRule.isCipherColumn(tableName, originalColumnMetaData.getName())) {
             return originalColumnMetaData;
         }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java
index bc4b157..0dad0ae 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java
@@ -23,8 +23,8 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.junit.Before;
@@ -98,7 +98,7 @@ public final class EncryptMetaDataLoaderTest {
     public void assertLoadByExistedTable() throws SQLException {
         EncryptRule rule = createEncryptRule();
         EncryptMetaDataLoader loader = (EncryptMetaDataLoader) OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(rule), ShardingSphereMetaDataLoader.class).get(rule);
-        Optional<PhysicalTableMetaData> actual = loader.load(TABLE_NAME, databaseType, Collections.singletonMap("logic_db", dataSource), new DataNodes(Collections.singletonList(rule)), rule, props);
+        Optional<TableMetaData> actual = loader.load(TABLE_NAME, databaseType, Collections.singletonMap("logic_db", dataSource), new DataNodes(Collections.singletonList(rule)), rule, props);
         assertTrue(actual.isPresent());
         assertThat(actual.get().getColumnMetaData(0).getName(), is("id"));
         assertThat(actual.get().getColumnMetaData(1).getName(), is("pwd_cipher"));
@@ -109,7 +109,7 @@ public final class EncryptMetaDataLoaderTest {
     public void assertLoadByNotExistedTable() throws SQLException {
         EncryptRule rule = createEncryptRule();
         EncryptMetaDataLoader loader = new EncryptMetaDataLoader();
-        Optional<PhysicalTableMetaData> actual = loader.load(
+        Optional<TableMetaData> actual = loader.load(
                 "not_existed_table", databaseType, Collections.singletonMap("logic_db", dataSource), new DataNodes(Collections.singletonList(rule)), rule, props);
         assertFalse(actual.isPresent());
     }
@@ -118,7 +118,7 @@ public final class EncryptMetaDataLoaderTest {
     public void assertDecorate() {
         EncryptRule rule = createEncryptRule();
         EncryptMetaDataLoader loader = (EncryptMetaDataLoader) OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(rule), ShardingSphereMetaDataLoader.class).get(rule);
-        PhysicalTableMetaData actual = loader.decorate("t_encrypt", createTableMetaData(), rule);
+        TableMetaData actual = loader.decorate("t_encrypt", createTableMetaData(), rule);
         assertThat(actual.getColumns().size(), is(2));
         assertTrue(actual.getColumns().containsKey("id"));
         assertTrue(actual.getColumns().containsKey("pwd"));
@@ -133,9 +133,9 @@ public final class EncryptMetaDataLoaderTest {
         return result;
     }
     
-    private PhysicalTableMetaData createTableMetaData() {
-        Collection<PhysicalColumnMetaData> columns = Arrays.asList(new PhysicalColumnMetaData("id", 1, "int", true, true, true),
-                new PhysicalColumnMetaData("pwd_cipher", 2, "varchar", false, false, true), new PhysicalColumnMetaData("pwd_plain", 2, "varchar", false, false, true));
-        return new PhysicalTableMetaData(columns, Collections.emptyList());
+    private TableMetaData createTableMetaData() {
+        Collection<ColumnMetaData> columns = Arrays.asList(new ColumnMetaData("id", 1, "int", true, true, true),
+                new ColumnMetaData("pwd_cipher", 2, "varchar", false, false, true), new ColumnMetaData("pwd_plain", 2, "varchar", false, false, true));
+        return new TableMetaData(columns, Collections.emptyList());
     }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/main/java/org/apache/shardingsphere/encrypt/merge/dal/impl/EncryptColumnsMergedResult.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/main/java/org/apache/shardingsphere/encrypt/merge/dal/impl/EncryptColumnsMergedResult.java
index 7908a1e..748348d 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/main/java/org/apache/shardingsphere/encrypt/merge/dal/impl/EncryptColumnsMergedResult.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/main/java/org/apache/shardingsphere/encrypt/merge/dal/impl/EncryptColumnsMergedResult.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.encrypt.merge.dal.impl;
 
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.encrypt.metadata.EncryptColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
@@ -82,7 +82,7 @@ public abstract class EncryptColumnsMergedResult implements MergedResult {
     
     private Collection<EncryptColumnMetaData> getTableEncryptColumnMetaDataList() {
         Collection<EncryptColumnMetaData> result = new LinkedList<>();
-        for (Entry<String, PhysicalColumnMetaData> entry : schema.get(tableName).getColumns().entrySet()) {
+        for (Entry<String, ColumnMetaData> entry : schema.get(tableName).getColumns().entrySet()) {
             if (entry.getValue() instanceof EncryptColumnMetaData) {
                 result.add((EncryptColumnMetaData) entry.getValue());
             }
@@ -101,7 +101,7 @@ public abstract class EncryptColumnsMergedResult implements MergedResult {
     }
     
     private Optional<String> getLogicColumnOfCipher(final String cipherColumn) {
-        for (Entry<String, PhysicalColumnMetaData> entry : schema.get(tableName).getColumns().entrySet()) {
+        for (Entry<String, ColumnMetaData> entry : schema.get(tableName).getColumns().entrySet()) {
             if (entry.getValue() instanceof EncryptColumnMetaData) {
                 EncryptColumnMetaData encryptColumnMetaData = (EncryptColumnMetaData) entry.getValue();
                 if (encryptColumnMetaData.getCipherColumnName().equalsIgnoreCase(cipherColumn)) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dal/MergedEncryptColumnsMergedResultTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dal/MergedEncryptColumnsMergedResultTest.java
index 40dd0ce..e996e7a 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dal/MergedEncryptColumnsMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dal/MergedEncryptColumnsMergedResultTest.java
@@ -21,7 +21,7 @@ import org.apache.shardingsphere.encrypt.merge.dal.impl.MergedEncryptColumnsMerg
 import org.apache.shardingsphere.encrypt.metadata.EncryptColumnMetaData;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
@@ -53,8 +53,8 @@ public final class MergedEncryptColumnsMergedResultTest {
     @Test
     public void assertNextWithTableEncryptColumnMetaDataListEmpty() throws SQLException {
         when(queryResult.next()).thenReturn(true);
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList());
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(1, 1);
+        TableMetaData tableMetaData = new TableMetaData(Collections.emptyList(), Collections.emptyList());
+        Map<String, TableMetaData> tables = new HashMap<>(1, 1);
         tables.put("test", tableMetaData);
         assertTrue(createMergedEncryptColumnsMergedResult(queryResult, new ShardingSphereSchema(tables)).next());
     }
@@ -69,8 +69,8 @@ public final class MergedEncryptColumnsMergedResultTest {
         when(queryResult.next()).thenReturn(true).thenReturn(false);
         when(queryResult.getValue(1, String.class)).thenReturn("assistedQuery");
         EncryptColumnMetaData encryptColumnMetaData = new EncryptColumnMetaData("id", Types.VARCHAR, "varchar", true, "cipher", "plain", "assistedQuery");
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(1, 1);
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
+        Map<String, TableMetaData> tables = new HashMap<>(1, 1);
         tables.put("test", tableMetaData);
         assertFalse(createMergedEncryptColumnsMergedResult(queryResult, new ShardingSphereSchema(tables)).next());
     }
@@ -79,8 +79,8 @@ public final class MergedEncryptColumnsMergedResultTest {
     public void assertGetValueWithCipherColumn() throws SQLException {
         when(queryResult.getValue(1, String.class)).thenReturn("cipher");
         EncryptColumnMetaData encryptColumnMetaData = new EncryptColumnMetaData("id", Types.VARCHAR, "varchar", true, "cipher", "plain", "assistedQuery");
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(1);
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
+        Map<String, TableMetaData> tables = new HashMap<>(1);
         tables.put("test", tableMetaData);
         assertThat(createMergedEncryptColumnsMergedResult(queryResult, new ShardingSphereSchema(tables)).getValue(1, String.class), is("id"));
     }
@@ -89,8 +89,8 @@ public final class MergedEncryptColumnsMergedResultTest {
     public void assertGetValueWithOtherColumn() throws SQLException {
         when(queryResult.getValue(1, String.class)).thenReturn("assistedQuery");
         EncryptColumnMetaData encryptColumnMetaData = new EncryptColumnMetaData("id", Types.VARCHAR, "varchar", true, "cipher", "plain", "assistedQuery");
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(1, 1);
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
+        Map<String, TableMetaData> tables = new HashMap<>(1, 1);
         tables.put("test", tableMetaData);
         assertThat(createMergedEncryptColumnsMergedResult(queryResult, new ShardingSphereSchema(tables)).getValue(1, String.class), is("assistedQuery"));
     }
@@ -99,8 +99,8 @@ public final class MergedEncryptColumnsMergedResultTest {
     public void assertGetValueWithOtherIndex() throws SQLException {
         when(queryResult.getValue(2, String.class)).thenReturn("id");
         EncryptColumnMetaData encryptColumnMetaData = new EncryptColumnMetaData("id", Types.VARCHAR, "varchar", true, "cipher", "plain", "assistedQuery");
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(1, 1);
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(encryptColumnMetaData), Collections.emptyList());
+        Map<String, TableMetaData> tables = new HashMap<>(1, 1);
         tables.put("test", tableMetaData);
         assertThat(createMergedEncryptColumnsMergedResult(queryResult, new ShardingSphereSchema(tables)).getValue(2, String.class), is("id"));
     }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/impl/EncryptColumnsMergedResultTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/impl/EncryptColumnsMergedResultTest.java
index da9c2f2..66fe8be 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/impl/EncryptColumnsMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-merge/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/impl/EncryptColumnsMergedResultTest.java
@@ -21,9 +21,9 @@ import com.google.common.collect.Lists;
 import org.apache.shardingsphere.encrypt.merge.dql.fixture.EncryptColumnsMergedResultFixture;
 import org.apache.shardingsphere.encrypt.merge.dql.fixture.TableAvailableAndSqlStatementContextFixture;
 import org.apache.shardingsphere.encrypt.metadata.EncryptColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
@@ -57,13 +57,13 @@ public final class EncryptColumnsMergedResultTest {
     private ShardingSphereSchema schema;
     
     @Mock
-    private PhysicalTableMetaData tableMetaData;
+    private TableMetaData tableMetaData;
     
     private EncryptColumnsMergedResultFixture encryptColumnsMergedResultFixture;
     
     @Before
     public void setUp() { 
-        Map<String, PhysicalColumnMetaData> columns = new HashMap<>(1, 1);
+        Map<String, ColumnMetaData> columns = new HashMap<>(1, 1);
         EncryptColumnMetaData encryptColumnMetaData = new EncryptColumnMetaData("order", 1, "Integer", false, "status", "status", "status");
         columns.put("", encryptColumnMetaData);
         SimpleTableSegment simpleTableSegment = mock(SimpleTableSegment.class);
@@ -97,7 +97,7 @@ public final class EncryptColumnsMergedResultTest {
     
     @Test
     public void assertContainerColumnName() throws SQLException {
-        Map<String, PhysicalColumnMetaData> columns = new HashMap<>(1, 1);
+        Map<String, ColumnMetaData> columns = new HashMap<>(1, 1);
         EncryptColumnMetaData encryptColumnMetaData = new EncryptColumnMetaData("order", 1, "Integer", false, "status", "status", "status");
         columns.put("", encryptColumnMetaData);
         when(schema.get(anyString())).thenReturn(tableMetaData);
@@ -110,7 +110,7 @@ public final class EncryptColumnsMergedResultTest {
     
     @Test
     public void assertGetValueWithColumnIndex() throws SQLException {
-        Map<String, PhysicalColumnMetaData> columns = new HashMap<>(1, 1);
+        Map<String, ColumnMetaData> columns = new HashMap<>(1, 1);
         EncryptColumnMetaData encryptColumnMetaData = new EncryptColumnMetaData("order", 1, "Integer", false, "status", "status", "status");
         columns.put("key", encryptColumnMetaData);
         when(schema.get(anyString())).thenReturn(tableMetaData);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java
index 4962258..be7720e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java
@@ -27,9 +27,9 @@ import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.schema.loader.physical.PhysicalTableMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
@@ -62,8 +62,8 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
     private static final int FUTURE_GET_TIME_OUT_SECOND = 5;
     
     @Override
-    public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
-                                                final ShardingRule rule, final ConfigurationProperties props) throws SQLException {
+    public Optional<TableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
+                                        final ShardingRule rule, final ConfigurationProperties props) throws SQLException {
         if (!rule.findTableRule(tableName).isPresent()) {
             return Optional.empty();
         }
@@ -74,7 +74,7 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
             DataNode dataNode = dataNodes.getDataNodes(tableName).iterator().next();
             return PhysicalTableMetaDataLoader.load(dataSourceMap.get(dataNode.getDataSourceName()), dataNode.getTableName(), databaseType);
         }
-        Map<String, PhysicalTableMetaData> actualTableMetaDataMap = parallelLoadTables(databaseType, dataSourceMap, dataNodes, tableName, maxConnectionsSizePerQuery);
+        Map<String, TableMetaData> actualTableMetaDataMap = parallelLoadTables(databaseType, dataSourceMap, dataNodes, tableName, maxConnectionsSizePerQuery);
         if (actualTableMetaDataMap.isEmpty()) {
             return Optional.empty();
         }
@@ -82,15 +82,15 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
         return Optional.of(actualTableMetaDataMap.values().iterator().next());
     }
     
-    private Map<String, PhysicalTableMetaData> parallelLoadTables(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
-                                                                  final String tableName, final int maxConnectionsSizePerQuery) {
+    private Map<String, TableMetaData> parallelLoadTables(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
+                                                          final String tableName, final int maxConnectionsSizePerQuery) {
         Map<String, List<DataNode>> dataNodeGroups = dataNodes.getDataNodeGroups(tableName);
-        Map<String, PhysicalTableMetaData> result = new HashMap<>(dataNodeGroups.size(), 1);
-        Map<String, Future<Optional<PhysicalTableMetaData>>> tableFutureMap = new HashMap<>(dataNodeGroups.size(), 1);
+        Map<String, TableMetaData> result = new HashMap<>(dataNodeGroups.size(), 1);
+        Map<String, Future<Optional<TableMetaData>>> tableFutureMap = new HashMap<>(dataNodeGroups.size(), 1);
         ExecutorService executorService = Executors.newFixedThreadPool(Math.min(CPU_CORES * 2, dataNodeGroups.size() * maxConnectionsSizePerQuery));
         for (Entry<String, List<DataNode>> entry : dataNodeGroups.entrySet()) {
             for (DataNode each : entry.getValue()) {
-                Future<Optional<PhysicalTableMetaData>> futures = executorService.submit(() -> loadTableByDataNode(each, databaseType, dataSourceMap));
+                Future<Optional<TableMetaData>> futures = executorService.submit(() -> loadTableByDataNode(each, databaseType, dataSourceMap));
                 tableFutureMap.put(each.getTableName(), futures);
             }
         }
@@ -105,11 +105,11 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
         return result;
     }
     
-    private Optional<PhysicalTableMetaData> getTableMetaData(final Future<Optional<PhysicalTableMetaData>> value) throws InterruptedException, ExecutionException, TimeoutException {
+    private Optional<TableMetaData> getTableMetaData(final Future<Optional<TableMetaData>> value) throws InterruptedException, ExecutionException, TimeoutException {
         return value.get(FUTURE_GET_TIME_OUT_SECOND, TimeUnit.SECONDS);
     }
     
-    private Optional<PhysicalTableMetaData> loadTableByDataNode(final DataNode dataNode, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap) {
+    private Optional<TableMetaData> loadTableByDataNode(final DataNode dataNode, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap) {
         try {
             return PhysicalTableMetaDataLoader.load(dataSourceMap.get(dataNode.getDataSourceName()), dataNode.getTableName(), databaseType);
         } catch (final SQLException ex) {
@@ -117,8 +117,8 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
         }
     }
     
-    private void checkUniformed(final String logicTableName, final Map<String, PhysicalTableMetaData> actualTableMetaDataMap, final ShardingRule shardingRule) {
-        PhysicalTableMetaData sample = decorate(logicTableName, actualTableMetaDataMap.values().iterator().next(), shardingRule);
+    private void checkUniformed(final String logicTableName, final Map<String, TableMetaData> actualTableMetaDataMap, final ShardingRule shardingRule) {
+        TableMetaData sample = decorate(logicTableName, actualTableMetaDataMap.values().iterator().next(), shardingRule);
         Collection<TableMetaDataViolation> violations = actualTableMetaDataMap.entrySet().stream()
                 .filter(entry -> !sample.equals(decorate(logicTableName, entry.getValue(), shardingRule)))
                 .map(entry -> new TableMetaDataViolation(entry.getKey(), entry.getValue())).collect(Collectors.toList());
@@ -137,20 +137,20 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
     }
 
     @Override
-    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final ShardingRule shardingRule) {
+    public TableMetaData decorate(final String tableName, final TableMetaData tableMetaData, final ShardingRule shardingRule) {
         return shardingRule.findTableRule(tableName).map(
-            tableRule -> new PhysicalTableMetaData(getColumnMetaDataList(tableMetaData, tableRule), getIndexMetaDataList(tableMetaData, tableRule))).orElse(tableMetaData);
+            tableRule -> new TableMetaData(getColumnMetaDataList(tableMetaData, tableRule), getIndexMetaDataList(tableMetaData, tableRule))).orElse(tableMetaData);
     }
     
-    private Collection<PhysicalColumnMetaData> getColumnMetaDataList(final PhysicalTableMetaData tableMetaData, final TableRule tableRule) {
+    private Collection<ColumnMetaData> getColumnMetaDataList(final TableMetaData tableMetaData, final TableRule tableRule) {
         Optional<String> generateKeyColumn = tableRule.getGenerateKeyColumn();
         if (!generateKeyColumn.isPresent()) {
             return tableMetaData.getColumns().values();
         }
-        Collection<PhysicalColumnMetaData> result = new LinkedList<>();
-        for (Entry<String, PhysicalColumnMetaData> entry : tableMetaData.getColumns().entrySet()) {
+        Collection<ColumnMetaData> result = new LinkedList<>();
+        for (Entry<String, ColumnMetaData> entry : tableMetaData.getColumns().entrySet()) {
             if (entry.getKey().equalsIgnoreCase(generateKeyColumn.get())) {
-                result.add(new PhysicalColumnMetaData(
+                result.add(new ColumnMetaData(
                         entry.getValue().getName(), entry.getValue().getDataType(), entry.getValue().getDataTypeName(), entry.getValue().isPrimaryKey(), true, entry.getValue().isCaseSensitive()));
             } else {
                 result.add(entry.getValue());
@@ -159,11 +159,11 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
         return result;
     }
     
-    private Collection<PhysicalIndexMetaData> getIndexMetaDataList(final PhysicalTableMetaData tableMetaData, final TableRule tableRule) {
-        Collection<PhysicalIndexMetaData> result = new HashSet<>();
-        for (Entry<String, PhysicalIndexMetaData> entry : tableMetaData.getIndexes().entrySet()) {
+    private Collection<IndexMetaData> getIndexMetaDataList(final TableMetaData tableMetaData, final TableRule tableRule) {
+        Collection<IndexMetaData> result = new HashSet<>();
+        for (Entry<String, IndexMetaData> entry : tableMetaData.getIndexes().entrySet()) {
             for (DataNode each : tableRule.getActualDataNodes()) {
-                getLogicIndex(entry.getKey(), each.getTableName()).ifPresent(logicIndex -> result.add(new PhysicalIndexMetaData(logicIndex)));
+                getLogicIndex(entry.getKey(), each.getTableName()).ifPresent(logicIndex -> result.add(new IndexMetaData(logicIndex)));
             }
         }
         return result;
@@ -190,6 +190,6 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
         
         private final String actualTableName;
         
-        private final PhysicalTableMetaData tableMetaData;
+        private final TableMetaData tableMetaData;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
index e9106df..c765100 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
@@ -26,9 +26,9 @@ import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryQueryResul
 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;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.Projection;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.AggregationDistinctProjection;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.AggregationProjection;
@@ -128,8 +128,8 @@ public final class GroupByMemoryMergedResult extends MemoryMergedResult<Sharding
                                                     final SelectStatementContext selectStatementContext, final ShardingSphereSchema schema, final int columnIndex) throws SQLException {
         for (SimpleTableSegment each : selectStatementContext.getSimpleTableSegments()) {
             String tableName = each.getTableName().getIdentifier().getValue();
-            PhysicalTableMetaData tableMetaData = schema.get(tableName);
-            Map<String, PhysicalColumnMetaData> columns = tableMetaData.getColumns();
+            TableMetaData tableMetaData = schema.get(tableName);
+            Map<String, ColumnMetaData> columns = tableMetaData.getColumns();
             String columnName = queryResult.getColumnName(columnIndex);
             if (columns.containsKey(columnName)) {
                 return columns.get(columnName).isCaseSensitive();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java
index 0a9bc39..cfa8f69 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/main/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByValue.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.sharding.merge.dql.orderby;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.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.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
@@ -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();
-            PhysicalTableMetaData tableMetaData = schema.get(tableName);
-            Map<String, PhysicalColumnMetaData> columns = tableMetaData.getColumns();
+            TableMetaData tableMetaData = schema.get(tableName);
+            Map<String, ColumnMetaData> 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-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java
index 432c4de..8e2fb7a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowCreateTableMergedResultTest.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sharding.merge.dal.show;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
@@ -57,8 +57,8 @@ public final class ShowCreateTableMergedResultTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>(1, 1);
-        tableMetaDataMap.put("table", new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList()));
+        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(1, 1);
+        tableMetaDataMap.put("table", new TableMetaData(Collections.emptyList(), Collections.emptyList()));
         return new ShardingSphereSchema(tableMetaDataMap);
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java
index d5dedb8..0fc491e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dal/show/ShowTablesMergedResultTest.java
@@ -21,7 +21,7 @@ import com.google.common.collect.Lists;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
@@ -58,8 +58,8 @@ public final class ShowTablesMergedResultTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>(1, 1);
-        tableMetaDataMap.put("table", new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList()));
+        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(1, 1);
+        tableMetaDataMap.put("table", new TableMetaData(Collections.emptyList(), Collections.emptyList()));
         return new ShardingSphereSchema(tableMetaDataMap);
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java
index 2cfc348..263f531 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/ShardingDQLResultMergerTest.java
@@ -28,9 +28,9 @@ import org.apache.shardingsphere.sharding.merge.dql.orderby.OrderByStreamMergedR
 import org.apache.shardingsphere.sharding.merge.dql.pagination.LimitDecoratorMergedResult;
 import org.apache.shardingsphere.sharding.merge.dql.pagination.RowNumberDecoratorMergedResult;
 import org.apache.shardingsphere.sharding.merge.dql.pagination.TopAndRowNumberDecoratorMergedResult;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.segment.select.groupby.GroupByContext;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByContext;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByItem;
@@ -336,10 +336,10 @@ public final class ShardingDQLResultMergerTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        PhysicalColumnMetaData columnMetaData1 = new PhysicalColumnMetaData("col1", 0, "dataType", false, false, false);
-        PhysicalColumnMetaData columnMetaData2 = new PhysicalColumnMetaData("col2", 0, "dataType", false, false, false);
-        PhysicalColumnMetaData columnMetaData3 = new PhysicalColumnMetaData("col3", 0, "dataType", false, false, false);
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList());
+        ColumnMetaData columnMetaData1 = new ColumnMetaData("col1", 0, "dataType", false, false, false);
+        ColumnMetaData columnMetaData2 = new ColumnMetaData("col2", 0, "dataType", false, false, false);
+        ColumnMetaData columnMetaData3 = new ColumnMetaData("col3", 0, "dataType", false, false, false);
+        TableMetaData tableMetaData = new TableMetaData(Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList());
         return new ShardingSphereSchema(ImmutableMap.of("tbl", tableMetaData));
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java
index b39b435..fe209f8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByStreamMergedResultTest.java
@@ -22,9 +22,9 @@ import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.sharding.merge.dql.ShardingDQLResultMerger;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.segment.select.groupby.GroupByContext;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByContext;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByItem;
@@ -178,10 +178,10 @@ public final class GroupByStreamMergedResultTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        PhysicalColumnMetaData columnMetaData1 = new PhysicalColumnMetaData("col1", 0, "dataType", false, false, false);
-        PhysicalColumnMetaData columnMetaData2 = new PhysicalColumnMetaData("col2", 0, "dataType", false, false, false);
-        PhysicalColumnMetaData columnMetaData3 = new PhysicalColumnMetaData("col3", 0, "dataType", false, false, false);
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList());
+        ColumnMetaData columnMetaData1 = new ColumnMetaData("col1", 0, "dataType", false, false, false);
+        ColumnMetaData columnMetaData2 = new ColumnMetaData("col2", 0, "dataType", false, false, false);
+        ColumnMetaData columnMetaData3 = new ColumnMetaData("col3", 0, "dataType", false, false, false);
+        TableMetaData tableMetaData = new TableMetaData(Arrays.asList(columnMetaData1, columnMetaData2, columnMetaData3), Collections.emptyList());
         return new ShardingSphereSchema(ImmutableMap.of("tbl", tableMetaData));
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java
index 2080b79..204a33a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-merge/src/test/java/org/apache/shardingsphere/sharding/merge/dql/orderby/OrderByStreamMergedResultTest.java
@@ -22,9 +22,9 @@ import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.sharding.merge.dql.ShardingDQLResultMerger;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.segment.select.groupby.GroupByContext;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByContext;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByItem;
@@ -183,9 +183,9 @@ public final class OrderByStreamMergedResultTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        PhysicalColumnMetaData columnMetaData1 = new PhysicalColumnMetaData("col1", 0, "dataType", false, false, true);
-        PhysicalColumnMetaData columnMetaData2 = new PhysicalColumnMetaData("col2", 0, "dataType", false, false, false);
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Arrays.asList(columnMetaData1, columnMetaData2), Collections.emptyList());
+        ColumnMetaData columnMetaData1 = new ColumnMetaData("col1", 0, "dataType", false, false, true);
+        ColumnMetaData columnMetaData2 = new ColumnMetaData("col2", 0, "dataType", false, false, false);
+        TableMetaData tableMetaData = new TableMetaData(Arrays.asList(columnMetaData1, columnMetaData2), Collections.emptyList());
         return new ShardingSphereSchema(ImmutableMap.of("tbl", tableMetaData));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
index 415d6d5..e9cf0f1 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
@@ -25,10 +25,10 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
@@ -102,29 +102,29 @@ public final class MixSQLRewriterParameterizedTest extends AbstractSQLRewriterPa
     private ShardingSphereSchema mockSchema() {
         ShardingSphereSchema result = mock(ShardingSphereSchema.class);
         when(result.getAllTableNames()).thenReturn(Arrays.asList("t_account", "t_account_bak", "t_account_detail"));
-        PhysicalTableMetaData accountTableMetaData = mock(PhysicalTableMetaData.class);
+        TableMetaData accountTableMetaData = mock(TableMetaData.class);
         when(accountTableMetaData.getColumns()).thenReturn(createColumnMetaDataMap());
-        Map<String, PhysicalIndexMetaData> indexMetaDataMap = new HashMap<>(1, 1);
-        indexMetaDataMap.put("index_name", new PhysicalIndexMetaData("index_name"));
+        Map<String, IndexMetaData> indexMetaDataMap = new HashMap<>(1, 1);
+        indexMetaDataMap.put("index_name", new IndexMetaData("index_name"));
         when(accountTableMetaData.getIndexes()).thenReturn(indexMetaDataMap);
         when(result.containsTable("t_account")).thenReturn(true);
         when(result.get("t_account")).thenReturn(accountTableMetaData);
-        PhysicalTableMetaData accountBakTableMetaData = mock(PhysicalTableMetaData.class);
+        TableMetaData accountBakTableMetaData = mock(TableMetaData.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(PhysicalTableMetaData.class));
+        when(result.get("t_account_detail")).thenReturn(mock(TableMetaData.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 result;
     }
     
-    private Map<String, PhysicalColumnMetaData> createColumnMetaDataMap() {
-        Map<String, PhysicalColumnMetaData> result = new LinkedHashMap<>(4, 1);
-        result.put("account_id", new PhysicalColumnMetaData("account_id", Types.INTEGER, "INT", true, true, false));
-        result.put("password", mock(PhysicalColumnMetaData.class));
-        result.put("amount", mock(PhysicalColumnMetaData.class));
-        result.put("status", mock(PhysicalColumnMetaData.class));
+    private Map<String, ColumnMetaData> createColumnMetaDataMap() {
+        Map<String, ColumnMetaData> result = new LinkedHashMap<>(4, 1);
+        result.put("account_id", new ColumnMetaData("account_id", Types.INTEGER, "INT", true, true, false));
+        result.put("password", mock(ColumnMetaData.class));
+        result.put("amount", mock(ColumnMetaData.class));
+        result.put("status", mock(ColumnMetaData.class));
         return result;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
index a65099b..b99981c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
@@ -25,10 +25,10 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
@@ -102,23 +102,23 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
     private ShardingSphereSchema mockSchema() {
         ShardingSphereSchema result = mock(ShardingSphereSchema.class);
         when(result.getAllTableNames()).thenReturn(Arrays.asList("t_account", "t_account_detail"));
-        PhysicalTableMetaData accountTableMetaData = mock(PhysicalTableMetaData.class);
+        TableMetaData accountTableMetaData = mock(TableMetaData.class);
         when(accountTableMetaData.getColumns()).thenReturn(createColumnMetaDataMap());
-        Map<String, PhysicalIndexMetaData> indexMetaDataMap = new HashMap<>(1, 1);
-        indexMetaDataMap.put("index_name", new PhysicalIndexMetaData("index_name"));
+        Map<String, IndexMetaData> indexMetaDataMap = new HashMap<>(1, 1);
+        indexMetaDataMap.put("index_name", new IndexMetaData("index_name"));
         when(accountTableMetaData.getIndexes()).thenReturn(indexMetaDataMap);
         when(result.containsTable("t_account")).thenReturn(true);
         when(result.get("t_account")).thenReturn(accountTableMetaData);
-        when(result.get("t_account_detail")).thenReturn(mock(PhysicalTableMetaData.class));
+        when(result.get("t_account_detail")).thenReturn(mock(TableMetaData.class));
         when(result.getAllColumnNames("t_account")).thenReturn(Arrays.asList("account_id", "amount", "status"));
         return result;
     }
     
-    private Map<String, PhysicalColumnMetaData> createColumnMetaDataMap() {
-        Map<String, PhysicalColumnMetaData> result = new LinkedHashMap<>(3, 1);
-        result.put("account_id", new PhysicalColumnMetaData("account_id", Types.INTEGER, "INT", true, true, false));
-        result.put("amount", mock(PhysicalColumnMetaData.class));
-        result.put("status", mock(PhysicalColumnMetaData.class));
+    private Map<String, ColumnMetaData> createColumnMetaDataMap() {
+        Map<String, ColumnMetaData> result = new LinkedHashMap<>(3, 1);
+        result.put("account_id", new ColumnMetaData("account_id", Types.INTEGER, "INT", true, true, false));
+        result.put("amount", mock(ColumnMetaData.class));
+        result.put("status", mock(ColumnMetaData.class));
         return result;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java
index f3d7da2..d98380f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java
@@ -24,9 +24,9 @@ import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
@@ -66,7 +66,7 @@ public final class ShardingTableBroadcastRoutingEngineTest {
     private ShardingSphereSchema schema;
     
     @Mock
-    private PhysicalTableMetaData tableMetaData;
+    private TableMetaData tableMetaData;
     
     private ShardingTableBroadcastRoutingEngine tableBroadcastRoutingEngine;
     
@@ -81,8 +81,8 @@ public final class ShardingTableBroadcastRoutingEngineTest {
         when(tablesContext.getTableNames()).thenReturn(Lists.newArrayList("t_order"));
         when(schema.getAllTableNames()).thenReturn(Lists.newArrayList("t_order"));
         when(schema.get("t_order")).thenReturn(tableMetaData);
-        Map<String, PhysicalIndexMetaData> indexMetaDataMap = new HashMap<>(1, 1);
-        indexMetaDataMap.put("index_name", new PhysicalIndexMetaData("index_name"));
+        Map<String, IndexMetaData> indexMetaDataMap = new HashMap<>(1, 1);
+        indexMetaDataMap.put("index_name", new IndexMetaData("index_name"));
         when(tableMetaData.getIndexes()).thenReturn(indexMetaDataMap);
         tableBroadcastRoutingEngine = new ShardingTableBroadcastRoutingEngine(schema, sqlStatementContext);
         shardingRule = new ShardingRule(shardingRuleConfig, Arrays.asList("ds0", "ds1"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java
index d6f304e..9967678 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.sharding.route.engine.type.single;
 
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
@@ -79,14 +79,14 @@ public final class SingleTableRoutingEngineTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(2, 1);
+        Map<String, TableMetaData> tables = new HashMap<>(2, 1);
         tables.put("t_order", buildTableMetaData());
         tables.put("t_order_item", buildTableMetaData());
         return new ShardingSphereSchema(tables);
     }
     
-    private PhysicalTableMetaData buildTableMetaData() {
-        PhysicalTableMetaData result = new PhysicalTableMetaData();
+    private TableMetaData buildTableMetaData() {
+        TableMetaData result = new TableMetaData();
         result.getAddressingDataSources().add("ds_0");
         return result;
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
index 448eed3..a4cfaa9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
@@ -24,9 +24,9 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
@@ -63,17 +63,17 @@ public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
-        tableMetaDataMap.put("t_order", new PhysicalTableMetaData(Arrays.asList(new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false),
-                new PhysicalColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
-                new PhysicalColumnMetaData("status", Types.INTEGER, "int", false, false, false)), Collections.emptySet()));
-        tableMetaDataMap.put("t_order_item", new PhysicalTableMetaData(Arrays.asList(new PhysicalColumnMetaData("item_id", Types.INTEGER, "int", true, false, false),
-                new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", false, false, false),
-                new PhysicalColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
-                new PhysicalColumnMetaData("status", Types.VARCHAR, "varchar", false, false, false),
-                new PhysicalColumnMetaData("c_date", Types.TIMESTAMP, "timestamp", false, false, false)), Collections.emptySet()));
-        tableMetaDataMap.put("t_other", new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false)), Collections.emptySet()));
-        tableMetaDataMap.put("t_category", new PhysicalTableMetaData());
+        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
+        tableMetaDataMap.put("t_order", new TableMetaData(Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, "int", true, false, false),
+                new ColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
+                new ColumnMetaData("status", Types.INTEGER, "int", false, false, false)), Collections.emptySet()));
+        tableMetaDataMap.put("t_order_item", new TableMetaData(Arrays.asList(new ColumnMetaData("item_id", Types.INTEGER, "int", true, false, false),
+                new ColumnMetaData("order_id", Types.INTEGER, "int", false, false, false),
+                new ColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
+                new ColumnMetaData("status", Types.VARCHAR, "varchar", false, false, false),
+                new ColumnMetaData("c_date", Types.TIMESTAMP, "timestamp", false, false, false)), Collections.emptySet()));
+        tableMetaDataMap.put("t_other", new TableMetaData(Collections.singletonList(new ColumnMetaData("order_id", Types.INTEGER, "int", true, false, false)), Collections.emptySet()));
+        tableMetaDataMap.put("t_category", new TableMetaData());
         tableMetaDataMap.get("t_category").getAddressingDataSources().add("single_db");
         return new ShardingSphereSchema(tableMetaDataMap);
     }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-common/src/main/java/org/apache/shardingsphere/governance/core/yaml/swapper/LogicSchemaMetaDataYamlSwapper.java b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-common/src/main/java/org/apache/shardingsphere/governance/core/yaml/swapper/LogicSchemaMetaDataYamlSwapper.java
index e2fe76b..fa024c4 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-common/src/main/java/org/apache/shardingsphere/governance/core/yaml/swapper/LogicSchemaMetaDataYamlSwapper.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-common/src/main/java/org/apache/shardingsphere/governance/core/yaml/swapper/LogicSchemaMetaDataYamlSwapper.java
@@ -22,10 +22,10 @@ import org.apache.shardingsphere.governance.core.yaml.config.metadata.YamlIndexM
 import org.apache.shardingsphere.governance.core.yaml.config.metadata.YamlLogicSchemaMetaData;
 import org.apache.shardingsphere.governance.core.yaml.config.metadata.YamlSchemaMetaData;
 import org.apache.shardingsphere.governance.core.yaml.config.metadata.YamlTableMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlSwapper;
 
 import java.util.Collection;
@@ -58,24 +58,24 @@ public final class LogicSchemaMetaDataYamlSwapper implements YamlSwapper<YamlLog
                 .collect(Collectors.toMap(Entry::getKey, entry -> convertTable(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new)));
     }
     
-    private PhysicalTableMetaData convertTable(final YamlTableMetaData table) {
-        return new PhysicalTableMetaData(convertColumns(table.getColumns()), convertIndexes(table.getIndexes()));
+    private TableMetaData convertTable(final YamlTableMetaData table) {
+        return new TableMetaData(convertColumns(table.getColumns()), convertIndexes(table.getIndexes()));
     }
 
-    private Collection<PhysicalIndexMetaData> convertIndexes(final Map<String, YamlIndexMetaData> indexes) {
+    private Collection<IndexMetaData> convertIndexes(final Map<String, YamlIndexMetaData> indexes) {
         return null == indexes ? Collections.emptyList() : indexes.values().stream().map(this::convertIndex).collect(Collectors.toList());
     }
 
-    private PhysicalIndexMetaData convertIndex(final YamlIndexMetaData index) {
-        return new PhysicalIndexMetaData(index.getName());
+    private IndexMetaData convertIndex(final YamlIndexMetaData index) {
+        return new IndexMetaData(index.getName());
     }
 
-    private Collection<PhysicalColumnMetaData> convertColumns(final Map<String, YamlColumnMetaData> indexes) {
+    private Collection<ColumnMetaData> convertColumns(final Map<String, YamlColumnMetaData> indexes) {
         return null == indexes ? Collections.emptyList() : indexes.values().stream().map(this::convertColumn).collect(Collectors.toList());
     }
 
-    private PhysicalColumnMetaData convertColumn(final YamlColumnMetaData column) {
-        return new PhysicalColumnMetaData(column.getName(), column.getDataType(), column.getDataTypeName(), column.isPrimaryKey(), column.isGenerated(), column.isCaseSensitive());
+    private ColumnMetaData convertColumn(final YamlColumnMetaData column) {
+        return new ColumnMetaData(column.getName(), column.getDataType(), column.getDataTypeName(), column.isPrimaryKey(), column.isGenerated(), column.isCaseSensitive());
     }
     
     private YamlSchemaMetaData convertYamlSchema(final ShardingSphereSchema schema) {
@@ -86,28 +86,28 @@ public final class LogicSchemaMetaDataYamlSwapper implements YamlSwapper<YamlLog
         return result;
     }
     
-    private YamlTableMetaData convertYamlTable(final PhysicalTableMetaData table) {
+    private YamlTableMetaData convertYamlTable(final TableMetaData table) {
         YamlTableMetaData result = new YamlTableMetaData();
         result.setColumns(convertYamlColumns(table.getColumns()));
         result.setIndexes(convertYamlIndexes(table.getIndexes()));
         return result;
     }
 
-    private Map<String, YamlIndexMetaData> convertYamlIndexes(final Map<String, PhysicalIndexMetaData> indexes) {
+    private Map<String, YamlIndexMetaData> convertYamlIndexes(final Map<String, IndexMetaData> indexes) {
         return indexes.entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> convertYamlIndex(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
     
-    private YamlIndexMetaData convertYamlIndex(final PhysicalIndexMetaData index) {
+    private YamlIndexMetaData convertYamlIndex(final IndexMetaData index) {
         YamlIndexMetaData result = new YamlIndexMetaData();
         result.setName(index.getName());
         return result;
     }
     
-    private Map<String, YamlColumnMetaData> convertYamlColumns(final Map<String, PhysicalColumnMetaData> columns) {
+    private Map<String, YamlColumnMetaData> convertYamlColumns(final Map<String, ColumnMetaData> columns) {
         return columns.entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> convertYamlColumn(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
     
-    private YamlColumnMetaData convertYamlColumn(final PhysicalColumnMetaData column) {
+    private YamlColumnMetaData convertYamlColumn(final ColumnMetaData column) {
         YamlColumnMetaData result = new YamlColumnMetaData();
         result.setName(column.getName());
         result.setCaseSensitive(column.isCaseSensitive());
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 91d4b75..bd7a77f 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
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.binder.segment.insert.keygen.engine;
 
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.SetAssignmentSegment;
@@ -62,7 +62,7 @@ public final class GeneratedKeyContextEngine {
         if (!schema.containsTable(tableName)) {
             return Optional.empty();
         }
-        for (Entry<String, PhysicalColumnMetaData> entry : schema.get(tableName).getColumns().entrySet()) {
+        for (Entry<String, ColumnMetaData> 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 735a9aa..265ca8e 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.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 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;
@@ -62,8 +62,8 @@ public final class GeneratedKeyContextEngineTest {
     
     @Before
     public void setUp() {
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("id", Types.INTEGER, "INT", true, true, false)), Collections.emptyList());
-        Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>(1, 1);
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(new ColumnMetaData("id", Types.INTEGER, "INT", true, true, false)), Collections.emptyList());
+        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(1, 1);
         tableMetaDataMap.put("tbl", tableMetaData);
         schema = new ShardingSphereSchema(tableMetaDataMap);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchema.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchema.java
index 8d273b3..2855c81 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchema.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchema.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.metadata.schema;
 
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -31,14 +31,14 @@ import java.util.concurrent.ConcurrentHashMap;
  */
 public final class ShardingSphereSchema {
     
-    private final Map<String, PhysicalTableMetaData> tables;
+    private final Map<String, TableMetaData> tables;
     
     @SuppressWarnings("CollectionWithoutInitialCapacity")
     public ShardingSphereSchema() {
         tables = new ConcurrentHashMap<>();
     }
     
-    public ShardingSphereSchema(final Map<String, PhysicalTableMetaData> tables) {
+    public ShardingSphereSchema(final Map<String, TableMetaData> tables) {
         this.tables = new ConcurrentHashMap<>(tables.size(), 1);
         tables.forEach((key, value) -> this.tables.put(key.toLowerCase(), value));
     }
@@ -58,7 +58,7 @@ public final class ShardingSphereSchema {
      * @param tableName tableName table name
      * @return table mata data
      */
-    public PhysicalTableMetaData get(final String tableName) {
+    public TableMetaData get(final String tableName) {
         return tables.get(tableName.toLowerCase());
     }
     
@@ -68,7 +68,7 @@ public final class ShardingSphereSchema {
      * @param tableName table name
      * @param tableMetaData table meta data
      */
-    public void put(final String tableName, final PhysicalTableMetaData tableMetaData) {
+    public void put(final String tableName, final TableMetaData tableMetaData) {
         tables.put(tableName.toLowerCase(), tableMetaData);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java
index 9427e96..8db4239 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.loader.addressing.TableAddressingMapperDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 
@@ -76,7 +76,7 @@ public final class SchemaMetaDataLoader {
         for (Entry<String, Collection<String>> entry : TableAddressingMapperDataLoader.load(databaseType, dataSourceMap, rules).entrySet()) {
             String tableName = entry.getKey();
             if (!schema.containsTable(tableName)) {
-                schema.put(tableName, new PhysicalTableMetaData());
+                schema.put(tableName, new TableMetaData());
             }
             schema.get(tableName).getAddressingDataSources().addAll(entry.getValue());
         }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java
index 3a2943e..43b4229 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
@@ -57,21 +57,21 @@ public final class TableMetaDataLoader {
      * @return table meta data
      * @throws SQLException SQL exception
      */
-    public static Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, 
-                                                       final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
-        Optional<PhysicalTableMetaData> tableMetaData = loadTableMetaData(tableName, databaseType, dataSourceMap, rules, props);
+    public static Optional<TableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
+                                               final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
+        Optional<TableMetaData> tableMetaData = loadTableMetaData(tableName, databaseType, dataSourceMap, rules, props);
         return tableMetaData.map(optional -> decorateTableMetaData(tableName, optional, rules));
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private static Optional<PhysicalTableMetaData> loadTableMetaData(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, 
-                                                                     final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
+    private static Optional<TableMetaData> loadTableMetaData(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
+                                                             final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
         DataNodes dataNodes = new DataNodes(rules);
         for (Entry<ShardingSphereRule, ShardingSphereMetaDataLoader> entry : OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataLoader.class).entrySet()) {
             if (entry.getKey() instanceof TableContainedRule) {
                 TableContainedRule rule = (TableContainedRule) entry.getKey();
                 ShardingSphereMetaDataLoader loader = entry.getValue();
-                Optional<PhysicalTableMetaData> tableMetaData = loader.load(tableName, databaseType, dataSourceMap, dataNodes, rule, props);
+                Optional<TableMetaData> tableMetaData = loader.load(tableName, databaseType, dataSourceMap, dataNodes, rule, props);
                 if (tableMetaData.isPresent()) {
                     return tableMetaData;
                 }
@@ -81,8 +81,8 @@ public final class TableMetaDataLoader {
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private static PhysicalTableMetaData decorateTableMetaData(final String tableName, final PhysicalTableMetaData tableMetaData, final Collection<ShardingSphereRule> rules) {
-        PhysicalTableMetaData result = null;
+    private static TableMetaData decorateTableMetaData(final String tableName, final TableMetaData tableMetaData, final Collection<ShardingSphereRule> rules) {
+        TableMetaData result = null;
         for (Entry<ShardingSphereRule, ShardingSphereMetaDataLoader> entry : OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataLoader.class).entrySet()) {
             if (entry.getKey() instanceof TableContainedRule) {
                 result = entry.getValue().decorate(tableName, null == result ? tableMetaData : result, (TableContainedRule) entry.getKey());
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalColumnMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalColumnMetaDataLoader.java
index 79c8cc8..9c79fd6 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalColumnMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalColumnMetaDataLoader.java
@@ -22,7 +22,7 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.loader.physical.jdbc.handler.DatabaseMetaDataDialectHandler;
 import org.apache.shardingsphere.infra.metadata.schema.loader.physical.jdbc.handler.DatabaseMetaDataDialectHandlerFactory;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 
 import java.sql.Connection;
@@ -58,8 +58,8 @@ public final class PhysicalColumnMetaDataLoader {
      * @return column meta data list
      * @throws SQLException SQL exception
      */
-    public static Collection<PhysicalColumnMetaData> load(final Connection connection, final String tableNamePattern, final DatabaseType databaseType) throws SQLException {
-        Collection<PhysicalColumnMetaData> result = new LinkedList<>();
+    public static Collection<ColumnMetaData> load(final Connection connection, final String tableNamePattern, final DatabaseType databaseType) throws SQLException {
+        Collection<ColumnMetaData> result = new LinkedList<>();
         Collection<String> primaryKeys = loadPrimaryKeys(connection, tableNamePattern);
         List<String> columnNames = new ArrayList<>();
         List<Integer> columnTypes = new ArrayList<>();
@@ -85,7 +85,7 @@ public final class PhysicalColumnMetaDataLoader {
         }
         for (int i = 0; i < columnNames.size(); i++) {
             // TODO load auto generated from database meta data
-            result.add(new PhysicalColumnMetaData(columnNames.get(i), columnTypes.get(i), columnTypeNames.get(i), isPrimaryKeys.get(i), false, isCaseSensitives.get(i)));
+            result.add(new ColumnMetaData(columnNames.get(i), columnTypes.get(i), columnTypeNames.get(i), isPrimaryKeys.get(i), false, isCaseSensitives.get(i)));
         }
         return result;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalIndexMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalIndexMetaDataLoader.java
index 735ac2b..a32acdb 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalIndexMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalIndexMetaDataLoader.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.metadata.schema.loader.physical;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 
 import java.sql.Connection;
 import java.sql.ResultSet;
@@ -44,13 +44,13 @@ public final class PhysicalIndexMetaDataLoader {
      * @return index meta data list
      * @throws SQLException SQL exception
      */
-    public static Collection<PhysicalIndexMetaData> load(final Connection connection, final String table) throws SQLException {
-        Collection<PhysicalIndexMetaData> result = new HashSet<>();
+    public static Collection<IndexMetaData> load(final Connection connection, final String table) throws SQLException {
+        Collection<IndexMetaData> result = new HashSet<>();
         try (ResultSet resultSet = connection.getMetaData().getIndexInfo(connection.getCatalog(), connection.getSchema(), table, false, false)) {
             while (resultSet.next()) {
                 String indexName = resultSet.getString(INDEX_NAME);
                 if (null != indexName) {
-                    result.add(new PhysicalIndexMetaData(indexName));
+                    result.add(new IndexMetaData(indexName));
                 }
             }
         }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalTableMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalTableMetaDataLoader.java
index e35458e..71fae97 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalTableMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/PhysicalTableMetaDataLoader.java
@@ -22,7 +22,7 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.loader.physical.jdbc.MetaDataConnectionAdapter;
 import org.apache.shardingsphere.infra.metadata.schema.loader.physical.jdbc.handler.DatabaseMetaDataDialectHandlerFactory;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import javax.sql.DataSource;
 import java.sql.Connection;
@@ -45,11 +45,11 @@ public final class PhysicalTableMetaDataLoader {
      * @return table meta data
      * @throws SQLException SQL exception
      */
-    public static Optional<PhysicalTableMetaData> load(final DataSource dataSource, final String tableNamePattern, final DatabaseType databaseType) throws SQLException {
+    public static Optional<TableMetaData> load(final DataSource dataSource, final String tableNamePattern, final DatabaseType databaseType) throws SQLException {
         try (MetaDataConnectionAdapter connectionAdapter = new MetaDataConnectionAdapter(databaseType, dataSource.getConnection())) {
             String formattedTableNamePattern = formatTableNamePattern(tableNamePattern, databaseType);
             return isTableExist(connectionAdapter, formattedTableNamePattern)
-                    ? Optional.of(new PhysicalTableMetaData(PhysicalColumnMetaDataLoader.load(
+                    ? Optional.of(new TableMetaData(PhysicalColumnMetaDataLoader.load(
                             connectionAdapter, formattedTableNamePattern, databaseType), PhysicalIndexMetaDataLoader.load(connectionAdapter, formattedTableNamePattern)))
                     : Optional.empty();
         }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java
index ab9872d..2306ceb 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.infra.metadata.schema.loader.spi;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPI;
 
@@ -48,8 +48,8 @@ public interface ShardingSphereMetaDataLoader<T extends TableContainedRule> exte
      * @return meta data
      * @throws SQLException SQL exception
      */
-    Optional<PhysicalTableMetaData> load(String tableName, 
-                                         DatabaseType databaseType, Map<String, DataSource> dataSourceMap, DataNodes dataNodes, T rule, ConfigurationProperties props) throws SQLException;
+    Optional<TableMetaData> load(String tableName,
+                                 DatabaseType databaseType, Map<String, DataSource> dataSourceMap, DataNodes dataNodes, T rule, ConfigurationProperties props) throws SQLException;
     
     /**
      * Decorate table meta data.
@@ -59,5 +59,5 @@ public interface ShardingSphereMetaDataLoader<T extends TableContainedRule> exte
      * @param rule rule
      * @return decorated table meta data
      */
-    PhysicalTableMetaData decorate(String tableName, PhysicalTableMetaData tableMetaData, T rule);
+    TableMetaData decorate(String tableName, TableMetaData tableMetaData, T rule);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalColumnMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/ColumnMetaData.java
similarity index 89%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalColumnMetaData.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/ColumnMetaData.java
index 307acc9..1883f49 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalColumnMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/ColumnMetaData.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.model.physical;
+package org.apache.shardingsphere.infra.metadata.schema.model;
 
 import lombok.EqualsAndHashCode;
 import lombok.Getter;
@@ -23,13 +23,13 @@ import lombok.RequiredArgsConstructor;
 import lombok.ToString;
 
 /**
- * Physical column meta data.
+ * Column meta data.
  */
 @RequiredArgsConstructor
 @Getter
 @EqualsAndHashCode
 @ToString
-public class PhysicalColumnMetaData {
+public class ColumnMetaData {
     
     private final String name;
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalIndexMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/IndexMetaData.java
similarity index 87%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalIndexMetaData.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/IndexMetaData.java
index 03e6716..e374580 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalIndexMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/IndexMetaData.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.model.physical;
+package org.apache.shardingsphere.infra.metadata.schema.model;
 
 import lombok.EqualsAndHashCode;
 import lombok.Getter;
@@ -23,13 +23,13 @@ import lombok.RequiredArgsConstructor;
 import lombok.ToString;
 
 /**
- * Physical index meta data.
+ * Index meta data.
  */
 @RequiredArgsConstructor
 @Getter
 @EqualsAndHashCode
 @ToString
-public final class PhysicalIndexMetaData {
+public final class IndexMetaData {
     
     private final String name;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalTableMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/TableMetaData.java
similarity index 73%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalTableMetaData.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/TableMetaData.java
index 7cd441e..cb3a049 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalTableMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/TableMetaData.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.model.physical;
+package org.apache.shardingsphere.infra.metadata.schema.model;
 
 import lombok.AccessLevel;
 import lombok.EqualsAndHashCode;
@@ -31,16 +31,16 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Physical table meta data.
+ * Table meta data.
  */
 @Getter
 @EqualsAndHashCode
 @ToString
-public final class PhysicalTableMetaData {
+public final class TableMetaData {
     
-    private final Map<String, PhysicalColumnMetaData> columns;
+    private final Map<String, ColumnMetaData> columns;
     
-    private final Map<String, PhysicalIndexMetaData> indexes;
+    private final Map<String, IndexMetaData> indexes;
     
     @Getter(AccessLevel.NONE)
     private final List<String> columnNames = new ArrayList<>();
@@ -49,18 +49,18 @@ public final class PhysicalTableMetaData {
     
     private final Collection<String> addressingDataSources = new LinkedHashSet<>();
     
-    public PhysicalTableMetaData() {
+    public TableMetaData() {
         this(Collections.emptyList(), Collections.emptyList());
     }
     
-    public PhysicalTableMetaData(final Collection<PhysicalColumnMetaData> columnMetaDataList, final Collection<PhysicalIndexMetaData> indexMetaDataList) {
+    public TableMetaData(final Collection<ColumnMetaData> columnMetaDataList, final Collection<IndexMetaData> indexMetaDataList) {
         columns = getColumns(columnMetaDataList);
         indexes = getIndexes(indexMetaDataList);
     }
     
-    private Map<String, PhysicalColumnMetaData> getColumns(final Collection<PhysicalColumnMetaData> columnMetaDataList) {
-        Map<String, PhysicalColumnMetaData> result = new LinkedHashMap<>(columnMetaDataList.size(), 1);
-        for (PhysicalColumnMetaData each : columnMetaDataList) {
+    private Map<String, ColumnMetaData> getColumns(final Collection<ColumnMetaData> columnMetaDataList) {
+        Map<String, ColumnMetaData> result = new LinkedHashMap<>(columnMetaDataList.size(), 1);
+        for (ColumnMetaData each : columnMetaDataList) {
             String lowerColumnName = each.getName().toLowerCase();
             columnNames.add(lowerColumnName);
             result.put(lowerColumnName, each);
@@ -71,9 +71,9 @@ public final class PhysicalTableMetaData {
         return Collections.synchronizedMap(result);
     }
     
-    private Map<String, PhysicalIndexMetaData> getIndexes(final Collection<PhysicalIndexMetaData> indexMetaDataList) {
-        Map<String, PhysicalIndexMetaData> result = new LinkedHashMap<>(indexMetaDataList.size(), 1);
-        for (PhysicalIndexMetaData each : indexMetaDataList) {
+    private Map<String, IndexMetaData> getIndexes(final Collection<IndexMetaData> indexMetaDataList) {
+        Map<String, IndexMetaData> result = new LinkedHashMap<>(indexMetaDataList.size(), 1);
+        for (IndexMetaData each : indexMetaDataList) {
             result.put(each.getName().toLowerCase(), each);
         }
         return Collections.synchronizedMap(result);
@@ -85,7 +85,7 @@ public final class PhysicalTableMetaData {
      * @param columnIndex column index
      * @return column meta data
      */
-    public PhysicalColumnMetaData getColumnMetaData(final int columnIndex) {
+    public ColumnMetaData getColumnMetaData(final int columnIndex) {
         return columns.get(columnNames.get(columnIndex));
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/TableMetaDataLoaderCallback.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/TableMetaDataLoaderCallback.java
index a00f3d1..f243bf1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/TableMetaDataLoaderCallback.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/TableMetaDataLoaderCallback.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.refresh;
 
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import java.sql.SQLException;
 import java.util.Optional;
@@ -34,5 +34,5 @@ public interface TableMetaDataLoaderCallback {
      * @return table meta data
      * @throws SQLException SQL exception
      */
-    Optional<PhysicalTableMetaData> load(String tableName) throws SQLException;
+    Optional<TableMetaData> load(String tableName) throws SQLException;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateIndexStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateIndexStatementMetaDataRefreshStrategy.java
index 6d10359..c71ce4e 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateIndexStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateIndexStatementMetaDataRefreshStrategy.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.TableMetaDataLoaderCallback;
@@ -39,6 +39,6 @@ public final class CreateIndexStatementMetaDataRefreshStrategy implements MetaDa
         }
         String indexName = sqlStatement.getIndex().getIdentifier().getValue();
         String tableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
-        schema.get(tableName).getIndexes().put(indexName, new PhysicalIndexMetaData(indexName));
+        schema.get(tableName).getIndexes().put(indexName, new IndexMetaData(indexName));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java
index c631af7..b7cda31 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
@@ -37,11 +37,11 @@ public final class CreateTableStatementMetaDataRefreshStrategy implements MetaDa
     public void refreshMetaData(final ShardingSphereSchema schema, final DatabaseType databaseType, final Collection<String> routeDataSourceNames,
                                 final CreateTableStatement sqlStatement, final TableMetaDataLoaderCallback callback) throws SQLException {
         String tableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
-        Optional<PhysicalTableMetaData> tableMetaData = callback.load(tableName);
+        Optional<TableMetaData> tableMetaData = callback.load(tableName);
         if (tableMetaData.isPresent()) {
             schema.put(tableName, tableMetaData.get());
         } else {
-            schema.put(tableName, new PhysicalTableMetaData());
+            schema.put(tableName, new TableMetaData());
         }
         schema.get(tableName).getAddressingDataSources().addAll(routeDataSourceNames);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java
index 24573ec..a45c812 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
@@ -35,7 +35,7 @@ public final class CreateViewStatementMetaDataRefreshStrategy implements MetaDat
     public void refreshMetaData(final ShardingSphereSchema schema, final DatabaseType databaseType, final Collection<String> routeDataSourceNames,
                                 final CreateViewStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
         String viewName = sqlStatement.getView().getTableName().getIdentifier().getValue();
-        schema.put(viewName, new PhysicalTableMetaData());
+        schema.put(viewName, new TableMetaData());
         schema.get(viewName).getAddressingDataSources().addAll(routeDataSourceNames);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategy.java
index c7423a8..ce33f20 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategy.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.TableMetaDataLoaderCallback;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -42,7 +42,7 @@ public final class DropIndexStatementMetaDataRefreshStrategy implements MetaData
         Collection<String> indexNames = getIndexNames(sqlStatement);
         Optional<SimpleTableSegment> simpleTableSegment = DropIndexStatementHandler.getSimpleTableSegment(sqlStatement);
         String tableName = simpleTableSegment.map(tableSegment -> tableSegment.getTableName().getIdentifier().getValue()).orElse("");
-        PhysicalTableMetaData tableMetaData = schema.get(tableName);
+        TableMetaData tableMetaData = schema.get(tableName);
         if (simpleTableSegment.isPresent()) {
             indexNames.forEach(each -> tableMetaData.getIndexes().remove(each));
         }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchemaTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchemaTest.java
index aa74f22..40639e9 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchemaTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchemaTest.java
@@ -19,8 +19,8 @@ package org.apache.shardingsphere.infra.metadata.schema;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.junit.Test;
 
 import java.util.Collections;
@@ -35,50 +35,50 @@ public final class ShardingSphereSchemaTest {
     
     @Test
     public void assertGetAllTableNames() {
-        assertThat(new ShardingSphereSchema(ImmutableMap.of("tbl", mock(PhysicalTableMetaData.class))).getAllTableNames(), is(Sets.newHashSet("tbl")));
+        assertThat(new ShardingSphereSchema(ImmutableMap.of("tbl", mock(TableMetaData.class))).getAllTableNames(), is(Sets.newHashSet("tbl")));
     }
     
     @Test
     public void assertGet() {
-        PhysicalTableMetaData tableMetaData = mock(PhysicalTableMetaData.class);
+        TableMetaData tableMetaData = mock(TableMetaData.class);
         assertThat(new ShardingSphereSchema(ImmutableMap.of("tbl", tableMetaData)).get("tbl"), is(tableMetaData));
     }
     
     @Test
     public void assertPut() {
         ShardingSphereSchema actual = new ShardingSphereSchema(Collections.emptyMap());
-        PhysicalTableMetaData tableMetaData = mock(PhysicalTableMetaData.class);
+        TableMetaData tableMetaData = mock(TableMetaData.class);
         actual.put("tbl", tableMetaData);
         assertThat(actual.get("tbl"), is(tableMetaData));
     }
     
     @Test
     public void assertRemove() {
-        ShardingSphereSchema actual = new ShardingSphereSchema(ImmutableMap.of("tbl", mock(PhysicalTableMetaData.class)));
+        ShardingSphereSchema actual = new ShardingSphereSchema(ImmutableMap.of("tbl", mock(TableMetaData.class)));
         actual.remove("tbl");
         assertNull(actual.get("tbl"));
     }
     
     @Test
     public void assertContainsTable() {
-        assertTrue(new ShardingSphereSchema(ImmutableMap.of("tbl", mock(PhysicalTableMetaData.class))).containsTable("tbl"));
+        assertTrue(new ShardingSphereSchema(ImmutableMap.of("tbl", mock(TableMetaData.class))).containsTable("tbl"));
     }
     
     @Test
     public void assertContainsColumn() {
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("col", 0, "dataType", false, false, false)), Collections.emptyList());
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(new ColumnMetaData("col", 0, "dataType", false, false, false)), Collections.emptyList());
         assertTrue(new ShardingSphereSchema(ImmutableMap.of("tbl", tableMetaData)).containsColumn("tbl", "col"));
     }
     
     @Test
     public void assertGetAllColumnNamesWhenContainsKey() {
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("col", 0, "dataType", false, false, false)), Collections.emptyList());
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(new ColumnMetaData("col", 0, "dataType", false, false, false)), Collections.emptyList());
         assertThat(new ShardingSphereSchema(ImmutableMap.of("tbl", tableMetaData)).getAllColumnNames("tbl"), is(Collections.singletonList("col")));
     }
     
     @Test
     public void assertGetAllColumnNamesWhenNotContainsKey() {
-        PhysicalTableMetaData tableMetaData = new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("col", 0, "dataType", false, false, false)), Collections.emptyList());
+        TableMetaData tableMetaData = new TableMetaData(Collections.singletonList(new ColumnMetaData("col", 0, "dataType", false, false, false)), Collections.emptyList());
         assertThat(new ShardingSphereSchema(ImmutableMap.of("tbl1", tableMetaData)).getAllColumnNames("tbl2"), is(Collections.<String>emptyList()));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java
index 020ece1..58517bc 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java
@@ -22,7 +22,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.fixture.rule.CommonFixtureRule;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import javax.sql.DataSource;
 import java.util.Map;
@@ -31,13 +31,13 @@ import java.util.Optional;
 public final class CommonFixtureLogicMetaDataLoader implements ShardingSphereMetaDataLoader<CommonFixtureRule> {
     
     @Override
-    public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                                final DataNodes dataNodes, final CommonFixtureRule rule, final ConfigurationProperties props) {
+    public Optional<TableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
+                                        final DataNodes dataNodes, final CommonFixtureRule rule, final ConfigurationProperties props) {
         return Optional.empty();
     }
     
     @Override
-    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final CommonFixtureRule rule) {
+    public TableMetaData decorate(final String tableName, final TableMetaData tableMetaData, final CommonFixtureRule rule) {
         return tableMetaData;
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeContainedFixtureLogicMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeContainedFixtureLogicMetaDataLoader.java
index 2c7d010..a0d901a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeContainedFixtureLogicMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeContainedFixtureLogicMetaDataLoader.java
@@ -22,8 +22,8 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.fixture.rule.DataNodeContainedFixtureRule;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import javax.sql.DataSource;
 import java.util.Collections;
@@ -33,16 +33,16 @@ import java.util.Optional;
 public final class DataNodeContainedFixtureLogicMetaDataLoader implements ShardingSphereMetaDataLoader<DataNodeContainedFixtureRule> {
     
     @Override
-    public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                                final DataNodes dataNodes, final DataNodeContainedFixtureRule rule, final ConfigurationProperties props) {
+    public Optional<TableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
+                                        final DataNodes dataNodes, final DataNodeContainedFixtureRule rule, final ConfigurationProperties props) {
         return ("data_node_routed_table_0".equals(tableName) || "data_node_routed_table_1".equals(tableName))
-                ? Optional.of(new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList())) : Optional.empty();
+                ? Optional.of(new TableMetaData(Collections.emptyList(), Collections.emptyList())) : Optional.empty();
     }
     
     @Override
-    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final DataNodeContainedFixtureRule rule) {
-        PhysicalColumnMetaData columnMetaData = new PhysicalColumnMetaData("id", 1, "INT", true, true, false);
-        return new PhysicalTableMetaData(Collections.singletonList(columnMetaData), Collections.emptyList());
+    public TableMetaData decorate(final String tableName, final TableMetaData tableMetaData, final DataNodeContainedFixtureRule rule) {
+        ColumnMetaData columnMetaData = new ColumnMetaData("id", 1, "INT", true, true, false);
+        return new TableMetaData(Collections.singletonList(columnMetaData), Collections.emptyList());
     }
     
     @Override
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/ColumnMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/ColumnMetaDataLoaderTest.java
index 2233b84..1f2bcf6 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/ColumnMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/ColumnMetaDataLoaderTest.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.metadata.schema.loader.physical;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -94,14 +94,14 @@ public final class ColumnMetaDataLoaderTest {
     
     @Test
     public void assertLoad() throws SQLException {
-        Collection<PhysicalColumnMetaData> actual = PhysicalColumnMetaDataLoader.load(connection, TEST_TABLE, databaseType);
+        Collection<ColumnMetaData> actual = PhysicalColumnMetaDataLoader.load(connection, TEST_TABLE, databaseType);
         assertThat(actual.size(), is(2));
-        Iterator<PhysicalColumnMetaData> columnMetaDataIterator = actual.iterator();
+        Iterator<ColumnMetaData> columnMetaDataIterator = actual.iterator();
         assertColumnMetaData(columnMetaDataIterator.next(), "pk_col", Types.INTEGER, "INT", true, true);
         assertColumnMetaData(columnMetaDataIterator.next(), "col", Types.VARCHAR, "VARCHAR", false, false);
     }
     
-    private void assertColumnMetaData(final PhysicalColumnMetaData actual, final String name, final int dataType, final String typeName, final boolean primaryKey, final boolean caseSensitive) {
+    private void assertColumnMetaData(final ColumnMetaData actual, final String name, final int dataType, final String typeName, final boolean primaryKey, final boolean caseSensitive) {
         assertThat(actual.getName(), is(name));
         assertThat(actual.getDataType(), is(dataType));
         assertThat(actual.getDataTypeName(), is(typeName));
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/IndexMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/IndexMetaDataLoaderTest.java
index 9d52ad2..8b0a81c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/IndexMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/IndexMetaDataLoaderTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.loader.physical;
 
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -61,9 +61,9 @@ public final class IndexMetaDataLoaderTest {
         when(databaseMetaData.getIndexInfo(TEST_CATALOG, null, TEST_TABLE, false, false)).thenReturn(indexResultSet);
         when(indexResultSet.next()).thenReturn(true, true, false);
         when(indexResultSet.getString("INDEX_NAME")).thenReturn("my_index");
-        Collection<PhysicalIndexMetaData> actual = PhysicalIndexMetaDataLoader.load(connection, TEST_TABLE);
+        Collection<IndexMetaData> actual = PhysicalIndexMetaDataLoader.load(connection, TEST_TABLE);
         assertThat(actual.size(), is(1));
-        PhysicalIndexMetaData indexMetaData = actual.iterator().next();
+        IndexMetaData indexMetaData = actual.iterator().next();
         assertThat(indexMetaData.getName(), is("my_index"));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/TableMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/TableMetaDataLoaderTest.java
index ce30105..76823a5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/TableMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/physical/TableMetaDataLoaderTest.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.infra.metadata.schema.loader.physical;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -117,13 +117,13 @@ public final class TableMetaDataLoaderTest {
     
     @Test
     public void assertLoad() throws SQLException {
-        Optional<PhysicalTableMetaData> actual = PhysicalTableMetaDataLoader.load(dataSource, TEST_TABLE, databaseType);
+        Optional<TableMetaData> actual = PhysicalTableMetaDataLoader.load(dataSource, TEST_TABLE, databaseType);
         assertTrue(actual.isPresent());
-        Map<String, PhysicalColumnMetaData> columnMetaDataMap = actual.get().getColumns();
+        Map<String, ColumnMetaData> columnMetaDataMap = actual.get().getColumns();
         assertThat(columnMetaDataMap.size(), is(2));
         assertColumnMetaData(columnMetaDataMap.get("pk_col"), "pk_col", Types.INTEGER, "INT", true, true);
         assertColumnMetaData(columnMetaDataMap.get("col"), "col", Types.VARCHAR, "VARCHAR", false, false);
-        Map<String, PhysicalIndexMetaData> indexMetaDataMap = actual.get().getIndexes();
+        Map<String, IndexMetaData> indexMetaDataMap = actual.get().getIndexes();
         assertThat(indexMetaDataMap.size(), is(1));
         assertTrue(indexMetaDataMap.containsKey("my_index"));
     }
@@ -132,11 +132,11 @@ public final class TableMetaDataLoaderTest {
     public void assertTableNotExist() throws SQLException {
         when(databaseMetaData.getTables(TEST_CATALOG, null, TEST_TABLE, null)).thenReturn(tableNotExistResultSet);
         when(tableNotExistResultSet.next()).thenReturn(false);
-        Optional<PhysicalTableMetaData> actual = PhysicalTableMetaDataLoader.load(dataSource, TEST_TABLE, databaseType);
+        Optional<TableMetaData> actual = PhysicalTableMetaDataLoader.load(dataSource, TEST_TABLE, databaseType);
         assertFalse(actual.isPresent());
     }
     
-    private void assertColumnMetaData(final PhysicalColumnMetaData actual, final String name, final int dataType, final String typeName, final boolean primaryKey, final boolean caseSensitive) {
+    private void assertColumnMetaData(final ColumnMetaData actual, final String name, final int dataType, final String typeName, final boolean primaryKey, final boolean caseSensitive) {
         assertThat(actual.getName(), is(name));
         assertThat(actual.getDataType(), is(dataType));
         assertThat(actual.getDataTypeName(), is(typeName));
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/TableMetaDataTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/TableMetaDataTest.java
index cd539db..4f397a7 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/TableMetaDataTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/TableMetaDataTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.model.physical;
 
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -30,16 +32,16 @@ import static org.junit.Assert.assertTrue;
 
 public final class TableMetaDataTest {
     
-    private PhysicalTableMetaData tableMetaData;
+    private TableMetaData tableMetaData;
     
     @Before
     public void setUp() {
-        tableMetaData = new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("test", Types.INTEGER, "INT", true, false, true)), Collections.emptyList());
+        tableMetaData = new TableMetaData(Collections.singletonList(new ColumnMetaData("test", Types.INTEGER, "INT", true, false, true)), Collections.emptyList());
     }
     
     @Test
     public void assertGetColumnMetaData() {
-        PhysicalColumnMetaData actual = tableMetaData.getColumnMetaData(0);
+        ColumnMetaData actual = tableMetaData.getColumnMetaData(0);
         assertThat(actual.getName(), is("test"));
         assertThat(actual.getDataType(), is(Types.INTEGER));
         assertThat(actual.getDataTypeName(), is("INT"));
@@ -62,7 +64,7 @@ public final class TableMetaDataTest {
     
     @Test
     public void assertEmptyColumnsWithDefaultConstructor() {
-        tableMetaData = new PhysicalTableMetaData();
+        tableMetaData = new TableMetaData();
         assertThat(tableMetaData.getColumns(), is(Collections.emptyMap()));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java
index 20329fa..f9c5297 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java
@@ -19,10 +19,10 @@ package org.apache.shardingsphere.infra.metadata.schema.refresh;
 
 import com.google.common.collect.ImmutableMap;
 import lombok.Getter;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.junit.Before;
 
 import java.util.Collections;
@@ -38,8 +38,8 @@ public abstract class AbstractMetaDataRefreshStrategyTest {
     }
     
     private ShardingSphereSchema buildSchema() {
-        return new ShardingSphereSchema(ImmutableMap.of("t_order", new PhysicalTableMetaData(
-                Collections.singletonList(new PhysicalColumnMetaData("order_id", 1, "String", false, false, false)), Collections.singletonList(new PhysicalIndexMetaData("index")))));
+        return new ShardingSphereSchema(ImmutableMap.of("t_order", new TableMetaData(
+                Collections.singletonList(new ColumnMetaData("order_id", 1, "String", false, false, false)), Collections.singletonList(new IndexMetaData("index")))));
     }
 }
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/AlterTableStatementMetaDataRefreshStrategyTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/AlterTableStatementMetaDataRefreshStrategyTest.java
index 479e738..e176407 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/AlterTableStatementMetaDataRefreshStrategyTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/AlterTableStatementMetaDataRefreshStrategyTest.java
@@ -18,9 +18,9 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.AbstractMetaDataRefreshStrategyTest;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -71,9 +71,9 @@ public final class AlterTableStatementMetaDataRefreshStrategyTest extends Abstra
     private void refreshMetaData(final AlterTableStatement alterTableStatement) throws SQLException {
         MetaDataRefreshStrategy<AlterTableStatement> metaDataRefreshStrategy = new AlterTableStatementMetaDataRefreshStrategy();
         alterTableStatement.setTable(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order"))));
-        metaDataRefreshStrategy.refreshMetaData(getSchema(), mock(DatabaseType.class), Collections.emptyList(), alterTableStatement, tableName -> Optional.of(new PhysicalTableMetaData(
-                Collections.singletonList(new PhysicalColumnMetaData("order_id", 1, "String", true, false, false)),
-                Collections.singletonList(new PhysicalIndexMetaData("index_alter")))));
+        metaDataRefreshStrategy.refreshMetaData(getSchema(), mock(DatabaseType.class), Collections.emptyList(), alterTableStatement, tableName -> Optional.of(new TableMetaData(
+                Collections.singletonList(new ColumnMetaData("order_id", 1, "String", true, false, false)),
+                Collections.singletonList(new IndexMetaData("index_alter")))));
         assertTrue(getSchema().get("t_order").getIndexes().containsKey("index_alter"));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategyTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategyTest.java
index 1fe6225..0dafadc 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategyTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategyTest.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.AbstractMetaDataRefreshStrategyTest;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -79,9 +79,9 @@ public final class CreateTableStatementMetaDataRefreshStrategyTest extends Abstr
     private void refreshMetaData(final CreateTableStatement createTableStatement) throws SQLException {
         createTableStatement.setTable(new SimpleTableSegment(new TableNameSegment(1, 3, new IdentifierValue("t_order_0"))));
         MetaDataRefreshStrategy<CreateTableStatement> metaDataRefreshStrategy = new CreateTableStatementMetaDataRefreshStrategy();
-        metaDataRefreshStrategy.refreshMetaData(getSchema(), mock(DatabaseType.class), Collections.emptyList(), createTableStatement, tableName -> Optional.of(new PhysicalTableMetaData(
-                Collections.singletonList(new PhysicalColumnMetaData("order_id", 1, "String", true, false, false)),
-                Collections.singletonList(new PhysicalIndexMetaData("index")))));
+        metaDataRefreshStrategy.refreshMetaData(getSchema(), mock(DatabaseType.class), Collections.emptyList(), createTableStatement, tableName -> Optional.of(new TableMetaData(
+                Collections.singletonList(new ColumnMetaData("order_id", 1, "String", true, false, false)),
+                Collections.singletonList(new IndexMetaData("index")))));
         assertTrue(getSchema().containsTable("t_order_0"));
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategyTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategyTest.java
index 7f61fa2..4626340 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategyTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropIndexStatementMetaDataRefreshStrategyTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.AbstractMetaDataRefreshStrategyTest;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
@@ -79,9 +79,9 @@ public final class DropIndexStatementMetaDataRefreshStrategyTest extends Abstrac
         dropIndexStatement.getIndexes().add(new IndexSegment(1, 2, new IdentifierValue("index")));
         dropIndexStatement.getIndexes().add(new IndexSegment(2, 3, new IdentifierValue("t_order_index")));
         dropIndexStatement.getIndexes().add(new IndexSegment(3, 4, new IdentifierValue("order_id_index")));
-        Map<String, PhysicalIndexMetaData> actualIndex = getSchema().get("t_order").getIndexes();
-        actualIndex.put("t_order_index", new PhysicalIndexMetaData("t_order_index"));
-        actualIndex.put("order_id_index", new PhysicalIndexMetaData("order_id_index"));
+        Map<String, IndexMetaData> actualIndex = getSchema().get("t_order").getIndexes();
+        actualIndex.put("t_order_index", new IndexMetaData("t_order_index"));
+        actualIndex.put("order_id_index", new IndexMetaData("order_id_index"));
         MetaDataRefreshStrategy<DropIndexStatement> metaDataRefreshStrategy = new DropIndexStatementMetaDataRefreshStrategy();
         metaDataRefreshStrategy.refreshMetaData(getSchema(), mock(DatabaseType.class), Collections.emptyList(), dropIndexStatement, tableName -> Optional.empty());
         assertFalse(getSchema().get("t_order").getIndexes().containsKey("index"));
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilder.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilder.java
index 1530f7c..123218f 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilder.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 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.SQLRewriteResult;
@@ -121,7 +121,7 @@ public final class ExecutionContextBuilder {
     private static List<PrimaryKeyMetaData> getPrimaryKeyColumns(final ShardingSphereSchema schema, final List<String> actualTableNames) {
         List<PrimaryKeyMetaData> result = new LinkedList<>();
         for (String each: actualTableNames) {
-            PhysicalTableMetaData tableMetaData = schema.get(each);
+            TableMetaData tableMetaData = schema.get(each);
             if (null != tableMetaData) {
                 result.add(new PrimaryKeyMetaData(each, tableMetaData.getPrimaryKeyColumns()));
             }
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 b1f27f6..ab268f8 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.metadata.resource.CachedDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.DataSourcesMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 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;
@@ -105,25 +105,25 @@ public final class ExecutionContextBuilderTest {
     }
     
     private ShardingSphereSchema buildSchemaWithoutPrimaryKey() {
-        Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
-        tableMetaDataMap.put("logicName1", new PhysicalTableMetaData(Arrays.asList(new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false),
-                new PhysicalColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
-                new PhysicalColumnMetaData("status", Types.INTEGER, "int", false, false, false)), Collections.emptySet()));
-        tableMetaDataMap.put("t_other", new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false)), Collections.emptySet()));
+        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
+        tableMetaDataMap.put("logicName1", new TableMetaData(Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, "int", true, false, false),
+                new ColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
+                new ColumnMetaData("status", Types.INTEGER, "int", false, false, false)), Collections.emptySet()));
+        tableMetaDataMap.put("t_other", new TableMetaData(Collections.singletonList(new ColumnMetaData("order_id", Types.INTEGER, "int", true, false, false)), Collections.emptySet()));
         return new ShardingSphereSchema(tableMetaDataMap);
     }
     
     private ShardingSphereSchema buildSchema() {
-        Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
-        tableMetaDataMap.put("logicName1", new PhysicalTableMetaData(Arrays.asList(new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false),
-                new PhysicalColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
-                new PhysicalColumnMetaData("status", Types.INTEGER, "int", false, false, false)), Collections.emptySet()));
-        tableMetaDataMap.put("logicName2", new PhysicalTableMetaData(Arrays.asList(new PhysicalColumnMetaData("item_id", Types.INTEGER, "int", true, false, false),
-                new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", false, false, false),
-                new PhysicalColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
-                new PhysicalColumnMetaData("status", Types.VARCHAR, "varchar", false, false, false),
-                new PhysicalColumnMetaData("c_date", Types.TIMESTAMP, "timestamp", false, false, false)), Collections.emptySet()));
-        tableMetaDataMap.put("t_other", new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false)), Collections.emptySet()));
+        Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
+        tableMetaDataMap.put("logicName1", new TableMetaData(Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, "int", true, false, false),
+                new ColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
+                new ColumnMetaData("status", Types.INTEGER, "int", false, false, false)), Collections.emptySet()));
+        tableMetaDataMap.put("logicName2", new TableMetaData(Arrays.asList(new ColumnMetaData("item_id", Types.INTEGER, "int", true, false, false),
+                new ColumnMetaData("order_id", Types.INTEGER, "int", false, false, false),
+                new ColumnMetaData("user_id", Types.INTEGER, "int", false, false, false),
+                new ColumnMetaData("status", Types.VARCHAR, "varchar", false, false, false),
+                new ColumnMetaData("c_date", Types.TIMESTAMP, "timestamp", false, false, false)), Collections.emptySet()));
+        tableMetaDataMap.put("t_other", new TableMetaData(Collections.singletonList(new ColumnMetaData("order_id", Types.INTEGER, "int", true, false, false)), Collections.emptySet()));
         return new ShardingSphereSchema(tableMetaDataMap);
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
index f2b5ab98..95f49a9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
@@ -32,7 +32,7 @@ import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.loader.TableMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategyFactory;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
@@ -110,7 +110,7 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
         }
     }
     
-    private Optional<PhysicalTableMetaData> loadTableMetaData(final String tableName) throws SQLException {
+    private Optional<TableMetaData> loadTableMetaData(final String tableName) throws SQLException {
         return TableMetaDataLoader.load(tableName, ProxyContext.getInstance().getMetaDataContexts().getDatabaseType(), 
                 metaData.getResource().getDataSources(), metaData.getRuleMetaData().getRules(), ProxyContext.getInstance().getMetaDataContexts().getProps());
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java
index 4f68923..9d1c798 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java
@@ -22,7 +22,7 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.executor.sql.raw.execute.result.query.QueryHeader;
 import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.Projection;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.ProjectionsContext;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ColumnProjection;
@@ -74,7 +74,7 @@ public final class QueryHeaderBuilder {
         boolean primaryKey;
         if (null != actualTableName && dataNodeContainedRule.isPresent()) {
             tableName = dataNodeContainedRule.get().findLogicTableByActualTable(actualTableName).orElse("");
-            PhysicalTableMetaData tableMetaData = metaData.getSchema().get(tableName);
+            TableMetaData tableMetaData = metaData.getSchema().get(tableName);
             primaryKey = null != tableMetaData && tableMetaData.getColumns().get(columnName.toLowerCase()).isPrimaryKey();
         } else {
             tableName = actualTableName;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java
index 7f90df5..a42b78c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java
@@ -24,10 +24,10 @@ import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
 import org.apache.shardingsphere.infra.executor.sql.raw.execute.result.query.QueryHeader;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.DataSourcesMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.Test;
 
@@ -127,9 +127,9 @@ public final class QueryHeaderBuilderTest {
     
     private ShardingSphereMetaData createMetaData() {
         ShardingSphereMetaData result = mock(ShardingSphereMetaData.class, RETURNS_DEEP_STUBS);
-        PhysicalColumnMetaData columnMetaData = new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false);
+        ColumnMetaData columnMetaData = new ColumnMetaData("order_id", Types.INTEGER, "int", true, false, false);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
-        when(schema.get("t_logic_order")).thenReturn(new PhysicalTableMetaData(Collections.singletonList(columnMetaData), Collections.singletonList(new PhysicalIndexMetaData("order_id"))));
+        when(schema.get("t_logic_order")).thenReturn(new TableMetaData(Collections.singletonList(columnMetaData), Collections.singletonList(new IndexMetaData("order_id"))));
         DataSourcesMetaData dataSourcesMetaData = mock(DataSourcesMetaData.class);
         when(dataSourcesMetaData.getDataSourceMetaData("ds_0")).thenReturn(mock(DataSourceMetaData.class));
         when(result.getResource().getDataSourcesMetaData()).thenReturn(dataSourcesMetaData);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
index 49c3796..b09a42e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
@@ -28,7 +28,7 @@ import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
@@ -202,7 +202,7 @@ public final class ShardingCTLHintBackendHandlerTest {
     
     private Map<String, ShardingSphereMetaData> getMetaDataMap() {
         ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
-        when(metaData.getSchema()).thenReturn(new ShardingSphereSchema(ImmutableMap.of("user", mock(PhysicalTableMetaData.class))));
+        when(metaData.getSchema()).thenReturn(new ShardingSphereSchema(ImmutableMap.of("user", mock(TableMetaData.class))));
         when(metaData.isComplete()).thenReturn(true);
         return Collections.singletonMap("schema", metaData);
     }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/execute/executor/dumper/AbstractJDBCDumper.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/execute/executor/dumper/AbstractJDBCDumper.java
index 8162708..3217946 100755
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/execute/executor/dumper/AbstractJDBCDumper.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/execute/executor/dumper/AbstractJDBCDumper.java
@@ -39,7 +39,7 @@ import org.apache.shardingsphere.scaling.core.job.position.PlaceholderInventoryP
 import org.apache.shardingsphere.scaling.core.job.position.PrimaryKeyPosition;
 import org.apache.shardingsphere.scaling.core.metadata.MetaDataManager;
 import org.apache.shardingsphere.scaling.core.utils.RdbmsConfigurationUtil;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import java.sql.Connection;
 import java.sql.PreparedStatement;
@@ -58,7 +58,7 @@ public abstract class AbstractJDBCDumper extends AbstractShardingScalingExecutor
     
     private final DataSourceManager dataSourceManager;
     
-    private final PhysicalTableMetaData tableMetaData;
+    private final TableMetaData tableMetaData;
     
     @Setter
     private Channel channel;
@@ -72,7 +72,7 @@ public abstract class AbstractJDBCDumper extends AbstractShardingScalingExecutor
         tableMetaData = createTableMetaData();
     }
     
-    private PhysicalTableMetaData createTableMetaData() {
+    private TableMetaData createTableMetaData() {
         MetaDataManager metaDataManager = new MetaDataManager(dataSourceManager.getDataSource(inventoryDumperConfiguration.getDataSourceConfiguration()));
         return metaDataManager.getTableMetaData(inventoryDumperConfiguration.getTableName());
     }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/job/preparer/splitter/InventoryDataTaskSplitter.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/job/preparer/splitter/InventoryDataTaskSplitter.java
index ee1123f..92aa621 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/job/preparer/splitter/InventoryDataTaskSplitter.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/job/preparer/splitter/InventoryDataTaskSplitter.java
@@ -31,7 +31,7 @@ import org.apache.shardingsphere.scaling.core.job.task.DefaultSyncTaskFactory;
 import org.apache.shardingsphere.scaling.core.job.task.ScalingTask;
 import org.apache.shardingsphere.scaling.core.job.task.SyncTaskFactory;
 import org.apache.shardingsphere.scaling.core.metadata.MetaDataManager;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import javax.sql.DataSource;
 import java.sql.Connection;
@@ -92,7 +92,7 @@ public final class InventoryDataTaskSplitter {
     }
     
     private boolean isSpiltByPrimaryKeyRange(final InventoryDumperConfiguration inventoryDumperConfig, final MetaDataManager metaDataManager) {
-        PhysicalTableMetaData tableMetaData = metaDataManager.getTableMetaData(inventoryDumperConfig.getTableName());
+        TableMetaData tableMetaData = metaDataManager.getTableMetaData(inventoryDumperConfig.getTableName());
         if (null == tableMetaData) {
             log.warn("Can't split range for table {}, reason: can not get table metadata ", inventoryDumperConfig.getTableName());
             return false;
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManager.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManager.java
index d2d7477..1ea16ec 100755
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManager.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManager.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.scaling.core.metadata;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.loader.physical.PhysicalTableMetaDataLoader;
 
 import javax.sql.DataSource;
@@ -35,7 +35,7 @@ public final class MetaDataManager {
     
     private final DataSource dataSource;
     
-    private final Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>();
+    private final Map<String, TableMetaData> tableMetaDataMap = new HashMap<>();
     
     /**
      * Get table meta data by table name.
@@ -43,7 +43,7 @@ public final class MetaDataManager {
      * @param tableName table name
      * @return table meta data
      */
-    public PhysicalTableMetaData getTableMetaData(final String tableName) {
+    public TableMetaData getTableMetaData(final String tableName) {
         if (!tableMetaDataMap.containsKey(tableName)) {
             try {
                 PhysicalTableMetaDataLoader.load(dataSource, tableName, DatabaseTypeRegistry.getActualDatabaseType("MySQL")).ifPresent(tableMetaData -> tableMetaDataMap.put(tableName, tableMetaData));
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/test/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManagerTest.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/test/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManagerTest.java
index 9a248bc..59a1629 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/test/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManagerTest.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/test/java/org/apache/shardingsphere/scaling/core/metadata/MetaDataManagerTest.java
@@ -17,8 +17,8 @@
 
 package org.apache.shardingsphere.scaling.core.metadata;
 
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -118,14 +118,14 @@ public final class MetaDataManagerTest {
         assertThat(actual.get(0), is("id"));
     }
     
-    private void assertColumnMetaData(final PhysicalTableMetaData actual) {
+    private void assertColumnMetaData(final TableMetaData actual) {
         assertThat(actual.getColumns().size(), is(3));
         assertColumnMetaData(actual.getColumnMetaData(0), "id", Types.BIGINT, "BIGINT");
         assertColumnMetaData(actual.getColumnMetaData(1), "name", Types.VARCHAR, "VARCHAR");
         assertColumnMetaData(actual.getColumnMetaData(2), "age", Types.INTEGER, "INTEGER");
     }
     
-    private void assertColumnMetaData(final PhysicalColumnMetaData actual, final String expectedName, final int expectedType, final String expectedTypeName) {
+    private void assertColumnMetaData(final ColumnMetaData actual, final String expectedName, final int expectedType, final String expectedTypeName) {
         assertThat(actual.getName(), is(expectedName));
         assertThat(actual.getDataType(), is(expectedType));
         assertThat(actual.getDataTypeName(), is(expectedTypeName));
diff --git a/shardingsphere-scaling/shardingsphere-scaling-mysql/src/main/java/org/apache/shardingsphere/scaling/mysql/MySQLBinlogDumper.java b/shardingsphere-scaling/shardingsphere-scaling-mysql/src/main/java/org/apache/shardingsphere/scaling/mysql/MySQLBinlogDumper.java
index 5bdb67c..b9eb885 100755
--- a/shardingsphere-scaling/shardingsphere-scaling-mysql/src/main/java/org/apache/shardingsphere/scaling/mysql/MySQLBinlogDumper.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-mysql/src/main/java/org/apache/shardingsphere/scaling/mysql/MySQLBinlogDumper.java
@@ -44,7 +44,7 @@ import org.apache.shardingsphere.scaling.mysql.binlog.event.UpdateRowsEvent;
 import org.apache.shardingsphere.scaling.mysql.binlog.event.WriteRowsEvent;
 import org.apache.shardingsphere.scaling.mysql.client.ConnectInfo;
 import org.apache.shardingsphere.scaling.mysql.client.MySQLClient;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import java.io.Serializable;
 import java.security.SecureRandom;
@@ -117,7 +117,7 @@ public final class MySQLBinlogDumper extends AbstractShardingScalingExecutor<Bin
     }
     
     private void handleWriteRowsEvent(final WriteRowsEvent event) {
-        PhysicalTableMetaData tableMetaData = metaDataManager.getTableMetaData(event.getTableName());
+        TableMetaData tableMetaData = metaDataManager.getTableMetaData(event.getTableName());
         for (Serializable[] each : event.getAfterRows()) {
             DataRecord record = createDataRecord(event, each.length);
             record.setType(ScalingConstant.INSERT);
@@ -129,7 +129,7 @@ public final class MySQLBinlogDumper extends AbstractShardingScalingExecutor<Bin
     }
     
     private void handleUpdateRowsEvent(final UpdateRowsEvent event) {
-        PhysicalTableMetaData tableMetaData = metaDataManager.getTableMetaData(event.getTableName());
+        TableMetaData tableMetaData = metaDataManager.getTableMetaData(event.getTableName());
         for (int i = 0; i < event.getBeforeRows().size(); i++) {
             Serializable[] beforeValues = event.getBeforeRows().get(i);
             Serializable[] afterValues = event.getAfterRows().get(i);
@@ -145,7 +145,7 @@ public final class MySQLBinlogDumper extends AbstractShardingScalingExecutor<Bin
     }
     
     private void handleDeleteRowsEvent(final DeleteRowsEvent event) {
-        PhysicalTableMetaData tableMetaData = metaDataManager.getTableMetaData(event.getTableName());
+        TableMetaData tableMetaData = metaDataManager.getTableMetaData(event.getTableName());
         for (Serializable[] each : event.getBeforeRows()) {
             DataRecord record = createDataRecord(event, each.length);
             record.setType(ScalingConstant.DELETE);
diff --git a/shardingsphere-scaling/shardingsphere-scaling-postgresql/src/main/java/org/apache/shardingsphere/scaling/postgresql/wal/WalEventConverter.java b/shardingsphere-scaling/shardingsphere-scaling-postgresql/src/main/java/org/apache/shardingsphere/scaling/postgresql/wal/WalEventConverter.java
index 06010a4..281c009 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-postgresql/src/main/java/org/apache/shardingsphere/scaling/postgresql/wal/WalEventConverter.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-postgresql/src/main/java/org/apache/shardingsphere/scaling/postgresql/wal/WalEventConverter.java
@@ -33,7 +33,7 @@ import org.apache.shardingsphere.scaling.postgresql.wal.event.DeleteRowEvent;
 import org.apache.shardingsphere.scaling.postgresql.wal.event.PlaceholderEvent;
 import org.apache.shardingsphere.scaling.postgresql.wal.event.UpdateRowEvent;
 import org.apache.shardingsphere.scaling.postgresql.wal.event.WriteRowEvent;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 
 import java.util.List;
 
@@ -122,7 +122,7 @@ public final class WalEventConverter {
         return result;
     }
     
-    private void putColumnsIntoDataRecord(final DataRecord dataRecord, final PhysicalTableMetaData tableMetaData, final List<Object> values) {
+    private void putColumnsIntoDataRecord(final DataRecord dataRecord, final TableMetaData tableMetaData, final List<Object> values) {
         for (int i = 0; i < values.size(); i++) {
             dataRecord.addColumn(new Column(tableMetaData.getColumnMetaData(i).getName(), values.get(i), true, tableMetaData.isPrimaryKey(i)));
         }