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 2023/04/19 09:16:05 UTC

[shardingsphere] branch master updated: Support mysql load data, load xml statement for single table, broadcast table (#25231)

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 f504f90e78d Support mysql load data, load xml statement for single table, broadcast table (#25231)
f504f90e78d is described below

commit f504f90e78db48ab991b8747c70506a6a17d9519
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Wed Apr 19 17:15:52 2023 +0800

    Support mysql load data, load xml statement for single table, broadcast table (#25231)
---
 .../ShardingStatementValidatorFactory.java         | 10 ++++
 .../impl/ShardingLoadDataStatementValidator.java   | 50 ++++++++++++++++++++
 .../impl/ShardingLoadXMLStatementValidator.java    | 50 ++++++++++++++++++++
 .../infra/binder/SQLStatementContextFactory.java   | 10 ++++
 .../statement/dml/LoadDataStatementContext.java    | 46 +++++++++++++++++++
 .../statement/dml/LoadXMLStatementContext.java     | 46 +++++++++++++++++++
 .../impl/MySQLDMLStatementSQLVisitor.java          | 21 +++++++++
 .../core/database/visitor/SQLVisitorRule.java      |  2 +
 .../mysql/dml/MySQLLoadDataStatement.java          | 35 ++++++++++++++
 .../statement/mysql/dml/MySQLLoadXMLStatement.java | 35 ++++++++++++++
 .../asserts/statement/dml/DMLStatementAssert.java  | 10 ++++
 .../dml/impl/LoadDataStatementAssert.java          | 53 ++++++++++++++++++++++
 .../statement/dml/impl/LoadXMLStatementAssert.java | 53 ++++++++++++++++++++++
 .../cases/parser/jaxb/RootSQLParserTestCases.java  |  8 ++++
 .../statement/dml/LoadDataStatementTestCase.java   | 36 +++++++++++++++
 .../statement/dml/LoadXMLStatementTestCase.java    | 36 +++++++++++++++
 .../src/main/resources/case/dml/load-data.xml      | 37 +++++++++++++++
 .../src/main/resources/case/dml/load-xml.xml       | 34 ++++++++++++++
 .../main/resources/sql/supported/dml/load-data.xml | 25 ++++++++++
 .../main/resources/sql/supported/dml/load-xml.xml  | 24 ++++++++++
 20 files changed, 621 insertions(+)

diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
index 2007677a9b2..1eb36fe2e60 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
@@ -35,6 +35,8 @@ import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.Shardi
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingCopyStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingDeleteStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingInsertStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingLoadDataStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingLoadXMLStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingUpdateStatementValidator;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterIndexStatement;
@@ -55,6 +57,8 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatemen
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
 
 import java.util.Optional;
 
@@ -134,6 +138,12 @@ public final class ShardingStatementValidatorFactory {
         if (sqlStatement instanceof CopyStatement) {
             return Optional.of(new ShardingCopyStatementValidator());
         }
+        if (sqlStatement instanceof MySQLLoadDataStatement) {
+            return Optional.of(new ShardingLoadDataStatementValidator());
+        }
+        if (sqlStatement instanceof MySQLLoadXMLStatement) {
+            return Optional.of(new ShardingLoadXMLStatementValidator());
+        }
         return Optional.empty();
     }
 }
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingLoadDataStatementValidator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingLoadDataStatementValidator.java
new file mode 100644
index 00000000000..a5a7d7e6273
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingLoadDataStatementValidator.java
@@ -0,0 +1,50 @@
+/*
+ * 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.dml.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.hint.HintValueContext;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.exception.syntax.UnsupportedShardingOperationException;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
+
+import java.util.List;
+
+/**
+ * Sharding load data statement validator.
+ */
+public final class ShardingLoadDataStatementValidator extends ShardingDMLStatementValidator<MySQLLoadDataStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<MySQLLoadDataStatement> sqlStatementContext,
+                            final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props) {
+        String tableName = sqlStatementContext.getSqlStatement().getTableSegment().getTableName().getIdentifier().getValue();
+        if (shardingRule.isShardingTable(tableName)) {
+            throw new UnsupportedShardingOperationException("LOAD DATA", tableName);
+        }
+    }
+    
+    @Override
+    public void postValidate(final ShardingRule shardingRule, final SQLStatementContext<MySQLLoadDataStatement> sqlStatementContext, final HintValueContext hintValueContext,
+                             final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props, final RouteContext routeContext) {
+    }
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingLoadXMLStatementValidator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingLoadXMLStatementValidator.java
new file mode 100644
index 00000000000..42b0f2a97d9
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingLoadXMLStatementValidator.java
@@ -0,0 +1,50 @@
+/*
+ * 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.dml.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.hint.HintValueContext;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.exception.syntax.UnsupportedShardingOperationException;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
+
+import java.util.List;
+
+/**
+ * Sharding load xml statement validator.
+ */
+public final class ShardingLoadXMLStatementValidator extends ShardingDMLStatementValidator<MySQLLoadXMLStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<MySQLLoadXMLStatement> sqlStatementContext,
+                            final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props) {
+        String tableName = sqlStatementContext.getSqlStatement().getTableSegment().getTableName().getIdentifier().getValue();
+        if (shardingRule.isShardingTable(tableName)) {
+            throw new UnsupportedShardingOperationException("LOAD XML", tableName);
+        }
+    }
+    
+    @Override
+    public void postValidate(final ShardingRule shardingRule, final SQLStatementContext<MySQLLoadXMLStatement> sqlStatementContext, final HintValueContext hintValueContext,
+                             final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props, final RouteContext routeContext) {
+    }
+}
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
index db4b121ccbe..525cb0d491a 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
@@ -59,6 +59,8 @@ import org.apache.shardingsphere.infra.binder.statement.dml.CopyStatementContext
 import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.DoStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.LoadDataStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.LoadXMLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
