You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by tu...@apache.org on 2023/06/06 06:21:37 UTC

[shardingsphere] branch master updated: Validate table exist when execute insert, delete, update and select statement (#26066)

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

tuichenchuxin 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 77c230ca6a9 Validate table exist when execute insert, delete, update and select statement (#26066)
77c230ca6a9 is described below

commit 77c230ca6a9472e77ed31e6d26f615c68d157a4e
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Tue Jun 6 14:21:29 2023 +0800

    Validate table exist when execute insert, delete, update and select statement (#26066)
    
    * Validate table exist when execute insert, delete, update and select statement
    
    * fix unit test
    
    * fix sql rewrite test case
---
 .../dql/groupby/GroupByMemoryMergedResult.java     |  2 +
 .../sharding/route/engine/ShardingSQLRouter.java   |  1 -
 .../InsertClauseShardingConditionEngine.java       |  3 +
 .../ShardingStatementValidatorFactory.java         |  5 ++
 .../dml/impl/ShardingDeleteStatementValidator.java |  2 +
 .../dml/impl/ShardingInsertStatementValidator.java |  2 +
 ....java => ShardingSelectStatementValidator.java} | 18 ++----
 .../dml/impl/ShardingUpdateStatementValidator.java |  2 +
 .../dql/groupby/GroupByMemoryMergedResultTest.java |  3 +-
 .../InsertClauseShardingConditionEngineTest.java   |  1 +
 .../standard/assertion/ShardingRouteAssert.java    |  3 +
 .../dml/ShardingDeleteStatementValidatorTest.java  | 13 +++-
 .../dml/ShardingInsertStatementValidatorTest.java  | 50 +++++++++++-----
 .../dml/ShardingUpdateStatementValidatorTest.java  | 27 +++++++--
 .../ShardingSphereMetaDataValidateUtils.java       | 51 ++++++++++++++++
 kernel/single/core/pom.xml                         |  5 ++
 .../single/route/SingleSQLRouter.java              | 18 ++++--
 .../validator/SingleMetaDataValidatorFactory.java  |  1 -
 .../single/route/SingleSQLRouterTest.java          |  9 ++-
 .../engine/scenario/EncryptSQLRewriterIT.java      |  7 +++
 .../rewrite/engine/scenario/MixSQLRewriterIT.java  |  4 +-
 .../engine/scenario/ShardingSQLRewriterIT.java     | 10 ++++
 .../query-with-cipher/ddl/create/create-table.xml  | 24 ++++----
 .../scenario/mix/config/query-with-cipher.yaml     | 23 +++++++
 .../scenario/sharding/case/ddl/create-table.xml    | 70 +++++++++++-----------
 .../scenario/sharding/config/sharding-rule.yaml    | 15 ++++-
 26 files changed, 277 insertions(+), 92 deletions(-)

diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
index 729dc896b2c..62901fbf5fc 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResult.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.sharding.merge.dql.groupby;
 
+import org.apache.shardingsphere.dialect.exception.syntax.table.NoSuchTableException;
 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,6 +129,7 @@ public final class GroupByMemoryMergedResult extends MemoryMergedResult<Sharding
                                                     final SelectStatementContext selectStatementContext, final ShardingSphereSchema schema, final int columnIndex) throws SQLException {
         for (SimpleTableSegment each : selectStatementContext.getAllTables()) {
             String tableName = each.getTableName().getIdentifier().getValue();
+            ShardingSpherePreconditions.checkState(schema.containsTable(tableName), () -> new NoSuchTableException(tableName));
             ShardingSphereTable table = schema.getTable(tableName);
             String columnName = queryResult.getMetaData().getColumnName(columnIndex);
             if (table.containsColumn(columnName)) {
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
index 740362579d9..751082ffd14 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
@@ -59,7 +59,6 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
         return createRouteContext0(queryContext, globalRuleMetaData, database, rule, props, connectionContext);
     }
     
-    @SuppressWarnings({"rawtypes", "unchecked"})
     private RouteContext createRouteContext0(final QueryContext queryContext, final ShardingSphereRuleMetaData globalRuleMetaData, final ShardingSphereDatabase database, final ShardingRule rule,
                                              final ConfigurationProperties props, final ConnectionContext connectionContext) {
         SQLStatement sqlStatement = queryContext.getSqlStatementContext().getSqlStatement();
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java
index 48ada0ec921..f59f782679a 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngine.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.sharding.route.engine.condition.engine;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.dialect.exception.data.InsertColumnsAndValuesMismatchedException;
+import org.apache.shardingsphere.dialect.exception.syntax.table.NoSuchTableException;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.infra.binder.segment.insert.values.InsertValueContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
@@ -26,6 +27,7 @@ import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementConte
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.sharding.route.engine.condition.ExpressionConditionUtils;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
 import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
@@ -92,6 +94,7 @@ public final class InsertClauseShardingConditionEngine {
         String defaultSchemaName = DatabaseTypeEngine.getDefaultSchemaName(sqlStatementContext.getDatabaseType(), database.getName());
         ShardingSphereSchema schema = sqlStatementContext.getTablesContext().getSchemaName().map(database::getSchema).orElseGet(() -> database.getSchema(defaultSchemaName));
         String tableName = sqlStatementContext.getSqlStatement().getTable().getTableName().getIdentifier().getValue();
+        ShardingSpherePreconditions.checkState(schema.containsTable(tableName), () -> new NoSuchTableException(tableName));
         Collection<String> allColumnNames = schema.getTable(tableName).getColumnNames();
         if (columnNames.size() == allColumnNames.size()) {
             return;
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
index 1eb36fe2e60..b7b0622e89e 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
@@ -37,6 +37,7 @@ import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.Shardi
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingInsertStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingLoadDataStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingLoadXMLStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingSelectStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingUpdateStatementValidator;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterIndexStatement;
@@ -56,6 +57,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStateme
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
@@ -135,6 +137,9 @@ public final class ShardingStatementValidatorFactory {
         if (sqlStatement instanceof DeleteStatement) {
             return Optional.of(new ShardingDeleteStatementValidator());
         }
+        if (sqlStatement instanceof SelectStatement) {
+            return Optional.of(new ShardingSelectStatementValidator());
+        }
         if (sqlStatement instanceof CopyStatement) {
             return Optional.of(new ShardingCopyStatementValidator());
         }
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
index 9b49d39f934..0b12ab1a869 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.connection.validator.ShardingSphereMetaDataValidateUtils;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.sharding.exception.syntax.DMLMultipleDataNodesWithLimitException;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
@@ -38,6 +39,7 @@ public final class ShardingDeleteStatementValidator extends ShardingDMLStatement
     @Override
     public void preValidate(final ShardingRule shardingRule, final SQLStatementContext sqlStatementContext,
                             final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props) {
+        ShardingSphereMetaDataValidateUtils.validateTableExist(sqlStatementContext, database);
         validateMultipleTable(shardingRule, sqlStatementContext);
     }
     
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingInsertStatementValidator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingInsertStatementValidator.java
index 3fe8023ea0c..106ba1d3aaf 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingInsertStatementValidator.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingInsertStatementValidator.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.connection.validator.ShardingSphereMetaDataValidateUtils;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -58,6 +59,7 @@ public final class ShardingInsertStatementValidator extends ShardingDMLStatement
     @Override
     public void preValidate(final ShardingRule shardingRule, final SQLStatementContext sqlStatementContext,
                             final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props) {
+        ShardingSphereMetaDataValidateUtils.validateTableExist(sqlStatementContext, database);
         if (null == ((InsertStatementContext) sqlStatementContext).getInsertSelectContext()) {
             validateMultipleTable(shardingRule, sqlStatementContext);
         }
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingSelectStatementValidator.java
similarity index 65%
copy from features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
copy to features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingSelectStatementValidator.java
index 9b49d39f934..268bf0fa7e6 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingSelectStatementValidator.java
@@ -19,34 +19,28 @@ package org.apache.shardingsphere.sharding.route.engine.validator.dml.impl;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.connection.validator.ShardingSphereMetaDataValidateUtils;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.exception.syntax.DMLMultipleDataNodesWithLimitException;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.DeleteStatementHandler;
 
 import java.util.List;
 
 /**
- * Sharding delete statement validator.
+ * Sharding select statement validator.
  */
-public final class ShardingDeleteStatementValidator extends ShardingDMLStatementValidator {
+public final class ShardingSelectStatementValidator extends ShardingDMLStatementValidator {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext sqlStatementContext,
-                            final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props) {
-        validateMultipleTable(shardingRule, sqlStatementContext);
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext sqlStatementContext, final List<Object> params, final ShardingSphereDatabase database,
+                            final ConfigurationProperties props) {
+        ShardingSphereMetaDataValidateUtils.validateTableExist(sqlStatementContext, database);
     }
     
     @Override
     public void postValidate(final ShardingRule shardingRule, final SQLStatementContext sqlStatementContext, final HintValueContext hintValueContext, final List<Object> params,
                              final ShardingSphereDatabase database, final ConfigurationProperties props, final RouteContext routeContext) {
-        if (!shardingRule.isBroadcastTable(sqlStatementContext.getTablesContext().getTableNames().iterator().next())
-                && DeleteStatementHandler.getLimitSegment((DeleteStatement) sqlStatementContext.getSqlStatement()).isPresent() && routeContext.getRouteUnits().size() > 1) {
-            throw new DMLMultipleDataNodesWithLimitException("DELETE");
-        }
     }
 }
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingUpdateStatementValidator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingUpdateStatementValidator.java
index b0728d8041c..75808a7f183 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingUpdateStatementValidator.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingUpdateStatementValidator.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.sharding.route.engine.validator.dml.impl;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.connection.validator.ShardingSphereMetaDataValidateUtils;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -43,6 +44,7 @@ public final class ShardingUpdateStatementValidator extends ShardingDMLStatement
     @Override
     public void preValidate(final ShardingRule shardingRule, final SQLStatementContext sqlStatementContext, final List<Object> params, final ShardingSphereDatabase database,
                             final ConfigurationProperties props) {
+        ShardingSphereMetaDataValidateUtils.validateTableExist(sqlStatementContext, database);
         validateMultipleTable(shardingRule, sqlStatementContext);
     }
     
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java
index 0672ca3cc5a..565ee6c4973 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/merge/dql/groupby/GroupByMemoryMergedResultTest.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.sharding.merge.dql.groupby;
 
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
@@ -29,6 +28,7 @@ import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
+import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.sharding.merge.dql.ShardingDQLResultMerger;
 import org.apache.shardingsphere.sql.parser.sql.common.enums.AggregationType;
@@ -205,6 +205,7 @@ class GroupByMemoryMergedResultTest {
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
         ShardingSphereTable table = mock(ShardingSphereTable.class);
         when(schema.getTable("t_order")).thenReturn(table);
+        when(schema.containsTable("t_order")).thenReturn(true);
         when(table.getColumns()).thenReturn(Collections.emptyList());
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         when(database.getSchema(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngineTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngineTest.java
index 67f3c65905e..eb7ac907af3 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngineTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/InsertClauseShardingConditionEngineTest.java
@@ -92,6 +92,7 @@ class InsertClauseShardingConditionEngineTest {
         ShardingSphereDatabase result = mock(ShardingSphereDatabase.class);
         when(result.getName()).thenReturn(DefaultDatabase.LOGIC_NAME);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS);
+        when(schema.containsTable("foo_table")).thenReturn(true);
         when(schema.getTable("foo_table").getColumnNames()).thenReturn(Arrays.asList("foo_col_1", "foo_col_2"));
         when(result.getSchema(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
         return result;
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/assertion/ShardingRouteAssert.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/assertion/ShardingRouteAssert.java
index d359cbf334a..b5f9265f695 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/assertion/ShardingRouteAssert.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/assertion/ShardingRouteAssert.java
@@ -105,6 +105,9 @@ public final class ShardingRouteAssert {
         tables.put("t_other", new ShardingSphereTable("t_other", Collections.singletonList(
                 new ShardingSphereColumn("order_id", Types.INTEGER, true, false, false, true, false)), Collections.emptyList(), Collections.emptyList()));
         tables.put("t_category", new ShardingSphereTable("t_category", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        tables.put("t_product", new ShardingSphereTable("t_product", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        tables.put("t_user", new ShardingSphereTable("t_user", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        tables.put("t_hint_test", new ShardingSphereTable("t_hint_test", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
         return Collections.singletonMap(DefaultDatabase.LOGIC_NAME, new ShardingSphereSchema(tables, Collections.emptyMap()));
     }
 }
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
index 1b0b75cddab..86d8611f693 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.sharding.route.engine.validator.dml;
 import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.sharding.exception.syntax.DMLWithMultipleShardingTablesException;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingDeleteStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -44,6 +45,7 @@ import java.util.Collections;
 import java.util.HashSet;
 
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -84,10 +86,17 @@ class ShardingDeleteStatementValidatorTest {
         tableSegment.getActualDeleteTables().add(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("order"))));
         tableSegment.getActualDeleteTables().add(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("order_item"))));
         sqlStatement.setTable(tableSegment);
-        DeleteStatementContext sqlStatementContext = new DeleteStatementContext(sqlStatement);
         Collection<String> tableNames = new HashSet<>(Arrays.asList("user", "order", "order_item"));
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(false);
         when(shardingRule.tableRuleExists(tableNames)).thenReturn(true);
-        new ShardingDeleteStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class));
+        ShardingSphereDatabase database = mock(ShardingSphereDatabase.class);
+        ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
+        when(schema.containsTable("user")).thenReturn(true);
+        when(schema.containsTable("order")).thenReturn(true);
+        when(schema.containsTable("order_item")).thenReturn(true);
+        DeleteStatementContext sqlStatementContext = new DeleteStatementContext(sqlStatement);
+        new ShardingDeleteStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class));
     }
 }
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingInsertStatementValidatorTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingInsertStatementValidatorTest.java
index 05d97472cfd..a3fd8ff825f 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingInsertStatementValidatorTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingInsertStatementValidatorTest.java
@@ -75,7 +75,7 @@ import java.util.Optional;
 
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -91,18 +91,26 @@ class ShardingInsertStatementValidatorTest {
     @Mock
     private ShardingConditions shardingConditions;
     
+    @Mock
+    private ShardingSphereDatabase database;
+    
+    @Mock
+    private ShardingSphereSchema schema;
+    
     @Test
     void assertPreValidateWhenInsertMultiTables() {
         SQLStatementContext sqlStatementContext = createInsertStatementContext(Collections.singletonList(1), createInsertStatement());
         Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(false);
         when(shardingRule.tableRuleExists(tableNames)).thenReturn(true);
+        when(schema.containsTable("user")).thenReturn(true);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
         assertThrows(DMLWithMultipleShardingTablesException.class, () -> new ShardingInsertStatementValidator(shardingConditions).preValidate(shardingRule,
-                sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class)));
+                sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class)));
     }
     
     private InsertStatementContext createInsertStatementContext(final List<Object> params, final InsertStatement insertStatement) {
-        ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         when(database.getSchema(DefaultDatabase.LOGIC_NAME)).thenReturn(mock(ShardingSphereSchema.class));
         ShardingSphereMetaData metaData =
                 new ShardingSphereMetaData(Collections.singletonMap(DefaultDatabase.LOGIC_NAME, database), mock(ShardingSphereRuleMetaData.class), mock(ConfigurationProperties.class));
@@ -115,8 +123,11 @@ class ShardingInsertStatementValidatorTest {
         when(shardingRule.isGenerateKeyColumn("id", "user")).thenReturn(false);
         SQLStatementContext sqlStatementContext = createInsertStatementContext(Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTableNames().addAll(createSingleTablesContext().getTableNames());
+        when(schema.containsTable("user")).thenReturn(true);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
         assertThrows(MissingGenerateKeyColumnWithInsertSelectException.class, () -> new ShardingInsertStatementValidator(shardingConditions).preValidate(shardingRule,
-                sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class)));
+                sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class)));
     }
     
     @Test
@@ -125,8 +136,11 @@ class ShardingInsertStatementValidatorTest {
         when(shardingRule.isGenerateKeyColumn("id", "user")).thenReturn(true);
         SQLStatementContext sqlStatementContext = createInsertStatementContext(Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTableNames().addAll(createSingleTablesContext().getTableNames());
+        when(schema.containsTable("user")).thenReturn(true);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
         assertDoesNotThrow(() -> new ShardingInsertStatementValidator(shardingConditions).preValidate(
-                shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class)));
+                shardingRule, sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class)));
     }
     
     @Test
@@ -138,8 +152,12 @@ class ShardingInsertStatementValidatorTest {
         when(shardingRule.tableRuleExists(multiTablesContext.getTableNames())).thenReturn(true);
         SQLStatementContext sqlStatementContext = createInsertStatementContext(Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTableNames().addAll(multiTablesContext.getTableNames());
+        when(schema.containsTable("user")).thenReturn(true);
+        when(schema.containsTable("order")).thenReturn(true);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
         assertThrows(InsertSelectTableViolationException.class, () -> new ShardingInsertStatementValidator(shardingConditions).preValidate(
-                shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class)));
+                shardingRule, sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class)));
     }
     
     @Test
