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/06/28 11:04:44 UTC

[shardingsphere] branch master updated: support replace statement (#6178)

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 54a5b50  support replace statement (#6178)
54a5b50 is described below

commit 54a5b50e545572c9f91b14042460f0b99e988bfb
Author: DuanZhengqiang <st...@gmail.com>
AuthorDate: Sun Jun 28 19:04:32 2020 +0800

    support replace statement (#6178)
    
    * support replace statement
    
    * add SQL rewritten test cases
---
 .../src/test/resources/sharding/insert.xml         | 104 +++++++++++++
 .../binary/prepare/MySQLComStmtPrepareChecker.java |   3 +-
 .../prepare/MySQLComStmtPrepareCheckerTest.java    |   3 +-
 .../parser/mysql/visitor/impl/MySQLDMLVisitor.java |  39 +++--
 .../segment/dml/column/ReplaceColumnsSegment.java  |  38 -----
 .../parser/sql/statement/dml/ReplaceStatement.java | 161 ---------------------
 .../asserts/statement/dml/DMLStatementAssert.java  |   7 +-
 .../statement/dml/impl/ReplaceStatementAssert.java |  83 -----------
 .../jaxb/cases/domain/SQLParserTestCases.java      |   9 +-
 .../statement/dml/ReplaceStatementTestCase.java    |  48 ------
 .../src/test/resources/case/dml/replace.xml        | 116 +++++++--------
 11 files changed, 184 insertions(+), 427 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/insert.xml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/insert.xml
index a2ed3ca..731727e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/insert.xml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/insert.xml
@@ -131,4 +131,108 @@
         <output sql="INSERT INTO t_account_0 VALUES (100, 1000, 'OK'), (102, 1000, 'OK')  ON DUPLICATE KEY UPDATE status = ?" parameters="OK_UPDATE" />
         <output sql="INSERT INTO t_account_1 VALUES (101, 2000, 'OK')  ON DUPLICATE KEY UPDATE status = ?" parameters="OK_UPDATE" />
     </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_with_columns_with_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (account_id, amount, status) VALUES (?, ?, ?)" parameters="100, 1000, OK" />
+        <output sql="REPLACE INTO t_account_0 (account_id, amount, status) VALUES (?, ?, ?)" parameters="100, 1000, OK" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_with_columns_with_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (account_id, amount, status) VALUES (100, 1000, 'OK')" />
+        <output sql="REPLACE INTO t_account_0 (account_id, amount, status) VALUES (100, 1000, 'OK')" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_with_columns_without_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (amount, status) VALUES (?, ?)" parameters="1000, OK" />
+        <output sql="REPLACE INTO t_account_1 (amount, status, account_id) VALUES (?, ?, ?)" parameters="1000, OK, 1" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_with_columns_without_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (amount, status) VALUES (1000, 'OK')" />
+        <output sql="REPLACE INTO t_account_1 (amount, status, account_id) VALUES (1000, 'OK', 1)" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_without_columns_with_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (?, ?, ?)" parameters="100, 1000, OK" />
+        <output sql="REPLACE INTO t_account_0 VALUES (?, ?, ?)" parameters="100, 1000, OK" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_without_columns_with_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (100, 1000, 'OK')" />
+        <output sql="REPLACE INTO t_account_0 VALUES (100, 1000, 'OK')" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_without_columns_without_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (?, ?)" parameters="1000, OK" />
+        <output sql="REPLACE INTO t_account_1(amount, status, account_id) VALUES (?, ?, ?)" parameters="1000, OK, 1" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_values_without_columns_without_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (1000, 'OK')" />
+        <output sql="REPLACE INTO t_account_1(amount, status, account_id) VALUES (1000, 'OK', 1)" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_with_columns_with_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (account_id, amount, status) VALUES (100, 0, 'OK'), (?, ?, ?), (102, 2000, 'OK'), (?, ?, ?), (?, ?, ?), (105, 5000, 'OK')" parameters="101, 1000, OK, 103, 3000, OK, 104, 4000, OK" />
+        <output sql="REPLACE INTO t_account_0 (account_id, amount, status) VALUES (100, 0, 'OK'), (102, 2000, 'OK'), (?, ?, ?)" parameters="104, 4000, OK" />
+        <output sql="REPLACE INTO t_account_1 (account_id, amount, status) VALUES (?, ?, ?), (?, ?, ?), (105, 5000, 'OK')" parameters="101, 1000, OK, 103, 3000, OK" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_with_columns_with_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (account_id, amount, status) VALUES (100, 0, 'OK'), (101, 1000, 'OK'), (102, 2000, 'OK'), (103, 3000, 'OK'), (104, 4000, 'OK'), (105, 5000, 'OK')" />
+        <output sql="REPLACE INTO t_account_0 (account_id, amount, status) VALUES (100, 0, 'OK'), (102, 2000, 'OK'), (104, 4000, 'OK')" />
+        <output sql="REPLACE INTO t_account_1 (account_id, amount, status) VALUES (101, 1000, 'OK'), (103, 3000, 'OK'), (105, 5000, 'OK')" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_with_columns_without_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (amount, status) VALUES (?, ?), (1001, 'OK')" parameters="1000, OK" />
+        <output sql="REPLACE INTO t_account_1 (amount, status, account_id) VALUES (?, ?, ?), (1001, 'OK', 1)" parameters="1000, OK, 1" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_with_columns_without_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account (amount, status) VALUES (1000, 'OK'), (1001, 'OK')" />
+        <output sql="REPLACE INTO t_account_1 (amount, status, account_id) VALUES (1000, 'OK', 1), (1001, 'OK', 1)" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_without_columns_with_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (?, ?, ?), (101, 1000, 'OK')" parameters="100, 0, OK" />
+        <output sql="REPLACE INTO t_account_0 VALUES (?, ?, ?)" parameters="100, 0, OK" />
+        <output sql="REPLACE INTO t_account_1 VALUES (101, 1000, 'OK')" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_without_columns_with_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (100, 0, 'OK'), (101, 1000, 'OK')" />
+        <output sql="REPLACE INTO t_account_0 VALUES (100, 0, 'OK')" />
+        <output sql="REPLACE INTO t_account_1 VALUES (101, 1000, 'OK')" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_without_columns_without_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (?, ?), (1001, 'OK')" parameters="1000, OK" />
+        <output sql="REPLACE INTO t_account_1(amount, status, account_id) VALUES (?, ?, ?), (1001, 'OK', 1)" parameters="1000, OK, 1" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_multiple_values_without_columns_without_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account VALUES (1000, 'OK'), (1001, 'OK')" />
+        <output sql="REPLACE INTO t_account_1(amount, status, account_id) VALUES (1000, 'OK', 1), (1001, 'OK', 1)" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_set_with_columns_with_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account SET account_id = ?, amount = ?, status = ?" parameters="100, 1000, OK" />
+        <output sql="REPLACE INTO t_account_0 SET account_id = ?, amount = ?, status = ?" parameters="100, 1000, OK" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_set_with_columns_with_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account SET account_id = 100, amount = 1000, status = 'OK'" />
+        <output sql="REPLACE INTO t_account_0 SET account_id = 100, amount = 1000, status = 'OK'" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_set_with_columns_without_id_for_parameters" db-type="MySQL">
+        <input sql="REPLACE INTO t_account SET amount = ?, status = ?" parameters="1000, OK" />
+        <output sql="REPLACE INTO t_account_1 SET amount = ?, status = ?, account_id = ?" parameters="1000, OK, 1" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="replace_set_with_columns_without_id_for_literals" db-type="MySQL">
+        <input sql="REPLACE INTO t_account SET amount = 1000, status = 'OK'" />
+        <output sql="REPLACE INTO t_account_1 SET amount = 1000, status = 'OK', account_id = 1" />
+    </rewrite-assertion>
 </rewrite-assertions>
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareChecker.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareChecker.java
index 594afc5..2326ce4 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareChecker.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareChecker.java
@@ -61,7 +61,6 @@ import org.apache.shardingsphere.sql.parser.sql.statement.dml.CallStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DoStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.InsertStatement;
-import org.apache.shardingsphere.sql.parser.sql.statement.dml.ReplaceStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.UpdateStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.rl.ChangeMasterStatement;
@@ -89,7 +88,7 @@ public final class MySQLComStmtPrepareChecker {
             CreateTableStatement.class, DropTableStatement.class, CreateUserStatement.class, RenameUserStatement.class, DropUserStatement.class,
             CreateViewStatement.class, DropViewStatement.class, DeleteStatement.class, DoStatement.class, FlushStatement.class,
             GrantStatement.class, InsertStatement.class, InstallPluginStatement.class, KillStatement.class, LoadIndexInfoStatement.class,
-            OptimizeTableStatement.class, RenameTableStatement.class, RepairTableStatement.class, ReplaceStatement.class, ResetStatement.class,
+            OptimizeTableStatement.class, RenameTableStatement.class, RepairTableStatement.class, ResetStatement.class,
             RevokeStatement.class, SelectStatement.class, SetStatement.class, ShowWarningsStatement.class, ShowErrorsStatement.class,
             ShowBinlogStatement.class, ShowCreateProcedureStatement.class, ShowCreateFunctionStatement.class, ShowCreateEventStatement.class,
             ShowCreateTableStatement.class, ShowCreateViewStatement.class, ShowBinaryLogsStatement.class, ShowStatusStatement.class,
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareCheckerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareCheckerTest.java
index 3ddb5da..c67d4eb 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareCheckerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareCheckerTest.java
@@ -62,7 +62,6 @@ import org.apache.shardingsphere.sql.parser.sql.statement.dml.CallStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DoStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.InsertStatement;
-import org.apache.shardingsphere.sql.parser.sql.statement.dml.ReplaceStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.UpdateStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.rl.ChangeMasterStatement;
@@ -87,7 +86,7 @@ public final class MySQLComStmtPrepareCheckerTest {
             new CreateDatabaseStatement(), new DropDatabaseStatement(), new CreateTableStatement(mock(SimpleTableSegment.class)), new DropTableStatement(), new CreateUserStatement(),
             new RenameUserStatement(), new DropUserStatement(), new CreateViewStatement(), new DropViewStatement(), new DeleteStatement(), new DoStatement(), new FlushStatement(),
             new GrantStatement(), new InsertStatement(), new InstallPluginStatement(), new KillStatement(), new LoadIndexInfoStatement(), new OptimizeTableStatement(), new RenameTableStatement(),
-            new RepairTableStatement(), new ReplaceStatement(), new ResetStatement(), new RevokeStatement(), new SelectStatement(), new SetStatement(), new ShowWarningsStatement(),
+            new RepairTableStatement(), new ResetStatement(), new RevokeStatement(), new SelectStatement(), new SetStatement(), new ShowWarningsStatement(),
             new ShowErrorsStatement(), new ShowBinlogStatement(), new ShowCreateProcedureStatement(), new ShowCreateFunctionStatement(), new ShowCreateEventStatement(),
             new ShowCreateTableStatement(), new ShowCreateViewStatement(), new ShowBinaryLogsStatement(), new ShowStatusStatement(), new StartSlaveStatement(), new StopSlaveStatement(),
             new TruncateStatement(), new UninstallPluginStatement(), new UpdateStatement());
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 fef53c7..c92c750 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
@@ -36,7 +36,6 @@ import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.FromCla
 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.InsertValuesClauseContext;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ReplaceValuesClauseContext;
 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;
@@ -51,6 +50,7 @@ import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.Project
 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.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;
@@ -107,7 +107,6 @@ import org.apache.shardingsphere.sql.parser.sql.statement.dml.CallStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DoStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.InsertStatement;
-import org.apache.shardingsphere.sql.parser.sql.statement.dml.ReplaceStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.UpdateStatement;
 import org.apache.shardingsphere.sql.parser.sql.value.collection.CollectionValue;
@@ -155,17 +154,10 @@ public final class MySQLDMLVisitor extends MySQLVisitor implements DMLVisitor {
         return result;
     }
     
-    @SuppressWarnings("unchecked")
     @Override
     public ASTNode visitInsertValuesClause(final InsertValuesClauseContext ctx) {
         InsertStatement result = new InsertStatement();
-        if (null != ctx.columnNames()) {
-            ColumnNamesContext columnNames = ctx.columnNames();
-            CollectionValue<ColumnSegment> columnSegments = (CollectionValue<ColumnSegment>) visit(columnNames);
-            result.setInsertColumns(new InsertColumnsSegment(columnNames.start.getStartIndex(), columnNames.stop.getStopIndex(), columnSegments.getValue()));
-        } else {
-            result.setInsertColumns(new InsertColumnsSegment(ctx.start.getStartIndex() - 1, ctx.start.getStartIndex() - 1, Collections.emptyList()));
-        }
+        result.setInsertColumns(createInsertColumns(ctx.columnNames(), ctx.start.getStartIndex()));
         result.getValues().addAll(createInsertValuesSegments(ctx.assignmentValues()));
         return result;
     }
@@ -192,11 +184,11 @@ public final class MySQLDMLVisitor extends MySQLVisitor implements DMLVisitor {
     @Override
     public ASTNode visitReplace(final ReplaceContext ctx) {
         // TODO :FIXME, since there is no segment for replaceValuesClause, ReplaceStatement is created by sub rule.
-        ReplaceStatement result;
+        InsertStatement result;
         if (null != ctx.replaceValuesClause()) {
-            result = (ReplaceStatement) visit(ctx.replaceValuesClause());
+            result = (InsertStatement) visit(ctx.replaceValuesClause());
         } else {
-            result = new ReplaceStatement();
+            result = new InsertStatement();
             result.setSetAssignment((SetAssignmentSegment) visit(ctx.setAssignmentsClause()));
         }
         result.setTable((SimpleTableSegment) visit(ctx.tableName()));
@@ -204,19 +196,22 @@ public final class MySQLDMLVisitor extends MySQLVisitor implements DMLVisitor {
         return result;
     }
     
-    @SuppressWarnings("unchecked")
     @Override
     public ASTNode visitReplaceValuesClause(final ReplaceValuesClauseContext ctx) {
-        ReplaceStatement result = new ReplaceStatement();
-        if (null != ctx.columnNames()) {
-            ColumnNamesContext columnNames = ctx.columnNames();
+        InsertStatement result = new InsertStatement();
+        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);
-            result.setReplaceColumns(new InsertColumnsSegment(columnNames.start.getStartIndex(), columnNames.stop.getStopIndex(), columnSegments.getValue()));
+            return new InsertColumnsSegment(columnNames.start.getStartIndex(), columnNames.stop.getStopIndex(), columnSegments.getValue());
         } else {
-            result.setReplaceColumns(new InsertColumnsSegment(ctx.start.getStartIndex() - 1, ctx.start.getStartIndex() - 1, Collections.emptyList()));
+            return new InsertColumnsSegment(startIndex - 1, startIndex - 1, Collections.emptyList());
         }
-        result.getValues().addAll(createReplaceValuesSegments(ctx.assignmentValues()));
-        return result;
     }
     
     private Collection<InsertValuesSegment> createReplaceValuesSegments(final Collection<AssignmentValuesContext> assignmentValuesContexts) {
@@ -571,7 +566,7 @@ public final class MySQLDMLVisitor extends MySQLVisitor implements DMLVisitor {
         }
         if (null != ctx.USING()) {
             Collection<ColumnSegment> columnSegmentList = new LinkedList<>();
-            for (MySQLStatementParser.ColumnNameContext cname :ctx.columnNames().columnName()) {
+            for (MySQLStatementParser.ColumnNameContext cname : ctx.columnNames().columnName()) {
                 columnSegmentList.add((ColumnSegment) visit(cname));
             }
             result.getUsingColumns().addAll(columnSegmentList);
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/segment/dml/column/ReplaceColumnsSegment.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/segment/dml/column/ReplaceColumnsSegment.java
deleted file mode 100644
index 9385bfd..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/segment/dml/column/ReplaceColumnsSegment.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.segment.dml.column;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.sql.parser.sql.segment.SQLSegment;
-
-import java.util.Collection;
-
-/**
- * Replace columns segment.
- */
-@RequiredArgsConstructor
-@Getter
-public final class ReplaceColumnsSegment implements SQLSegment {
-    
-    private final int startIndex;
-    
-    private final int stopIndex;
-    
-    private final Collection<ColumnSegment> columns;
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/statement/dml/ReplaceStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/statement/dml/ReplaceStatement.java
deleted file mode 100644
index 477fb63..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/statement/dml/ReplaceStatement.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * 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.statement.dml;
-
-import lombok.Getter;
-import lombok.Setter;
-import org.apache.shardingsphere.sql.parser.sql.segment.dml.assignment.AssignmentSegment;
-import org.apache.shardingsphere.sql.parser.sql.segment.dml.assignment.InsertValuesSegment;
-import org.apache.shardingsphere.sql.parser.sql.segment.dml.assignment.SetAssignmentSegment;
-import org.apache.shardingsphere.sql.parser.sql.segment.dml.column.ColumnSegment;
-import org.apache.shardingsphere.sql.parser.sql.segment.dml.column.InsertColumnsSegment;
-import org.apache.shardingsphere.sql.parser.sql.segment.dml.expr.ExpressionSegment;
-import org.apache.shardingsphere.sql.parser.sql.segment.generic.table.SimpleTableSegment;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Optional;
-
-/**
- * Replace statement.
- */
-@Getter
-@Setter
-public final class ReplaceStatement extends DMLStatement {
-    
-    private SimpleTableSegment table;
-    
-    private InsertColumnsSegment replaceColumns;
-    
-    private SetAssignmentSegment setAssignment;
-    
-    private final Collection<InsertValuesSegment> values = new LinkedList<>();
-    
-    /**
-     * Get replace columns segment.
-     * 
-     * @return replace columns segment
-     */
-    public Optional<InsertColumnsSegment> getReplaceColumns() {
-        return Optional.ofNullable(replaceColumns);
-    }
-    
-    /**
-     * Get columns.
-     * 
-     * @return columns
-     */
-    public Collection<ColumnSegment> getColumns() {
-        return null == replaceColumns ? Collections.emptyList() : replaceColumns.getColumns();
-    }
-    
-    /**
-     * Get set assignment segment.
-     * 
-     * @return set assignment segment
-     */
-    public Optional<SetAssignmentSegment> getSetAssignment() {
-        return Optional.ofNullable(setAssignment);
-    }
-    
-    /**
-     * Judge is use default columns or not.
-     * 
-     * @return is use default columns or not
-     */
-    public boolean useDefaultColumns() {
-        return getColumns().isEmpty() && null == setAssignment;
-    }
-    
-    /**
-     * Get column names.
-     *
-     * @return column names
-     */
-    public List<String> getColumnNames() {
-        return null == setAssignment ? getColumnNamesForReplaceColumns() : getColumnNamesForSetAssignment();
-    }
-    
-    private List<String> getColumnNamesForReplaceColumns() {
-        List<String> result = new LinkedList<>();
-        for (ColumnSegment each : getColumns()) {
-            result.add(each.getIdentifier().getValue().toLowerCase());
-        }
-        return result;
-    }
-    
-    private List<String> getColumnNamesForSetAssignment() {
-        List<String> result = new LinkedList<>();
-        for (AssignmentSegment each : setAssignment.getAssignments()) {
-            result.add(each.getColumn().getIdentifier().getValue().toLowerCase());
-        }
-        return result;
-    }
-    
-    /**
-     * Get value list count.
-     *
-     * @return value list count
-     */
-    public int getValueListCount() {
-        return null == setAssignment ? values.size() : 1;
-    }
-    
-    /**
-     * Get value count for per value list.
-     * 
-     * @return value count
-     */
-    public int getValueCountForPerGroup() {
-        if (!values.isEmpty()) {
-            return values.iterator().next().getValues().size();
-        }
-        if (null != setAssignment) {
-            return setAssignment.getAssignments().size();
-        }
-        return 0;
-    }
-    
-    /**
-     * Get all value expressions.
-     * 
-     * @return all value expressions
-     */
-    public List<List<ExpressionSegment>> getAllValueExpressions() {
-        return null == setAssignment ? getAllValueExpressionsFromValues() : Collections.singletonList(getAllValueExpressionsFromSetAssignment());
-    }
-    
-    private List<List<ExpressionSegment>> getAllValueExpressionsFromValues() {
-        List<List<ExpressionSegment>> result = new ArrayList<>(values.size());
-        for (InsertValuesSegment each : values) {
-            result.add(each.getValues());
-        }
-        return result;
-    }
-    
-    private List<ExpressionSegment> getAllValueExpressionsFromSetAssignment() {
-        List<ExpressionSegment> result = new ArrayList<>(setAssignment.getAssignments().size());
-        for (AssignmentSegment each : setAssignment.getAssignments()) {
-            result.add(each.getValue());
-        }
-        return result;
-    }
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/asserts/statement/dml/DMLStatementAssert.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/asserts/statement/dml/DMLStatementAssert.java
index 71dbbdf..04230c8 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/asserts/statement/dml/DMLStatementAssert.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/asserts/statement/dml/DMLStatementAssert.java
@@ -22,19 +22,16 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.sql.parser.integrate.asserts.SQLCaseAssertContext;
 import org.apache.shardingsphere.sql.parser.integrate.asserts.statement.dml.impl.DeleteStatementAssert;
 import org.apache.shardingsphere.sql.parser.integrate.asserts.statement.dml.impl.InsertStatementAssert;
-import org.apache.shardingsphere.sql.parser.integrate.asserts.statement.dml.impl.ReplaceStatementAssert;
 import org.apache.shardingsphere.sql.parser.integrate.asserts.statement.dml.impl.SelectStatementAssert;
 import org.apache.shardingsphere.sql.parser.integrate.asserts.statement.dml.impl.UpdateStatementAssert;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.SQLParserTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.DeleteStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.InsertStatementTestCase;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.ReplaceStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.SelectStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.UpdateStatementTestCase;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.InsertStatement;
-import org.apache.shardingsphere.sql.parser.sql.statement.dml.ReplaceStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.UpdateStatement;
 
@@ -46,7 +43,7 @@ public final class DMLStatementAssert {
     
     /**
      * Assert DML statement is correct with expected parser result.
-     * 
+     *
      * @param assertContext assert context
      * @param actual actual DML statement
      * @param expected expected parser result
@@ -60,8 +57,6 @@ public final class DMLStatementAssert {
             DeleteStatementAssert.assertIs(assertContext, (DeleteStatement) actual, (DeleteStatementTestCase) expected);
         } else if (actual instanceof InsertStatement) {
             InsertStatementAssert.assertIs(assertContext, (InsertStatement) actual, (InsertStatementTestCase) expected);
-        } else if (actual instanceof ReplaceStatement) {
-            ReplaceStatementAssert.assertIs(assertContext, (ReplaceStatement) actual, (ReplaceStatementTestCase) expected);
         }
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/asserts/statement/dml/impl/ReplaceStatementAssert.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/asserts/statement/dml/impl/ReplaceStatementAssert.java
deleted file mode 100644
index 1f68e2c..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/asserts/statement/dml/impl/ReplaceStatementAssert.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.integrate.asserts.statement.dml.impl;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.sql.parser.integrate.asserts.SQLCaseAssertContext;
-import org.apache.shardingsphere.sql.parser.integrate.asserts.segment.insert.InsertColumnsClauseAssert;
-import org.apache.shardingsphere.sql.parser.integrate.asserts.segment.insert.InsertValuesClauseAssert;
-import org.apache.shardingsphere.sql.parser.integrate.asserts.segment.set.SetClauseAssert;
-import org.apache.shardingsphere.sql.parser.integrate.asserts.segment.table.TableAssert;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.ReplaceStatementTestCase;
-import org.apache.shardingsphere.sql.parser.sql.statement.dml.ReplaceStatement;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Replace statement assert.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ReplaceStatementAssert {
-    
-    /**
-     * Assert insert statement is correct with expected parser result.
-     *
-     * @param assertContext assert context
-     * @param actual actual insert statement
-     * @param expected expected insert statement test case
-     */
-    public static void assertIs(final SQLCaseAssertContext assertContext, final ReplaceStatement actual, final ReplaceStatementTestCase expected) {
-        assertTable(assertContext, actual, expected);
-        assertReplaceColumnsClause(assertContext, actual, expected);
-        assertReplaceValuesClause(assertContext, actual, expected);
-        assertSetClause(assertContext, actual, expected);
-    }
-    
-    private static void assertTable(final SQLCaseAssertContext assertContext, final ReplaceStatement actual, final ReplaceStatementTestCase expected) {
-        TableAssert.assertIs(assertContext, actual.getTable(), expected.getTable());
-    }
-    
-    private static void assertReplaceColumnsClause(final SQLCaseAssertContext assertContext, final ReplaceStatement actual, final ReplaceStatementTestCase expected) {
-        if (null != expected.getReplaceColumnsClause()) {
-            assertTrue(assertContext.getText("Actual insert columns segment should exist."), actual.getReplaceColumns().isPresent());
-            InsertColumnsClauseAssert.assertIs(assertContext, actual.getReplaceColumns().get(), expected.getReplaceColumnsClause());    
-        } else {
-            assertFalse(assertContext.getText("Actual insert columns segment should not exist."), actual.getReplaceColumns().isPresent());
-        }
-    }
-    
-    private static void assertReplaceValuesClause(final SQLCaseAssertContext assertContext, final ReplaceStatement actual, final ReplaceStatementTestCase expected) {
-        if (null != expected.getReplaceValuesClause()) {
-            assertFalse(assertContext.getText("Actual insert values segment should exist."), actual.getValues().isEmpty());
-            InsertValuesClauseAssert.assertIs(assertContext, actual.getValues(), expected.getReplaceValuesClause());
-        } else {
-            assertTrue(assertContext.getText("Actual insert values segment should not exist."), actual.getValues().isEmpty());
-        }
-    }
-    
-    private static void assertSetClause(final SQLCaseAssertContext assertContext, final ReplaceStatement actual, final ReplaceStatementTestCase expected) {
-        if (null != expected.getSetClause()) {
-            assertTrue(assertContext.getText("Actual set assignment segment should exist."), actual.getSetAssignment().isPresent());
-            SetClauseAssert.assertIs(assertContext, actual.getSetAssignment().get(), expected.getSetClause());
-        } else {
-            assertFalse(assertContext.getText("Actual set assignment segment should not exist."), actual.getSetAssignment().isPresent());
-        }
-    }
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/jaxb/cases/domain/SQLParserTestCases.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/jaxb/cases/domain/SQLParserTestCases.java
index d795159..2901b7d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/jaxb/cases/domain/SQLParserTestCases.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/jaxb/cases/domain/SQLParserTestCases.java
@@ -22,6 +22,8 @@ import lombok.Getter;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.CommonStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.SQLParserTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.DescribeStatementTestCase;
+import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.SetVariableStatementTestCase;
+import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.ShowColumnsStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.ShowCreateTableStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.ShowDatabasesStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.ShowIndexStatementTestCase;
@@ -29,8 +31,6 @@ import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statemen
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.ShowTableStatusStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.ShowTablesStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.UseStatementTestCase;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.ShowColumnsStatementTestCase;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dal.SetVariableStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dcl.AlterLoginStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dcl.AlterRoleStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dcl.AlterUserStatementTestCase;
@@ -56,7 +56,6 @@ import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statemen
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.ddl.TruncateStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.DeleteStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.InsertStatementTestCase;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.ReplaceStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.SelectStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.dml.UpdateStatementTestCase;
 import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.tcl.BeginTransactionStatementTestCase;
@@ -94,9 +93,6 @@ public final class SQLParserTestCases {
     @XmlElement(name = "insert")
     private final List<InsertStatementTestCase> insertTestCases = new LinkedList<>();
 
-    @XmlElement(name = "replace")
-    private final List<ReplaceStatementTestCase> replaceTestCases = new LinkedList<>();
-
     @XmlElement(name = "create-table")
     private final List<CreateTableStatementTestCase> createTableTestCases = new LinkedList<>();
     
@@ -228,7 +224,6 @@ public final class SQLParserTestCases {
         putAll(updateTestCases, result);
         putAll(deleteTestCases, result);
         putAll(insertTestCases, result);
-        putAll(replaceTestCases, result);
         putAll(createTableTestCases, result);
         putAll(alterTableTestCases, result);
         putAll(dropTableTestCases, result);
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/jaxb/cases/domain/statement/dml/ReplaceStatementTestCase.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/jaxb/cases/domain/statement/dml/ReplaceStatementTestCase.java
deleted file mode 100644
index e37839e..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/java/org/apache/shardingsphere/sql/parser/integrate/jaxb/cases/domain/statement/dml/ReplaceStatementTestCase.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.integrate.jaxb.cases.domain.statement.dml;
-
-import lombok.Getter;
-import lombok.Setter;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.segment.impl.insert.ExpectedInsertColumnsClause;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.segment.impl.insert.ExpectedInsertValuesClause;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.segment.impl.set.ExpectedSetClause;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.segment.impl.table.ExpectedSimpleTable;
-import org.apache.shardingsphere.sql.parser.integrate.jaxb.cases.domain.statement.SQLParserTestCase;
-
-import javax.xml.bind.annotation.XmlElement;
-
-/**
- * Replace statement test case.
- */
-@Getter
-@Setter
-public final class ReplaceStatementTestCase extends SQLParserTestCase {
-    
-    @XmlElement
-    private ExpectedSimpleTable table;
-    
-    @XmlElement(name = "columns")
-    private ExpectedInsertColumnsClause replaceColumnsClause;
-    
-    @XmlElement(name = "values")
-    private ExpectedInsertValuesClause replaceValuesClause;
-    
-    @XmlElement(name = "set")
-    private ExpectedSetClause setClause;
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/resources/case/dml/replace.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/resources/case/dml/replace.xml
index 0f497c7..5c42559 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/resources/case/dml/replace.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/test/resources/case/dml/replace.xml
@@ -17,7 +17,7 @@
   -->
 
 <sql-parser-test-cases>
-    <replace sql-case-id="replace_with_all_placeholders" parameters="1, 1, 'init'">
+    <insert sql-case-id="replace_with_all_placeholders" parameters="1, 1, 'init'">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="21" stop-index="47">
             <column name="order_id" start-index="22" stop-index="29" />
@@ -40,9 +40,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_now_function" parameters="1, 1, 'init'">
+    <insert sql-case-id="replace_with_now_function" parameters="1, 1, 'init'">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <columns start-index="26" stop-index="76">
             <column name="item_id" start-index="27" stop-index="33" />
@@ -73,9 +73,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_without_parameters">
+    <insert sql-case-id="replace_without_parameters">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="21" stop-index="47">
             <column name="order_id" start-index="22" stop-index="29" />
@@ -98,9 +98,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_multiple_values">
+    <insert sql-case-id="replace_with_multiple_values">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="21" stop-index="47">
             <column name="order_id" start-index="22" stop-index="29" />
@@ -131,9 +131,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_special_characters">
+    <insert sql-case-id="replace_with_special_characters">
         <table name="t_order" start-delimiter="`" end-delimiter="`" start-index="13" stop-index="21" />
         <columns start-index="23" stop-index="55">
             <column name="order_id" start-delimiter="`" end-delimiter="`" start-index="24" stop-index="33" />
@@ -153,9 +153,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_all_placeholders_for_table_identifier" parameters="1, 1, 'init'">
+    <insert sql-case-id="replace_with_all_placeholders_for_table_identifier" parameters="1, 1, 'init'">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="21" stop-index="71">
             <column name="order_id" start-index="22" stop-index="37">
@@ -184,9 +184,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_without_columns_with_all_placeholders" parameters="1, 1, 'init'">
+    <insert sql-case-id="replace_without_columns_with_all_placeholders" parameters="1, 1, 'init'">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="20" stop-index="20" />
         <values>
@@ -205,9 +205,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_set_with_all_placeholders" parameters="1, 1, 'init'">
+    <insert sql-case-id="replace_set_with_all_placeholders" parameters="1, 1, 'init'">
         <table name="t_order" start-index="13" stop-index="19" />
         <set start-index="21" stop-index="61" literal-stop-index="66">
             <assignment>
@@ -232,9 +232,9 @@
                 </assignment-value>
             </assignment>
         </set>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_partial_placeholders" parameters="1, 1">
+    <insert sql-case-id="replace_with_partial_placeholders" parameters="1, 1">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="21" stop-index="47">
             <column name="order_id" start-index="22" stop-index="29" />
@@ -256,9 +256,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_set_with_partial_placeholders" parameters="1, 1">
+    <insert sql-case-id="replace_set_with_partial_placeholders" parameters="1, 1">
         <table name="t_order" start-index="13" stop-index="19" />
         <set start-index="21" stop-index="69">
             <assignment>
@@ -282,9 +282,9 @@
                 </assignment-value>
             </assignment>
         </set>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_generate_key_column" parameters="10000, 1000, 10">
+    <insert sql-case-id="replace_with_generate_key_column" parameters="10000, 1000, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <columns start-index="25" stop-index="75">
             <column name="item_id" start-index="26" stop-index="32" />
@@ -315,9 +315,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_without_generate_key_column" parameters="1000, 10">
+    <insert sql-case-id="replace_without_generate_key_column" parameters="1000, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <columns start-index="25" stop-index="66">
             <column name="order_id" start-index="26" stop-index="33" />
@@ -343,9 +343,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_set_with_generate_key_column" parameters="10000, 1000, 10">
+    <insert sql-case-id="replace_set_with_generate_key_column" parameters="10000, 1000, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <set start-index="26" stop-index="115" literal-stop-index="123">
             <assignment>
@@ -382,9 +382,9 @@
                 </assignment-value>
             </assignment>
         </set>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_set_without_generate_key_column" parameters="1000, 10">
+    <insert sql-case-id="replace_set_without_generate_key_column" parameters="1000, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <set start-index="26" stop-index="102" literal-stop-index="106">
             <assignment>
@@ -414,9 +414,9 @@
                 </assignment-value>
             </assignment>
         </set>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_set_with_all_placeholders_for_table_identifier" parameters="1, 1, 'init'">
+    <insert sql-case-id="replace_set_with_all_placeholders_for_table_identifier" parameters="1, 1, 'init'">
         <table name="t_order" start-index="13" stop-index="19" />
         <set start-index="21" stop-index="85" literal-stop-index="90">
             <assignment>
@@ -447,9 +447,9 @@
                 </assignment-value>
             </assignment>
         </set>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_batch" parameters="1000, 10, 'init', 1100, 11, 'init'">
+    <insert sql-case-id="replace_with_batch" parameters="1000, 10, 'init', 1100, 11, 'init'">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="21" stop-index="47">
             <column name="order_id" start-index="22" stop-index="29" />
@@ -486,9 +486,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_batch_and_irregular_parameters" parameters="1, 2, 2, 'init'">
+    <insert sql-case-id="replace_with_batch_and_irregular_parameters" parameters="1, 2, 2, 'init'">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="21" stop-index="47">
             <column name="order_id" start-index="22" stop-index="29" />
@@ -523,13 +523,13 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
     <!--<replace sql-case-id="replace_with_batch_and_composite_expression" parameters="1, 1, 'init', 2, 2, 'init'">-->
     <!--<table name="t_order" start-index="13" stop-index="19" />-->
     <!--</replace>-->
 
-    <replace sql-case-id="replace_with_batch_and_with_generate_key_column" parameters="10000, 1000, 10, 10010, 1001, 10">
+    <insert sql-case-id="replace_with_batch_and_with_generate_key_column" parameters="10000, 1000, 10, 10010, 1001, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <columns start-index="25" stop-index="75">
             <column name="item_id" start-index="26" stop-index="32" />
@@ -580,9 +580,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_batch_and_without_generate_key_column" parameters="1000, 10, 1001, 10">
+    <insert sql-case-id="replace_with_batch_and_without_generate_key_column" parameters="1000, 10, 1001, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <columns start-index="25" stop-index="66">
             <column name="order_id" start-index="26" stop-index="33" />
@@ -624,9 +624,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_without_columns_and_with_generate_key_column" parameters="10000, 1000, 10">
+    <insert sql-case-id="replace_without_columns_and_with_generate_key_column" parameters="10000, 1000, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <columns start-index="25" stop-index="25" />
         <values>
@@ -651,9 +651,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_without_columns_and_without_generate_key_column" parameters="1000, 10">
+    <insert sql-case-id="replace_without_columns_and_without_generate_key_column" parameters="1000, 10">
         <table name="t_order_item" start-index="13" stop-index="24" />
         <columns start-index="25" stop-index="25" />
         <values>
@@ -674,7 +674,7 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
     <!-- // TODO
     <replace sql-case-id="assertInsertSelect" sql="REPLACE INTO `order` ('order_id', 'state') (SELECT 1, 'RUNNING' FROM dual UNION ALL SELECT 2, 'RUNNING' FROM dual )"">
@@ -685,15 +685,15 @@
     </replace>
     -->
 
-    <replace sql-case-id="replace_with_one_auto_increment_column">
+    <insert sql-case-id="replace_with_one_auto_increment_column">
         <table name="t_auto_increment_table" start-index="13" stop-index="34" />
         <columns start-index="35" stop-index="35" />
         <values>
             <value />
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_double_value">
+    <insert sql-case-id="replace_with_double_value">
         <table name="t_double_test" start-index="13" stop-index="25" />
         <columns start-index="26" stop-index="31">
             <column name="col1" start-index="27" stop-index="30" />
@@ -705,9 +705,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_null_value">
+    <insert sql-case-id="replace_with_null_value">
         <table name="t_null_value_test" start-index="13" stop-index="29" />
         <columns start-index="30" stop-index="35">
             <column name="col1" start-index="31" stop-index="34" />
@@ -719,9 +719,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_blob_value">
+    <insert sql-case-id="replace_with_blob_value">
         <table name="t_blob_value_test" start-index="13" stop-index="29" />
         <columns start-index="30" stop-index="35">
             <column name="col1" start-index="31" stop-index="34" />
@@ -733,9 +733,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_function" parameters="1000, 10">
+    <insert sql-case-id="replace_with_function" parameters="1000, 10">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="20" stop-index="52">
             <column name="present_date" start-index="21" stop-index="32" />
@@ -757,9 +757,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_unix_timestamp_function" parameters="2019-10-19, 1000, 10">
+    <insert sql-case-id="replace_with_unix_timestamp_function" parameters="2019-10-19, 1000, 10">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="20" stop-index="46">
             <column name="status" start-index="21" stop-index="26" />
@@ -781,9 +781,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_str_to_date" parameters="2019-12-10, 1, 1">
+    <insert sql-case-id="replace_with_str_to_date" parameters="2019-12-10, 1, 1">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="20" stop-index="52">
             <column name="present_date" start-index="21" stop-index="32" />
@@ -806,9 +806,9 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 
-    <replace sql-case-id="replace_with_str_date_add" parameters="1, 1, 1">
+    <insert sql-case-id="replace_with_str_date_add" parameters="1, 1, 1">
         <table name="t_order" start-index="13" stop-index="19" />
         <columns start-index="20" stop-index="52">
             <column name="present_date" start-index="21" stop-index="32" />
@@ -831,5 +831,5 @@
                 </assignment-value>
             </value>
         </values>
-    </replace>
+    </insert>
 </sql-parser-test-cases>