@@ -106,6 +108,8 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowTableStatusStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowTablesStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussCursorStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sqlserver.dcl.SQLServerDenyUserStatement;
 
@@ -179,6 +183,12 @@ public final class SQLStatementContextFactory {
         if (sqlStatement instanceof DoStatement) {
             return new DoStatementContext((DoStatement) sqlStatement);
         }
+        if (sqlStatement instanceof MySQLLoadDataStatement) {
+            return new LoadDataStatementContext((MySQLLoadDataStatement) sqlStatement);
+        }
+        if (sqlStatement instanceof MySQLLoadXMLStatement) {
+            return new LoadXMLStatementContext((MySQLLoadXMLStatement) sqlStatement);
+        }
         throw new UnsupportedSQLOperationException(String.format("Unsupported SQL statement `%s`", sqlStatement.getClass().getSimpleName()));
     }
     
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/LoadDataStatementContext.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/LoadDataStatementContext.java
new file mode 100644
index 00000000000..8019dea2a6c
--- /dev/null
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/LoadDataStatementContext.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.binder.statement.dml;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
+
+import java.util.Collection;
+
+/**
+ * Load data statement context.
+ */
+@Getter
+public final class LoadDataStatementContext extends CommonSQLStatementContext<MySQLLoadDataStatement> implements TableAvailable {
+    
+    private final TablesContext tablesContext;
+    
+    public LoadDataStatementContext(final MySQLLoadDataStatement sqlStatement) {
+        super(sqlStatement);
+        tablesContext = new TablesContext(sqlStatement.getTableSegment(), getDatabaseType());
+    }
+    
+    @Override
+    public Collection<SimpleTableSegment> getAllTables() {
+        return tablesContext.getTables();
+    }
+}
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/LoadXMLStatementContext.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/LoadXMLStatementContext.java
new file mode 100644
index 00000000000..986d2a4b3d2
--- /dev/null
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/LoadXMLStatementContext.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.binder.statement.dml;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
+
+import java.util.Collection;
+
+/**
+ * Load xml statement context.
+ */
+@Getter
+public final class LoadXMLStatementContext extends CommonSQLStatementContext<MySQLLoadXMLStatement> implements TableAvailable {
+    
+    private final TablesContext tablesContext;
+    
+    public LoadXMLStatementContext(final MySQLLoadXMLStatement sqlStatement) {
+        super(sqlStatement);
+        tablesContext = new TablesContext(sqlStatement.getTableSegment(), getDatabaseType());
+    }
+    
+    @Override
+    public Collection<SimpleTableSegment> getAllTables() {
+        return tablesContext.getTables();
+    }
+}
diff --git a/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDMLStatementSQLVisitor.java b/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDMLStatementSQLVisitor.java
index fb221db80dd..4b99808cb9e 100644
--- a/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDMLStatementSQLVisitor.java
+++ b/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDMLStatementSQLVisitor.java
@@ -23,9 +23,15 @@ import org.apache.shardingsphere.sql.parser.api.visitor.operation.SQLStatementVi
 import org.apache.shardingsphere.sql.parser.api.visitor.type.DMLSQLVisitor;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.CallContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.DoStatementContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LoadDataStatementContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LoadStatementContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.LoadXmlStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLCallStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLDoStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -54,4 +60,19 @@ public final class MySQLDMLStatementSQLVisitor extends MySQLStatementSQLVisitor
         ctx.expr().forEach(each -> params.add((ExpressionSegment) visit(each)));
         return new MySQLDoStatement(params);
     }