@@ -149,8 +167,12 @@ class ShardingInsertStatementValidatorTest {
         TablesContext multiTablesContext = createMultiTablesContext();
         SQLStatementContext sqlStatementContext = createInsertStatementContext(Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTableNames().addAll(multiTablesContext.getTableNames());
+        when(schema.containsTable("user")).thenReturn(true);
+        when(schema.containsTable("order")).thenReturn(true);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
         assertDoesNotThrow(() -> new ShardingInsertStatementValidator(shardingConditions).preValidate(
-                shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class)));
+                shardingRule, sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class)));
     }
     
     @Test
@@ -158,7 +180,7 @@ class ShardingInsertStatementValidatorTest {
         SQLStatementContext sqlStatementContext = createInsertStatementContext(Collections.singletonList(1), createInsertStatement());
         when(routeContext.isSingleRouting()).thenReturn(true);
         assertDoesNotThrow(() -> new ShardingInsertStatementValidator(shardingConditions).postValidate(shardingRule, sqlStatementContext, new HintValueContext(),
-                Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class), routeContext));
+                Collections.emptyList(), database, mock(ConfigurationProperties.class), routeContext));
     }
     
     @Test
@@ -167,7 +189,7 @@ class ShardingInsertStatementValidatorTest {
         when(routeContext.isSingleRouting()).thenReturn(false);
         when(shardingRule.isBroadcastTable(((InsertStatement) sqlStatementContext.getSqlStatement()).getTable().getTableName().getIdentifier().getValue())).thenReturn(true);
         assertDoesNotThrow(() -> new ShardingInsertStatementValidator(shardingConditions).postValidate(shardingRule, sqlStatementContext, new HintValueContext(),
-                Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class), routeContext));
+                Collections.emptyList(), database, mock(ConfigurationProperties.class), routeContext));
     }
     
     @Test
