You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2023/04/14 05:10:25 UTC

[shardingsphere] branch master updated: Add xa transaction data source privilege checker (#25147)

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

duanzhengqiang 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 a06d032ad04 Add xa transaction data source privilege checker (#25147)
a06d032ad04 is described below

commit a06d032ad040d8f1a97d7144233cc1371b381acc
Author: ZhangCheng <fl...@outlook.com>
AuthorDate: Fri Apr 14 13:10:08 2023 +0800

    Add xa transaction data source privilege checker (#25147)
    
    * Add xa transaction data source privilege checker
    
    * fix
    
    * fix
    
    * fix
    
    * fix
    
    * fix
---
 docs/document/content/dev-manual/transaction.cn.md | 17 ++++
 docs/document/content/dev-manual/transaction.en.md | 17 ++++
 .../user-manual/error-code/sql-error-code.cn.md    |  3 +-
 .../user-manual/error-code/sql-error-code.en.md    |  2 +
 .../xa/XAShardingSphereTransactionManager.java     |  2 +
 .../checker/DataSourcePrivilegeChecker.java        | 35 ++++++++
 .../checker/DefaultDataSourcePrivilegeChecker.java | 35 ++++++++
 .../dialect/MySQLDataSourcePrivilegeChecker.java   | 82 ++++++++++++++++++
 ...XATransactionCheckPrivilegeFailedException.java | 35 ++++++++
 .../exception/XATransactionPrivilegeException.java | 33 ++++++++
 ...a.datasource.checker.DataSourcePrivilegeChecker | 19 +++++
 .../MySQLDataSourcePrivilegeCheckerTest.java       | 99 ++++++++++++++++++++++
 12 files changed, 378 insertions(+), 1 deletion(-)

diff --git a/docs/document/content/dev-manual/transaction.cn.md b/docs/document/content/dev-manual/transaction.cn.md
index 748ad11e273..2fa01bcbb90 100644
--- a/docs/document/content/dev-manual/transaction.cn.md
+++ b/docs/document/content/dev-manual/transaction.cn.md
@@ -76,3 +76,20 @@ XA 分布式事务管理器
 | *配置标识*                             | *详细说明*                 | *全限定类名*                                                                                                                                                                                                                                                                                                    |
 |------------------------------------|------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
 | com.zaxxer.hikari.HikariDataSource | 用于获取 HikariCP 连接池的标准属性 | [`org.apache.shardingsphere.transaction.xa.jta.datasource.swapper.impl.HikariCPPropertyProvider`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/swapper/impl/HikariCPPropertyProvider.java) | |
+
+## DataSourcePrivilegeChecker
+
+### 全限定类名
+
+[`org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DataSourcePrivilegeChecker`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DataSourcePrivilegeChecker.java)
+
+### 定义
+
+用于通过数据源校验事务所需的权限
+
+### 已知实现
+
+| *配置标识*  | *详细说明*           | *全限定类名*                                                                                                                                                                                                                                                                                                                              |
+|---------|------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| MySQL   | 校验 MySQL XA 事务所需的权限 | [`org.apache.shardingsphere.transaction.xa.jta.datasource.checker.dialect.MySQLDataSourcePrivilegeChecker`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/dialect/MySQLDataSourcePrivilegeChecker.java) |         |
+| Default | 默认不校验事务所需的权限     | [`org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DefaultDataSourcePrivilegeChecker`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/DefaultDataSourcePrivilegeChecker.java) |                 |
diff --git a/docs/document/content/dev-manual/transaction.en.md b/docs/document/content/dev-manual/transaction.en.md
index ebd90c3a51b..4fce0ffefa6 100644
--- a/docs/document/content/dev-manual/transaction.en.md
+++ b/docs/document/content/dev-manual/transaction.en.md
@@ -76,3 +76,20 @@ Data source property provider service definition
 | *Configuration Type*               | *Description*                               | *Fully-qualified class name*                                                                                                                                                                                                                                                                             |
 |------------------------------------|---------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
 | com.zaxxer.hikari.HikariDataSource | Used to get standard properties of HikariCP | [`org.apache.shardingsphere.transaction.xa.jta.datasource.swapper.impl.HikariCPPropertyProvider`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/swapper/impl/HikariCPPropertyProvider.java) |
+
+## DataSourcePrivilegeChecker
+
+### Fully-qualified class name
+
+[`org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DataSourcePrivilegeChecker`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DataSourcePrivilegeChecker.java)
+
+### Definition
+
+Check transaction privilege through the data source
+
+### Implementation classes
+
+| *配置标识*  | *详细说明*                                | *全限定类名*                                                                                                                                                                                                                                                                                                                              |
+|---------|---------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| MySQL   | Verify MySQL XA transaction privilege | [`org.apache.shardingsphere.transaction.xa.jta.datasource.checker.dialect.MySQLDataSourcePrivilegeChecker`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/dialect/MySQLDataSourcePrivilegeChecker.java) |         |
+| Default | Not verify transaction privilege by default    | [`org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DefaultDataSourcePrivilegeChecker`](https://github.com/apache/shardingsphere/blob/master/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/DefaultDataSourcePrivilegeChecker.java) |                 |
diff --git a/docs/document/content/user-manual/error-code/sql-error-code.cn.md b/docs/document/content/user-manual/error-code/sql-error-code.cn.md
index 2b00b54068b..0d9623d6214 100644
--- a/docs/document/content/user-manual/error-code/sql-error-code.cn.md
+++ b/docs/document/content/user-manual/error-code/sql-error-code.cn.md
@@ -70,6 +70,8 @@ SQL 错误码以标准的 SQL State,Vendor Code 和详细错误信息提供,
 | 25000     | 14200       | Can not start new XA transaction in a active transaction.                          |
 | 25000     | 14201       | Failed to create \`%s\` XA data source.                                            |
 | 25000     | 14202       | Max length of xa unique resource name \`%s\` exceeded: should be less than 45.     |
+| 25000     | 14203       | Check privileges failed on data source, reason is: \`%s\`     |
+| 25000     | 14204       | Failed to create XA transaction manager, requires \`%s\` privileges    |
 | 25000     | 14301       | ShardingSphere Seata-AT transaction has been disabled.                             |
 | 25000     | 14302       | Please config application id within seata.conf file.                               |
 
@@ -226,7 +228,6 @@ SQL 错误码以标准的 SQL State,Vendor Code 和详细错误信息提供,
 | 42S02     | 20296       | Not found load balance type in database \`%s\.`                                             |
 | 44000     | 20297       | Weight load balancer datasource name config does not match data sources in database \`%s\.` |
 
-
 ### 数据库高可用
 
 | SQL State | Vendor Code | 错误信息                                                                         |
diff --git a/docs/document/content/user-manual/error-code/sql-error-code.en.md b/docs/document/content/user-manual/error-code/sql-error-code.en.md
index 0ed3450df9b..62c465f7b56 100644
--- a/docs/document/content/user-manual/error-code/sql-error-code.en.md
+++ b/docs/document/content/user-manual/error-code/sql-error-code.en.md
@@ -70,6 +70,8 @@ SQL error codes provide by standard `SQL State`, `Vendor Code` and `Reason`, whi
 | 25000     | 14200       | Can not start new XA transaction in a active transaction.                          |
 | 25000     | 14201       | Failed to create \`%s\` XA data source.                                            |
 | 25000     | 14202       | Max length of xa unique resource name \`%s\` exceeded: should be less than 45.     |
+| 25000     | 14203       | Check privileges failed on data source, reason is: \`%s\`     |
+| 25000     | 14204       | Failed to create XA transaction manager, requires \`%s\` privileges    |
 | 25000     | 14301       | ShardingSphere Seata-AT transaction has been disabled.                             |
 | 25000     | 14302       | Please config application id within seata.conf file.                               |
 
diff --git a/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/XAShardingSphereTransactionManager.java b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/XAShardingSphereTransactionManager.java
index 35308dcff28..c038767f1b3 100644
--- a/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/XAShardingSphereTransactionManager.java
+++ b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/XAShardingSphereTransactionManager.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.transaction.core.ResourceDataSource;
 import org.apache.shardingsphere.transaction.exception.TransactionTimeoutException;
 import org.apache.shardingsphere.transaction.spi.ShardingSphereTransactionManager;
 import org.apache.shardingsphere.transaction.xa.jta.datasource.XATransactionDataSource;
+import org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DataSourcePrivilegeChecker;
 import org.apache.shardingsphere.transaction.xa.spi.XATransactionManagerProvider;
 
 import javax.sql.DataSource;
@@ -54,6 +55,7 @@ public final class XAShardingSphereTransactionManager implements ShardingSphereT
     
     @Override
     public void init(final Map<String, DatabaseType> databaseTypes, final Map<String, DataSource> dataSources, final String providerType) {
+        dataSources.forEach((key, value) -> TypedSPILoader.getService(DataSourcePrivilegeChecker.class, databaseTypes.get(key).getType()).checkPrivilege(value));
         xaTransactionManagerProvider = TypedSPILoader.getService(XATransactionManagerProvider.class, providerType);
         xaTransactionManagerProvider.init();
         Map<String, ResourceDataSource> resourceDataSources = getResourceDataSources(dataSources);
diff --git a/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DataSourcePrivilegeChecker.java b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DataSourcePrivilegeChecker.java
new file mode 100644
index 00000000000..64f754cbfa0
--- /dev/null
+++ b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DataSourcePrivilegeChecker.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.transaction.xa.jta.datasource.checker;
+
+import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPI;
+
+import javax.sql.DataSource;
+
+/**
+ * Data source privilege checker.
+ */
+public interface DataSourcePrivilegeChecker extends TypedSPI {
+    
+    /**
+     * Check privilege.
+     *
+     * @param dataSource data source
+     */
+    void checkPrivilege(DataSource dataSource);
+}
diff --git a/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DefaultDataSourcePrivilegeChecker.java b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DefaultDataSourcePrivilegeChecker.java
new file mode 100644
index 00000000000..f6ad1b80d1e
--- /dev/null
+++ b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/DefaultDataSourcePrivilegeChecker.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.transaction.xa.jta.datasource.checker;
+
+import javax.sql.DataSource;
+
+/**
+ * Default data source privilege checker.
+ */
+public final class DefaultDataSourcePrivilegeChecker implements DataSourcePrivilegeChecker {
+    
+    @Override
+    public void checkPrivilege(final DataSource dataSource) {
+    }
+    
+    @Override
+    public boolean isDefault() {
+        return true;
+    }
+}
diff --git a/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/dialect/MySQLDataSourcePrivilegeChecker.java b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/dialect/MySQLDataSourcePrivilegeChecker.java
new file mode 100644
index 00000000000..e380dfc870f
--- /dev/null
+++ b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/dialect/MySQLDataSourcePrivilegeChecker.java
@@ -0,0 +1,82 @@
+/*
+ * 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.transaction.xa.jta.datasource.checker.dialect;
+
+import org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DataSourcePrivilegeChecker;
+import org.apache.shardingsphere.transaction.xa.jta.exception.XATransactionCheckPrivilegeFailedException;
+import org.apache.shardingsphere.transaction.xa.jta.exception.XATransactionPrivilegeException;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+
+/**
+ * MySQL data source privilege checker.
+ */
+public final class MySQLDataSourcePrivilegeChecker implements DataSourcePrivilegeChecker {
+    
+    private static final String SHOW_GRANTS_SQL = "SHOW GRANTS";
+    
+    private static final String[][] REQUIRED_PRIVILEGES = {{"ALL PRIVILEGES", "ON *.*"}, {"XA_RECOVER_ADMIN", "ON *.*"}};
+    
+    private static final int MYSQL_MAJOR_VERSION_8 = 8;
+    
+    /**
+     * Check privilege.
+     *
+     * @param dataSource data source
+     * @throws XATransactionCheckPrivilegeFailedException XA transaction check privilege failed exception
+     */
+    public void checkPrivilege(final DataSource dataSource) {
+        try (Connection connection = dataSource.getConnection()) {
+            if (MYSQL_MAJOR_VERSION_8 == connection.getMetaData().getDatabaseMajorVersion()) {
+                checkPrivilege(connection);
+            }
+        } catch (final SQLException ex) {
+            throw new XATransactionCheckPrivilegeFailedException(ex);
+        }
+    }
+    
+    private void checkPrivilege(final Connection connection) {
+        try (
+                PreparedStatement preparedStatement = connection.prepareStatement(SHOW_GRANTS_SQL);
+                ResultSet resultSet = preparedStatement.executeQuery()) {
+            while (resultSet.next()) {
+                String privilege = resultSet.getString(1).toUpperCase();
+                if (matchPrivileges(privilege)) {
+                    return;
+                }
+            }
+        } catch (final SQLException ex) {
+            throw new XATransactionCheckPrivilegeFailedException(ex);
+        }
+        throw new XATransactionPrivilegeException("XA_RECOVER_ADMIN");
+    }
+    
+    private boolean matchPrivileges(final String privilege) {
+        return Arrays.stream(REQUIRED_PRIVILEGES).anyMatch(each -> Arrays.stream(each).allMatch(privilege::contains));
+    }
+    
+    @Override
+    public String getType() {
+        return "MySQL";
+    }
+}
diff --git a/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/exception/XATransactionCheckPrivilegeFailedException.java b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/exception/XATransactionCheckPrivilegeFailedException.java
new file mode 100644
index 00000000000..dc48dd2c671
--- /dev/null
+++ b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/exception/XATransactionCheckPrivilegeFailedException.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.transaction.xa.jta.exception;
+
+import org.apache.shardingsphere.infra.util.exception.external.sql.sqlstate.XOpenSQLState;
+import org.apache.shardingsphere.transaction.exception.TransactionSQLException;
+
+import java.sql.SQLException;
+
+/**
+ * XA transaction check privilege failed exception.
+ */
+public final class XATransactionCheckPrivilegeFailedException extends TransactionSQLException {
+    
+    private static final long serialVersionUID = 6073175429050058508L;
+    
+    public XATransactionCheckPrivilegeFailedException(final SQLException cause) {
+        super(XOpenSQLState.INVALID_TRANSACTION_STATE, 203, String.format("Check privileges failed on data source, reason is: %s", cause.getMessage()), cause);
+    }
+}
diff --git a/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/exception/XATransactionPrivilegeException.java b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/exception/XATransactionPrivilegeException.java
new file mode 100644
index 00000000000..bc796df5eee
--- /dev/null
+++ b/kernel/transaction/type/xa/core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/exception/XATransactionPrivilegeException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.transaction.xa.jta.exception;
+
+import org.apache.shardingsphere.infra.util.exception.external.sql.sqlstate.XOpenSQLState;
+import org.apache.shardingsphere.transaction.exception.TransactionSQLException;
+
+/**
+ * XA transaction privilege exception.
+ */
+public final class XATransactionPrivilegeException extends TransactionSQLException {
+    
+    private static final long serialVersionUID = -1565168229743080642L;
+    
+    public XATransactionPrivilegeException(final String errorMessage) {
+        super(XOpenSQLState.INVALID_TRANSACTION_STATE, 204, "Failed to create XA transaction manager, requires `%s` privileges", errorMessage);
+    }
+}
diff --git a/kernel/transaction/type/xa/core/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DataSourcePrivilegeChecker b/kernel/transaction/type/xa/core/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DataSourcePrivilegeChecker
new file mode 100644
index 00000000000..d67dbc8ae46
--- /dev/null
+++ b/kernel/transaction/type/xa/core/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DataSourcePrivilegeChecker
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.transaction.xa.jta.datasource.checker.DefaultDataSourcePrivilegeChecker
+org.apache.shardingsphere.transaction.xa.jta.datasource.checker.dialect.MySQLDataSourcePrivilegeChecker
diff --git a/kernel/transaction/type/xa/core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/MySQLDataSourcePrivilegeCheckerTest.java b/kernel/transaction/type/xa/core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/MySQLDataSourcePrivilegeCheckerTest.java
new file mode 100644
index 00000000000..0050bfcd8f9
--- /dev/null
+++ b/kernel/transaction/type/xa/core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/datasource/checker/MySQLDataSourcePrivilegeCheckerTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.transaction.xa.jta.datasource.checker;
+
+import org.apache.shardingsphere.transaction.xa.jta.datasource.checker.dialect.MySQLDataSourcePrivilegeChecker;
+import org.apache.shardingsphere.transaction.xa.jta.exception.XATransactionCheckPrivilegeFailedException;
+import org.apache.shardingsphere.transaction.xa.jta.exception.XATransactionPrivilegeException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import javax.sql.DataSource;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+class MySQLDataSourcePrivilegeCheckerTest {
+    
+    @Mock
+    private PreparedStatement preparedStatement;
+    
+    @Mock
+    private ResultSet resultSet;
+    
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private DataSource dataSource;
+    
+    @BeforeEach
+    void setUp() throws SQLException {
+        when(dataSource.getConnection().prepareStatement(anyString())).thenReturn(preparedStatement);
+    }
+    
+    @Test
+    void assertCheckPrivilegeWithParticularSuccessInMySQL8() throws SQLException {
+        when(preparedStatement.executeQuery()).thenReturn(resultSet);
+        when(dataSource.getConnection().getMetaData().getDatabaseMajorVersion()).thenReturn(8);
+        when(resultSet.next()).thenReturn(true);
+        when(resultSet.getString(1)).thenReturn("GRANT XA_RECOVER_ADMIN ON *.* TO '%'@'%'");
+        new MySQLDataSourcePrivilegeChecker().checkPrivilege(dataSource);
+        verify(preparedStatement).executeQuery();
+    }
+    
+    @Test
+    void assertUnCheckPrivilegeInMySQL5() throws SQLException {
+        when(dataSource.getConnection().getMetaData().getDatabaseMajorVersion()).thenReturn(5);
+        new MySQLDataSourcePrivilegeChecker().checkPrivilege(dataSource);
+        verify(preparedStatement, times(0)).executeQuery();
+    }
+    
+    @Test
+    void assertCheckPrivilegeWithAllSuccessInMySQL8() throws SQLException {
+        when(preparedStatement.executeQuery()).thenReturn(resultSet);
+        when(dataSource.getConnection().getMetaData().getDatabaseMajorVersion()).thenReturn(8);
+        when(resultSet.next()).thenReturn(true);
+        when(resultSet.getString(1)).thenReturn("GRANT ALL PRIVILEGES ON *.* TO '%'@'%'");
+        new MySQLDataSourcePrivilegeChecker().checkPrivilege(dataSource);
+        verify(preparedStatement).executeQuery();
+    }
+    
+    @Test
+    void assertCheckPrivilegeLackPrivilegesInMySQL8() throws SQLException {
+        when(preparedStatement.executeQuery()).thenReturn(resultSet);
+        when(dataSource.getConnection().getMetaData().getDatabaseMajorVersion()).thenReturn(8);
+        assertThrows(XATransactionPrivilegeException.class, () -> new MySQLDataSourcePrivilegeChecker().checkPrivilege(dataSource));
+    }
+    
+    @Test
+    void assertCheckPrivilegeFailureInMySQL8() throws SQLException {
+        when(preparedStatement.executeQuery()).thenReturn(resultSet);
+        when(dataSource.getConnection().getMetaData().getDatabaseMajorVersion()).thenReturn(8);
+        when(resultSet.next()).thenThrow(new SQLException(""));
+        assertThrows(XATransactionCheckPrivilegeFailedException.class, () -> new MySQLDataSourcePrivilegeChecker().checkPrivilege(dataSource));
+    }
+}