+    
+    @Override
+    public ASTNode visitLoadStatement(final LoadStatementContext ctx) {
+        return null != ctx.loadDataStatement() ? visit(ctx.loadDataStatement()) : visit(ctx.loadXmlStatement());
+    }
+    
+    @Override
+    public ASTNode visitLoadDataStatement(final LoadDataStatementContext ctx) {
+        return new MySQLLoadDataStatement((SimpleTableSegment) visit(ctx.tableName()));
+    }
+    
+    @Override
+    public ASTNode visitLoadXmlStatement(final LoadXmlStatementContext ctx) {
+        return new MySQLLoadXMLStatement((SimpleTableSegment) visit(ctx.tableName()));
+    }
 }
diff --git a/sql-parser/engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java b/sql-parser/engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
index eab0cdbb134..07df221528b 100644
--- a/sql-parser/engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
+++ b/sql-parser/engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
@@ -465,6 +465,8 @@ public enum SQLVisitorRule {
     
     CALL("Call", SQLStatementType.DML),
     
+    LOAD_STATEMENT("LoadStatement", SQLStatementType.DML),
+    
     CHANGE_MASTER("ChangeMaster", SQLStatementType.RL),
     
     START_SLAVE("StartSlave", SQLStatementType.RL),
diff --git a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dml/MySQLLoadDataStatement.java b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dml/MySQLLoadDataStatement.java
new file mode 100644
index 00000000000..4f6768048ba
--- /dev/null
+++ b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dml/MySQLLoadDataStatement.java
@@ -0,0 +1,35 @@
+/*
+ * 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.statement.mysql.dml;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+
+/**
+ * MySQL load data statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class MySQLLoadDataStatement extends AbstractSQLStatement implements DMLStatement, MySQLStatement {
+    
+    private final SimpleTableSegment tableSegment;
+}
diff --git a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dml/MySQLLoadXMLStatement.java b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dml/MySQLLoadXMLStatement.java
new file mode 100644
index 00000000000..9084fd5d907
--- /dev/null
+++ b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dml/MySQLLoadXMLStatement.java
@@ -0,0 +1,35 @@
+/*
+ * 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.statement.mysql.dml;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+
+/**
+ * MySQL load xml statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class MySQLLoadXMLStatement extends AbstractSQLStatement implements DMLStatement, MySQLStatement {
+    
+    private final SimpleTableSegment tableSegment;
+}
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/DMLStatementAssert.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/DMLStatementAssert.java
index 5bc3bfa8d8c..64d74600c80 100644
--- a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/DMLStatementAssert.java
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/DMLStatementAssert.java
@@ -28,12 +28,16 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertState
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.MergeStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.SQLCaseAssertContext;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.CallStatementAssert;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.CopyStatementAssert;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.DeleteStatementAssert;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.DoStatementAssert;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.InsertStatementAssert;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.LoadDataStatementAssert;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.LoadXMLStatementAssert;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.MergeStatementAssert;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.SelectStatementAssert;
 import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.dml.impl.UpdateStatementAssert;
@@ -43,6 +47,8 @@ import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.s
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.DeleteStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.DoStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.InsertStatementTestCase;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.LoadDataStatementTestCase;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.LoadXMLStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.MergeStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.SelectStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.UpdateStatementTestCase;
@@ -77,6 +83,10 @@ public final class DMLStatementAssert {
             CopyStatementAssert.assertIs(assertContext, (CopyStatement) actual, (CopyStatementTestCase) expected);
         } else if (actual instanceof DoStatement) {
             DoStatementAssert.assertIs(assertContext, (DoStatement) actual, (DoStatementTestCase) expected);
+        } else if (actual instanceof MySQLLoadDataStatement) {
+            LoadDataStatementAssert.assertIs(assertContext, (MySQLLoadDataStatement) actual, (LoadDataStatementTestCase) expected);
+        } else if (actual instanceof MySQLLoadXMLStatement) {
+            LoadXMLStatementAssert.assertIs(assertContext, (MySQLLoadXMLStatement) actual, (LoadXMLStatementTestCase) expected);
         }
     }
 }
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/LoadDataStatementAssert.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/LoadDataStatementAssert.java
new file mode 100644
index 00000000000..260c2959657
--- /dev/null
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/LoadDataStatementAssert.java
@@ -0,0 +1,53 @@
+/*
+ * 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.test.it.sql.parser.internal.asserts.statement.dml.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadDataStatement;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.segment.table.TableAssert;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.LoadDataStatementTestCase;
+
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+/**
+ * Load data statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class LoadDataStatementAssert {
+    
+    /**
+     * Assert load data statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual load data statement
+     * @param expected expected load data statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final MySQLLoadDataStatement actual, final LoadDataStatementTestCase expected) {
+        assertTable(assertContext, actual, expected);
+    }
+    
+    private static void assertTable(final SQLCaseAssertContext assertContext, final MySQLLoadDataStatement actual, final LoadDataStatementTestCase expected) {
+        if (null == expected.getTable()) {
+            assertNull(actual.getTableSegment(), assertContext.getText("Actual table should not exist."));
+        } else {
+            TableAssert.assertIs(assertContext, actual.getTableSegment(), expected.getTable());
+        }
+    }
+}
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/LoadXMLStatementAssert.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/LoadXMLStatementAssert.java
new file mode 100644
index 00000000000..c434d22fa6b
--- /dev/null
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/LoadXMLStatementAssert.java
@@ -0,0 +1,53 @@
+/*
+ * 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.test.it.sql.parser.internal.asserts.statement.dml.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLLoadXMLStatement;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.segment.table.TableAssert;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.LoadXMLStatementTestCase;
+
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+/**
+ * Load xml statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class LoadXMLStatementAssert {
+    
+    /**
+     * Assert load xml statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual load xml statement
+     * @param expected expected load xml statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final MySQLLoadXMLStatement actual, final LoadXMLStatementTestCase expected) {
+        assertTable(assertContext, actual, expected);
+    }
+    
+    private static void assertTable(final SQLCaseAssertContext assertContext, final MySQLLoadXMLStatement actual, final LoadXMLStatementTestCase expected) {
+        if (null == expected.getTable()) {
+            assertNull(actual.getTableSegment(), assertContext.getText("Actual table should not exist."));
+        } else {
+            TableAssert.assertIs(assertContext, actual.getTableSegment(), expected.getTable());
+        }
+    }
+}
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/RootSQLParserTestCases.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/RootSQLParserTestCases.java
index 87426f68b71..3f6505075dc 100644
--- a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/RootSQLParserTestCases.java
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/RootSQLParserTestCases.java
@@ -281,6 +281,8 @@ import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.s
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.DeleteStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.DoStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.InsertStatementTestCase;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.LoadDataStatementTestCase;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.LoadXMLStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.LockTableStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.MergeStatementTestCase;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml.SelectStatementTestCase;
@@ -821,6 +823,12 @@ public final class RootSQLParserTestCases {
     @XmlElement(name = "copy")
     private final List<CopyStatementTestCase> copyTestCases = new LinkedList<>();
     
+    @XmlElement(name = "load-data")
+    private final List<LoadDataStatementTestCase> loadDataTestCases = new LinkedList<>();
+    
+    @XmlElement(name = "load-xml")
+    private final List<LoadXMLStatementTestCase> loadXmlTestCases = new LinkedList<>();
+    
     @XmlElement(name = "xa")
     private final List<XATestCase> xaTestCases = new LinkedList<>();
     
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/statement/dml/LoadDataStatementTestCase.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/statement/dml/LoadDataStatementTestCase.java
new file mode 100644
index 00000000000..66ced3810bf
--- /dev/null
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/statement/dml/LoadDataStatementTestCase.java
@@ -0,0 +1,36 @@
+/*
+ * 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.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.SQLParserTestCase;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.segment.impl.table.ExpectedSimpleTable;
+
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * Load data statement test case.
+ */
+@Getter
+@Setter
+public final class LoadDataStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement
+    private ExpectedSimpleTable table;
+}
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/statement/dml/LoadXMLStatementTestCase.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/statement/dml/LoadXMLStatementTestCase.java
new file mode 100644
index 00000000000..87973704200
--- /dev/null
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/statement/dml/LoadXMLStatementTestCase.java
@@ -0,0 +1,36 @@
+/*
+ * 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.test.it.sql.parser.internal.cases.parser.jaxb.statement.dml;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.SQLParserTestCase;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.segment.impl.table.ExpectedSimpleTable;
+
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * Load xml statement test case.
+ */
+@Getter
+@Setter
+public final class LoadXMLStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement
+    private ExpectedSimpleTable table;
+}
diff --git a/test/it/parser/src/main/resources/case/dml/load-data.xml b/test/it/parser/src/main/resources/case/dml/load-data.xml
new file mode 100644
index 00000000000..f33468ea1c5
--- /dev/null
+++ b/test/it/parser/src/main/resources/case/dml/load-data.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<sql-parser-test-cases>
+    <load-data sql-case-id="load_data_into_table_from_server_file">
+        <table name="t_order" start-index="45" stop-index="51" />
+    </load-data>
+    <load-data sql-case-id="load_data_into_table_from_local_file">
+        <table name="t_order" start-index="51" stop-index="57" />
+    </load-data>
+    <load-data sql-case-id="load_data_into_table_with_schema_name">
+        <table name="t_order" start-index="45" stop-index="63">
+            <owner name="sharding_db" start-index="45" stop-index="55" />
+        </table>
+    </load-data>
+    <load-data sql-case-id="load_data_into_table_with_lines_starting">
+        <table name="t_order" start-index="44" stop-index="50" />
+    </load-data>
+    <load-data sql-case-id="load_data_into_table_with_ignore_lines">
+        <table name="t_order" start-index="44" stop-index="50" />
+    </load-data>
+</sql-parser-test-cases>
diff --git a/test/it/parser/src/main/resources/case/dml/load-xml.xml b/test/it/parser/src/main/resources/case/dml/load-xml.xml
new file mode 100644
index 00000000000..df79b4762eb
--- /dev/null
+++ b/test/it/parser/src/main/resources/case/dml/load-xml.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<sql-parser-test-cases>
+    <load-xml sql-case-id="load_xml_into_table_from_server_file">
+        <table name="t_order" start-index="44" stop-index="50" />
+    </load-xml>
+    <load-xml sql-case-id="load_xml_into_table_from_local_file">
+        <table name="t_order" start-index="50" stop-index="56" />
+    </load-xml>
+    <load-xml sql-case-id="load_xml_into_table_with_schema_name">
+        <table name="t_order" start-index="44" stop-index="62">
+            <owner name="sharding_db" start-index="44" stop-index="54" />
+        </table>
+    </load-xml>
+    <load-xml sql-case-id="load_xml_into_table_with_ignore_lines">
+        <table name="t_order" start-index="43" stop-index="49" />
+    </load-xml>
+</sql-parser-test-cases>
diff --git a/test/it/parser/src/main/resources/sql/supported/dml/load-data.xml b/test/it/parser/src/main/resources/sql/supported/dml/load-data.xml
new file mode 100644
index 00000000000..b6262af5197
--- /dev/null
+++ b/test/it/parser/src/main/resources/sql/supported/dml/load-data.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<sql-cases>
+    <sql-case id="load_data_into_table_from_server_file" value="LOAD DATA INFILE '/temp/test.txt' INTO TABLE t_order" db-types="MySQL" />
+    <sql-case id="load_data_into_table_from_local_file" value="LOAD DATA LOCAL INFILE '/temp/test.txt' INTO TABLE t_order" db-types="MySQL" />
+    <sql-case id="load_data_into_table_with_schema_name" value="LOAD DATA INFILE '/temp/test.txt' INTO TABLE sharding_db.t_order" db-types="MySQL" />
+    <sql-case id="load_data_into_table_with_lines_starting" value="LOAD DATA INFILE '/tmp/test.txt' INTO TABLE t_order FIELDS TERMINATED BY ','  LINES STARTING BY 'xxx'" db-types="MySQL" />
+    <sql-case id="load_data_into_table_with_ignore_lines" value="LOAD DATA INFILE '/tmp/test.txt' INTO TABLE t_order IGNORE 1 LINES" db-types="MySQL" />
+</sql-cases>
diff --git a/test/it/parser/src/main/resources/sql/supported/dml/load-xml.xml b/test/it/parser/src/main/resources/sql/supported/dml/load-xml.xml
new file mode 100644
index 00000000000..0c7fe4656c5
--- /dev/null
+++ b/test/it/parser/src/main/resources/sql/supported/dml/load-xml.xml
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<sql-cases>
+    <sql-case id="load_xml_into_table_from_server_file" value="LOAD XML INFILE '/temp/test.xml' INTO TABLE t_order" db-types="MySQL" />
+    <sql-case id="load_xml_into_table_from_local_file" value="LOAD XML LOCAL INFILE '/temp/test.xml' INTO TABLE t_order" db-types="MySQL" />
+    <sql-case id="load_xml_into_table_with_schema_name" value="LOAD XML INFILE '/temp/test.xml' INTO TABLE sharding_db.t_order" db-types="MySQL" />
+    <sql-case id="load_xml_into_table_with_ignore_lines" value="LOAD XML INFILE '/tmp/test.xml' INTO TABLE t_order IGNORE 1 LINES" db-types="MySQL" />
+</sql-cases>