@@ -177,7 +199,7 @@ class ShardingInsertStatementValidatorTest {
         when(shardingRule.isBroadcastTable(((InsertStatement) sqlStatementContext.getSqlStatement()).getTable().getTableName().getIdentifier().getValue())).thenReturn(false);
         when(routeContext.getOriginalDataNodes()).thenReturn(getSingleRouteDataNodes());
         assertDoesNotThrow(() -> new ShardingInsertStatementValidator(shardingConditions).postValidate(shardingRule, sqlStatementContext, new HintValueContext(),
-                Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class), routeContext));
+                Collections.emptyList(), database, mock(ConfigurationProperties.class), routeContext));
     }
     
     @Test
@@ -187,7 +209,7 @@ class ShardingInsertStatementValidatorTest {
         when(shardingRule.isBroadcastTable(((InsertStatement) sqlStatementContext.getSqlStatement()).getTable().getTableName().getIdentifier().getValue())).thenReturn(false);
         when(routeContext.getOriginalDataNodes()).thenReturn(getMultipleRouteDataNodes());
         assertThrows(DuplicateInsertDataRecordException.class, () -> new ShardingInsertStatementValidator(shardingConditions).postValidate(shardingRule, sqlStatementContext, new HintValueContext(),
-                Collections.emptyList(), mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class), routeContext));
+                Collections.emptyList(), database, mock(ConfigurationProperties.class), routeContext));
     }
     
     @Test
