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

[shardingsphere] branch master updated: support create, alter, drop, select view statement (#7740)

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

panjuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 101378e  support create, alter, drop, select view statement (#7740)
101378e is described below

commit 101378ed8d2aee68752c9523a3efc416c8ed6a5d
Author: DuanZhengqiang <st...@gmail.com>
AuthorDate: Tue Oct 13 11:19:45 2020 +0800

    support create, alter, drop, select view statement (#7740)
    
    * support create, alter, drop, select view statement
    
    * fix unit test error
    
    * extract ShardingDDLStatementValidator and ShardingDMLStatementValidator to carry common functions
    
    * modify route engine to ShardingUnconfiguredTablesRoutingEngine
---
 .../engine/type/ShardingRouteEngineFactory.java    | 20 +++---
 .../ShardingStatementValidatorFactory.java         | 52 ++++++++++----
 .../ddl/ShardingDDLStatementValidator.java         | 66 +++++++++++++++++
 .../impl/ShardingAlterViewStatementValidator.java} | 36 +++++-----
 .../ShardingCreateFunctionStatementValidator.java} | 36 +++++-----
 ...ShardingCreateProcedureStatementValidator.java} | 36 +++++-----
 .../ShardingCreateTableStatementValidator.java     | 19 +++--
 .../ShardingCreateViewStatementValidator.java}     | 36 +++++-----
 .../ShardingDMLStatementValidator.java}            | 33 ++++-----
 .../impl/ShardingDeleteStatementValidator.java     | 16 ++---
 .../impl/ShardingInsertStatementValidator.java     | 17 +++--
 .../impl/ShardingUpdateStatementValidator.java     | 13 ++--
 .../ShardingCreateFunctionStatementValidator.java  |  2 +-
 .../ShardingCreateProcedureStatementValidator.java |  2 +-
 .../ShardingAlterViewStatementValidatorTest.java   | 83 ++++++++++++++++++++++
 ...ardingCreateFunctionStatementValidatorTest.java | 33 ++-------
 ...rdingCreateProcedureStatementValidatorTest.java | 33 ++-------
 .../ShardingCreateTableStatementValidatorTest.java |  3 +-
 .../ShardingCreateViewStatementValidatorTest.java  | 83 ++++++++++++++++++++++
 .../ShardingDeleteStatementValidatorTest.java      |  5 +-
 .../ShardingInsertStatementValidatorTest.java      | 26 ++++---
 .../ShardingUpdateStatementValidatorTest.java      |  7 +-
 .../infra/binder/SQLStatementContextFactory.java   | 20 ++++++
 .../statement/ddl/AlterViewStatementContext.java   | 45 ++++++++++++
 .../ddl/CreateFunctionStatementContext.java        | 47 ++++++++++++
 .../ddl/CreateProcedureStatementContext.java       | 47 ++++++++++++
 .../statement/ddl/CreateViewStatementContext.java  | 45 ++++++++++++
 .../schema/model/schema/SchemaMetaDataLoader.java  |  4 +-
 .../model/schema/SchemaMetaDataLoaderTest.java     |  4 +-
 ...ernanceShardingSphereDataSourceFactoryTest.java |  4 +-
 .../sql/parser/autogen/MySQLStatement.g4           |  3 +
 .../parser/mysql/visitor/impl/MySQLDDLVisitor.java | 13 +++-
 .../visitor/impl/PostgreSQLDDLVisitor.java         |  7 ++
 .../sql/parser/core/visitor/VisitorRule.java       |  2 +
 .../statement/ddl/AlterViewStatement.java}         |  9 ++-
 .../handler/ddl/AlterViewStatementHandler.java     | 48 +++++++++++++
 .../handler/ddl/CreateViewStatementHandler.java    | 48 +++++++++++++
 ...Statement.java => MySQLAlterViewStatement.java} | 24 ++++++-
 .../mysql/ddl/MySQLCreateViewStatement.java        | 18 +++++
 .../ddl/PostgreSQLAlterViewStatement.java}         | 10 +--
 .../handler/ddl/AlterViewStatementHandlerTest.java | 46 ++++++++++++
 .../ddl/CreateViewStatementHandlerTest.java        | 46 ++++++++++++
 42 files changed, 909 insertions(+), 238 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
index 2160928..f009a54 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
@@ -19,8 +19,11 @@ package org.apache.shardingsphere.sharding.route.engine.type;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.model.rule.RuleSchemaMetaData;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
 import org.apache.shardingsphere.sharding.route.engine.type.broadcast.ShardingDataSourceGroupBroadcastRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.broadcast.ShardingDatabaseBroadcastRoutingEngine;
@@ -32,20 +35,20 @@ import org.apache.shardingsphere.sharding.route.engine.type.standard.ShardingSta
 import org.apache.shardingsphere.sharding.route.engine.type.unconfigured.ShardingUnconfiguredTablesRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.unicast.ShardingUnicastRoutingEngine;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.SetStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dcl.DCLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterProcedureStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DDLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropProcedureStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.TCLStatement;
