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/09 04:38:15 UTC

[shardingsphere] branch master updated: add create function and procedure validator (#7704)

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 e3bb018  add create function and procedure validator (#7704)
e3bb018 is described below

commit e3bb018d4366da9d5f1ba51197ae3403a2596c99
Author: DuanZhengqiang <st...@gmail.com>
AuthorDate: Fri Oct 9 12:37:59 2020 +0800

    add create function and procedure validator (#7704)
    
    * add create function and procedure validator
    
    * optimize create function and procedure validator
---
 .../mysql/constant/MySQLServerErrorCode.java       |   2 +
 .../engine/exception/NoSuchTableException.java     |  23 +-
 .../engine/type/ShardingRouteEngineFactory.java    |  21 +-
 .../ShardingStatementValidatorFactory.java         |  10 +
 .../ShardingCreateFunctionStatementValidator.java  |  79 +++
 .../ShardingCreateProcedureStatementValidator.java |  79 +++
 ...ardingCreateFunctionStatementValidatorTest.java | 142 +++++
 ...rdingCreateProcedureStatementValidatorTest.java | 142 +++++
 .../frontend/mysql/MySQLErrPacketFactory.java      |   4 +
 .../frontend/mysql/MySQLErrPacketFactoryTest.java  |  10 +
 .../sql/parser/mysql/visitor/MySQLVisitor.java     | 552 ++++++++++++++++++
 .../parser/mysql/visitor/impl/MySQLDDLVisitor.java | 211 ++++++-
 .../parser/mysql/visitor/impl/MySQLDMLVisitor.java | 626 ---------------------
 .../sql/common/extractor/TableExtractor.java       |  60 ++
 .../segment/ddl/routine/RoutineBodySegment.java}   |  22 +-
 .../segment/ddl/routine/ValidStatementSegment.java | 130 +++++
 .../ddl/CreateFunctionStatementHandler.java        |  48 ++
 .../ddl/CreateProcedureStatementHandler.java       |  48 ++
 .../mysql/ddl/MySQLCreateFunctionStatement.java    |  18 +
 .../mysql/ddl/MySQLCreateProcedureStatement.java   |  18 +
 .../ddl/AlterIndexStatementHandlerTest.java        |   2 +-
 ...ava => CreateFunctionStatementHandlerTest.java} |  26 +-
 ...va => CreateProcedureStatementHandlerTest.java} |  26 +-
 .../handler/ddl/DropIndexStatementHandlerTest.java |   2 +-
 .../handler/dml/DeleteStatementHandlerTest.java    |   2 +-
 .../handler/dml/InsertStatementHandlerTest.java    |   2 +-
 .../handler/dml/SelectStatementHandlerTest.java    |   2 +-
 .../handler/dml/UpdateStatementHandlerTest.java    |   2 +-
 28 files changed, 1619 insertions(+), 690 deletions(-)

diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/constant/MySQLServerErrorCode.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/constant/MySQLServerErrorCode.java
index 4a23ba1..adaa7bc 100644
--- a/shardingsphere-db-protocol/shardingsphere-db-protocol-mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/constant/MySQLServerErrorCode.java
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-mysql/src/main/java/org/apache/shardingsphere/db/protocol/mysql/constant/MySQLServerErrorCode.java
@@ -48,6 +48,8 @@ public enum MySQLServerErrorCode implements SQLErrorCode {
     
     ER_TABLE_EXISTS_ERROR(1050, "42S01", "Table '%s' already exists"),
     
+    ER_NO_SUCH_TABLE(1146, "42S02", "Table '%s.%s' doesn't exist"),
+    
     ER_NOT_SUPPORTED_YET(1235, "42000", "This version of ShardingProxy doesn't yet support this SQL. '%s'"),
     
     ER_SP_DOES_NOT_EXIST(1305, "42000", "Message: Datasource or ShardingSphere rule does not exist"),
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/exception/NoSuchTableException.java
similarity index 54%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/exception/NoSuchTableException.java
index 3cb41a9..8e9d842 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/exception/NoSuchTableException.java
@@ -15,13 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl;
+package org.apache.shardingsphere.sharding.route.engine.exception;
 
-import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+import lombok.Getter;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 
 /**
- * MySQL create function statement.
+ * No such table exception.
  */
-public final class MySQLCreateFunctionStatement extends CreateFunctionStatement implements MySQLStatement {
+@Getter
+public final class NoSuchTableException extends ShardingSphereException {
+    
+    private static final long serialVersionUID = 8311953084941769743L;
+    
+    private final String databaseName;
+    
+    private final String tableName;
+    
+    public NoSuchTableException(final String databaseName, final String tableName) {
+        super(String.format("Table '%s.%s' doesn't exist", databaseName, tableName));
+        this.databaseName = databaseName;
+        this.tableName = tableName;
+    }
 }
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 6fc5c52..42d3bea 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
@@ -32,13 +32,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.sql.parser.binder.metadata.schema.SchemaMetaData;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.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.CreateFunctionStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
 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.dml.DMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.TCLStatement;
@@ -73,7 +80,7 @@ public final class ShardingRouteEngineFactory {
             return new ShardingDatabaseBroadcastRoutingEngine();
         }
         if (sqlStatement instanceof DDLStatement) {
-            return new ShardingTableBroadcastRoutingEngine(metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), sqlStatementContext);
+            return getDDLRoutingEngine(metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), sqlStatementContext);
         }
         if (sqlStatement instanceof DALStatement) {
             return getDALRoutingEngine(shardingRule, metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap(), sqlStatement, tableNames);
@@ -93,7 +100,17 @@ public final class ShardingRouteEngineFactory {
         return getShardingRoutingEngine(shardingRule, shardingConditions, tableNames, props);
     }
     
-    private static ShardingRouteEngine getDALRoutingEngine(final ShardingRule shardingRule, 
+    private static ShardingRouteEngine getDDLRoutingEngine(final SchemaMetaData configuredSchemaMetaData, 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();
+        }
+        return new ShardingTableBroadcastRoutingEngine(configuredSchemaMetaData, sqlStatementContext);
+    }
+    
+    private static ShardingRouteEngine getDALRoutingEngine(final ShardingRule shardingRule,
                                                            final Map<String, Collection<String>> unconfiguredSchemaMetaDataMap, final SQLStatement sqlStatement, final Collection<String> tableNames) {
         if (sqlStatement instanceof MySQLUseStatement) {
             return new ShardingIgnoreRoutingEngine();
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 f534741..d572af2 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,11 +19,15 @@ 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.sql.parser.sql.common.statement.SQLStatement;
+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.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
@@ -56,6 +60,12 @@ public final class ShardingStatementValidatorFactory {
         if (sqlStatement instanceof CreateTableStatement) {
             return Optional.of(new ShardingCreateTableStatementValidator());
         }
+        if (sqlStatement instanceof CreateFunctionStatement) {
+            return Optional.of(new ShardingCreateFunctionStatementValidator());
+        }
+        if (sqlStatement instanceof CreateProcedureStatement) {
+            return Optional.of(new ShardingCreateProcedureStatementValidator());
+        }
         return Optional.empty();
     }
 }
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
new file mode 100644
index 0000000..9dad727
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+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.sql.parser.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;
+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.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Sharding create function statement validator.
+ */
+public final class ShardingCreateFunctionStatementValidator implements ShardingStatementValidator<CreateFunctionStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateFunctionStatement> sqlStatementContext,
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        Optional<RoutineBodySegment> routineBodySegment = CreateFunctionStatementHandler.getRoutineBodySegment(sqlStatementContext.getSqlStatement());
+        TableExtractor extractor = new TableExtractor();
+        routineBodySegment.ifPresent(routineBody -> validateShardingTableAndTableExist(metaData, extractor.extractExistTableFromRoutineBody(routineBody)));
+        routineBodySegment.ifPresent(routineBody -> validateTableNotExist(metaData, extractor.extractNotExistTableFromRoutineBody(routineBody)));
+    }
+    
+    @Override
+    public void postValidate(final CreateFunctionStatement sqlStatement, final RouteContext routeContext) {
+    }
+    
+    private void validateShardingTableAndTableExist(final ShardingSphereMetaData metaData, final Collection<SimpleTableSegment> simpleTableSegments) {
+        for (SimpleTableSegment each : simpleTableSegments) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new ShardingSphereException("Create function statement can not support sharding table '%s'.", tableName);
+            }
+            for (Map.Entry<String, Collection<String>> entry : metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().entrySet()) {
+                if (!entry.getValue().contains(tableName)) {
+                    throw new NoSuchTableException(entry.getKey(), tableName);
+                }
+            }
+        }
+    }
+    
+    private void validateTableNotExist(final ShardingSphereMetaData metaData, final Collection<SimpleTableSegment> simpleTableSegments) {
+        for (SimpleTableSegment each : simpleTableSegments) {
+            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/ShardingCreateProcedureStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidator.java
new file mode 100644
index 0000000..01c9351
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+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.sql.parser.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;
+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.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Sharding create procedure statement validator.
+ */
+public final class ShardingCreateProcedureStatementValidator implements ShardingStatementValidator<CreateProcedureStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateProcedureStatement> sqlStatementContext,
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        Optional<RoutineBodySegment> routineBodySegment = CreateProcedureStatementHandler.getRoutineBodySegment(sqlStatementContext.getSqlStatement());
+        TableExtractor extractor = new TableExtractor();
+        routineBodySegment.ifPresent(routineBody -> validateShardingTableAndTableExist(metaData, extractor.extractExistTableFromRoutineBody(routineBody)));
+        routineBodySegment.ifPresent(routineBody -> validateTableNotExist(metaData, extractor.extractNotExistTableFromRoutineBody(routineBody)));
+    }
+    
+    @Override
+    public void postValidate(final CreateProcedureStatement sqlStatement, final RouteContext routeContext) {
+    }
+    
+    private void validateShardingTableAndTableExist(final ShardingSphereMetaData metaData, final Collection<SimpleTableSegment> simpleTableSegments) {
+        for (SimpleTableSegment each : simpleTableSegments) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new ShardingSphereException("Create procedure statement can not support sharding table '%s'.", tableName);
+            }
+            for (Map.Entry<String, Collection<String>> entry : metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().entrySet()) {
+                if (!entry.getValue().contains(tableName)) {
+                    throw new NoSuchTableException(entry.getKey(), tableName);
+                }
+            }
+        }
+    }
+    
+    private void validateTableNotExist(final ShardingSphereMetaData metaData, final Collection<SimpleTableSegment> simpleTableSegments) {
+        for (SimpleTableSegment each : simpleTableSegments) {
+            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/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/impl/ShardingCreateFunctionStatementValidatorTest.java
new file mode 100644
index 0000000..184afb7
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidatorTest.java
@@ -0,0 +1,142 @@
+/*
+ * 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.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.binder.metadata.schema.SchemaMetaData;
+import org.apache.shardingsphere.sql.parser.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.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;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateFunctionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateTableStatement;
+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 ShardingCreateFunctionStatementValidatorTest {
+    
+    @Mock
+    private ShardingRule shardingRule;
+    
+    @Test
+    public void assertValidateCreateFunctionForMySQL() {
+        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
+        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order_item")));
+        MySQLCreateTableStatement createTableStatement = new MySQLCreateTableStatement();
+        createTableStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        ValidStatementSegment createTableValidStatementSegment = new ValidStatementSegment(0, 0);
+        createTableValidStatementSegment.setSqlStatement(createTableStatement);
+        ValidStatementSegment selectValidStatementSegment = new ValidStatementSegment(0, 0);
+        selectValidStatementSegment.setSqlStatement(selectStatement);
+        RoutineBodySegment routineBody = new RoutineBodySegment(0, 0);
+        routineBody.getValidStatements().add(createTableValidStatementSegment);
+        routineBody.getValidStatements().add(selectValidStatementSegment);
+        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.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);
+        when(ruleSchemaMetaData.getAllTableNames()).thenReturn(Collections.emptyList());
+        SQLStatementContext<CreateFunctionStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingCreateFunctionStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertValidateCreateFunctionWithShardingTableForMySQL() {
+        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.singleton("t_order"));
+        SQLStatementContext<CreateFunctionStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        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();
+        createTableStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        ValidStatementSegment validStatementSegment = new ValidStatementSegment(0, 0);
+        validStatementSegment.setSqlStatement(createTableStatement);
+        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);
+        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
+        when(ruleSchemaMetaData.getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
+        SQLStatementContext<CreateFunctionStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingCreateFunctionStatementValidator().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/ShardingCreateProcedureStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidatorTest.java
new file mode 100644
index 0000000..0a3506e
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateProcedureStatementValidatorTest.java
@@ -0,0 +1,142 @@
+/*
+ * 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.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.binder.metadata.schema.SchemaMetaData;
+import org.apache.shardingsphere.sql.parser.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.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;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateProcedureStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateTableStatement;
+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 ShardingCreateProcedureStatementValidatorTest {
+    
+    @Mock
+    private ShardingRule shardingRule;
+    
+    @Test
+    public void assertValidateCreateProcedureForMySQL() {
+        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
+        selectStatement.setFrom(new SimpleTableSegment(0, 0, new IdentifierValue("t_order_item")));
+        MySQLCreateTableStatement createTableStatement = new MySQLCreateTableStatement();
+        createTableStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        ValidStatementSegment createTableValidStatementSegment = new ValidStatementSegment(0, 0);
+        createTableValidStatementSegment.setSqlStatement(createTableStatement);
+        ValidStatementSegment selectValidStatementSegment = new ValidStatementSegment(0, 0);
+        selectValidStatementSegment.setSqlStatement(selectStatement);
+        RoutineBodySegment routineBody = new RoutineBodySegment(0, 0);
+        routineBody.getValidStatements().add(createTableValidStatementSegment);
+        routineBody.getValidStatements().add(selectValidStatementSegment);
+        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.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);
+        when(ruleSchemaMetaData.getAllTableNames()).thenReturn(Collections.emptyList());
+        SQLStatementContext<CreateProcedureStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingCreateProcedureStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertValidateCreateProcedureWithShardingTableForMySQL() {
+        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.singleton("t_order"));
+        SQLStatementContext<CreateProcedureStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        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();
+        createTableStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        ValidStatementSegment validStatementSegment = new ValidStatementSegment(0, 0);
+        validStatementSegment.setSqlStatement(createTableStatement);
+        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);
+        when(metaData.getRuleSchemaMetaData()).thenReturn(ruleSchemaMetaData);
+        when(ruleSchemaMetaData.getAllTableNames()).thenReturn(Collections.singletonList("t_order"));
+        SQLStatementContext<CreateProcedureStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
+        new ShardingCreateProcedureStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), metaData);
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactory.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactory.java
index a0bf7bd..066e03b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactory.java
@@ -35,6 +35,7 @@ import org.apache.shardingsphere.proxy.backend.text.sctl.exception.ShardingCTLEx
 import org.apache.shardingsphere.proxy.frontend.exception.UnsupportedCommandException;
 import org.apache.shardingsphere.proxy.frontend.exception.UnsupportedPreparedStatementException;
 import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
 import org.apache.shardingsphere.sql.parser.exception.SQLParsingException;
 
 import java.sql.SQLException;
@@ -79,6 +80,9 @@ public final class MySQLErrPacketFactory {
         if (cause instanceof TableExistsException) {
             return new MySQLErrPacket(1, MySQLServerErrorCode.ER_TABLE_EXISTS_ERROR, ((TableExistsException) cause).getTableName());
         }
+        if (cause instanceof NoSuchTableException) {
+            return new MySQLErrPacket(1, MySQLServerErrorCode.ER_NO_SUCH_TABLE, ((NoSuchTableException) cause).getDatabaseName(), ((NoSuchTableException) cause).getTableName());
+        }
         if (cause instanceof CircuitBreakException) {
             return new MySQLErrPacket(1, CommonErrorCode.CIRCUIT_BREAK_MODE);
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactoryTest.java
index 37f83c3..63ccb1d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLErrPacketFactoryTest.java
@@ -29,6 +29,7 @@ import org.apache.shardingsphere.proxy.backend.text.sctl.exception.InvalidShardi
 import org.apache.shardingsphere.proxy.backend.text.sctl.exception.UnsupportedShardingCTLTypeException;
 import org.apache.shardingsphere.proxy.frontend.exception.UnsupportedCommandException;
 import org.apache.shardingsphere.proxy.frontend.exception.UnsupportedPreparedStatementException;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
 import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
 import org.apache.shardingsphere.sql.parser.exception.SQLParsingException;
 import org.junit.Test;
@@ -152,6 +153,15 @@ public final class MySQLErrPacketFactoryTest {
     }
     
     @Test
+    public void assertNewInstanceWithNoSuchTableException() {
+        MySQLErrPacket actual = MySQLErrPacketFactory.newInstance(new NoSuchTableException("ds_0", "table_name"));
+        assertThat(actual.getSequenceId(), is(1));
+        assertThat(actual.getErrorCode(), is(1146));
+        assertThat(actual.getSqlState(), is("42S02"));
+        assertThat(actual.getErrorMessage(), is("Table 'ds_0.table_name' doesn't exist"));
+    }
+    
+    @Test
     public void assertNewInstanceWithCircuitBreakException() {
         MySQLErrPacket actual = MySQLErrPacketFactory.newInstance(new CircuitBreakException());
         assertThat(actual.getSequenceId(), is(1));
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/MySQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/MySQLVisitor.java
index 6e5fefe..9e2f23d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/MySQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/MySQLVisitor.java
@@ -23,6 +23,7 @@ import org.antlr.v4.runtime.misc.Interval;
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.shardingsphere.sql.parser.api.ASTNode;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementBaseVisitor;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AggregationFunctionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.BitExprContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.BitValueLiteralsContext;
@@ -70,7 +71,12 @@ import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.WindowF
 import org.apache.shardingsphere.sql.parser.sql.common.constant.AggregationType;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.OrderDirection;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.InsertValuesSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.SetAssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.InsertColumnsSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.OnDuplicateKeyColumnsSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExistsSubqueryExpression;
@@ -85,17 +91,34 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.subquery
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.subquery.SubquerySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.AggregationDistinctProjectionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.AggregationProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ColumnProjectionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ExpressionProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ShorthandProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.SubqueryProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.GroupBySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.OrderBySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.LimitSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.ParameterMarkerLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.LockSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.AliasSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.DataTypeLengthSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.DataTypeSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.OwnerSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.DeleteMultiTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.JoinTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SubqueryTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.util.SQLUtil;
 import org.apache.shardingsphere.sql.parser.sql.common.value.collection.CollectionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
@@ -105,7 +128,10 @@ import org.apache.shardingsphere.sql.parser.sql.common.value.literal.impl.Number
 import org.apache.shardingsphere.sql.parser.sql.common.value.literal.impl.OtherLiteralValue;
 import org.apache.shardingsphere.sql.parser.sql.common.value.literal.impl.StringLiteralValue;
 import org.apache.shardingsphere.sql.parser.sql.common.value.parametermarker.ParameterMarkerValue;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLDeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLUpdateStatement;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -690,4 +716,530 @@ public abstract class MySQLVisitor extends MySQLStatementBaseVisitor<ASTNode> {
         }
         return new ExpressionOrderByItemSegment(ctx.expr().getStart().getStartIndex(), ctx.expr().getStop().getStopIndex(), ctx.expr().getText(), orderDirection);
     }
+    
+    @Override
+    public ASTNode visitInsert(final MySQLStatementParser.InsertContext ctx) {
+        // TODO :FIXME, since there is no segment for insertValuesClause, InsertStatement is created by sub rule.
+        MySQLInsertStatement result;
+        if (null != ctx.insertValuesClause()) {
+            result = (MySQLInsertStatement) visit(ctx.insertValuesClause());
+        } else if (null != ctx.insertSelectClause()) {
+            result = (MySQLInsertStatement) visit(ctx.insertSelectClause());
+        } else {
+            result = new MySQLInsertStatement();
+            result.setSetAssignment((SetAssignmentSegment) visit(ctx.setAssignmentsClause()));
+        }
+        if (null != ctx.onDuplicateKeyClause()) {
+            result.setOnDuplicateKeyColumns((OnDuplicateKeyColumnsSegment) visit(ctx.onDuplicateKeyClause()));
+        }
+        result.setTable((SimpleTableSegment) visit(ctx.tableName()));
+        result.setParameterCount(getCurrentParameterIndex());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitInsertSelectClause(final MySQLStatementParser.InsertSelectClauseContext ctx) {
+        MySQLInsertStatement result = new MySQLInsertStatement();
+        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
+        result.setInsertSelect(createInsertSelectSegment(ctx));
+        return result;
+    }
+    
+    private SubquerySegment createInsertSelectSegment(final MySQLStatementParser.InsertSelectClauseContext ctx) {
+        MySQLSelectStatement selectStatement = (MySQLSelectStatement) visit(ctx.select());
+        return new SubquerySegment(ctx.select().start.getStartIndex(), ctx.select().stop.getStopIndex(), selectStatement);
+    }
+    
+    @Override
+    public ASTNode visitInsertValuesClause(final MySQLStatementParser.InsertValuesClauseContext ctx) {
+        MySQLInsertStatement result = new MySQLInsertStatement();
+        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
+        result.getValues().addAll(createInsertValuesSegments(ctx.assignmentValues()));
+        return result;
+    }
+    
+    private Collection<InsertValuesSegment> createInsertValuesSegments(final Collection<MySQLStatementParser.AssignmentValuesContext> assignmentValuesContexts) {
+        Collection<InsertValuesSegment> result = new LinkedList<>();
+        for (MySQLStatementParser.AssignmentValuesContext each : assignmentValuesContexts) {
+            result.add((InsertValuesSegment) visit(each));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitOnDuplicateKeyClause(final MySQLStatementParser.OnDuplicateKeyClauseContext ctx) {
+        Collection<AssignmentSegment> columns = new LinkedList<>();
+        for (MySQLStatementParser.AssignmentContext each : ctx.assignment()) {
+            columns.add((AssignmentSegment) visit(each));
+        }
+        return new OnDuplicateKeyColumnsSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), columns);
+    }
+    
+    @Override
+    public ASTNode visitReplace(final MySQLStatementParser.ReplaceContext ctx) {
+        // TODO :FIXME, since there is no segment for replaceValuesClause, ReplaceStatement is created by sub rule.
+        MySQLInsertStatement result;
+        if (null != ctx.replaceValuesClause()) {
+            result = (MySQLInsertStatement) visit(ctx.replaceValuesClause());
+        } else if (null != ctx.replaceSelectClause()) {
+            result = (MySQLInsertStatement) visit(ctx.replaceSelectClause());
+        } else {
+            result = new MySQLInsertStatement();
+            result.setSetAssignment((SetAssignmentSegment) visit(ctx.setAssignmentsClause()));
+        }
+        result.setTable((SimpleTableSegment) visit(ctx.tableName()));
+        result.setParameterCount(getCurrentParameterIndex());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitReplaceSelectClause(final MySQLStatementParser.ReplaceSelectClauseContext ctx) {
+        MySQLInsertStatement result = new MySQLInsertStatement();
+        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
+        result.setInsertSelect(createReplaceSelectSegment(ctx));
+        return result;
+    }
+    
+    private SubquerySegment createReplaceSelectSegment(final MySQLStatementParser.ReplaceSelectClauseContext ctx) {
+        MySQLSelectStatement selectStatement = (MySQLSelectStatement) visit(ctx.select());
+        return new SubquerySegment(ctx.select().start.getStartIndex(), ctx.select().stop.getStopIndex(), selectStatement);
+    }
+    
+    @Override
+    public ASTNode visitReplaceValuesClause(final MySQLStatementParser.ReplaceValuesClauseContext ctx) {
+        MySQLInsertStatement result = new MySQLInsertStatement();
+        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
+        result.getValues().addAll(createReplaceValuesSegments(ctx.assignmentValues()));
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    private InsertColumnsSegment createInsertColumns(final ColumnNamesContext columnNames, final int startIndex) {
+        if (null != columnNames) {
+            CollectionValue<ColumnSegment> columnSegments = (CollectionValue<ColumnSegment>) visit(columnNames);
+            return new InsertColumnsSegment(columnNames.start.getStartIndex(), columnNames.stop.getStopIndex(), columnSegments.getValue());
+        } else {
+            return new InsertColumnsSegment(startIndex - 1, startIndex - 1, Collections.emptyList());
+        }
+    }
+    
+    private Collection<InsertValuesSegment> createReplaceValuesSegments(final Collection<MySQLStatementParser.AssignmentValuesContext> assignmentValuesContexts) {
+        Collection<InsertValuesSegment> result = new LinkedList<>();
+        for (MySQLStatementParser.AssignmentValuesContext each : assignmentValuesContexts) {
+            result.add((InsertValuesSegment) visit(each));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitUpdate(final MySQLStatementParser.UpdateContext ctx) {
+        MySQLUpdateStatement result = new MySQLUpdateStatement();
+        TableSegment tableSegment = (TableSegment) visit(ctx.tableReferences());
+        result.setTableSegment(tableSegment);
+        result.setSetAssignment((SetAssignmentSegment) visit(ctx.setAssignmentsClause()));
+        if (null != ctx.whereClause()) {
+            result.setWhere((WhereSegment) visit(ctx.whereClause()));
+        }
+        if (null != ctx.orderByClause()) {
+            result.setOrderBy((OrderBySegment) visit(ctx.orderByClause()));
+        }
+        if (null != ctx.limitClause()) {
+            result.setLimit((LimitSegment) visit(ctx.limitClause()));
+        }
+        result.setParameterCount(getCurrentParameterIndex());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSetAssignmentsClause(final MySQLStatementParser.SetAssignmentsClauseContext ctx) {
+        Collection<AssignmentSegment> assignments = new LinkedList<>();
+        for (MySQLStatementParser.AssignmentContext each : ctx.assignment()) {
+            assignments.add((AssignmentSegment) visit(each));
+        }
+        return new SetAssignmentSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), assignments);
+    }
+    
+    @Override
+    public ASTNode visitAssignmentValues(final MySQLStatementParser.AssignmentValuesContext ctx) {
+        List<ExpressionSegment> segments = new LinkedList<>();
+        for (MySQLStatementParser.AssignmentValueContext each : ctx.assignmentValue()) {
+            segments.add((ExpressionSegment) visit(each));
+        }
+        return new InsertValuesSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), segments);
+    }
+    
+    @Override
+    public ASTNode visitAssignment(final MySQLStatementParser.AssignmentContext ctx) {
+        ColumnSegment column = (ColumnSegment) visitColumnName(ctx.columnName());
+        ExpressionSegment value = (ExpressionSegment) visit(ctx.assignmentValue());
+        return new AssignmentSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), column, value);
+    }
+    
+    @Override
+    public ASTNode visitAssignmentValue(final MySQLStatementParser.AssignmentValueContext ctx) {
+        ExprContext expr = ctx.expr();
+        if (null != expr) {
+            ASTNode result = visit(expr);
+            if (result instanceof ColumnSegment) {
+                return new CommonExpressionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+            } else {
+                return result;
+            }
+        }
+        return new CommonExpressionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+    }
+    
+    @Override
+    public ASTNode visitBlobValue(final MySQLStatementParser.BlobValueContext ctx) {
+        return new StringLiteralValue(ctx.STRING_().getText());
+    }
+    
+    @Override
+    public ASTNode visitDelete(final MySQLStatementParser.DeleteContext ctx) {
+        MySQLDeleteStatement result = new MySQLDeleteStatement();
+        if (null != ctx.multipleTablesClause()) {
+            result.setTableSegment((TableSegment) visit(ctx.multipleTablesClause()));
+        } else {
+            result.setTableSegment((TableSegment) visit(ctx.singleTableClause()));
+        }
+        if (null != ctx.whereClause()) {
+            result.setWhere((WhereSegment) visit(ctx.whereClause()));
+        }
+        if (null != ctx.orderByClause()) {
+            result.setOrderBy((OrderBySegment) visit(ctx.orderByClause()));
+        }
+        if (null != ctx.limitClause()) {
+            result.setLimit((LimitSegment) visit(ctx.limitClause()));
+        }
+        result.setParameterCount(getCurrentParameterIndex());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSingleTableClause(final MySQLStatementParser.SingleTableClauseContext ctx) {
+        SimpleTableSegment result = (SimpleTableSegment) visit(ctx.tableName());
+        if (null != ctx.alias()) {
+            result.setAlias((AliasSegment) visit(ctx.alias()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitMultipleTablesClause(final MySQLStatementParser.MultipleTablesClauseContext ctx) {
+        DeleteMultiTableSegment result = new DeleteMultiTableSegment();
+        TableSegment relateTableSource = (TableSegment) visit(ctx.tableReferences());
+        result.setRelationTable(relateTableSource);
+        result.setActualDeleteTables(generateTablesFromTableMultipleTableNames(ctx.multipleTableNames()));
+        return result;
+    }
+    
+    private List<SimpleTableSegment> generateTablesFromTableMultipleTableNames(final MySQLStatementParser.MultipleTableNamesContext ctx) {
+        List<SimpleTableSegment> result = new LinkedList<>();
+        for (TableNameContext each : ctx.tableName()) {
+            result.add((SimpleTableSegment) visit(each));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSelect(final MySQLStatementParser.SelectContext ctx) {
+        // TODO :Unsupported for withClause.
+        MySQLSelectStatement result = (MySQLSelectStatement) visit(ctx.unionClause());
+        result.setParameterCount(getCurrentParameterIndex());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitUnionClause(final MySQLStatementParser.UnionClauseContext ctx) {
+        // TODO :Unsupported for union SQL.
+        return visit(ctx.selectClause(0));
+    }
+    
+    @Override
+    public ASTNode visitSelectClause(final MySQLStatementParser.SelectClauseContext ctx) {
+        MySQLSelectStatement result = new MySQLSelectStatement();
+        result.setProjections((ProjectionsSegment) visit(ctx.projections()));
+        if (null != ctx.selectSpecification()) {
+            result.getProjections().setDistinctRow(isDistinct(ctx));
+        }
+        if (null != ctx.fromClause()) {
+            TableSegment tableSource = (TableSegment) visit(ctx.fromClause().tableReferences());
+            result.setFrom(tableSource);
+        }
+        if (null != ctx.whereClause()) {
+            result.setWhere((WhereSegment) visit(ctx.whereClause()));
+        }
+        if (null != ctx.groupByClause()) {
+            result.setGroupBy((GroupBySegment) visit(ctx.groupByClause()));
+        }
+        if (null != ctx.orderByClause()) {
+            result.setOrderBy((OrderBySegment) visit(ctx.orderByClause()));
+        }
+        if (null != ctx.limitClause()) {
+            result.setLimit((LimitSegment) visit(ctx.limitClause()));
+        }
+        if (null != ctx.lockClause()) {
+            result.setLock((LockSegment) visit(ctx.lockClause()));
+        }
+        return result;
+    }
+    
+    private boolean isDistinct(final MySQLStatementParser.SelectClauseContext ctx) {
+        for (MySQLStatementParser.SelectSpecificationContext each : ctx.selectSpecification()) {
+            if (((BooleanLiteralValue) visit(each)).getValue()) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    @Override
+    public ASTNode visitSelectSpecification(final MySQLStatementParser.SelectSpecificationContext ctx) {
+        if (null != ctx.duplicateSpecification()) {
+            return visit(ctx.duplicateSpecification());
+        }
+        return new BooleanLiteralValue(false);
+    }
+    
+    @Override
+    public ASTNode visitDuplicateSpecification(final MySQLStatementParser.DuplicateSpecificationContext ctx) {
+        String text = ctx.getText();
+        if ("DISTINCT".equalsIgnoreCase(text) || "DISTINCTROW".equalsIgnoreCase(text)) {
+            return new BooleanLiteralValue(true);
+        }
+        return new BooleanLiteralValue(false);
+    }
+    
+    @Override
+    public ASTNode visitProjections(final MySQLStatementParser.ProjectionsContext ctx) {
+        Collection<ProjectionSegment> projections = new LinkedList<>();
+        if (null != ctx.unqualifiedShorthand()) {
+            projections.add(new ShorthandProjectionSegment(ctx.unqualifiedShorthand().getStart().getStartIndex(), ctx.unqualifiedShorthand().getStop().getStopIndex()));
+        }
+        for (MySQLStatementParser.ProjectionContext each : ctx.projection()) {
+            projections.add((ProjectionSegment) visit(each));
+        }
+        ProjectionsSegment result = new ProjectionsSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex());
+        result.getProjections().addAll(projections);
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitProjection(final MySQLStatementParser.ProjectionContext ctx) {
+        // FIXME :The stop index of project is the stop index of projection, instead of alias.
+        if (null != ctx.qualifiedShorthand()) {
+            MySQLStatementParser.QualifiedShorthandContext shorthand = ctx.qualifiedShorthand();
+            ShorthandProjectionSegment result = new ShorthandProjectionSegment(shorthand.getStart().getStartIndex(), shorthand.getStop().getStopIndex());
+            IdentifierValue identifier = new IdentifierValue(shorthand.identifier().getText());
+            result.setOwner(new OwnerSegment(shorthand.identifier().getStart().getStartIndex(), shorthand.identifier().getStop().getStopIndex(), identifier));
+            return result;
+        }
+        AliasSegment alias = null == ctx.alias() ? null : (AliasSegment) visit(ctx.alias());
+        ASTNode exprProjection = visit(ctx.expr());
+        if (exprProjection instanceof ColumnSegment) {
+            ColumnProjectionSegment result = new ColumnProjectionSegment((ColumnSegment) exprProjection);
+            result.setAlias(alias);
+            return result;
+        }
+        if (exprProjection instanceof SubquerySegment) {
+            SubqueryProjectionSegment result = new SubqueryProjectionSegment((SubquerySegment) exprProjection);
+            result.setAlias(alias);
+            return result;
+        }
+        return createProjection(ctx, alias, exprProjection);
+    }
+    
+    @Override
+    public ASTNode visitAlias(final MySQLStatementParser.AliasContext ctx) {
+        if (null != ctx.identifier()) {
+            return new AliasSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), (IdentifierValue) visit(ctx.identifier()));
+        }
+        return new AliasSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(ctx.STRING_().getText()));
+    }
+    
+    private ASTNode createProjection(final MySQLStatementParser.ProjectionContext ctx, final AliasSegment alias, final ASTNode projection) {
+        if (projection instanceof AggregationProjectionSegment) {
+            ((AggregationProjectionSegment) projection).setAlias(alias);
+            return projection;
+        }
+        if (projection instanceof ExpressionProjectionSegment) {
+            ((ExpressionProjectionSegment) projection).setAlias(alias);
+            return projection;
+        }
+        if (projection instanceof CommonExpressionSegment) {
+            CommonExpressionSegment segment = (CommonExpressionSegment) projection;
+            ExpressionProjectionSegment result = new ExpressionProjectionSegment(segment.getStartIndex(), segment.getStopIndex(), segment.getText());
+            result.setAlias(alias);
+            return result;
+        }
+        // FIXME :For DISTINCT()
+        if (projection instanceof ColumnSegment) {
+            ExpressionProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), ctx.getText());
+            result.setAlias(alias);
+            return result;
+        }
+        if (projection instanceof SubqueryExpressionSegment) {
+            SubqueryProjectionSegment result = new SubqueryProjectionSegment(((SubqueryExpressionSegment) projection).getSubquery());
+            result.setAlias(alias);
+            return result;
+        }
+        if (projection instanceof BinaryOperationExpression) {
+            int startIndex = ((BinaryOperationExpression) projection).getStartIndex();
+            int stopIndex = null != alias ? alias.getStopIndex() : ((BinaryOperationExpression) projection).getStopIndex();
+            ExpressionProjectionSegment result = new ExpressionProjectionSegment(startIndex, stopIndex, ((BinaryOperationExpression) projection).getText());
+            result.setAlias(alias);
+            return result;
+        }
+        LiteralExpressionSegment column = (LiteralExpressionSegment) projection;
+        ExpressionProjectionSegment result = null == alias ? new ExpressionProjectionSegment(column.getStartIndex(), column.getStopIndex(), String.valueOf(column.getLiterals()))
+                : new ExpressionProjectionSegment(column.getStartIndex(), ctx.alias().stop.getStopIndex(), String.valueOf(column.getLiterals()));
+        result.setAlias(alias);
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitFromClause(final MySQLStatementParser.FromClauseContext ctx) {
+        return visit(ctx.tableReferences());
+    }
+    
+    @Override
+    public ASTNode visitTableReferences(final MySQLStatementParser.TableReferencesContext ctx) {
+        TableSegment result = (TableSegment) visit(ctx.escapedTableReference(0));
+        if (ctx.escapedTableReference().size() > 1) {
+            for (int i = 1; i < ctx.escapedTableReference().size(); i++) {
+                result = generateJoinTableSourceFromEscapedTableReference(ctx.escapedTableReference(i), result);
+            }
+        }
+        return result;
+    }
+    
+    private JoinTableSegment generateJoinTableSourceFromEscapedTableReference(final MySQLStatementParser.EscapedTableReferenceContext ctx, final TableSegment tableSegment) {
+        JoinTableSegment result = new JoinTableSegment();
+        result.setStartIndex(tableSegment.getStartIndex());
+        result.setStopIndex(ctx.stop.getStopIndex());
+        result.setLeft(tableSegment);
+        result.setRight((TableSegment) visit(ctx));
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitEscapedTableReference(final MySQLStatementParser.EscapedTableReferenceContext ctx) {
+        return visit(ctx.tableReference());
+    }
+    
+    @Override
+    public ASTNode visitTableReference(final MySQLStatementParser.TableReferenceContext ctx) {
+        TableSegment result;
+        TableSegment left;
+        left = (TableSegment) visit(ctx.tableFactor());
+        if (!ctx.joinedTable().isEmpty()) {
+            for (MySQLStatementParser.JoinedTableContext each : ctx.joinedTable()) {
+                left = visitJoinedTable(each, left);
+            }
+        }
+        result = left;
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitTableFactor(final MySQLStatementParser.TableFactorContext ctx) {
+        if (null != ctx.subquery()) {
+            MySQLSelectStatement subquery = (MySQLSelectStatement) visit(ctx.subquery());
+            SubquerySegment subquerySegment = new SubquerySegment(ctx.subquery().start.getStartIndex(), ctx.subquery().stop.getStopIndex(), subquery);
+            SubqueryTableSegment result = new SubqueryTableSegment(subquerySegment);
+            if (null != ctx.alias()) {
+                result.setAlias((AliasSegment) visit(ctx.alias()));
+            }
+            return result;
+        }
+        if (null != ctx.tableName()) {
+            SimpleTableSegment result = (SimpleTableSegment) visit(ctx.tableName());
+            if (null != ctx.alias()) {
+                result.setAlias((AliasSegment) visit(ctx.alias()));
+            }
+            return result;
+        }
+        return visit(ctx.tableReferences());
+    }
+    
+    private JoinTableSegment visitJoinedTable(final MySQLStatementParser.JoinedTableContext ctx, final TableSegment tableSegment) {
+        JoinTableSegment result = new JoinTableSegment();
+        result.setLeft(tableSegment);
+        result.setStartIndex(tableSegment.getStartIndex());
+        result.setStopIndex(ctx.stop.getStopIndex());
+        TableSegment right = (TableSegment) visit(ctx.tableFactor());
+        result.setRight(right);
+        if (null != ctx.joinSpecification()) {
+            result = visitJoinSpecification(ctx.joinSpecification(), result);
+        }
+        return result;
+    }
+    
+    private JoinTableSegment visitJoinSpecification(final MySQLStatementParser.JoinSpecificationContext ctx, final JoinTableSegment joinTableSource) {
+        if (null != ctx.expr()) {
+            ExpressionSegment condition = (ExpressionSegment) visit(ctx.expr());
+            joinTableSource.setCondition(condition);
+        }
+        if (null != ctx.USING()) {
+            List<ColumnSegment> columnSegmentList = new LinkedList<>();
+            for (MySQLStatementParser.ColumnNameContext cname : ctx.columnNames().columnName()) {
+                columnSegmentList.add((ColumnSegment) visit(cname));
+            }
+            joinTableSource.setUsing(columnSegmentList);
+        }
+        return joinTableSource;
+    }
+    
+    @Override
+    public ASTNode visitWhereClause(final MySQLStatementParser.WhereClauseContext ctx) {
+        ASTNode segment = visit(ctx.expr());
+        return new WhereSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), (ExpressionSegment) segment);
+    }
+    
+    @Override
+    public ASTNode visitGroupByClause(final MySQLStatementParser.GroupByClauseContext ctx) {
+        Collection<OrderByItemSegment> items = new LinkedList<>();
+        for (OrderByItemContext each : ctx.orderByItem()) {
+            items.add((OrderByItemSegment) visit(each));
+        }
+        return new GroupBySegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), items);
+    }
+    
+    @Override
+    public ASTNode visitLimitClause(final MySQLStatementParser.LimitClauseContext ctx) {
+        if (null == ctx.limitOffset()) {
+            return new LimitSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), null, (PaginationValueSegment) visit(ctx.limitRowCount()));
+        }
+        PaginationValueSegment rowCount;
+        PaginationValueSegment offset;
+        if (null != ctx.OFFSET()) {
+            rowCount = (PaginationValueSegment) visit(ctx.limitRowCount());
+            offset = (PaginationValueSegment) visit(ctx.limitOffset());
+        } else {
+            offset = (PaginationValueSegment) visit(ctx.limitOffset());
+            rowCount = (PaginationValueSegment) visit(ctx.limitRowCount());
+        }
+        return new LimitSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), offset, rowCount);
+    }
+    
+    @Override
+    public ASTNode visitLimitRowCount(final MySQLStatementParser.LimitRowCountContext ctx) {
+        if (null != ctx.numberLiterals()) {
+            return new NumberLiteralLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((NumberLiteralValue) visit(ctx.numberLiterals())).getValue().longValue());
+        }
+        return new ParameterMarkerLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((ParameterMarkerValue) visit(ctx.parameterMarker())).getValue());
+    }
+    
+    @Override
+    public ASTNode visitLimitOffset(final MySQLStatementParser.LimitOffsetContext ctx) {
+        if (null != ctx.numberLiterals()) {
+            return new NumberLiteralLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((NumberLiteralValue) visit(ctx.numberLiterals())).getValue().longValue());
+        }
+        return new ParameterMarkerLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((ParameterMarkerValue) visit(ctx.parameterMarker())).getValue());
+    }
+    
+    @Override
+    public ASTNode visitLockClause(final MySQLStatementParser.LockClauseContext ctx) {
+        return new LockSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex());
+    }
 }
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 400227f..702c9b6 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,9 +32,12 @@ 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.BeginStatementContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CaseStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ChangeColumnSpecificationContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CheckConstraintDefinitionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ColumnDefinitionContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CompoundStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ConstraintDefinitionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CreateDatabaseContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CreateDefinitionClauseContext;
@@ -62,17 +65,25 @@ import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.DropTab
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.DropTriggerContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.DropViewContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.FirstOrAfterColumnContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.FlowControlStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ForeignKeyOptionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.GeneratedOptionContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.IfStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.IndexDefinition_Context;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.KeyPart_Context;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.KeyParts_Context;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LoopStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ModifyColumnSpecificationContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ReferenceDefinitionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.RenameColumnSpecificationContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.RenameTableSpecificationContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.RepeatStatementContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.RoutineBodyContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.SimpleStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.StorageOptionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TruncateTableContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ValidStatementContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.WhileStatementContext;
 import org.apache.shardingsphere.sql.parser.mysql.visitor.MySQLVisitor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.AlterDefinitionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.CreateDefinitionSegment;
@@ -87,9 +98,12 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.column.positi
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.constraint.ConstraintDefinitionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.constraint.DropPrimaryKeySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
+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.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.DataTypeSegment;
 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.value.collection.CollectionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLAlterDatabaseStatement;
@@ -122,6 +136,10 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLDropViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLRenameTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLTruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLDeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLUpdateStatement;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -461,92 +479,245 @@ public final class MySQLDDLVisitor extends MySQLVisitor implements DDLVisitor {
         }
         return result;
     }
-
+    
     @Override
     public ASTNode visitCreateProcedure(final CreateProcedureContext ctx) {
-        return new MySQLCreateProcedureStatement();
+        MySQLCreateProcedureStatement result = new MySQLCreateProcedureStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
     @Override
     public ASTNode visitAlterProcedure(final AlterProcedureContext ctx) {
         return new MySQLAlterProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitDropProcedure(final DropProcedureContext ctx) {
         return new MySQLDropProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateFunction(final CreateFunctionContext ctx) {
-        return new MySQLCreateFunctionStatement();
+        MySQLCreateFunctionStatement result = new MySQLCreateFunctionStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitRoutineBody(final RoutineBodyContext ctx) {
+        RoutineBodySegment result = new RoutineBodySegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        CollectionValue<ValidStatementSegment> validStatements;
+        if (null != ctx.simpleStatement()) {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.simpleStatement());
+        } else {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.compoundStatement());
+        }
+        result.getValidStatements().addAll(validStatements.getValue());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSimpleStatement(final SimpleStatementContext ctx) {
+        return visit(ctx.validStatement());
+    }
+    
+    @Override
+    public ASTNode visitCompoundStatement(final CompoundStatementContext ctx) {
+        return visit(ctx.beginStatement());
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitBeginStatement(final BeginStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitValidStatement(final ValidStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = createValidStatementSegment(ctx);
+        if (null != validStatement.getSqlStatement()) {
+            result.getValue().add(validStatement);
+        }
+        if (null != ctx.beginStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.beginStatement()));
+        }
+        if (null != ctx.flowControlStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.flowControlStatement()));
+        }
+        return result;
+    }
+    
+    private ValidStatementSegment createValidStatementSegment(final ValidStatementContext ctx) {
+        ValidStatementSegment result = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        SQLStatement sqlStatement = null;
+        if (null != ctx.createTable()) {
+            sqlStatement = (MySQLCreateTableStatement) visit(ctx.createTable());
+        } else if (null != ctx.alterTable()) {
+            sqlStatement = (MySQLAlterTableStatement) visit(ctx.alterTable());
+        } else if (null != ctx.dropTable()) {
+            sqlStatement = (MySQLDropTableStatement) visit(ctx.dropTable());
+        } else if (null != ctx.truncateTable()) {
+            sqlStatement = (MySQLTruncateStatement) visit(ctx.truncateTable());
+        } else if (null != ctx.insert()) {
+            sqlStatement = (MySQLInsertStatement) visit(ctx.insert());
+        } else if (null != ctx.replace()) {
+            sqlStatement = (MySQLInsertStatement) visit(ctx.replace());
+        } else if (null != ctx.update()) {
+            sqlStatement = (MySQLUpdateStatement) visit(ctx.update());
+        } else if (null != ctx.delete()) {
+            sqlStatement = (MySQLDeleteStatement) visit(ctx.delete());
+        } else if (null != ctx.select()) {
+            sqlStatement = (MySQLSelectStatement) visit(ctx.select());
+        }
+        result.setSqlStatement(sqlStatement);
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitFlowControlStatement(final FlowControlStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        if (null != ctx.caseStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.caseStatement()));
+        }
+        if (null != ctx.ifStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.ifStatement()));
+        }
+        if (null != ctx.loopStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.loopStatement()));
+        }
+        if (null != ctx.repeatStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.repeatStatement()));
+        }
+        if (null != ctx.whileStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.whileStatement()));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitCaseStatement(final CaseStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitIfStatement(final IfStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitLoopStatement(final LoopStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitRepeatStatement(final RepeatStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitWhileStatement(final WhileStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
     @Override
     public ASTNode visitAlterFunction(final AlterFunctionContext ctx) {
         return new MySQLAlterFunctionStatement();
     }
-
+    
     @Override
     public ASTNode visitDropFunction(final DropFunctionContext ctx) {
         return new MySQLDropFunctionStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateEvent(final CreateEventContext ctx) {
         return new MySQLCreateEventStatement();
     }
-
+    
     @Override
     public ASTNode visitAlterEvent(final AlterEventContext ctx) {
         return new MySQLAlterEventStatement();
     }
-
+    
     @Override
     public ASTNode visitDropEvent(final DropEventContext ctx) {
         return new MySQLDropEventStatement();
     }
-
+    
     @Override
     public ASTNode visitAlterInstance(final AlterInstanceContext ctx) {
         return new MySQLAlterInstanceStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateLogfileGroup(final CreateLogfileGroupContext ctx) {
         return new MySQLCreateLogfileGroupStatement();
     }
-
+    
     @Override
     public ASTNode visitAlterLogfileGroup(final AlterLogfileGroupContext ctx) {
         return new MySQLAlterLogfileGroupStatement();
     }
-
+    
     @Override
     public ASTNode visitDropLogfileGroup(final DropLogfileGroupContext ctx) {
         return new MySQLDropLogfileGroupStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateServer(final CreateServerContext ctx) {
         return new MySQLCreateServerStatement();
     }
-
+    
     @Override
     public ASTNode visitAlterServer(final AlterServerContext ctx) {
         return new MySQLAlterServerStatement();
     }
-
+    
     @Override
     public ASTNode visitDropServer(final DropServerContext ctx) {
         return new MySQLDropServerStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateTrigger(final CreateTriggerContext ctx) {
         return new MySQLCreateTriggerStatement();
     }
-
+    
     @Override
     public ASTNode visitDropTrigger(final DropTriggerContext ctx) {
         return new MySQLDropTriggerStatement();
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/MySQLDMLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDMLVisitor.java
index 797fd4a..50d0259 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDMLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDMLVisitor.java
@@ -19,106 +19,11 @@ package org.apache.shardingsphere.sql.parser.mysql.visitor.impl;
 
 import org.apache.shardingsphere.sql.parser.api.ASTNode;
 import org.apache.shardingsphere.sql.parser.api.visitor.statement.DMLVisitor;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AliasContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AssignmentContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AssignmentValueContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AssignmentValuesContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.BlobValueContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CallContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ColumnNamesContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.DeleteContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.DoStatementContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.DuplicateSpecificationContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.EscapedTableReferenceContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ExprContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.FromClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.GroupByClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.InsertContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.InsertSelectClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.InsertValuesClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.JoinSpecificationContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.JoinedTableContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LimitClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LimitOffsetContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LimitRowCountContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LockClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.MultipleTableNamesContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.MultipleTablesClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.OnDuplicateKeyClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.OrderByItemContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ProjectionContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ProjectionsContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.QualifiedShorthandContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ReplaceContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ReplaceSelectClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ReplaceValuesClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.SelectClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.SelectContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.SelectSpecificationContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.SetAssignmentsClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.SingleTableClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.SubqueryContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableFactorContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableNameContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableReferenceContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableReferencesContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.UnionClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.UpdateContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.WhereClauseContext;
 import org.apache.shardingsphere.sql.parser.mysql.visitor.MySQLVisitor;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.InsertValuesSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.SetAssignmentSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.InsertColumnsSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.OnDuplicateKeyColumnsSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.subquery.SubqueryExpressionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.subquery.SubquerySegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.AggregationProjectionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ColumnProjectionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ExpressionProjectionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ShorthandProjectionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.SubqueryProjectionSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.GroupBySegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.OrderBySegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.LimitSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.ParameterMarkerLimitValueSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.LockSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.AliasSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.OwnerSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.DeleteMultiTableSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.JoinTableSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SubqueryTableSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableSegment;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLUpdateStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.value.collection.CollectionValue;
-import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
-import org.apache.shardingsphere.sql.parser.sql.common.value.literal.impl.BooleanLiteralValue;
-import org.apache.shardingsphere.sql.parser.sql.common.value.literal.impl.NumberLiteralValue;
-import org.apache.shardingsphere.sql.parser.sql.common.value.literal.impl.StringLiteralValue;
-import org.apache.shardingsphere.sql.parser.sql.common.value.parametermarker.ParameterMarkerValue;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLCallStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLDeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLDoStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
 
 /**
  * DML visitor for MySQL.
@@ -134,535 +39,4 @@ public final class MySQLDMLVisitor extends MySQLVisitor implements DMLVisitor {
     public ASTNode visitDoStatement(final DoStatementContext ctx) {
         return new MySQLDoStatement();
     }
-    
-    @Override
-    public ASTNode visitInsert(final InsertContext ctx) {
-        // TODO :FIXME, since there is no segment for insertValuesClause, InsertStatement is created by sub rule.
-        MySQLInsertStatement result;
-        if (null != ctx.insertValuesClause()) {
-            result = (MySQLInsertStatement) visit(ctx.insertValuesClause());
-        } else if (null != ctx.insertSelectClause()) {
-            result = (MySQLInsertStatement) visit(ctx.insertSelectClause());
-        } else {
-            result = new MySQLInsertStatement();
-            result.setSetAssignment((SetAssignmentSegment) visit(ctx.setAssignmentsClause()));
-        }
-        if (null != ctx.onDuplicateKeyClause()) {
-            result.setOnDuplicateKeyColumns((OnDuplicateKeyColumnsSegment) visit(ctx.onDuplicateKeyClause()));
-        }
-        result.setTable((SimpleTableSegment) visit(ctx.tableName()));
-        result.setParameterCount(getCurrentParameterIndex());
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitInsertSelectClause(final InsertSelectClauseContext ctx) {
-        MySQLInsertStatement result = new MySQLInsertStatement();
-        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
-        result.setInsertSelect(createInsertSelectSegment(ctx));
-        return result;
-    }
-    
-    private SubquerySegment createInsertSelectSegment(final InsertSelectClauseContext ctx) {
-        MySQLSelectStatement selectStatement = (MySQLSelectStatement) visit(ctx.select());
-        return new SubquerySegment(ctx.select().start.getStartIndex(), ctx.select().stop.getStopIndex(), selectStatement);
-    }
-    
-    @Override
-    public ASTNode visitInsertValuesClause(final InsertValuesClauseContext ctx) {
-        MySQLInsertStatement result = new MySQLInsertStatement();
-        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
-        result.getValues().addAll(createInsertValuesSegments(ctx.assignmentValues()));
-        return result;
-    }
-    
-    private Collection<InsertValuesSegment> createInsertValuesSegments(final Collection<AssignmentValuesContext> assignmentValuesContexts) {
-        Collection<InsertValuesSegment> result = new LinkedList<>();
-        for (AssignmentValuesContext each : assignmentValuesContexts) {
-            result.add((InsertValuesSegment) visit(each));
-        }
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitOnDuplicateKeyClause(final OnDuplicateKeyClauseContext ctx) {
-        Collection<AssignmentSegment> columns = new LinkedList<>();
-        for (AssignmentContext each : ctx.assignment()) {
-            columns.add((AssignmentSegment) visit(each));
-        }
-        return new OnDuplicateKeyColumnsSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), columns);
-    }
-    
-    @Override
-    public ASTNode visitReplace(final ReplaceContext ctx) {
-        // TODO :FIXME, since there is no segment for replaceValuesClause, ReplaceStatement is created by sub rule.
-        MySQLInsertStatement result;
-        if (null != ctx.replaceValuesClause()) {
-            result = (MySQLInsertStatement) visit(ctx.replaceValuesClause());
-        } else if (null != ctx.replaceSelectClause()) {
-            result = (MySQLInsertStatement) visit(ctx.replaceSelectClause());
-        } else {
-            result = new MySQLInsertStatement();
-            result.setSetAssignment((SetAssignmentSegment) visit(ctx.setAssignmentsClause()));
-        }
-        result.setTable((SimpleTableSegment) visit(ctx.tableName()));
-        result.setParameterCount(getCurrentParameterIndex());
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitReplaceSelectClause(final ReplaceSelectClauseContext ctx) {
-        MySQLInsertStatement result = new MySQLInsertStatement();
-        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
-        result.setInsertSelect(createReplaceSelectSegment(ctx));
-        return result;
-    }
-    
-    private SubquerySegment createReplaceSelectSegment(final ReplaceSelectClauseContext ctx) {
-        MySQLSelectStatement selectStatement = (MySQLSelectStatement) visit(ctx.select());
-        return new SubquerySegment(ctx.select().start.getStartIndex(), ctx.select().stop.getStopIndex(), selectStatement);
-    }
-    
-    @Override
-    public ASTNode visitReplaceValuesClause(final ReplaceValuesClauseContext ctx) {
-        MySQLInsertStatement result = new MySQLInsertStatement();
-        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
-        result.getValues().addAll(createReplaceValuesSegments(ctx.assignmentValues()));
-        return result;
-    }
-    
-    @SuppressWarnings("unchecked")
-    private InsertColumnsSegment createInsertColumns(final ColumnNamesContext columnNames, final int startIndex) {
-        if (null != columnNames) {
-            CollectionValue<ColumnSegment> columnSegments = (CollectionValue<ColumnSegment>) visit(columnNames);
-            return new InsertColumnsSegment(columnNames.start.getStartIndex(), columnNames.stop.getStopIndex(), columnSegments.getValue());
-        } else {
-            return new InsertColumnsSegment(startIndex - 1, startIndex - 1, Collections.emptyList());
-        }
-    }
-    
-    private Collection<InsertValuesSegment> createReplaceValuesSegments(final Collection<AssignmentValuesContext> assignmentValuesContexts) {
-        Collection<InsertValuesSegment> result = new LinkedList<>();
-        for (AssignmentValuesContext each : assignmentValuesContexts) {
-            result.add((InsertValuesSegment) visit(each));
-        }
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitUpdate(final UpdateContext ctx) {
-        MySQLUpdateStatement result = new MySQLUpdateStatement();
-        TableSegment tableSegment = (TableSegment) visit(ctx.tableReferences());
-        result.setTableSegment(tableSegment);
-        result.setSetAssignment((SetAssignmentSegment) visit(ctx.setAssignmentsClause()));
-        if (null != ctx.whereClause()) {
-            result.setWhere((WhereSegment) visit(ctx.whereClause()));
-        }
-        if (null != ctx.orderByClause()) {
-            result.setOrderBy((OrderBySegment) visit(ctx.orderByClause()));
-        }
-        if (null != ctx.limitClause()) {
-            result.setLimit((LimitSegment) visit(ctx.limitClause()));
-        }
-        result.setParameterCount(getCurrentParameterIndex());
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitSetAssignmentsClause(final SetAssignmentsClauseContext ctx) {
-        Collection<AssignmentSegment> assignments = new LinkedList<>();
-        for (AssignmentContext each : ctx.assignment()) {
-            assignments.add((AssignmentSegment) visit(each));
-        }
-        return new SetAssignmentSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), assignments);
-    }
-    
-    @Override
-    public ASTNode visitAssignmentValues(final AssignmentValuesContext ctx) {
-        List<ExpressionSegment> segments = new LinkedList<>();
-        for (AssignmentValueContext each : ctx.assignmentValue()) {
-            segments.add((ExpressionSegment) visit(each));
-        }
-        return new InsertValuesSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), segments);
-    }
-    
-    @Override
-    public ASTNode visitAssignment(final AssignmentContext ctx) {
-        ColumnSegment column = (ColumnSegment) visitColumnName(ctx.columnName());
-        ExpressionSegment value = (ExpressionSegment) visit(ctx.assignmentValue());
-        return new AssignmentSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), column, value);
-    }
-    
-    @Override
-    public ASTNode visitAssignmentValue(final AssignmentValueContext ctx) {
-        ExprContext expr = ctx.expr();
-        if (null != expr) {
-            ASTNode result = visit(expr);
-            if (result instanceof ColumnSegment) {
-                return new CommonExpressionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
-            } else {
-                return result;
-            }
-        }
-        return new CommonExpressionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
-    }
-    
-    @Override
-    public ASTNode visitBlobValue(final BlobValueContext ctx) {
-        return new StringLiteralValue(ctx.STRING_().getText());
-    }
-    
-    @Override
-    public ASTNode visitDelete(final DeleteContext ctx) {
-        MySQLDeleteStatement result = new MySQLDeleteStatement();
-        if (null != ctx.multipleTablesClause()) {
-            result.setTableSegment((TableSegment) visit(ctx.multipleTablesClause()));
-        } else {
-            result.setTableSegment((TableSegment) visit(ctx.singleTableClause()));
-        }
-        if (null != ctx.whereClause()) {
-            result.setWhere((WhereSegment) visit(ctx.whereClause()));
-        }
-        if (null != ctx.orderByClause()) {
-            result.setOrderBy((OrderBySegment) visit(ctx.orderByClause()));
-        }
-        if (null != ctx.limitClause()) {
-            result.setLimit((LimitSegment) visit(ctx.limitClause()));
-        }
-        result.setParameterCount(getCurrentParameterIndex());
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitSingleTableClause(final SingleTableClauseContext ctx) {
-        SimpleTableSegment result = (SimpleTableSegment) visit(ctx.tableName());
-        if (null != ctx.alias()) {
-            result.setAlias((AliasSegment) visit(ctx.alias()));
-        }
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitMultipleTablesClause(final MultipleTablesClauseContext ctx) {
-        DeleteMultiTableSegment result = new DeleteMultiTableSegment();
-        TableSegment relateTableSource = (TableSegment) visit(ctx.tableReferences());
-        result.setRelationTable(relateTableSource);
-        result.setActualDeleteTables(generateTablesFromTableMultipleTableNames(ctx.multipleTableNames()));
-        return result;
-    }
-    
-    private List<SimpleTableSegment> generateTablesFromTableMultipleTableNames(final MultipleTableNamesContext ctx) {
-        List<SimpleTableSegment> result = new LinkedList<>();
-        for (TableNameContext each : ctx.tableName()) {
-            result.add((SimpleTableSegment) visit(each));
-        }
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitSelect(final SelectContext ctx) {
-        // TODO :Unsupported for withClause.
-        MySQLSelectStatement result = (MySQLSelectStatement) visit(ctx.unionClause());
-        result.setParameterCount(getCurrentParameterIndex());
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitUnionClause(final UnionClauseContext ctx) {
-        // TODO :Unsupported for union SQL.
-        return visit(ctx.selectClause(0));
-    }
-    
-    @Override
-    public ASTNode visitSelectClause(final SelectClauseContext ctx) {
-        MySQLSelectStatement result = new MySQLSelectStatement();
-        result.setProjections((ProjectionsSegment) visit(ctx.projections()));
-        if (null != ctx.selectSpecification()) {
-            result.getProjections().setDistinctRow(isDistinct(ctx));
-        }
-        if (null != ctx.fromClause()) {
-            TableSegment tableSource = (TableSegment) visit(ctx.fromClause().tableReferences());
-            result.setFrom(tableSource);
-        }
-        if (null != ctx.whereClause()) {
-            result.setWhere((WhereSegment) visit(ctx.whereClause()));
-        }
-        if (null != ctx.groupByClause()) {
-            result.setGroupBy((GroupBySegment) visit(ctx.groupByClause()));
-        }
-        if (null != ctx.orderByClause()) {
-            result.setOrderBy((OrderBySegment) visit(ctx.orderByClause()));
-        }
-        if (null != ctx.limitClause()) {
-            result.setLimit((LimitSegment) visit(ctx.limitClause()));
-        }
-        if (null != ctx.lockClause()) {
-            result.setLock((LockSegment) visit(ctx.lockClause()));
-        }
-        return result;
-    }
-
-    private boolean isDistinct(final SelectClauseContext ctx) {
-        for (SelectSpecificationContext each : ctx.selectSpecification()) {
-            if (((BooleanLiteralValue) visit(each)).getValue()) {
-                return true;
-            }
-        }
-        return false;
-    }
-    
-    @Override
-    public ASTNode visitSelectSpecification(final SelectSpecificationContext ctx) {
-        if (null != ctx.duplicateSpecification()) {
-            return visit(ctx.duplicateSpecification());
-        }
-        return new BooleanLiteralValue(false);
-    }
-    
-    @Override
-    public ASTNode visitDuplicateSpecification(final DuplicateSpecificationContext ctx) {
-        String text = ctx.getText();
-        if ("DISTINCT".equalsIgnoreCase(text) || "DISTINCTROW".equalsIgnoreCase(text)) {
-            return new BooleanLiteralValue(true);
-        }
-        return new BooleanLiteralValue(false);
-    }
-    
-    @Override
-    public ASTNode visitProjections(final ProjectionsContext ctx) {
-        Collection<ProjectionSegment> projections = new LinkedList<>();
-        if (null != ctx.unqualifiedShorthand()) {
-            projections.add(new ShorthandProjectionSegment(ctx.unqualifiedShorthand().getStart().getStartIndex(), ctx.unqualifiedShorthand().getStop().getStopIndex()));
-        }
-        for (ProjectionContext each : ctx.projection()) {
-            projections.add((ProjectionSegment) visit(each));
-        }
-        ProjectionsSegment result = new ProjectionsSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex());
-        result.getProjections().addAll(projections);
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitProjection(final ProjectionContext ctx) {
-        // FIXME :The stop index of project is the stop index of projection, instead of alias.
-        if (null != ctx.qualifiedShorthand()) {
-            QualifiedShorthandContext shorthand = ctx.qualifiedShorthand();
-            ShorthandProjectionSegment result = new ShorthandProjectionSegment(shorthand.getStart().getStartIndex(), shorthand.getStop().getStopIndex());
-            IdentifierValue identifier = new IdentifierValue(shorthand.identifier().getText());
-            result.setOwner(new OwnerSegment(shorthand.identifier().getStart().getStartIndex(), shorthand.identifier().getStop().getStopIndex(), identifier));
-            return result;
-        }
-        AliasSegment alias = null == ctx.alias() ? null : (AliasSegment) visit(ctx.alias());
-        ASTNode exprProjection = visit(ctx.expr());
-        if (exprProjection instanceof ColumnSegment) {
-            ColumnProjectionSegment result = new ColumnProjectionSegment((ColumnSegment) exprProjection);
-            result.setAlias(alias);
-            return result;
-        }
-        if (exprProjection instanceof SubquerySegment) {
-            SubqueryProjectionSegment result = new SubqueryProjectionSegment((SubquerySegment) exprProjection);
-            result.setAlias(alias);
-            return result;
-        }
-        return createProjection(ctx, alias, exprProjection);
-    }
-    
-    @Override
-    public ASTNode visitAlias(final AliasContext ctx) {
-        if (null != ctx.identifier()) {
-            return new AliasSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), (IdentifierValue) visit(ctx.identifier()));
-        }
-        return new AliasSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(ctx.STRING_().getText()));
-    }
-    
-    private ASTNode createProjection(final ProjectionContext ctx, final AliasSegment alias, final ASTNode projection) {
-        if (projection instanceof AggregationProjectionSegment) {
-            ((AggregationProjectionSegment) projection).setAlias(alias);
-            return projection;
-        }
-        if (projection instanceof ExpressionProjectionSegment) {
-            ((ExpressionProjectionSegment) projection).setAlias(alias);
-            return projection;
-        }
-        if (projection instanceof CommonExpressionSegment) {
-            CommonExpressionSegment segment = (CommonExpressionSegment) projection;
-            ExpressionProjectionSegment result = new ExpressionProjectionSegment(segment.getStartIndex(), segment.getStopIndex(), segment.getText());
-            result.setAlias(alias);
-            return result;
-        }
-        // FIXME :For DISTINCT()
-        if (projection instanceof ColumnSegment) {
-            ExpressionProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), ctx.getText());
-            result.setAlias(alias);
-            return result;
-        }
-        if (projection instanceof SubqueryExpressionSegment) {
-            SubqueryProjectionSegment result = new SubqueryProjectionSegment(((SubqueryExpressionSegment) projection).getSubquery());
-            result.setAlias(alias);
-            return result;
-        }
-        if (projection instanceof BinaryOperationExpression) {
-            int startIndex = ((BinaryOperationExpression) projection).getStartIndex();
-            int stopIndex = null != alias ? alias.getStopIndex() : ((BinaryOperationExpression) projection).getStopIndex();
-            ExpressionProjectionSegment result = new ExpressionProjectionSegment(startIndex, stopIndex, ((BinaryOperationExpression) projection).getText());
-            result.setAlias(alias);
-            return result;
-        }
-        LiteralExpressionSegment column = (LiteralExpressionSegment) projection;
-        ExpressionProjectionSegment result = null == alias ? new ExpressionProjectionSegment(column.getStartIndex(), column.getStopIndex(), String.valueOf(column.getLiterals()))
-                : new ExpressionProjectionSegment(column.getStartIndex(), ctx.alias().stop.getStopIndex(), String.valueOf(column.getLiterals()));
-        result.setAlias(alias);
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitFromClause(final FromClauseContext ctx) {
-        return visit(ctx.tableReferences());
-    }
-    
-    @Override
-    public ASTNode visitTableReferences(final TableReferencesContext ctx) {
-        TableSegment result = (TableSegment) visit(ctx.escapedTableReference(0));
-        if (ctx.escapedTableReference().size() > 1) {
-            for (int i = 1; i < ctx.escapedTableReference().size(); i++) {
-                result = generateJoinTableSourceFromEscapedTableReference(ctx.escapedTableReference(i), result);
-            }
-        }
-        return result;
-    }
-    
-    private JoinTableSegment generateJoinTableSourceFromEscapedTableReference(final EscapedTableReferenceContext ctx, final TableSegment tableSegment) {
-        JoinTableSegment result = new JoinTableSegment();
-        result.setStartIndex(tableSegment.getStartIndex());
-        result.setStopIndex(ctx.stop.getStopIndex());
-        result.setLeft(tableSegment);
-        result.setRight((TableSegment) visit(ctx));
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitEscapedTableReference(final EscapedTableReferenceContext ctx) {
-        return visit(ctx.tableReference());
-    }
-    
-    @Override
-    public ASTNode visitTableReference(final TableReferenceContext ctx) {
-        TableSegment result;
-        TableSegment left;
-        left = (TableSegment) visit(ctx.tableFactor());
-        if (!ctx.joinedTable().isEmpty()) {
-            for (JoinedTableContext each : ctx.joinedTable()) {
-                left = visitJoinedTable(each, left);
-            }
-        }
-        result = left;
-        return result;
-    }
-    
-    @Override
-    public ASTNode visitTableFactor(final TableFactorContext ctx) {
-        if (null != ctx.subquery()) {
-            MySQLSelectStatement subquery = (MySQLSelectStatement) visit(ctx.subquery());
-            SubquerySegment subquerySegment = new SubquerySegment(ctx.subquery().start.getStartIndex(), ctx.subquery().stop.getStopIndex(), subquery);
-            SubqueryTableSegment result = new SubqueryTableSegment(subquerySegment);
-            if (null != ctx.alias()) {
-                result.setAlias((AliasSegment) visit(ctx.alias()));
-            }
-            return result;
-        }
-        if (null != ctx.tableName()) {
-            SimpleTableSegment result = (SimpleTableSegment) visit(ctx.tableName());
-            if (null != ctx.alias()) {
-                result.setAlias((AliasSegment) visit(ctx.alias()));
-            }
-            return result;
-        }
-        return visit(ctx.tableReferences());
-    }
-    
-    private JoinTableSegment visitJoinedTable(final JoinedTableContext ctx, final TableSegment tableSegment) {
-        JoinTableSegment result = new JoinTableSegment();
-        result.setLeft(tableSegment);
-        result.setStartIndex(tableSegment.getStartIndex());
-        result.setStopIndex(ctx.stop.getStopIndex());
-        TableSegment right = (TableSegment) visit(ctx.tableFactor());
-        result.setRight(right);
-        if (null != ctx.joinSpecification()) {
-            result = visitJoinSpecification(ctx.joinSpecification(), result);
-        }
-        return result;
-    }
-    
-    private JoinTableSegment visitJoinSpecification(final JoinSpecificationContext ctx, final JoinTableSegment joinTableSource) {
-        if (null != ctx.expr()) {
-            ExpressionSegment condition = (ExpressionSegment) visit(ctx.expr());
-            joinTableSource.setCondition(condition);
-        }
-        if (null != ctx.USING()) {
-            List<ColumnSegment> columnSegmentList = new LinkedList<>();
-            for (MySQLStatementParser.ColumnNameContext cname : ctx.columnNames().columnName()) {
-                columnSegmentList.add((ColumnSegment) visit(cname));
-            }
-            joinTableSource.setUsing(columnSegmentList);
-        }
-        return joinTableSource;
-    }
-    
-    @Override
-    public ASTNode visitWhereClause(final WhereClauseContext ctx) {
-        ASTNode segment = visit(ctx.expr());
-        return new WhereSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), (ExpressionSegment) segment);
-    }
-    
-    @Override
-    public ASTNode visitGroupByClause(final GroupByClauseContext ctx) {
-        Collection<OrderByItemSegment> items = new LinkedList<>();
-        for (OrderByItemContext each : ctx.orderByItem()) {
-            items.add((OrderByItemSegment) visit(each));
-        }
-        return new GroupBySegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), items);
-    }
-    
-    @Override
-    public ASTNode visitLimitClause(final LimitClauseContext ctx) {
-        if (null == ctx.limitOffset()) {
-            return new LimitSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), null, (PaginationValueSegment) visit(ctx.limitRowCount()));
-        }
-        PaginationValueSegment rowCount;
-        PaginationValueSegment offset;
-        if (null != ctx.OFFSET()) {
-            rowCount = (PaginationValueSegment) visit(ctx.limitRowCount());
-            offset = (PaginationValueSegment) visit(ctx.limitOffset());
-        } else {
-            offset = (PaginationValueSegment) visit(ctx.limitOffset());
-            rowCount = (PaginationValueSegment) visit(ctx.limitRowCount());
-        }
-        return new LimitSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), offset, rowCount);
-    }
-    
-    @Override
-    public ASTNode visitLimitRowCount(final LimitRowCountContext ctx) {
-        if (null != ctx.numberLiterals()) {
-            return new NumberLiteralLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((NumberLiteralValue) visit(ctx.numberLiterals())).getValue().longValue());
-        }
-        return new ParameterMarkerLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((ParameterMarkerValue) visit(ctx.parameterMarker())).getValue());
-    }
-    
-    @Override
-    public ASTNode visitLimitOffset(final LimitOffsetContext ctx) {
-        if (null != ctx.numberLiterals()) {
-            return new NumberLiteralLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((NumberLiteralValue) visit(ctx.numberLiterals())).getValue().longValue());
-        }
-        return new ParameterMarkerLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((ParameterMarkerValue) visit(ctx.parameterMarker())).getValue());
-    }
-    
-    @Override
-    public ASTNode visitSubquery(final SubqueryContext ctx) {
-        return visit(ctx.unionClause());
-    }
-    
-    @Override
-    public ASTNode visitLockClause(final LockClauseContext ctx) {
-        return new LockSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex());
-    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
index b182a09..066b839 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
@@ -18,6 +18,8 @@
 package org.apache.shardingsphere.sql.parser.sql.common.extractor;
 
 import lombok.Getter;
+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.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
@@ -38,10 +40,12 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.Joi
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SubqueryTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateTableStatementHandler;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -204,6 +208,7 @@ public final class TableExtractor {
     
     /**
      * Check if the table needs to be overwritten.
+     *
      * @param owner OwnerSegment.
      * @return boolean.
      */
@@ -215,4 +220,59 @@ public final class TableExtractor {
         }
         return true;
     }
+    
+    /**
+     * Extract the tables that should exist from RoutineBodySegment.
+     *
+     * @param routineBody RoutineBodySegment
+     * @return the tables that should exist
+     */
+    public Collection<SimpleTableSegment> extractExistTableFromRoutineBody(final RoutineBodySegment routineBody) {
+        Collection<SimpleTableSegment> result = new LinkedList<>();
+        for (ValidStatementSegment each : routineBody.getValidStatements()) {
+            if (each.getAlterTable().isPresent()) {
+                result.add(each.getAlterTable().get().getTable());
+            }
+            if (each.getDropTable().isPresent()) {
+                result.addAll(each.getDropTable().get().getTables());
+            }
+            if (each.getTruncate().isPresent()) {
+                result.addAll(each.getTruncate().get().getTables());
+            }
+            result.addAll(extractExistTableFromDMLStatement(each));
+        }
+        return result;
+    }
+    
+    private Collection<SimpleTableSegment> extractExistTableFromDMLStatement(final ValidStatementSegment validStatementSegment) {
+        if (validStatementSegment.getInsert().isPresent()) {
+            extractTablesFromInsert(validStatementSegment.getInsert().get());
+        } else if (validStatementSegment.getReplace().isPresent()) {
+            extractTablesFromInsert(validStatementSegment.getReplace().get());
+        } else if (validStatementSegment.getUpdate().isPresent()) {
+            extractTablesFromUpdate(validStatementSegment.getUpdate().get());
+        } else if (validStatementSegment.getDelete().isPresent()) {
+            extractTablesFromDelete(validStatementSegment.getDelete().get());
+        } else if (validStatementSegment.getSelect().isPresent()) {
+            extractTablesFromSelect(validStatementSegment.getSelect().get());
+        }
+        return rewriteTables;
+    }
+    
+    /**
+     * Extract the tables that should not exist from RoutineBodySegment.
+     *
+     * @param routineBody RoutineBodySegment
+     * @return the tables that should not exist
+     */
+    public Collection<SimpleTableSegment> extractNotExistTableFromRoutineBody(final RoutineBodySegment routineBody) {
+        Collection<SimpleTableSegment> result = new LinkedList<>();
+        for (ValidStatementSegment each : routineBody.getValidStatements()) {
+            Optional<CreateTableStatement> createTable = each.getCreateTable();
+            if (createTable.isPresent() && !CreateTableStatementHandler.containsIfNotExistClause(createTable.get())) {
+                result.add(createTable.get().getTable());
+            }
+        }
+        return result;
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/RoutineBodySegment.java
similarity index 59%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/RoutineBodySegment.java
index 3cb41a9..dbde6a2 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/RoutineBodySegment.java
@@ -15,13 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl;
+package org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
 
-import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+
+import java.util.Collection;
+import java.util.LinkedList;
 
 /**
- * MySQL create function statement.
+ * Routine body segment.
  */
-public final class MySQLCreateFunctionStatement extends CreateFunctionStatement implements MySQLStatement {
+@RequiredArgsConstructor
+@Getter
+public class RoutineBodySegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private final Collection<ValidStatementSegment> validStatements = new LinkedList<>();
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
new file mode 100644
index 0000000..792501b
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
@@ -0,0 +1,130 @@
+/*
+ * 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.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private SQLStatement sqlStatement;
+    
+    /**
+     * Get create table statement.
+     *
+     * @return create table statement
+     */
+    public Optional<CreateTableStatement> getCreateTable() {
+        return sqlStatement instanceof CreateTableStatement ? Optional.of((CreateTableStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get alter table statement.
+     *
+     * @return alter table statement
+     */
+    public Optional<AlterTableStatement> getAlterTable() {
+        return sqlStatement instanceof AlterTableStatement ? Optional.of((AlterTableStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get drop table statement.
+     *
+     * @return drop table statement
+     */
+    public Optional<DropTableStatement> getDropTable() {
+        return sqlStatement instanceof DropTableStatement ? Optional.of((DropTableStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get truncate statement.
+     *
+     * @return truncate statement
+     */
+    public Optional<TruncateStatement> getTruncate() {
+        return sqlStatement instanceof TruncateStatement ? Optional.of((TruncateStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get insert statement.
+     *
+     * @return insert statement
+     */
+    public Optional<InsertStatement> getInsert() {
+        return sqlStatement instanceof InsertStatement ? Optional.of((InsertStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get replace statement.
+     *
+     * @return replace statement
+     */
+    public Optional<InsertStatement> getReplace() {
+        return sqlStatement instanceof InsertStatement ? Optional.of((InsertStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get update statement.
+     *
+     * @return update statement
+     */
+    public Optional<UpdateStatement> getUpdate() {
+        return sqlStatement instanceof UpdateStatement ? Optional.of((UpdateStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get delete statement.
+     *
+     * @return delete statement
+     */
+    public Optional<DeleteStatement> getDelete() {
+        return sqlStatement instanceof DeleteStatement ? Optional.of((DeleteStatement) sqlStatement) : Optional.empty();
+    }
+    
+    /**
+     * Get select statement.
+     *
+     * @return select statement
+     */
+    public Optional<SelectStatement> getSelect() {
+        return sqlStatement instanceof SelectStatement ? Optional.of((SelectStatement) sqlStatement) : Optional.empty();
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateFunctionStatementHandler.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateFunctionStatementHandler.java
new file mode 100644
index 0000000..afdd736
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateFunctionStatementHandler.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.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+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.MySQLCreateFunctionStatement;
+
+import java.util.Optional;
+
+/**
+ * CreateFunctionStatement handler for different dialect SQLStatements.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateFunctionStatementHandler implements SQLStatementHandler {
+    
+    /**
+     * Get RoutineBodySegment.
+     *
+     * @param createFunctionStatement CreateFunctionStatement
+     * @return RoutineBodySegment
+     */
+    public static Optional<RoutineBodySegment> getRoutineBodySegment(final CreateFunctionStatement createFunctionStatement) {
+        if (createFunctionStatement instanceof MySQLStatement) {
+            return ((MySQLCreateFunctionStatement) createFunctionStatement).getRoutineBody();
+        }
+        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/CreateProcedureStatementHandler.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateProcedureStatementHandler.java
new file mode 100644
index 0000000..37f1b13
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateProcedureStatementHandler.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.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
+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.MySQLCreateProcedureStatement;
+
+import java.util.Optional;
+
+/**
+ * CreateProcedureStatement handler for different dialect SQLStatements.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateProcedureStatementHandler implements SQLStatementHandler {
+    
+    /**
+     * Get RoutineBodySegment.
+     *
+     * @param createProcedureStatement CreateProcedureStatement
+     * @return RoutineBodySegment
+     */
+    public static Optional<RoutineBodySegment> getRoutineBodySegment(final CreateProcedureStatement createProcedureStatement) {
+        if (createProcedureStatement instanceof MySQLStatement) {
+            return ((MySQLCreateProcedureStatement) createProcedureStatement).getRoutineBody();
+        }
+        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/MySQLCreateFunctionStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java
index 3cb41a9..23f3ede 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateFunctionStatement.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.segment.ddl.routine.RoutineBodySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
 
+import java.util.Optional;
+
 /**
  * MySQL create function statement.
  */
+@Getter
+@Setter
 public final class MySQLCreateFunctionStatement extends CreateFunctionStatement implements MySQLStatement {
+    
+    private RoutineBodySegment routineBody;
+    
+    /**
+     * Get routine body segment.
+     *
+     * @return routine body segment
+     */
+    public Optional<RoutineBodySegment> getRoutineBody() {
+        return Optional.ofNullable(routineBody);
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateProcedureStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateProcedureStatement.java
index ac4c529..29c9168 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateProcedureStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/ddl/MySQLCreateProcedureStatement.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.segment.ddl.routine.RoutineBodySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
 
+import java.util.Optional;
+
 /**
  * MySQL create procedure statement.
  */
+@Getter
+@Setter
 public final class MySQLCreateProcedureStatement extends CreateProcedureStatement implements MySQLStatement {
+    
+    private RoutineBodySegment routineBody;
+    
+    /**
+     * Get routine body segment.
+     *
+     * @return routine body segment
+     */
+    public Optional<RoutineBodySegment> getRoutineBody() {
+        return Optional.ofNullable(routineBody);
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java
index fc5f4f8..0074fc2 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java
@@ -27,7 +27,7 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class AlterIndexStatementHandlerTest {
+public final class AlterIndexStatementHandlerTest {
     
     @Test
     public void assertGetSimpleTableSegmentWithSimpleTableSegmentForSQLServer() {
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateFunctionStatementHandlerTest.java
similarity index 50%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateFunctionStatementHandlerTest.java
index fc5f4f8..33950c1 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateFunctionStatementHandlerTest.java
@@ -17,9 +17,8 @@
 
 package org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl;
 
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sqlserver.ddl.SQLServerAlterIndexStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateFunctionStatement;
 import org.junit.Test;
 
 import java.util.Optional;
@@ -27,20 +26,21 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class AlterIndexStatementHandlerTest {
+public final class CreateFunctionStatementHandlerTest {
     
     @Test
-    public void assertGetSimpleTableSegmentWithSimpleTableSegmentForSQLServer() {
-        SQLServerAlterIndexStatement alterIndexStatement = new SQLServerAlterIndexStatement();
-        alterIndexStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("")));
-        Optional<SimpleTableSegment> simpleTableSegment = AlterIndexStatementHandler.getSimpleTableSegment(alterIndexStatement);
-        assertTrue(simpleTableSegment.isPresent());
+    public void assertGetRoutineBodySegmentWithRoutineBodySegmentForMySQL() {
+        MySQLCreateFunctionStatement createFunctionStatement = new MySQLCreateFunctionStatement();
+        RoutineBodySegment routineBody = new RoutineBodySegment(0, 0);
+        createFunctionStatement.setRoutineBody(routineBody);
+        Optional<RoutineBodySegment> routineBodySegment = CreateFunctionStatementHandler.getRoutineBodySegment(createFunctionStatement);
+        assertTrue(routineBodySegment.isPresent());
     }
     
     @Test
-    public void assertGetSimpleTableSegmentWithoutSimpleTableSegmentForSQLServer() {
-        SQLServerAlterIndexStatement alterIndexStatement = new SQLServerAlterIndexStatement();
-        Optional<SimpleTableSegment> simpleTableSegment = AlterIndexStatementHandler.getSimpleTableSegment(alterIndexStatement);
-        assertFalse(simpleTableSegment.isPresent());
+    public void assertGetRoutineBodySegmentWithoutRoutineBodySegmentForMySQL() {
+        MySQLCreateFunctionStatement createFunctionStatement = new MySQLCreateFunctionStatement();
+        Optional<RoutineBodySegment> routineBodySegment = CreateFunctionStatementHandler.getRoutineBodySegment(createFunctionStatement);
+        assertFalse(routineBodySegment.isPresent());
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateProcedureStatementHandlerTest.java
similarity index 50%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateProcedureStatementHandlerTest.java
index fc5f4f8..1fbc55c 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/AlterIndexStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/CreateProcedureStatementHandlerTest.java
@@ -17,9 +17,8 @@
 
 package org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl;
 
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sqlserver.ddl.SQLServerAlterIndexStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateProcedureStatement;
 import org.junit.Test;
 
 import java.util.Optional;
@@ -27,20 +26,21 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class AlterIndexStatementHandlerTest {
+public final class CreateProcedureStatementHandlerTest {
     
     @Test
-    public void assertGetSimpleTableSegmentWithSimpleTableSegmentForSQLServer() {
-        SQLServerAlterIndexStatement alterIndexStatement = new SQLServerAlterIndexStatement();
-        alterIndexStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("")));
-        Optional<SimpleTableSegment> simpleTableSegment = AlterIndexStatementHandler.getSimpleTableSegment(alterIndexStatement);
-        assertTrue(simpleTableSegment.isPresent());
+    public void assertGetRoutineBodySegmentWithRoutineBodySegmentForMySQL() {
+        MySQLCreateProcedureStatement createProcedureStatement = new MySQLCreateProcedureStatement();
+        RoutineBodySegment routineBody = new RoutineBodySegment(0, 0);
+        createProcedureStatement.setRoutineBody(routineBody);
+        Optional<RoutineBodySegment> routineBodySegment = CreateProcedureStatementHandler.getRoutineBodySegment(createProcedureStatement);
+        assertTrue(routineBodySegment.isPresent());
     }
     
     @Test
-    public void assertGetSimpleTableSegmentWithoutSimpleTableSegmentForSQLServer() {
-        SQLServerAlterIndexStatement alterIndexStatement = new SQLServerAlterIndexStatement();
-        Optional<SimpleTableSegment> simpleTableSegment = AlterIndexStatementHandler.getSimpleTableSegment(alterIndexStatement);
-        assertFalse(simpleTableSegment.isPresent());
+    public void assertGetRoutineBodySegmentWithoutRoutineBodySegmentForMySQL() {
+        MySQLCreateProcedureStatement createProcedureStatement = new MySQLCreateProcedureStatement();
+        Optional<RoutineBodySegment> routineBodySegment = CreateProcedureStatementHandler.getRoutineBodySegment(createProcedureStatement);
+        assertFalse(routineBodySegment.isPresent());
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/DropIndexStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/DropIndexStatementHandlerTest.java
index fd261fa..27ca3bd 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/DropIndexStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/DropIndexStatementHandlerTest.java
@@ -28,7 +28,7 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class DropIndexStatementHandlerTest {
+public final class DropIndexStatementHandlerTest {
     
     @Test
     public void assertGetSimpleTableSegmentWithSimpleTableSegmentForMySQL() {
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/DeleteStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/DeleteStatementHandlerTest.java
index 9ea4aac..7a42fd3 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/DeleteStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/DeleteStatementHandlerTest.java
@@ -28,7 +28,7 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class DeleteStatementHandlerTest {
+public final class DeleteStatementHandlerTest {
     
     @Test
     public void assertGetOrderBySegmentWithOrderBySegmentForMySQL() {
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/InsertStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/InsertStatementHandlerTest.java
index 2cdf8f9..0c850ba 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/InsertStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/InsertStatementHandlerTest.java
@@ -31,7 +31,7 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class InsertStatementHandlerTest {
+public final class InsertStatementHandlerTest {
     
     @Test
     public void assertGetOnDuplicateKeyColumnsSegmentWithOnDuplicateKeyColumnsSegmentForMySQL() {
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/SelectStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/SelectStatementHandlerTest.java
index be12245..de1e398 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/SelectStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/SelectStatementHandlerTest.java
@@ -31,7 +31,7 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class SelectStatementHandlerTest {
+public final class SelectStatementHandlerTest {
     
     @Test
     public void assertGetLimitSegmentWithLimitSegmentForMySQL() {
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/UpdateStatementHandlerTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/UpdateStatementHandlerTest.java
index ffe32a9..ccd7dec 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/UpdateStatementHandlerTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/UpdateStatementHandlerTest.java
@@ -28,7 +28,7 @@ import java.util.Optional;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class UpdateStatementHandlerTest {
+public final class UpdateStatementHandlerTest {
     
     @Test
     public void assertGetOrderBySegmentWithOrderBySegmentForMySQL() {