@@ -197,7 +219,7 @@ class ShardingInsertStatementValidatorTest {
         when(routeContext.isSingleRouting()).thenReturn(true);
         InsertStatementContext insertStatementContext = createInsertStatementContext(params, createInsertStatement());
         assertDoesNotThrow(() -> new ShardingInsertStatementValidator(mock(ShardingConditions.class)).postValidate(
-                shardingRule, insertStatementContext, new HintValueContext(), params, mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class), routeContext));
+                shardingRule, insertStatementContext, new HintValueContext(), params, database, mock(ConfigurationProperties.class), routeContext));
     }
     
     @Test
@@ -206,7 +228,7 @@ class ShardingInsertStatementValidatorTest {
         List<Object> params = Collections.singletonList(1);
         InsertStatementContext insertStatementContext = createInsertStatementContext(params, createInsertStatement());
         assertDoesNotThrow(() -> new ShardingInsertStatementValidator(mock(ShardingConditions.class)).postValidate(shardingRule,
-                insertStatementContext, new HintValueContext(), params, mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class), createSingleRouteContext()));
+                insertStatementContext, new HintValueContext(), params, database, mock(ConfigurationProperties.class), createSingleRouteContext()));
     }
     
     @Test
@@ -215,7 +237,7 @@ class ShardingInsertStatementValidatorTest {
         List<Object> params = Collections.singletonList(1);
         InsertStatementContext insertStatementContext = createInsertStatementContext(params, createInsertStatement());
         assertThrows(UnsupportedUpdatingShardingValueException.class, () -> new ShardingInsertStatementValidator(mock(ShardingConditions.class)).postValidate(shardingRule,
-                insertStatementContext, new HintValueContext(), params, mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS), mock(ConfigurationProperties.class), createFullRouteContext()));
+                insertStatementContext, new HintValueContext(), params, database, mock(ConfigurationProperties.class), createFullRouteContext()));
     }
     
     private void mockShardingRuleForUpdateShardingColumn() {
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
index e2fcb713737..45de014ea99 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementConte
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
@@ -61,6 +62,7 @@ import java.util.Optional;
 
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -70,6 +72,12 @@ class ShardingUpdateStatementValidatorTest {
     @Mock
     private ShardingRule shardingRule;
     
+    @Mock
+    private ShardingSphereDatabase database;
+    
+    @Mock
+    private ShardingSphereSchema schema;
+    
     @Test
     void assertPreValidateWhenUpdateSingleTable() {
         UpdateStatement updateStatement = createUpdateStatement();
@@ -78,8 +86,11 @@ class ShardingUpdateStatementValidatorTest {
         Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(true);
         when(shardingRule.tableRuleExists(tableNames)).thenReturn(true);
+        when(schema.containsTable("user")).thenReturn(true);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
         assertDoesNotThrow(() -> new ShardingUpdateStatementValidator().preValidate(
-                shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class)));
+                shardingRule, sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class)));
     }
     
     @Test
@@ -93,22 +104,26 @@ class ShardingUpdateStatementValidatorTest {
         Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(false);
         when(shardingRule.tableRuleExists(tableNames)).thenReturn(true);
+        when(schema.containsTable("user")).thenReturn(true);
+        when(schema.containsTable("order")).thenReturn(true);
+        when(database.getSchema(any())).thenReturn(schema);
+        when(database.getName()).thenReturn("sharding_db");
         assertThrows(DMLWithMultipleShardingTablesException.class, () -> new ShardingUpdateStatementValidator().preValidate(
-                shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class)));
+                shardingRule, sqlStatementContext, Collections.emptyList(), database, mock(ConfigurationProperties.class)));
     }
     
     @Test
     void assertPostValidateWhenNotUpdateShardingColumn() {
         UpdateStatementContext sqlStatementContext = new UpdateStatementContext(createUpdateStatement());
         assertDoesNotThrow(() -> new ShardingUpdateStatementValidator().postValidate(shardingRule,
-                sqlStatementContext, new HintValueContext(), Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class), mock(RouteContext.class)));
+                sqlStatementContext, new HintValueContext(), Collections.emptyList(), database, mock(ConfigurationProperties.class), mock(RouteContext.class)));
     }
     
     @Test
     void assertPostValidateWhenUpdateShardingColumnWithSameRouteContext() {
         mockShardingRuleForUpdateShardingColumn();
         assertDoesNotThrow(() -> new ShardingUpdateStatementValidator().postValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()),
-                new HintValueContext(), Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class), createSingleRouteContext()));
+                new HintValueContext(), Collections.emptyList(), database, mock(ConfigurationProperties.class), createSingleRouteContext()));
     }
     
     @Test
@@ -116,7 +131,7 @@ class ShardingUpdateStatementValidatorTest {
         mockShardingRuleForUpdateShardingColumn();
         when(shardingRule.isBroadcastTable("user")).thenReturn(true);
         assertDoesNotThrow(() -> new ShardingUpdateStatementValidator().postValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()),
-                new HintValueContext(), Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class), createSingleRouteContext()));
+                new HintValueContext(), Collections.emptyList(), database, mock(ConfigurationProperties.class), createSingleRouteContext()));
     }
     
     @Test