@@ -80,7 +83,7 @@ public final class ShardingRouteEngineFactory {
             return new ShardingDatabaseBroadcastRoutingEngine();
         }
         if (sqlStatement instanceof DDLStatement) {
-            return getDDLRoutingEngine(metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), sqlStatementContext);
+            return getDDLRoutingEngine(tableNames, metaData.getRuleSchemaMetaData(), sqlStatementContext);
         }
         if (sqlStatement instanceof DALStatement) {
             return getDALRoutingEngine(shardingRule, metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap(), sqlStatement, tableNames);
@@ -100,14 +103,15 @@ public final class ShardingRouteEngineFactory {
         return getShardingRoutingEngine(shardingRule, shardingConditions, tableNames, props);
     }
     
-    private static ShardingRouteEngine getDDLRoutingEngine(final SchemaMetaData configuredSchemaMetaData, final SQLStatementContext sqlStatementContext) {
+    private static ShardingRouteEngine getDDLRoutingEngine(final Collection<String> tableNames, final RuleSchemaMetaData ruleSchemaMetaData, final SQLStatementContext sqlStatementContext) {
         SQLStatement sqlStatement = sqlStatementContext.getSqlStatement();
         boolean functionStatement = sqlStatement instanceof CreateFunctionStatement || sqlStatement instanceof AlterFunctionStatement || sqlStatement instanceof DropFunctionStatement;
         boolean procedureStatement = sqlStatement instanceof CreateProcedureStatement || sqlStatement instanceof AlterProcedureStatement || sqlStatement instanceof DropProcedureStatement;
-        if (functionStatement || procedureStatement) {
-            return new ShardingDatabaseBroadcastRoutingEngine();
+        boolean viewStatement = sqlStatement instanceof CreateViewStatement || sqlStatement instanceof AlterViewStatement || sqlStatement instanceof DropViewStatement;
+        if (functionStatement || procedureStatement || viewStatement) {
+            return new ShardingUnconfiguredTablesRoutingEngine(tableNames, ruleSchemaMetaData.getUnconfiguredSchemaMetaDataMap());
         }
-        return new ShardingTableBroadcastRoutingEngine(configuredSchemaMetaData, sqlStatementContext);
+        return new ShardingTableBroadcastRoutingEngine(ruleSchemaMetaData.getConfiguredSchemaMetaData(), sqlStatementContext);
     }
     
     private static ShardingRouteEngine getDALRoutingEngine(final ShardingRule shardingRule,
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
index d572af2..8aa0fe6 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
@@ -19,16 +19,22 @@ package org.apache.shardingsphere.sharding.route.engine.validator;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.sharding.route.engine.validator.impl.ShardingCreateFunctionStatementValidator;
-import org.apache.shardingsphere.sharding.route.engine.validator.impl.ShardingCreateProcedureStatementValidator;
-import org.apache.shardingsphere.sharding.route.engine.validator.impl.ShardingCreateTableStatementValidator;
-import org.apache.shardingsphere.sharding.route.engine.validator.impl.ShardingDeleteStatementValidator;
-import org.apache.shardingsphere.sharding.route.engine.validator.impl.ShardingInsertStatementValidator;
-import org.apache.shardingsphere.sharding.route.engine.validator.impl.ShardingUpdateStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterViewStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateFunctionStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateProcedureStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateTableStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateViewStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingDeleteStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingInsertStatementValidator;
+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.AlterViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DDLStatement;
+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.UpdateStatement;
@@ -48,15 +54,16 @@ public final class ShardingStatementValidatorFactory {
      * @return instance of sharding statement validator
      */
     public static Optional<ShardingStatementValidator> newInstance(final SQLStatement sqlStatement) {
-        if (sqlStatement instanceof InsertStatement) {
-            return Optional.of(new ShardingInsertStatementValidator());
-        }
-        if (sqlStatement instanceof UpdateStatement) {
-            return Optional.of(new ShardingUpdateStatementValidator());
+        if (sqlStatement instanceof DDLStatement) {
+            return getDDLStatementValidator(sqlStatement);
         }
-        if (sqlStatement instanceof DeleteStatement) {
-            return Optional.of(new ShardingDeleteStatementValidator());
+        if (sqlStatement instanceof DMLStatement) {
+            return getDMLStatementValidator(sqlStatement);
         }
+        return Optional.empty();
+    }
+    
+    private static Optional<ShardingStatementValidator> getDDLStatementValidator(final SQLStatement sqlStatement) {
         if (sqlStatement instanceof CreateTableStatement) {
             return Optional.of(new ShardingCreateTableStatementValidator());
         }
@@ -66,6 +73,25 @@ public final class ShardingStatementValidatorFactory {
         if (sqlStatement instanceof CreateProcedureStatement) {
             return Optional.of(new ShardingCreateProcedureStatementValidator());
         }
+        if (sqlStatement instanceof CreateViewStatement) {
+            return Optional.of(new ShardingCreateViewStatementValidator());
+        }
+        if (sqlStatement instanceof AlterViewStatement) {
+            return Optional.of(new ShardingAlterViewStatementValidator());
+        }
+        return Optional.empty();
+    }
+    
+    private static Optional<ShardingStatementValidator> getDMLStatementValidator(final SQLStatement sqlStatement) {
+        if (sqlStatement instanceof InsertStatement) {
+            return Optional.of(new ShardingInsertStatementValidator());
+        }
+        if (sqlStatement instanceof UpdateStatement) {
+            return Optional.of(new ShardingUpdateStatementValidator());
+        }
+        if (sqlStatement instanceof DeleteStatement) {
+            return Optional.of(new ShardingDeleteStatementValidator());
+        }
         return Optional.empty();
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDDLStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDDLStatementValidator.java
new file mode 100644
index 0000000..b6aa570
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDDLStatementValidator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.sharding.route.engine.validator.ddl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
+import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+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.DDLStatement;
+
+import java.util.Collection;
+
+/**
+ * Sharding ddl statement validator.
+ */
+public abstract class ShardingDDLStatementValidator<T extends DDLStatement> implements ShardingStatementValidator<T> {
+    
+    /**
+     * Validate sharding table.
+     *
+     * @param metaData meta data
+     * @param tables tables
+     * @param sqlStatement sqlStatement
+     */
+    protected void validateShardingTable(final ShardingSphereMetaData metaData, final Collection<SimpleTableSegment> tables, final SQLStatement sqlStatement) {
+        for (SimpleTableSegment each : tables) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new ShardingSphereException("Can not support sharding table '%s' for '%s'.", tableName, sqlStatement);
+            }
+        }
+    }
+    
+    /**
+     * Validate table not exist.
+     *
+     * @param metaData meta data
+     * @param tables tables
+     */
+    protected void validateTableNotExist(final ShardingSphereMetaData metaData, final Collection<SimpleTableSegment> tables) {
+        for (SimpleTableSegment each : tables) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new TableExistsException(tableName);
+            }
+        }
+    }
+    
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterViewStatementValidator.java
similarity index 56%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterViewStatementValidator.java
index dcba8bc..b87816a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterViewStatementValidator.java
@@ -15,36 +15,38 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl;
 
-import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.DeleteStatementHandler;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.AlterViewStatementHandler;
 
 import java.util.List;
+import java.util.Optional;
 
 /**
- * Sharding delete statement validator.
+ * Sharding alter view statement validator.
  */
-public final class ShardingDeleteStatementValidator implements ShardingStatementValidator<DeleteStatement> {
+public final class ShardingAlterViewStatementValidator extends ShardingDDLStatementValidator<AlterViewStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
-            throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
-        }
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterViewStatement> sqlStatementContext, 
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        Optional<SelectStatement> selectStatement = AlterViewStatementHandler.getSelectStatement(sqlStatementContext.getSqlStatement());
+        selectStatement.ifPresent(select -> {
+            TableExtractor extractor = new TableExtractor();
+            extractor.extractTablesFromSelect(select);
+            validateShardingTable(metaData, extractor.getRewriteTables(), sqlStatementContext.getSqlStatement());
+        });
     }
     
     @Override
-    public void postValidate(final DeleteStatement sqlStatement, final RouteContext routeContext) {
-        if (DeleteStatementHandler.getLimitSegment(sqlStatement).isPresent() && routeContext.getRouteUnits().size() > 1) {
-            throw new ShardingSphereException("DELETE ... LIMIT can not support sharding route to multiple data nodes.");
-        }
+    public void postValidate(final AlterViewStatement sqlStatement, final RouteContext routeContext) {
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateFunctionStatementValidator.java
similarity index 53%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateFunctionStatementValidator.java
index dcba8bc..7b092d9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateFunctionStatementValidator.java
@@ -15,36 +15,38 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl;
 
-import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.DeleteStatementHandler;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateFunctionStatementHandler;
 
 import java.util.List;
+import java.util.Optional;
 
 /**
- * Sharding delete statement validator.
+ * Sharding create function statement validator.
  */
-public final class ShardingDeleteStatementValidator implements ShardingStatementValidator<DeleteStatement> {
+public final class ShardingCreateFunctionStatementValidator extends ShardingDDLStatementValidator<CreateFunctionStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
-            throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
-        }
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateFunctionStatement> sqlStatementContext, 
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        Optional<RoutineBodySegment> routineBodySegment = CreateFunctionStatementHandler.getRoutineBodySegment(sqlStatementContext.getSqlStatement());
+        routineBodySegment.ifPresent(routineBody -> {
+            TableExtractor extractor = new TableExtractor();
+            validateShardingTable(metaData, extractor.extractExistTableFromRoutineBody(routineBody), sqlStatementContext.getSqlStatement());
+            validateTableNotExist(metaData, extractor.extractNotExistTableFromRoutineBody(routineBody));
+        });
     }
     
     @Override
-    public void postValidate(final DeleteStatement sqlStatement, final RouteContext routeContext) {
-        if (DeleteStatementHandler.getLimitSegment(sqlStatement).isPresent() && routeContext.getRouteUnits().size() > 1) {
-            throw new ShardingSphereException("DELETE ... LIMIT can not support sharding route to multiple data nodes.");
-        }
+    public void postValidate(final CreateFunctionStatement sqlStatement, final RouteContext routeContext) {
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateProcedureStatementValidator.java
similarity index 53%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateProcedureStatementValidator.java
index dcba8bc..c23a6c4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateProcedureStatementValidator.java
@@ -15,36 +15,38 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl;
 
-import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.DeleteStatementHandler;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateProcedureStatementHandler;
 
 import java.util.List;
+import java.util.Optional;
 
 /**
- * Sharding delete statement validator.
+ * Sharding create procedure statement validator.
  */
-public final class ShardingDeleteStatementValidator implements ShardingStatementValidator<DeleteStatement> {
+public final class ShardingCreateProcedureStatementValidator extends ShardingDDLStatementValidator<CreateProcedureStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
-            throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
-        }
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateProcedureStatement> sqlStatementContext,
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        Optional<RoutineBodySegment> routineBodySegment = CreateProcedureStatementHandler.getRoutineBodySegment(sqlStatementContext.getSqlStatement());
+        routineBodySegment.ifPresent(routineBody -> {
+            TableExtractor extractor = new TableExtractor();
+            validateTableNotExist(metaData, extractor.extractNotExistTableFromRoutineBody(routineBody));
+            validateShardingTable(metaData, extractor.extractExistTableFromRoutineBody(routineBody), sqlStatementContext.getSqlStatement());
+        });
     }
     
     @Override
-    public void postValidate(final DeleteStatement sqlStatement, final RouteContext routeContext) {
-        if (DeleteStatementHandler.getLimitSegment(sqlStatement).isPresent() && routeContext.getRouteUnits().size() > 1) {
-            throw new ShardingSphereException("DELETE ... LIMIT can not support sharding route to multiple data nodes.");
-        }
+    public void postValidate(final CreateProcedureStatement sqlStatement, final RouteContext routeContext) {
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateTableStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateTableStatementValidator.java
similarity index 69%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateTableStatementValidator.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateTableStatementValidator.java
index 3d74462..a0677ca 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateTableStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateTableStatementValidator.java
@@ -15,30 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateTableStatementHandler;
 
+import java.util.Collections;
 import java.util.List;
 
 /**
  * Sharding create table statement validator.
  */
-public final class ShardingCreateTableStatementValidator implements ShardingStatementValidator<CreateTableStatement> {
+public final class ShardingCreateTableStatementValidator extends ShardingDDLStatementValidator<CreateTableStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, 
-                            final SQLStatementContext<CreateTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        String tableName = sqlStatementContext.getSqlStatement().getTable().getTableName().getIdentifier().getValue();
-        if (!CreateTableStatementHandler.containsIfNotExistClause(sqlStatementContext.getSqlStatement()) && metaData.getRuleSchemaMetaData().getAllTableNames().contains(tableName)) {
-            throw new TableExistsException(tableName);
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateTableStatement> sqlStatementContext, 
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        if (!CreateTableStatementHandler.containsIfNotExistClause(sqlStatementContext.getSqlStatement())) {
+            validateTableNotExist(metaData, Collections.singletonList(sqlStatementContext.getSqlStatement().getTable()));
         }
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
similarity index 56%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
index dcba8bc..c2943f8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
@@ -15,36 +15,38 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl;
 
-import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.DeleteStatementHandler;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateViewStatementHandler;
 
 import java.util.List;
+import java.util.Optional;
 
 /**
- * Sharding delete statement validator.
+ * Sharding create view statement validator.
  */
-public final class ShardingDeleteStatementValidator implements ShardingStatementValidator<DeleteStatement> {
+public final class ShardingCreateViewStatementValidator extends ShardingDDLStatementValidator<CreateViewStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
-            throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
-        }
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateViewStatement> sqlStatementContext, 
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        Optional<SelectStatement> selectStatement = CreateViewStatementHandler.getSelectStatement(sqlStatementContext.getSqlStatement());
+        selectStatement.ifPresent(select -> {
+            TableExtractor extractor = new TableExtractor();
+            extractor.extractTablesFromSelect(select);
+            validateShardingTable(metaData, extractor.getRewriteTables(), sqlStatementContext.getSqlStatement());
+        });
     }
     
     @Override
-    public void postValidate(final DeleteStatement sqlStatement, final RouteContext routeContext) {
-        if (DeleteStatementHandler.getLimitSegment(sqlStatement).isPresent() && routeContext.getRouteUnits().size() > 1) {
-            throw new ShardingSphereException("DELETE ... LIMIT can not support sharding route to multiple data nodes.");
-        }
+    public void postValidate(final CreateViewStatement sqlStatement, final RouteContext routeContext) {
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDMLStatementValidator.java
similarity index 57%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDMLStatementValidator.java
index dcba8bc..4b3e915 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDMLStatementValidator.java
@@ -15,36 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.dml;
 
-import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
-import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
-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;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 /**
- * Sharding delete statement validator.
+ * Sharding dml statement validator.
  */
-public final class ShardingDeleteStatementValidator implements ShardingStatementValidator<DeleteStatement> {
+public abstract class ShardingDMLStatementValidator<T extends SQLStatement> implements ShardingStatementValidator<T> {
     
-    @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
+    /**
+     * Validate multiple table.
+     *
+     * @param sqlStatementContext sqlStatementContext
+     */
+    protected void validateMultipleTable(final SQLStatementContext<T> sqlStatementContext) {
         if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
             throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
         }
     }
-    
-    @Override
-    public void postValidate(final DeleteStatement sqlStatement, final RouteContext routeContext) {
-        if (DeleteStatementHandler.getLimitSegment(sqlStatement).isPresent() && routeContext.getRouteUnits().size() > 1) {
-            throw new ShardingSphereException("DELETE ... LIMIT can not support sharding route to multiple data nodes.");
-        }
-    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
similarity index 78%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
index dcba8bc..4865078 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingDeleteStatementValidator.java
@@ -15,15 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.dml.impl;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.DeleteStatementHandler;
 
@@ -32,13 +31,12 @@ import java.util.List;
 /**
  * Sharding delete statement validator.
  */
-public final class ShardingDeleteStatementValidator implements ShardingStatementValidator<DeleteStatement> {
+public final class ShardingDeleteStatementValidator extends ShardingDMLStatementValidator<DeleteStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
-            throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
-        }
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, 
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        validateMultipleTable(sqlStatementContext);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingInsertStatementValidator.java
similarity index 91%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidator.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingInsertStatementValidator.java
index db8ed25..af50649 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingInsertStatementValidator.java
@@ -15,17 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.dml.impl;
 
+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.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-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.binder.type.TableAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.OnDuplicateKeyColumnsSegment;
@@ -40,12 +39,12 @@ import java.util.Optional;
 /**
  * Sharding insert statement validator.
  */
-public final class ShardingInsertStatementValidator implements ShardingStatementValidator<InsertStatement> {
+public final class ShardingInsertStatementValidator extends ShardingDMLStatementValidator<InsertStatement> {
     
     @Override
     public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<InsertStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        if (null == ((InsertStatementContext) sqlStatementContext).getInsertSelectContext() && 1 != ((TableAvailable) sqlStatementContext).getTablesContext().getTables().size()) {
-            throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
+        if (null == ((InsertStatementContext) sqlStatementContext).getInsertSelectContext()) {
+            validateMultipleTable(sqlStatementContext);
         }
         InsertStatement sqlStatement = sqlStatementContext.getSqlStatement();
         Optional<OnDuplicateKeyColumnsSegment> onDuplicateKeyColumnsSegment = InsertStatementHandler.getOnDuplicateKeyColumnsSegment(sqlStatement);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingUpdateStatementValidator.java
similarity index 95%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidator.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingUpdateStatementValidator.java
index 3a69e5e..2e6f621 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingUpdateStatementValidator.java
@@ -15,15 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.dml.impl;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
@@ -42,13 +41,11 @@ import java.util.Optional;
 /**
  * Sharding update statement validator.
  */
-public final class ShardingUpdateStatementValidator implements ShardingStatementValidator<UpdateStatement> {
+public final class ShardingUpdateStatementValidator extends ShardingDMLStatementValidator<UpdateStatement> {
     
     @Override
     public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<UpdateStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereMetaData metaData) {
-        if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
-            throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
-        }
+        validateMultipleTable(sqlStatementContext);
         UpdateStatement sqlStatement = sqlStatementContext.getSqlStatement();
         String tableName = sqlStatementContext.getTablesContext().getTables().iterator().next().getTableName().getIdentifier().getValue();
         for (AssignmentSegment each : sqlStatement.getSetAssignment().getAssignments()) {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
index 3d6a003..36ece0f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -24,7 +25,6 @@ import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableExce
 import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
 import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidator.java
index 633860e..d221157 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidator.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -24,7 +25,6 @@ import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableExce
 import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
 import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterViewStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterViewStatementValidatorTest.java
new file mode 100644
index 0000000..1bca24c
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterViewStatementValidatorTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.sharding.route.engine.validator.ddl;
+
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.model.rule.RuleSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterViewStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.junit.Test;
+import org.mockito.Mock;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class ShardingAlterViewStatementValidatorTest {
+    
+    @Mock
+    private ShardingRule shardingRule;
+    
+    @Test
+    public void assertValidateAlterViewForMySQL() {
+        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
+        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order_item")));
+        MySQLAlterViewStatement sqlStatement = new MySQLAlterViewStatement();
+        sqlStatement.setSelect(selectStatement);
+        ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
+        RuleSchemaMetaData ruleSchemaMetaData = mock(RuleSchemaMetaData.class);
+        SchemaMetaData schemaMetaData = mock(SchemaMetaData.class);
+        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
+        when(ruleSchemaMetaData.getConfiguredSchemaMetaData()).thenReturn(schemaMetaData);
+        when(schemaMetaData.getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
+        Map<String, Collection<String>> unconfiguredSchemaMetaDataMap = new HashMap<>(1, 1);
+        unconfiguredSchemaMetaDataMap.put("ds_0", Collections.singleton("t_order_item"));
+        when(ruleSchemaMetaData.getUnconfiguredSchemaMetaDataMap()).thenReturn(unconfiguredSchemaMetaDataMap);
+        SQLStatementContext<AlterViewStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingAlterViewStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertValidateAlterViewWithShardingTableForMySQL() {
+        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
+        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        MySQLAlterViewStatement sqlStatement = new MySQLAlterViewStatement();
+        sqlStatement.setSelect(selectStatement);
+        ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
+        RuleSchemaMetaData ruleSchemaMetaData = mock(RuleSchemaMetaData.class);
+        SchemaMetaData schemaMetaData = mock(SchemaMetaData.class);
+        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
+        when(ruleSchemaMetaData.getConfiguredSchemaMetaData()).thenReturn(schemaMetaData);
+        when(schemaMetaData.getAllTableNames()).thenReturn(Collections.singleton("t_order"));
+        SQLStatementContext<AlterViewStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingAlterViewStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateFunctionStatementValidatorTest.java
similarity index 80%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidatorTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateFunctionStatementValidatorTest.java
index 1bd594e..764aa3f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateFunctionStatementValidatorTest.java
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl;
 
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.model.rule.RuleSchemaMetaData;
-import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
 import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateFunctionStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
-import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.ValidStatementSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -99,29 +99,6 @@ public final class ShardingCreateFunctionStatementValidatorTest {
         new ShardingCreateFunctionStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
     }
     
-    @Test(expected = NoSuchTableException.class)
-    public void assertValidateCreateFunctionWithNoSuchTableForMySQL() {
-        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
-        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
-        ValidStatementSegment validStatementSegment = new ValidStatementSegment(0, 0);
-        validStatementSegment.setSqlStatement(selectStatement);
-        RoutineBodySegment routineBody = new RoutineBodySegment(0, 0);
-        routineBody.getValidStatements().add(validStatementSegment);
-        MySQLCreateFunctionStatement sqlStatement = new MySQLCreateFunctionStatement();
-        sqlStatement.setRoutineBody(routineBody);
-        ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
-        RuleSchemaMetaData ruleSchemaMetaData = mock(RuleSchemaMetaData.class);
-        SchemaMetaData schemaMetaData = mock(SchemaMetaData.class);
-        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
-        when(ruleSchemaMetaData.getConfiguredSchemaMetaData()).thenReturn(schemaMetaData);
-        when(schemaMetaData.getAllTableNames()).thenReturn(Collections.emptyList());
-        Map<String, Collection<String>> unconfiguredSchemaMetaDataMap = new HashMap<>(1, 1);
-        unconfiguredSchemaMetaDataMap.put("ds_0", Collections.emptyList());
-        when(ruleSchemaMetaData.getUnconfiguredSchemaMetaDataMap()).thenReturn(unconfiguredSchemaMetaDataMap);
-        SQLStatementContext<CreateFunctionStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
-        new ShardingCreateFunctionStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
-    }
-    
     @Test(expected = TableExistsException.class)
     public void assertValidateCreateFunctionWithTableExistsForMySQL() {
         MySQLCreateTableStatement createTableStatement = new MySQLCreateTableStatement();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateProcedureStatementValidatorTest.java
similarity index 80%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidatorTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateProcedureStatementValidatorTest.java
index 46e6ff6..9011223 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateProcedureStatementValidatorTest.java
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl;
 
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.model.rule.RuleSchemaMetaData;
-import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
 import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateProcedureStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
-import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.ValidStatementSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
@@ -99,29 +99,6 @@ public final class ShardingCreateProcedureStatementValidatorTest {
         new ShardingCreateProcedureStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
     }
     
-    @Test(expected = NoSuchTableException.class)
-    public void assertValidateCreateProcedureWithNoSuchTableForMySQL() {
-        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
-        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
-        ValidStatementSegment validStatementSegment = new ValidStatementSegment(0, 0);
-        validStatementSegment.setSqlStatement(selectStatement);
-        RoutineBodySegment routineBody = new RoutineBodySegment(0, 0);
-        routineBody.getValidStatements().add(validStatementSegment);
-        MySQLCreateProcedureStatement sqlStatement = new MySQLCreateProcedureStatement();
-        sqlStatement.setRoutineBody(routineBody);
-        ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
-        RuleSchemaMetaData ruleSchemaMetaData = mock(RuleSchemaMetaData.class);
-        SchemaMetaData schemaMetaData = mock(SchemaMetaData.class);
-        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
-        when(ruleSchemaMetaData.getConfiguredSchemaMetaData()).thenReturn(schemaMetaData);
-        when(schemaMetaData.getAllTableNames()).thenReturn(Collections.emptyList());
-        Map<String, Collection<String>> unconfiguredSchemaMetaDataMap = new HashMap<>(1, 1);
-        unconfiguredSchemaMetaDataMap.put("ds_0", Collections.emptyList());
-        when(ruleSchemaMetaData.getUnconfiguredSchemaMetaDataMap()).thenReturn(unconfiguredSchemaMetaDataMap);
-        SQLStatementContext<CreateProcedureStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
-        new ShardingCreateProcedureStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
-    }
-    
     @Test(expected = TableExistsException.class)
     public void assertValidateCreateProcedureWithTableExistsForMySQL() {
         MySQLCreateTableStatement createTableStatement = new MySQLCreateTableStatement();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateTableStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateTableStatementValidatorTest.java
similarity index 98%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateTableStatementValidatorTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateTableStatementValidatorTest.java
index b788b44..3229ff0 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateTableStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateTableStatementValidatorTest.java
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.ddl;
 
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.model.rule.RuleSchemaMetaData;
 import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateTableStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateTableStatementContext;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
new file mode 100644
index 0000000..a12001c
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.sharding.route.engine.validator.ddl;
+
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.model.rule.RuleSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateViewStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.junit.Test;
+import org.mockito.Mock;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class ShardingCreateViewStatementValidatorTest {
+    
+    @Mock
+    private ShardingRule shardingRule;
+    
+    @Test
+    public void assertValidateCreateViewForMySQL() {
+        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
+        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order_item")));
+        MySQLCreateViewStatement sqlStatement = new MySQLCreateViewStatement();
+        sqlStatement.setSelect(selectStatement);
+        ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
+        RuleSchemaMetaData ruleSchemaMetaData = mock(RuleSchemaMetaData.class);
+        SchemaMetaData schemaMetaData = mock(SchemaMetaData.class);
+        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
+        when(ruleSchemaMetaData.getConfiguredSchemaMetaData()).thenReturn(schemaMetaData);
+        when(schemaMetaData.getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
+        Map<String, Collection<String>> unconfiguredSchemaMetaDataMap = new HashMap<>(1, 1);
+        unconfiguredSchemaMetaDataMap.put("ds_0", Collections.singleton("t_order_item"));
+        when(ruleSchemaMetaData.getUnconfiguredSchemaMetaDataMap()).thenReturn(unconfiguredSchemaMetaDataMap);
+        SQLStatementContext<CreateViewStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingCreateViewStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertValidateCreateViewWithShardingTableForMySQL() {
+        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
+        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        MySQLCreateViewStatement sqlStatement = new MySQLCreateViewStatement();
+        sqlStatement.setSelect(selectStatement);
+        ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
+        RuleSchemaMetaData ruleSchemaMetaData = mock(RuleSchemaMetaData.class);
+        SchemaMetaData schemaMetaData = mock(SchemaMetaData.class);
+        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
+        when(ruleSchemaMetaData.getConfiguredSchemaMetaData()).thenReturn(schemaMetaData);
+        when(schemaMetaData.getAllTableNames()).thenReturn(Collections.singleton("t_order"));
+        SQLStatementContext<CreateViewStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingCreateViewStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
similarity index 97%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidatorTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
index 5568079..03999ed 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
@@ -15,12 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.dml;
 
+import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingDeleteStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.DeleteMultiTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingInsertStatementValidatorTest.java
similarity index 93%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidatorTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingInsertStatementValidatorTest.java
index b87565f..4b950a0 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingInsertStatementValidatorTest.java
@@ -15,15 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.dml;
 
-import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
-import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
 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.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.model.schema.model.schema.SchemaMetaData;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingInsertStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.InsertColumnsSegment;
@@ -31,6 +32,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.OnDupl
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.ParameterMarkerExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.subquery.SubquerySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.OwnerSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
@@ -59,22 +61,21 @@ public final class ShardingInsertStatementValidatorTest {
     
     @Test(expected = ShardingSphereException.class)
     public void assertValidateInsertModifyMultiTables() {
-        SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement());
-        sqlStatementContext.getTablesContext().getTables().addAll(createMultiTablesContext().getTables());
+        SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement(true));
         new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class));
     }
     
     @Test
     public void assertValidateOnDuplicateKeyWithoutShardingKey() {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(false);
-        SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement());
+        SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement(false));
         new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class));
     }
     
     @Test(expected = ShardingSphereException.class)
     public void assertValidateOnDuplicateKeyWithShardingKey() {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(true);
-        SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement());
+        SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement(false));
         new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class));
     }
     
@@ -118,10 +119,13 @@ public final class ShardingInsertStatementValidatorTest {
         new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class));
     }
     
-    private InsertStatement createInsertStatement() {
+    private InsertStatement createInsertStatement(final boolean includeMultiTable) {
         MySQLInsertStatement result = new MySQLInsertStatement();
         result.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("user")));
         ColumnSegment columnSegment = new ColumnSegment(0, 0, new IdentifierValue("id"));
+        if (includeMultiTable) {
+            columnSegment.setOwner(new OwnerSegment(0, 0, new IdentifierValue("t")));
+        }
         AssignmentSegment assignmentSegment = new AssignmentSegment(0, 0, columnSegment, new ParameterMarkerExpressionSegment(0, 0, 1));
         result.setOnDuplicateKeyColumns(new OnDuplicateKeyColumnsSegment(0, 0, Collections.singletonList(assignmentSegment)));
         Collection<ColumnSegment> columns = new LinkedList<>();
@@ -131,7 +135,7 @@ public final class ShardingInsertStatementValidatorTest {
     }
 
     private InsertStatement createInsertSelectStatement() {
-        InsertStatement result = createInsertStatement();
+        InsertStatement result = createInsertStatement(false);
         SelectStatement selectStatement = new MySQLSelectStatement();
         selectStatement.setProjections(new ProjectionsSegment(0, 0));
         result.setInsertSelect(new SubquerySegment(0, 0, selectStatement));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
similarity index 98%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidatorTest.java
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
index ea799ca..71362b8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
@@ -15,13 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sharding.route.engine.validator.impl;
+package org.apache.shardingsphere.sharding.route.engine.validator.dml;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingUpdateStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.SetAssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
index 246850a..db7c886 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
@@ -31,8 +31,12 @@ import org.apache.shardingsphere.infra.binder.statement.dcl.GrantStatementContex
 import org.apache.shardingsphere.infra.binder.statement.dcl.RevokeStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.AlterIndexStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.AlterTableStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.AlterViewStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CreateFunctionStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateIndexStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CreateProcedureStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateTableStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CreateViewStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.DropIndexStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.DropTableStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.TruncateStatementContext;
@@ -48,8 +52,12 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.dcl.GrantStatem
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dcl.RevokeStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateIndexStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DDLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
@@ -139,6 +147,18 @@ public final class SQLStatementContextFactory {
         if (sqlStatement instanceof TruncateStatement) {
             return new TruncateStatementContext((TruncateStatement) sqlStatement);
         }
+        if (sqlStatement instanceof CreateFunctionStatement) {
+            return new CreateFunctionStatementContext((CreateFunctionStatement) sqlStatement);
+        }
+        if (sqlStatement instanceof CreateProcedureStatement) {
+            return new CreateProcedureStatementContext((CreateProcedureStatement) sqlStatement);
+        }
+        if (sqlStatement instanceof CreateViewStatement) {
+            return new CreateViewStatementContext((CreateViewStatement) sqlStatement);
+        }
+        if (sqlStatement instanceof AlterViewStatement) {
+            return new AlterViewStatementContext((AlterViewStatement) sqlStatement);
+        }
         return new CommonSQLStatementContext<>(sqlStatement);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/AlterViewStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/AlterViewStatementContext.java
new file mode 100644
index 0000000..43be81d
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/AlterViewStatementContext.java
@@ -0,0 +1,45 @@
+/*
+ * 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.binder.statement.ddl;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.AlterViewStatementHandler;
+
+import java.util.Optional;
+
+/**
+ * Alter view statement context.
+ */
+@Getter
+public final class AlterViewStatementContext extends CommonSQLStatementContext<AlterViewStatement> {
+    
+    private final TablesContext tablesContext;
+    
+    public AlterViewStatementContext(final AlterViewStatement sqlStatement) {
+        super(sqlStatement);
+        Optional<SelectStatement> selectStatement = AlterViewStatementHandler.getSelectStatement(sqlStatement);
+        TableExtractor extractor = new TableExtractor();
+        selectStatement.ifPresent(extractor::extractTablesFromSelect);
+        tablesContext = new TablesContext(extractor.getRewriteTables());
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateFunctionStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateFunctionStatementContext.java
new file mode 100644
index 0000000..c6f645c
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateFunctionStatementContext.java
@@ -0,0 +1,47 @@
+/*
+ * 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.binder.statement.ddl;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateFunctionStatementHandler;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+
+/**
+ * Create function statement context.
+ */
+@Getter
+public final class CreateFunctionStatementContext extends CommonSQLStatementContext<CreateFunctionStatement> {
+    
+    private final TablesContext tablesContext;
+    
+    public CreateFunctionStatementContext(final CreateFunctionStatement sqlStatement) {
+        super(sqlStatement);
+        Optional<RoutineBodySegment> routineBodySegment = CreateFunctionStatementHandler.getRoutineBodySegment(sqlStatement);
+        Collection<SimpleTableSegment> tables = routineBodySegment.map(bodySegment -> new TableExtractor().extractExistTableFromRoutineBody(bodySegment)).orElse(Collections.emptyList()); 
+        tablesContext = new TablesContext(tables);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateProcedureStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateProcedureStatementContext.java
new file mode 100644
index 0000000..da62da3
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateProcedureStatementContext.java
@@ -0,0 +1,47 @@
+/*
+ * 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.binder.statement.ddl;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateProcedureStatementHandler;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+
+/**
+ * Create procedure statement context.
+ */
+@Getter
+public final class CreateProcedureStatementContext extends CommonSQLStatementContext<CreateProcedureStatement> {
+    
+    private final TablesContext tablesContext;
+    
+    public CreateProcedureStatementContext(final CreateProcedureStatement sqlStatement) {
+        super(sqlStatement);
+        Optional<RoutineBodySegment> routineBodySegment = CreateProcedureStatementHandler.getRoutineBodySegment(sqlStatement);
+        Collection<SimpleTableSegment> tables = routineBodySegment.map(bodySegment -> new TableExtractor().extractExistTableFromRoutineBody(bodySegment)).orElse(Collections.emptyList());
+        tablesContext = new TablesContext(tables);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateViewStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateViewStatementContext.java
new file mode 100644
index 0000000..0d61d17
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CreateViewStatementContext.java
@@ -0,0 +1,45 @@
+/*
+ * 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.binder.statement.ddl;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateViewStatementHandler;
+
+import java.util.Optional;
+
+/**
+ * Create view statement context.
+ */
+@Getter
+public final class CreateViewStatementContext extends CommonSQLStatementContext<CreateViewStatement> {
+    
+    private final TablesContext tablesContext;
+    
+    public CreateViewStatementContext(final CreateViewStatement sqlStatement) {
+        super(sqlStatement);
+        Optional<SelectStatement> selectStatement = CreateViewStatementHandler.getSelectStatement(sqlStatement);
+        TableExtractor extractor = new TableExtractor();
+        selectStatement.ifPresent(extractor::extractTablesFromSelect);
+        tablesContext = new TablesContext(extractor.getRewriteTables());
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoader.java
index 7c23137..43aeeb8 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoader.java
@@ -42,6 +42,8 @@ public final class SchemaMetaDataLoader {
     
     private static final String TABLE_TYPE = "TABLE";
     
+    private static final String VIEW_TYPE = "VIEW";
+    
     private static final String TABLE_NAME = "TABLE_NAME";
     
     /**
@@ -68,7 +70,7 @@ public final class SchemaMetaDataLoader {
     
     private static List<String> loadAllTableNames(final Connection connection) throws SQLException {
         List<String> result = new LinkedList<>();
-        try (ResultSet resultSet = connection.getMetaData().getTables(connection.getCatalog(), connection.getSchema(), null, new String[]{TABLE_TYPE})) {
+        try (ResultSet resultSet = connection.getMetaData().getTables(connection.getCatalog(), connection.getSchema(), null, new String[]{TABLE_TYPE, VIEW_TYPE})) {
             while (resultSet.next()) {
                 String table = resultSet.getString(TABLE_NAME);
                 if (!isSystemTable(table)) {
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoaderTest.java
index 81b56b6..062a9a1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/model/schema/model/schema/SchemaMetaDataLoaderTest.java
@@ -44,6 +44,8 @@ public final class SchemaMetaDataLoaderTest {
     
     private static final String TABLE_TYPE = "TABLE";
     
+    private static final String VIEW_TYPE = "VIEW";
+    
     private final DatabaseType databaseType = DatabaseTypes.getTrunkDatabaseType("Oracle");
     
     @Mock
@@ -63,7 +65,7 @@ public final class SchemaMetaDataLoaderTest {
         when(dataSource.getConnection()).thenReturn(connection);
         when(connection.getCatalog()).thenReturn(TEST_CATALOG);
         when(connection.getMetaData()).thenReturn(databaseMetaData);
-        when(databaseMetaData.getTables(TEST_CATALOG, null, null, new String[]{TABLE_TYPE})).thenReturn(tableExistResultSet);
+        when(databaseMetaData.getTables(TEST_CATALOG, null, null, new String[]{TABLE_TYPE, VIEW_TYPE})).thenReturn(tableExistResultSet);
     }
     
     @Test
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/test/java/org/apache/shardingsphere/driver/governance/api/GovernanceShardingSphereDataSourceFactoryTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/test/java/org/apache/shardingsphere/driver/governance/api/GovernanceShardingSphereDataSourceFactoryTest.java
index e076088..a3250e5 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/test/java/org/apache/shardingsphere/driver/governance/api/GovernanceShardingSphereDataSourceFactoryTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/test/java/org/apache/shardingsphere/driver/governance/api/GovernanceShardingSphereDataSourceFactoryTest.java
@@ -41,6 +41,8 @@ public final class GovernanceShardingSphereDataSourceFactoryTest {
     
     private static final String TABLE_TYPE = "TABLE";
     
+    private static final String VIEW_TYPE = "VIEW";
+    
     @Test
     public void assertCreateDataSourceWhenRuleConfigurationsNotEmpty() throws SQLException {
         DataSource dataSource = GovernanceShardingSphereDataSourceFactory.createDataSource(createDataSourceMap(), Collections.singletonList(mock(RuleConfiguration.class)),
@@ -68,7 +70,7 @@ public final class GovernanceShardingSphereDataSourceFactoryTest {
         when(connection.getMetaData()).thenReturn(databaseMetaData);
         when(databaseMetaData.getURL()).thenReturn("jdbc:mysql://localhost:3306/mysql?serverTimezone=GMT%2B8");
         ResultSet resultSet = mock(ResultSet.class);
-        when(databaseMetaData.getTables(null, null, null, new String[]{TABLE_TYPE})).thenReturn(resultSet);
+        when(databaseMetaData.getTables(null, null, null, new String[]{TABLE_TYPE, VIEW_TYPE})).thenReturn(resultSet);
         when(result.getConnection()).thenReturn(connection);
         return result;
     }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/MySQLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/MySQLStatement.g4
index b5ad704..1e6e5cf 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/MySQLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/MySQLStatement.g4
@@ -52,6 +52,9 @@ execute
     | createServer
     | alterServer
     | dropServer
+    | createView
+    | alterView
+    | dropView
     | createTrigger
     | dropTrigger
     | setTransaction
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
index 702c9b6..1bbef03 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AlterPr
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AlterServerContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AlterSpecificationContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AlterTableContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AlterViewContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.BeginStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CaseStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ChangeColumnSpecificationContext;
@@ -114,6 +115,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterProcedureStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterServerStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateDatabaseStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateEventStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateFunctionStatement;
@@ -153,7 +155,16 @@ public final class MySQLDDLVisitor extends MySQLVisitor implements DDLVisitor {
     
     @Override
     public ASTNode visitCreateView(final CreateViewContext ctx) {
-        return new MySQLCreateViewStatement();
+        MySQLCreateViewStatement result = new MySQLCreateViewStatement();
+        result.setSelect((MySQLSelectStatement) visit(ctx.select()));
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitAlterView(final AlterViewContext ctx) {
+        MySQLAlterViewStatement result = new MySQLAlterViewStatement();
+        result.setSelect((MySQLSelectStatement) visit(ctx.select()));
+        return result;
     }
     
     @Override
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/impl/PostgreSQLDDLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/impl/PostgreSQLDDLVisitor.java
index cc59c17..4423d56 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/impl/PostgreSQLDDLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/impl/PostgreSQLDDLVisitor.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Al
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterProcedureContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterTableActionContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterTableContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterViewContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.ColumnConstraintContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.ColumnDefinitionContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CreateDatabaseContext;
@@ -70,6 +71,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterProcedureStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLCreateDatabaseStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLCreateFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLCreateIndexStatement;
@@ -341,6 +343,11 @@ public final class PostgreSQLDDLVisitor extends PostgreSQLVisitor implements DDL
     }
     
     @Override
+    public ASTNode visitAlterView(final AlterViewContext ctx) {
+        return new PostgreSQLAlterViewStatement();
+    }
+    
+    @Override
     public ASTNode visitDropDatabase(final DropDatabaseContext ctx) {
         PostgreSQLDropDatabaseStatement result = new PostgreSQLDropDatabaseStatement();
         result.setDatabaseName(((IdentifierValue) visit(ctx.name())).getValue());
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/visitor/VisitorRule.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/visitor/VisitorRule.java
index ae07c49..4e37f2d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/visitor/VisitorRule.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/visitor/VisitorRule.java
@@ -96,6 +96,8 @@ public enum VisitorRule {
     
     CREATE_VIEW("CreateView", SQLStatementType.DDL),
     
+    ALTER_VIEW("AlterView", SQLStatementType.DDL),
+    
     DROP_VIEW("DropView", SQLStatementType.DDL),
     
     SET_TRANSACTION("SetTransaction", SQLStatementType.TCL),
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/ddl/AlterViewStatement.java
similarity index 73%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/ddl/AlterViewStatement.java
index e979725..691dc46 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/ddl/AlterViewStatement.java
@@ -15,13 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl;
+package org.apache.shardingsphere.sql.parser.sql.common.statement.ddl;
 
-import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
 
 /**
- * MySQL create view statement.
+ * Alter view statement.
  */
-public final class MySQLCreateViewStatement extends CreateViewStatement implements MySQLStatement {
+public abstract class AlterViewStatement extends AbstractSQLStatement implements DDLStatement {
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterViewStatementHandler.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterViewStatementHandler.java
new file mode 100644
index 0000000..00eb662
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterViewStatementHandler.java
@@ -0,0 +1,48 @@
+/*
+ * 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.sql.parser.sql.dialect.handler.ddl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.SQLStatementHandler;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterViewStatement;
+
+import java.util.Optional;
+
+/**
+ * AlterViewStatement handler for different dialect SQLStatements.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class AlterViewStatementHandler implements SQLStatementHandler {
+    
+    /**
+     * Get select statement.
+     *
+     * @param alterViewStatement AlterViewStatement
+     * @return select statement
+     */
+    public static Optional<SelectStatement> getSelectStatement(final AlterViewStatement alterViewStatement) {
+        if (alterViewStatement instanceof MySQLStatement) {
+            return ((MySQLAlterViewStatement) alterViewStatement).getSelect();
+        }
+        return Optional.empty();
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateViewStatementHandler.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateViewStatementHandler.java
new file mode 100644
index 0000000..722ce3b
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateViewStatementHandler.java
@@ -0,0 +1,48 @@
+/*
+ * 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.sql.parser.sql.dialect.handler.ddl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.SQLStatementHandler;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateViewStatement;
+
+import java.util.Optional;
+
+/**
+ * CreateViewStatement handler for different dialect SQLStatements.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateViewStatementHandler implements SQLStatementHandler {
+    
+    /**
+     * Get select statement.
+     *
+     * @param createViewStatement CreateViewStatement
+     * @return select statement
+     */
+    public static Optional<SelectStatement> getSelectStatement(final CreateViewStatement createViewStatement) {
+        if (createViewStatement instanceof MySQLStatement) {
+            return ((MySQLCreateViewStatement) createViewStatement).getSelect();
+        }
+        return Optional.empty();
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLAlterViewStatement.java
similarity index 65%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLAlterViewStatement.java
index e979725..a129504 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLAlterViewStatement.java
@@ -17,11 +17,29 @@
 
 package org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl;
 
-import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
 
+import java.util.Optional;
+
 /**
- * MySQL create view statement.
+ * MySQL alter view statement.
  */
-public final class MySQLCreateViewStatement extends CreateViewStatement implements MySQLStatement {
+@Getter
+@Setter
+public class MySQLAlterViewStatement extends AlterViewStatement implements MySQLStatement {
+    
+    private SelectStatement select;
+    
+    /**
+     * Get select statement.
+     *
+     * @return select statement
+     */
+    public Optional<SelectStatement> getSelect() {
+        return Optional.ofNullable(select);
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
index e979725..917a0ac 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
@@ -17,11 +17,29 @@
 
 package org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl;
 
+import lombok.Getter;
+import lombok.Setter;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
 
+import java.util.Optional;
+
 /**
  * MySQL create view statement.
  */
+@Getter
+@Setter
 public final class MySQLCreateViewStatement extends CreateViewStatement implements MySQLStatement {
+    
+    private SelectStatement select;
+    
+    /**
+     * Get select statement.
+     *
+     * @return select statement
+     */
+    public Optional<SelectStatement> getSelect() {
+        return Optional.ofNullable(select);
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/ddl/PostgreSQLAlterViewStatement.java
similarity index 82%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/ddl/PostgreSQLAlterViewStatement.java
index e979725..1686be0 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateViewStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/ddl/PostgreSQLAlterViewStatement.java
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl;
+package org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl;
 
-import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.PostgreSQLStatement;
 
 /**
- * MySQL create view statement.
+ * PostgreSQL alter view statement.
  */
-public final class MySQLCreateViewStatement extends CreateViewStatement implements MySQLStatement {
+public class PostgreSQLAlterViewStatement extends AlterViewStatement implements PostgreSQLStatement {
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterViewStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterViewStatementHandlerTest.java
new file mode 100644
index 0000000..4894e99
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterViewStatementHandlerTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.sql.parser.sql.dialect.handler.ddl;
+
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public final class AlterViewStatementHandlerTest {
+    
+    @Test
+    public void assertGetSelectStatementWithSelectStatementForMySQL() {
+        MySQLAlterViewStatement createViewStatement = new MySQLAlterViewStatement();
+        createViewStatement.setSelect(new MySQLSelectStatement());
+        Optional<SelectStatement> selectStatement = AlterViewStatementHandler.getSelectStatement(createViewStatement);
+        assertTrue(selectStatement.isPresent());
+    }
+    
+    @Test
+    public void assertGetSelectStatementWithoutSelectStatementForMySQL() {
+        MySQLAlterViewStatement createViewStatement = new MySQLAlterViewStatement();
+        Optional<SelectStatement> selectStatement = AlterViewStatementHandler.getSelectStatement(createViewStatement);
+        assertFalse(selectStatement.isPresent());
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateViewStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateViewStatementHandlerTest.java
new file mode 100644
index 0000000..6ee05f4
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateViewStatementHandlerTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.sql.parser.sql.dialect.handler.ddl;
+
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public final class CreateViewStatementHandlerTest {
+    
+    @Test
+    public void assertGetSelectStatementWithSelectStatementForMySQL() {
+        MySQLCreateViewStatement createViewStatement = new MySQLCreateViewStatement();
+        createViewStatement.setSelect(new MySQLSelectStatement());
+        Optional<SelectStatement> selectStatement = CreateViewStatementHandler.getSelectStatement(createViewStatement);
+        assertTrue(selectStatement.isPresent());
+    }
+    
+    @Test
+    public void assertGetSelectStatementWithoutSelectStatementForMySQL() {
+        MySQLCreateViewStatement createViewStatement = new MySQLCreateViewStatement();
+        Optional<SelectStatement> selectStatement = CreateViewStatementHandler.getSelectStatement(createViewStatement);
+        assertFalse(selectStatement.isPresent());
+    }
+}