@@ -124,7 +139,7 @@ class ShardingUpdateStatementValidatorTest {
         mockShardingRuleForUpdateShardingColumn();
         assertThrows(UnsupportedUpdatingShardingValueException.class,
                 () -> new ShardingUpdateStatementValidator().postValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()), new HintValueContext(),
-                        Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class), createFullRouteContext()));
+                        Collections.emptyList(), database, mock(ConfigurationProperties.class), createFullRouteContext()));
     }
     
     private void mockShardingRuleForUpdateShardingColumn() {
diff --git a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/validator/ShardingSphereMetaDataValidateUtils.java b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/validator/ShardingSphereMetaDataValidateUtils.java
new file mode 100644
index 00000000000..f47d6b3f4a6
--- /dev/null
+++ b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/validator/ShardingSphereMetaDataValidateUtils.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.connection.validator;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.dialect.exception.syntax.table.NoSuchTableException;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
+
+/**
+ * ShardingSphere meta data validate utility class.
+ */
+// TODO consider add common ShardingSphereMetaDataValidateEngine for all features
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ShardingSphereMetaDataValidateUtils {
+    
+    /**
+     * Validate table exist.
+     *
+     * @param sqlStatementContext sql statement context
+     * @param database database
+     * @throws NoSuchTableException no such table exception
+     */
+    public static void validateTableExist(final SQLStatementContext sqlStatementContext, final ShardingSphereDatabase database) {
+        String defaultSchemaName = DatabaseTypeEngine.getDefaultSchemaName(sqlStatementContext.getDatabaseType(), database.getName());
+        ShardingSphereSchema schema = sqlStatementContext.getTablesContext().getSchemaName().map(database::getSchema).orElseGet(() -> database.getSchema(defaultSchemaName));
+        for (String each : sqlStatementContext.getTablesContext().getTableNames()) {
+            if (!schema.containsTable(each)) {
+                throw new NoSuchTableException(each);
+            }
+        }
+    }
+}
diff --git a/kernel/single/core/pom.xml b/kernel/single/core/pom.xml
index 005c5a32de9..5e14d1999ff 100644
--- a/kernel/single/core/pom.xml
+++ b/kernel/single/core/pom.xml
@@ -39,6 +39,11 @@
             <artifactId>shardingsphere-infra-route</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-infra-context</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java
index c74f83b6009..db64109fa6d 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java
@@ -18,13 +18,12 @@
 package org.apache.shardingsphere.single.route;
 
 import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.binder.type.IndexAvailable;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
-import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
+import org.apache.shardingsphere.infra.connection.validator.ShardingSphereMetaDataValidateUtils;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -35,13 +34,20 @@ import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
+import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.single.constant.SingleOrder;
 import org.apache.shardingsphere.single.route.engine.SingleRouteEngineFactory;
 import org.apache.shardingsphere.single.route.validator.SingleMetaDataValidator;
 import org.apache.shardingsphere.single.route.validator.SingleMetaDataValidatorFactory;
 import org.apache.shardingsphere.single.rule.SingleRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -61,13 +67,17 @@ public final class SingleSQLRouter implements SQLRouter<SingleRule> {
         }
         RouteContext result = new RouteContext();
         SQLStatementContext sqlStatementContext = queryContext.getSqlStatementContext();
-        Optional<SingleMetaDataValidator> validator = SingleMetaDataValidatorFactory.newInstance(sqlStatementContext.getSqlStatement());
+        SQLStatement sqlStatement = sqlStatementContext.getSqlStatement();
+        Optional<SingleMetaDataValidator> validator = SingleMetaDataValidatorFactory.newInstance(sqlStatement);
         validator.ifPresent(optional -> optional.validate(rule, sqlStatementContext, database));
         Collection<QualifiedTable> singleTableNames = getSingleTableNames(sqlStatementContext, database, rule, result);
         if (!singleTableNames.isEmpty()) {
+            if (sqlStatement instanceof InsertStatement || sqlStatement instanceof DeleteStatement || sqlStatement instanceof UpdateStatement || sqlStatement instanceof SelectStatement) {
+                ShardingSphereMetaDataValidateUtils.validateTableExist(sqlStatementContext, database);
+            }
             validateSameDataSource(sqlStatementContext, rule, props, singleTableNames, result);
         }
-        SingleRouteEngineFactory.newInstance(singleTableNames, sqlStatementContext.getSqlStatement()).ifPresent(optional -> optional.route(result, rule));
+        SingleRouteEngineFactory.newInstance(singleTableNames, sqlStatement).ifPresent(optional -> optional.route(result, rule));
         return result;
     }
     
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/validator/SingleMetaDataValidatorFactory.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/validator/SingleMetaDataValidatorFactory.java
index 3861aad0dec..d8568b47297 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/validator/SingleMetaDataValidatorFactory.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/validator/SingleMetaDataValidatorFactory.java
@@ -39,7 +39,6 @@ public final class SingleMetaDataValidatorFactory {
      * @param sqlStatement SQL statement
      * @return created instance
      */
-    @SuppressWarnings("rawtypes")
     public static Optional<SingleMetaDataValidator> newInstance(final SQLStatement sqlStatement) {
         if (sqlStatement instanceof DropSchemaStatement) {
             return Optional.of(new SingleDropSchemaMetaDataValidator());
diff --git a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
index 23c1b003f6a..57cda75be26 100644
--- a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
+++ b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
@@ -17,20 +17,21 @@
 
 package org.apache.shardingsphere.single.route;
 
-import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateTableStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResourceMetaData;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
+import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.util.spi.type.ordered.OrderedSPILoader;
 import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
 import org.apache.shardingsphere.single.rule.SingleRule;
@@ -57,6 +58,7 @@ import java.util.Properties;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -148,6 +150,9 @@ class SingleSQLRouterTest {
         dataSourceMap.put("ds_1", new MockedDataSource());
         when(result.getResourceMetaData().getDataSources()).thenReturn(dataSourceMap);
         when(result.getName()).thenReturn(DefaultDatabase.LOGIC_NAME);
+        ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
+        when(schema.containsTable("t_order")).thenReturn(true);
+        when(result.getSchema(any())).thenReturn(schema);
         return result;
     }
     
diff --git a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/EncryptSQLRewriterIT.java b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/EncryptSQLRewriterIT.java
index 1ad26d24261..620ad795fc5 100644
--- a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/EncryptSQLRewriterIT.java
+++ b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/EncryptSQLRewriterIT.java
@@ -71,7 +71,14 @@ class EncryptSQLRewriterIT extends SQLRewriterIT {
         when(result.getVisibleColumnNames("t_account_bak")).thenReturn(Arrays.asList("account_id", "certificate_number", "password", "amount"));
         when(result.getVisibleColumnNames("t_account_detail")).thenReturn(Arrays.asList("account_id", "certificate_number", "password", "amount"));
         when(result.getVisibleColumnNames("t_order")).thenReturn(Arrays.asList("ORDER_ID", "USER_ID", "CONTENT"));
+        when(result.getTable("t_account")).thenReturn(new ShardingSphereTable("t_account", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        when(result.getTable("t_account_bak")).thenReturn(new ShardingSphereTable("t_account_bak", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        when(result.getTable("t_account_detail")).thenReturn(new ShardingSphereTable("t_account_detail", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
         when(result.getTable("t_order")).thenReturn(new ShardingSphereTable("t_order", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
+        when(result.containsTable("t_account")).thenReturn(true);
+        when(result.containsTable("t_account_bak")).thenReturn(true);
+        when(result.containsTable("t_account_detail")).thenReturn(true);
+        when(result.containsTable("t_order")).thenReturn(true);
         return Collections.singletonMap(schemaName, result);
     }
     
diff --git a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/MixSQLRewriterIT.java b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/MixSQLRewriterIT.java
index 3c16a368360..1fa0dd31049 100644
--- a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/MixSQLRewriterIT.java
+++ b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/MixSQLRewriterIT.java
@@ -61,10 +61,12 @@ class MixSQLRewriterIT extends SQLRewriterIT {
         when(accountTable.getColumns()).thenReturn(createColumns());
         when(accountTable.getIndexes()).thenReturn(Collections.singletonList(new ShardingSphereIndex("index_name")));
         when(result.containsTable("t_account")).thenReturn(true);
-        when(result.getTable("t_account")).thenReturn(accountTable);
+        when(result.containsTable("t_account_bak")).thenReturn(true);
+        when(result.containsTable("t_account_detail")).thenReturn(true);
         ShardingSphereTable accountBakTable = mock(ShardingSphereTable.class);
         when(accountBakTable.getColumns()).thenReturn(createColumns());
         when(result.containsTable("t_account_bak")).thenReturn(true);
+        when(result.getTable("t_account")).thenReturn(accountTable);
         when(result.getTable("t_account_bak")).thenReturn(accountBakTable);
         when(result.getTable("t_account_detail")).thenReturn(mock(ShardingSphereTable.class));
         when(result.getAllColumnNames("t_account")).thenReturn(Arrays.asList("account_id", "password", "amount", "status"));
diff --git a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/ShardingSQLRewriterIT.java b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/ShardingSQLRewriterIT.java
index 807202a09c4..f51be7870a2 100644
--- a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/ShardingSQLRewriterIT.java
+++ b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/scenario/ShardingSQLRewriterIT.java
@@ -85,6 +85,16 @@ class ShardingSQLRewriterIT extends SQLRewriterIT {
         when(result.getVisibleColumnNames("t_user")).thenReturn(new ArrayList<>(Arrays.asList("id", "content")));
         when(result.getVisibleColumnNames("t_user_extend")).thenReturn(new ArrayList<>(Arrays.asList("user_id", "content")));
         when(result.containsColumn("t_account", "account_id")).thenReturn(true);
+        when(result.containsTable("t_account")).thenReturn(true);
+        when(result.containsTable("t_account_detail")).thenReturn(true);
+        when(result.containsTable("t_user")).thenReturn(true);
+        when(result.containsTable("t_user_extend")).thenReturn(true);
+        when(result.containsTable("t_single")).thenReturn(true);
+        when(result.containsTable("t_single_extend")).thenReturn(true);
+        when(result.containsTable("t_config")).thenReturn(true);
+        when(result.containsTable("T_ROLE")).thenReturn(true);
+        when(result.containsTable("T_ROLE_ADMIN")).thenReturn(true);
+        when(result.containsTable("t_account_view")).thenReturn(true);
         return Collections.singletonMap(schemaName, result);
     }
     
diff --git a/test/it/rewriter/src/test/resources/scenario/mix/case/query-with-cipher/ddl/create/create-table.xml b/test/it/rewriter/src/test/resources/scenario/mix/case/query-with-cipher/ddl/create/create-table.xml
index 1f35503d624..b1b86154753 100644
--- a/test/it/rewriter/src/test/resources/scenario/mix/case/query-with-cipher/ddl/create/create-table.xml
+++ b/test/it/rewriter/src/test/resources/scenario/mix/case/query-with-cipher/ddl/create/create-table.xml
@@ -18,26 +18,26 @@
 
 <rewrite-assertions yaml-rule="scenario/mix/config/query-with-cipher.yaml">
     <rewrite-assertion id="create_table_with_single_cipher_column" db-types="MySQL">
-        <input sql="CREATE TABLE t_account_detail(account_id INT, password VARCHAR(100));" />
-        <output sql="CREATE TABLE t_account_detail_0(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100));" />
-        <output sql="CREATE TABLE t_account_detail_1(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100));" />
+        <input sql="CREATE TABLE t_account_detail_new(account_id INT, password VARCHAR(100));" />
+        <output sql="CREATE TABLE t_account_detail_new_0(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100));" />
+        <output sql="CREATE TABLE t_account_detail_new_1(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100));" />
     </rewrite-assertion>
     
     <rewrite-assertion id="create_table_with_multi_cipher_column" db-types="MySQL">
-        <input sql="CREATE TABLE t_account_detail(account_id INT, password VARCHAR(100), amount VARCHAR(100));" />
-        <output sql="CREATE TABLE t_account_detail_0(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
-        <output sql="CREATE TABLE t_account_detail_1(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
+        <input sql="CREATE TABLE t_account_detail_new(account_id INT, password VARCHAR(100), amount VARCHAR(100));" />
+        <output sql="CREATE TABLE t_account_detail_new_0(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
+        <output sql="CREATE TABLE t_account_detail_new_1(account_id INT, cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
     </rewrite-assertion>
     
     <rewrite-assertion id="create_table_with_all_cipher_column" db-types="MySQL">
-        <input sql="CREATE TABLE t_account_detail(password VARCHAR(100), amount VARCHAR(100));" />
-        <output sql="CREATE TABLE t_account_detail_0(cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
-        <output sql="CREATE TABLE t_account_detail_1(cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
+        <input sql="CREATE TABLE t_account_detail_new(password VARCHAR(100), amount VARCHAR(100));" />
+        <output sql="CREATE TABLE t_account_detail_new_0(cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
+        <output sql="CREATE TABLE t_account_detail_new_1(cipher_password VARCHAR(100), assisted_query_password VARCHAR(100), cipher_amount VARCHAR(100));" />
     </rewrite-assertion>
     
     <rewrite-assertion id="create_table_with_random_blank_cipher_column" db-types="MySQL">
-        <input sql="CREATE TABLE t_account_detail( account_id INT  , password VARCHAR(100) ,  amount VARCHAR(100)); " />
-        <output sql="CREATE TABLE t_account_detail_0( account_id INT  , cipher_password VARCHAR(100) ,  assisted_query_password VARCHAR(100) ,  cipher_amount VARCHAR(100)); " />
-        <output sql="CREATE TABLE t_account_detail_1( account_id INT  , cipher_password VARCHAR(100) ,  assisted_query_password VARCHAR(100) ,  cipher_amount VARCHAR(100)); " />
+        <input sql="CREATE TABLE t_account_detail_new( account_id INT  , password VARCHAR(100) ,  amount VARCHAR(100)); " />
+        <output sql="CREATE TABLE t_account_detail_new_0( account_id INT  , cipher_password VARCHAR(100) ,  assisted_query_password VARCHAR(100) ,  cipher_amount VARCHAR(100)); " />
+        <output sql="CREATE TABLE t_account_detail_new_1( account_id INT  , cipher_password VARCHAR(100) ,  assisted_query_password VARCHAR(100) ,  cipher_amount VARCHAR(100)); " />
     </rewrite-assertion>
 </rewrite-assertions>
diff --git a/test/it/rewriter/src/test/resources/scenario/mix/config/query-with-cipher.yaml b/test/it/rewriter/src/test/resources/scenario/mix/config/query-with-cipher.yaml
index 72a8cd2c0cf..7da5ab6519f 100644
--- a/test/it/rewriter/src/test/resources/scenario/mix/config/query-with-cipher.yaml
+++ b/test/it/rewriter/src/test/resources/scenario/mix/config/query-with-cipher.yaml
@@ -46,6 +46,12 @@ rules:
         standard:
           shardingColumn: account_id
           shardingAlgorithmName: t_account_detail_inline
+    t_account_detail_new:
+      actualDataNodes: db.t_account_detail_new_${0..1}
+      tableStrategy:
+        standard:
+          shardingColumn: account_id
+          shardingAlgorithmName: t_account_detail_new_inline
   bindingTables:
     - t_account, t_account_detail
   
@@ -62,6 +68,10 @@ rules:
       type: INLINE
       props:
         algorithm-expression: t_account_detail_${account_id % 2}
+    t_account_detail_new_inline:
+      type: INLINE
+      props:
+        algorithm-expression: t_account_detail_new_${account_id % 2}
   
   keyGenerators:
     rewrite_keygen_fixture:
@@ -108,6 +118,19 @@ rules:
           cipher:
             name: cipher_amount
             encryptorName: rewrite_normal_fixture
+    t_account_detail_new:
+      columns:
+        password:
+          cipher:
+            name: cipher_password
+            encryptorName: rewrite_normal_fixture
+          assistedQuery:
+            name: assisted_query_password
+            encryptorName: rewrite_assisted_query_fixture
+        amount:
+          cipher:
+            name: cipher_amount
+            encryptorName: rewrite_normal_fixture
   encryptors:
     rewrite_normal_fixture:
       type: REWRITE.NORMAL.FIXTURE
diff --git a/test/it/rewriter/src/test/resources/scenario/sharding/case/ddl/create-table.xml b/test/it/rewriter/src/test/resources/scenario/sharding/case/ddl/create-table.xml
index b732c6b7bd5..b0d211ebc88 100644
--- a/test/it/rewriter/src/test/resources/scenario/sharding/case/ddl/create-table.xml
+++ b/test/it/rewriter/src/test/resources/scenario/sharding/case/ddl/create-table.xml
@@ -18,69 +18,69 @@
 
 <rewrite-assertions yaml-rule="scenario/sharding/config/sharding-rule.yaml">
     <rewrite-assertion id="create_table_with_single_data_node_binding_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_order(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk FOREIGN KEY (order_id) REFERENCES t_order_item (order_id))" />
-        <output sql="CREATE TABLE t_order_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk_t_order_0 FOREIGN KEY (order_id) REFERENCES t_order_item_0 (order_id))" />
+        <input sql="CREATE TABLE t_order_new(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk FOREIGN KEY (order_id) REFERENCES t_order_item (order_id))" />
+        <output sql="CREATE TABLE t_order_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk_t_order_new_0 FOREIGN KEY (order_id) REFERENCES t_order_item_0 (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_single_data_node_unbinding_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_order(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk FOREIGN KEY (order_id) REFERENCES t_order_extend (order_id))" />
-        <output sql="CREATE TABLE t_order_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk_t_order_0 FOREIGN KEY (order_id) REFERENCES t_order_extend_0 (order_id))" />
+        <input sql="CREATE TABLE t_order_new(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk FOREIGN KEY (order_id) REFERENCES t_order_extend (order_id))" />
+        <output sql="CREATE TABLE t_order_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk_t_order_new_0 FOREIGN KEY (order_id) REFERENCES t_order_extend_0 (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_multi_data_node_and_single_data_node_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_account_detail(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_order (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_0 FOREIGN KEY (account_id) REFERENCES t_order_0 (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_1 FOREIGN KEY (account_id) REFERENCES t_order_0 (account_id))" />
+        <input sql="CREATE TABLE t_account_detail_new(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_order_new (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_0 FOREIGN KEY (account_id) REFERENCES t_order_new_0 (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_1 FOREIGN KEY (account_id) REFERENCES t_order_new_0 (account_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_multi_data_node_binding_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_account_detail(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_account_detail (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_0 FOREIGN KEY (account_id) REFERENCES t_account_detail_0 (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_1 FOREIGN KEY (account_id) REFERENCES t_account_detail_1 (account_id))" />
+        <input sql="CREATE TABLE t_account_detail_new(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_account_detail_new (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_0 FOREIGN KEY (account_id) REFERENCES t_account_detail_new_0 (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_1 FOREIGN KEY (account_id) REFERENCES t_account_detail_new_1 (account_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_single_data_node_and_broadcast_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_order(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk FOREIGN KEY (order_id) REFERENCES t_config (order_id))" />
-        <output sql="CREATE TABLE t_order_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk_t_order_0 FOREIGN KEY (order_id) REFERENCES t_config (order_id))" />
+        <input sql="CREATE TABLE t_order_new(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk FOREIGN KEY (order_id) REFERENCES t_config_new (order_id))" />
+        <output sql="CREATE TABLE t_order_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk_t_order_new_0 FOREIGN KEY (order_id) REFERENCES t_config_new (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_multi_data_node_and_broadcast_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_account_detail(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_config (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_0 FOREIGN KEY (account_id) REFERENCES t_config (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_1 FOREIGN KEY (account_id) REFERENCES t_config (account_id))" />
+        <input sql="CREATE TABLE t_account_detail_new(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_config_new (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_0 FOREIGN KEY (account_id) REFERENCES t_config_new (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_1 FOREIGN KEY (account_id) REFERENCES t_config_new (account_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_single_data_node_and_single_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_order(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk FOREIGN KEY (order_id) REFERENCES t_single (order_id))" />
-        <output sql="CREATE TABLE t_order_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_fk_t_order_0 FOREIGN KEY (order_id) REFERENCES t_single (order_id))" />
+        <input sql="CREATE TABLE t_order_new(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk FOREIGN KEY (order_id) REFERENCES t_single_new (order_id))" />
+        <output sql="CREATE TABLE t_order_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_order_new_fk_t_order_new_0 FOREIGN KEY (order_id) REFERENCES t_single_new (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_multi_data_node_and_single_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_account_detail(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_single (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_0 FOREIGN KEY (account_id) REFERENCES t_single (account_id))" />
-        <output sql="CREATE TABLE t_account_detail_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_1 FOREIGN KEY (account_id) REFERENCES t_single (account_id))" />
+        <input sql="CREATE TABLE t_account_detail_new(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk FOREIGN KEY (account_id) REFERENCES t_single_new (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_0(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_0 FOREIGN KEY (account_id) REFERENCES t_single_new (account_id))" />
+        <output sql="CREATE TABLE t_account_detail_new_1(order_id INT PRIMARY KEY, CONSTRAINT t_account_fk_t_account_detail_new_1 FOREIGN KEY (account_id) REFERENCES t_single_new (account_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_broadcast_and_single_data_node_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_config(order_id INT PRIMARY KEY, CONSTRAINT t_config_fk FOREIGN KEY (order_id) REFERENCES t_order (order_id))" />
-        <output sql="CREATE TABLE t_config(order_id INT PRIMARY KEY, CONSTRAINT t_config_fk_t_config FOREIGN KEY (order_id) REFERENCES t_order_0 (order_id))" />
+        <input sql="CREATE TABLE t_config_new(order_id INT PRIMARY KEY, CONSTRAINT t_config_new_fk FOREIGN KEY (order_id) REFERENCES t_order_new (order_id))" />
+        <output sql="CREATE TABLE t_config_new(order_id INT PRIMARY KEY, CONSTRAINT t_config_new_fk_t_config_new FOREIGN KEY (order_id) REFERENCES t_order_new_0 (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_broadcast_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_config(order_id INT PRIMARY KEY, CONSTRAINT t_config_fk FOREIGN KEY (order_id) REFERENCES t_order_type (order_id))" />
-        <output sql="CREATE TABLE t_config(order_id INT PRIMARY KEY, CONSTRAINT t_config_fk_t_config FOREIGN KEY (order_id) REFERENCES t_order_type (order_id))" />
+        <input sql="CREATE TABLE t_config_new(order_id INT PRIMARY KEY, CONSTRAINT t_config_new_fk FOREIGN KEY (order_id) REFERENCES t_order_new_type (order_id))" />
+        <output sql="CREATE TABLE t_config_new(order_id INT PRIMARY KEY, CONSTRAINT t_config_new_fk_t_config_new FOREIGN KEY (order_id) REFERENCES t_order_new_type (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_broadcast_and_single_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_config(order_id INT PRIMARY KEY, CONSTRAINT t_config_fk FOREIGN KEY (order_id) REFERENCES t_single (order_id))" />
-        <output sql="CREATE TABLE t_config(order_id INT PRIMARY KEY, CONSTRAINT t_config_fk_t_config FOREIGN KEY (order_id) REFERENCES t_single (order_id))" />
+        <input sql="CREATE TABLE t_config_new(order_id INT PRIMARY KEY, CONSTRAINT t_config_new_fk FOREIGN KEY (order_id) REFERENCES t_single_new (order_id))" />
+        <output sql="CREATE TABLE t_config_new(order_id INT PRIMARY KEY, CONSTRAINT t_config_new_fk_t_config_new FOREIGN KEY (order_id) REFERENCES t_single_new (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_single_and_single_data_node_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_single(order_id INT PRIMARY KEY, CONSTRAINT t_single_fk FOREIGN KEY (order_id) REFERENCES t_order (order_id))" />
-        <output sql="CREATE TABLE t_single(order_id INT PRIMARY KEY, CONSTRAINT t_single_fk FOREIGN KEY (order_id) REFERENCES t_order_0 (order_id))" />
+        <input sql="CREATE TABLE t_single_new(order_id INT PRIMARY KEY, CONSTRAINT t_single_new_fk FOREIGN KEY (order_id) REFERENCES t_order_new (order_id))" />
+        <output sql="CREATE TABLE t_single_new(order_id INT PRIMARY KEY, CONSTRAINT t_single_new_fk FOREIGN KEY (order_id) REFERENCES t_order_new_0 (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_single_and_broadcast_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_single(order_id INT PRIMARY KEY, CONSTRAINT t_single_fk FOREIGN KEY (order_id) REFERENCES t_config (order_id))" />
-        <output sql="CREATE TABLE t_single(order_id INT PRIMARY KEY, CONSTRAINT t_single_fk FOREIGN KEY (order_id) REFERENCES t_config (order_id))" />
+        <input sql="CREATE TABLE t_single_new(order_id INT PRIMARY KEY, CONSTRAINT t_single_new_fk FOREIGN KEY (order_id) REFERENCES t_config_new (order_id))" />
+        <output sql="CREATE TABLE t_single_new(order_id INT PRIMARY KEY, CONSTRAINT t_single_new_fk FOREIGN KEY (order_id) REFERENCES t_config_new (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_single_table_with_add_foreign_constraint" db-types="MySQL,PostgreSQL,openGauss">
-        <input sql="CREATE TABLE t_single(order_id INT PRIMARY KEY, CONSTRAINT t_single_fk FOREIGN KEY (order_id) REFERENCES t_single_extend (order_id))" />
-        <output sql="CREATE TABLE t_single(order_id INT PRIMARY KEY, CONSTRAINT t_single_fk_t_single FOREIGN KEY (order_id) REFERENCES t_single_extend (order_id))" />
+        <input sql="CREATE TABLE t_single_new(order_id INT PRIMARY KEY, CONSTRAINT t_single_new_fk FOREIGN KEY (order_id) REFERENCES t_single_extend (order_id))" />
+        <output sql="CREATE TABLE t_single_new(order_id INT PRIMARY KEY, CONSTRAINT t_single_new_fk_t_single_new FOREIGN KEY (order_id) REFERENCES t_single_extend (order_id))" />
     </rewrite-assertion>
     <rewrite-assertion id="create_table_with_multi_data_node_with_storage_parameter" db-types="openGauss">
-        <input sql="CREATE TABLE t_account_detail (order_id INT,account_id INT) WITH (FILLFACTOR = 80, ORIENTATION=ROW)" />
-        <output sql="CREATE TABLE t_account_detail_0 (order_id INT,account_id INT) WITH (FILLFACTOR = 80, ORIENTATION=ROW)" />
-        <output sql="CREATE TABLE t_account_detail_1 (order_id INT,account_id INT) WITH (FILLFACTOR = 80, ORIENTATION=ROW)" />
+        <input sql="CREATE TABLE t_account_detail_new (order_id INT,account_id INT) WITH (FILLFACTOR = 80, ORIENTATION=ROW)" />
+        <output sql="CREATE TABLE t_account_detail_new_0 (order_id INT,account_id INT) WITH (FILLFACTOR = 80, ORIENTATION=ROW)" />
+        <output sql="CREATE TABLE t_account_detail_new_1 (order_id INT,account_id INT) WITH (FILLFACTOR = 80, ORIENTATION=ROW)" />
     </rewrite-assertion>
     <rewrite-assertion id="create_view_with_sharding_table" db-types="MySQL,PostgreSQL,openGauss">
         <input sql="CREATE VIEW t_account_view AS SELECT account_id, amount FROM t_account WHERE status = 'OK'" />
diff --git a/test/it/rewriter/src/test/resources/scenario/sharding/config/sharding-rule.yaml b/test/it/rewriter/src/test/resources/scenario/sharding/config/sharding-rule.yaml
index 5eb3424ec10..eeb79fed9da 100644
--- a/test/it/rewriter/src/test/resources/scenario/sharding/config/sharding-rule.yaml
+++ b/test/it/rewriter/src/test/resources/scenario/sharding/config/sharding-rule.yaml
@@ -37,8 +37,16 @@ rules:
         standard:
           shardingColumn: account_id
           shardingAlgorithmName: t_account_detail_inline
+    t_account_detail_new:
+      actualDataNodes: db.t_account_detail_new_${0..1}
+      tableStrategy:
+        standard:
+          shardingColumn: account_id
+          shardingAlgorithmName: t_account_detail_new_inline
     t_order:
       actualDataNodes: db.t_order_0
+    t_order_new:
+      actualDataNodes: db.t_order_new_0
     t_order_item:
       actualDataNodes: db.t_order_item_0 
     t_order_extend:
@@ -67,11 +75,12 @@ rules:
           shardingAlgorithmName: t_account_view_inline
   bindingTables:
     - t_account, t_account_detail, t_account_view
-    - t_order, t_order_item
+    - t_order, t_order_new, t_order_item
     - t_user, t_user_extend
     - T_ROLE, T_ROLE_ADMIN
   broadcastTables:
     - t_config
+    - t_config_new
     - t_order_type
   shardingAlgorithms:
     t_account_inline:
@@ -82,6 +91,10 @@ rules:
       type: INLINE
       props:
         algorithm-expression: t_account_detail_${account_id % 2}
+    t_account_detail_new_inline:
+      type: INLINE
+      props:
+        algorithm-expression: t_account_detail_new_${account_id % 2}
     t_user_inline:
       type: INLINE
       props: