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 2022/07/05 00:37:45 UTC

[shardingsphere] branch master updated: support PREPARE TRANSACTION statement in PostgreSQL (#18826)

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 16f3e46b04d support PREPARE TRANSACTION statement in PostgreSQL (#18826)
16f3e46b04d is described below

commit 16f3e46b04daa504a6cb6e2e71db9b3936fd1bb5
Author: RunQi Zhao <40...@users.noreply.github.com>
AuthorDate: Tue Jul 5 08:37:39 2022 +0800

    support PREPARE TRANSACTION statement in PostgreSQL (#18826)
    
    * support PREPARE TRANSACTION statement
    
    * support PREPARE TRANSACTION statement
---
 .../main/antlr4/imports/postgresql/TCLStatement.g4 |  8 ++---
 .../parser/autogen/PostgreSQLStatementParser.g4    |  1 +
 .../impl/PostgreSQLTCLStatementSQLVisitor.java     |  7 ++++
 .../core/database/visitor/SQLVisitorRule.java      |  2 ++
 .../statement/tcl/PrepareTransactionStatement.java | 26 ++++++++++++++
 .../tcl/PostgreSQLPrepareTransactionStatement.java | 29 +++++++++++++++
 .../asserts/statement/tcl/TCLStatementAssert.java  |  5 +++
 .../impl/PrepareTransactionStatementAssert.java    | 41 ++++++++++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  5 +++
 .../statement/tcl/PrepareTransactionTestCase.java  | 26 ++++++++++++++
 .../resources/case/tcl/prepare-transaction.xml     | 21 +++++++++++
 .../sql/supported/tcl/prepare-transaction.xml      | 21 +++++++++++
 .../main/resources/sql/unsupported/unsupported.xml | 22 ------------
 13 files changed, 188 insertions(+), 26 deletions(-)

diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/TCLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/TCLStatement.g4
index 10155a45e0f..8dd368fde12 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/TCLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/TCLStatement.g4
@@ -64,10 +64,6 @@ rollbackToSavepoint
     : ROLLBACK (WORK | TRANSACTION)? TO SAVEPOINT? colId
     ;
 
-prepareTransaction
-    : PREPARE TRANSACTION STRING_
-    ;
-
 commitPrepared
     : COMMIT PREPARED STRING_
     ;
@@ -102,3 +98,7 @@ lockType
     | EXCLUSIVE
     | ACCESS EXCLUSIVE
     ;
+
+prepareTransaction
+    : PREPARE TRANSACTION STRING_
+    ;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/PostgreSQLStatementParser.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/PostgreSQLStatementParser.g4
index 29c1aaf9427..d97e1491295 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/PostgreSQLStatementParser.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/PostgreSQLStatementParser.g4
@@ -162,5 +162,6 @@ execute
     | alterRoutine
     | alterRule
     | createCollation
+    | prepareTransaction
     ) SEMI_? EOF
     ;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLTCLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLTCLStatementSQLVisitor.java
index b0754b22770..ac42fbc66fc 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLTCLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLTCLStatementSQLVisitor.java
@@ -36,11 +36,13 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Sa
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.SetConstraintsContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.SetTransactionContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.StartTransactionContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.PrepareTransactionContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLBeginTransactionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLCommitPreparedStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLCommitStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLLockStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLPrepareTransactionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLReleaseSavepointStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLRollbackPreparedStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLRollbackStatement;
@@ -154,4 +156,9 @@ public final class PostgreSQLTCLStatementSQLVisitor extends PostgreSQLStatementS
         }
         return result;
     }
+    
+    @Override
+    public ASTNode visitPrepareTransaction(final PrepareTransactionContext ctx) {
+        return new PostgreSQLPrepareTransactionStatement();
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
index 509139555a7..caedb134202 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
@@ -590,6 +590,8 @@ public enum SQLVisitorRule {
     
     DO("DoStatement", SQLStatementType.DML),
     
+    PREPARE_TRANSACTION("PrepareTransaction", SQLStatementType.TCL),
+    
     CREATE_COLLATION("CreateCollation", SQLStatementType.DDL);
     
     private final String name;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/tcl/PrepareTransactionStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/tcl/PrepareTransactionStatement.java
new file mode 100644
index 00000000000..a90d799843f
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/tcl/PrepareTransactionStatement.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sql.parser.sql.common.statement.tcl;
+
+import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
+
+/**
+ * Prepare transaction statement.
+ */
+public abstract class PrepareTransactionStatement extends AbstractSQLStatement implements TCLStatement {
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/tcl/PostgreSQLPrepareTransactionStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/tcl/PostgreSQLPrepareTransactionStatement.java
new file mode 100644
index 00000000000..74ea106a485
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/tcl/PostgreSQLPrepareTransactionStatement.java
@@ -0,0 +1,29 @@
+/*
+ * 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.postgresql.tcl;
+
+import lombok.ToString;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.PrepareTransactionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.PostgreSQLStatement;
+
+/**
+ * PostgreSQL prepare transaction statement.
+ */
+@ToString
+public final class PostgreSQLPrepareTransactionStatement extends PrepareTransactionStatement implements PostgreSQLStatement {
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/TCLStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/TCLStatementAssert.java
index 3756899e9ee..0a32cbb8199 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/TCLStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/TCLStatementAssert.java
@@ -22,6 +22,7 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.BeginTransactionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.CommitStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.LockStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.PrepareTransactionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.RollbackStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.SavepointStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.SetAutoCommitStatement;
@@ -33,6 +34,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAs
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.tcl.impl.BeginTransactionStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.tcl.impl.CommitStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.tcl.impl.LockStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.tcl.impl.PrepareTransactionStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.tcl.impl.RollbackStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.tcl.impl.SavepointStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.tcl.impl.SetAutoCommitStatementAssert;
@@ -43,6 +45,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.BeginTransactionStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.CommitStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.LockStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.PrepareTransactionTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.RollbackStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.SavepointStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.SetAutoCommitStatementTestCase;
@@ -82,6 +85,8 @@ public final class TCLStatementAssert {
             UnlockStatementAssert.assertIs(assertContext, (UnlockStatement) actual, (UnlockStatementTestCase) expected);
         } else if (actual instanceof LockStatement) {
             LockStatementAssert.assertIs(assertContext, (LockStatement) actual, (LockStatementTestCase) expected);
+        } else if (actual instanceof PrepareTransactionStatement) {
+            PrepareTransactionStatementAssert.assertIs(assertContext, (PrepareTransactionStatement) actual, (PrepareTransactionTestCase) expected);
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/impl/PrepareTransactionStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/impl/PrepareTransactionStatementAssert.java
new file mode 100644
index 00000000000..c8c09ea483d
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/impl/PrepareTransactionStatementAssert.java
@@ -0,0 +1,41 @@
+/*
+ * 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.sql.parser.parameterized.asserts.statement.tcl.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.PrepareTransactionStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.PrepareTransactionTestCase;
+
+/**
+ * Prepare transaction statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class PrepareTransactionStatementAssert {
+    
+    /**
+     * Assert prepare transaction statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual prepare transaction statement
+     * @param expected expected prepare transaction statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final PrepareTransactionStatement actual, final PrepareTransactionTestCase expected) {
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
index 75d89f27941..eec0b2454d3 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
@@ -377,6 +377,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.BeginTransactionStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.CommitStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.LockStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.PrepareTransactionTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.RollbackStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.SavepointStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.SetAutoCommitStatementTestCase;
@@ -1500,6 +1501,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "create-collation")
     private final List<CreateCollationStatementTestCase> createCollationStatementTestCases = new LinkedList<>();
     
+    @XmlElement(name = "prepare-transaction")
+    private final List<PrepareTransactionTestCase> prepareTransactionTestCases = new LinkedList<>();
+    
     /**
      * Get all SQL parser test cases.
      *
@@ -1874,6 +1878,7 @@ public final class SQLParserTestCases {
         putAll(clusterStatementTestCases, result);
         putAll(createAccessMethodTestCases, result);
         putAll(createCollationStatementTestCases, result);
+        putAll(prepareTransactionTestCases, result);
         return result;
     }
     // CHECKSTYLE:ON
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/tcl/PrepareTransactionTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/tcl/PrepareTransactionTestCase.java
new file mode 100644
index 00000000000..9eec901fd6d
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/tcl/PrepareTransactionTestCase.java
@@ -0,0 +1,26 @@
+/*
+ * 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.sql.parser.parameterized.jaxb.cases.domain.statement.tcl;
+
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+
+/**
+ * Prepare transaction statement test case.
+ */
+public final class PrepareTransactionTestCase extends SQLParserTestCase {
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/tcl/prepare-transaction.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/tcl/prepare-transaction.xml
new file mode 100644
index 00000000000..9d4992f2800
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/tcl/prepare-transaction.xml
@@ -0,0 +1,21 @@
+<?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>
+    <prepare-transaction sql-case-id="prepare_transaction" />
+</sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/tcl/prepare-transaction.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/tcl/prepare-transaction.xml
new file mode 100644
index 00000000000..e9718c38f7a
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/tcl/prepare-transaction.xml
@@ -0,0 +1,21 @@
+<?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="prepare_transaction" value="PREPARE TRANSACTION 'foo1';" db-types="PostgreSQL" />
+</sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
index 854fb9f06c0..3aef72092cf 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
@@ -4627,18 +4627,6 @@
     <sql-case id="insert_by_postgresql_source_test_case395" value="INSERT INTO truncate_b DEFAULT VALUES;" db-types="PostgreSQL" />
     <sql-case id="insert_by_postgresql_source_test_case396" value="INSERT INTO truncate_b DEFAULT VALUES;" db-types="PostgreSQL" />
     <sql-case id="insert_by_postgresql_source_test_case428" value="INSERT INTO y2 (SELECT x, md5(x::text) FROM generate_series(0,20) x);" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case1" value="PREPARE TRANSACTION &apos;foo1&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case2" value="PREPARE TRANSACTION &apos;foo2&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case3" value="PREPARE TRANSACTION &apos;foo3&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case4" value="PREPARE TRANSACTION &apos;foo3&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case5" value="PREPARE TRANSACTION &apos;foo4&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case6" value="PREPARE TRANSACTION &apos;foo5&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case7" value="PREPARE TRANSACTION &apos;foo6&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case8" value="PREPARE TRANSACTION &apos;foobar&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case9" value="PREPARE TRANSACTION &apos;foobar&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case10" value="PREPARE TRANSACTION &apos;regress-one&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case11" value="PREPARE TRANSACTION &apos;regress-two&apos;;" db-types="PostgreSQL" />
-    <sql-case id="prepare_by_postgresql_source_test_case12" value="PREPARE TRANSACTION &apos;twophase_search&apos;;" db-types="PostgreSQL" />
     <sql-case id="reassign_by_postgresql_source_test_case1" value="REASSIGN OWNED BY regress_dep_user0 TO regress_dep_user1;" db-types="PostgreSQL" />
     <sql-case id="reassign_by_postgresql_source_test_case2" value="REASSIGN OWNED BY regress_dep_user1 TO regress_dep_user0;" db-types="PostgreSQL" />
     <sql-case id="reassign_by_postgresql_source_test_case3" value="REASSIGN OWNED BY regress_dep_user1 TO regress_dep_user2;" db-types="PostgreSQL" />
@@ -6872,16 +6860,6 @@
     <sql-case id="low_prepare_by_postgresql_source_test_case3" value="prepare q as   select &apos;some\more_text&apos; as &quot;a$title&quot;, E&apos;  #&lt;foo&gt;%&amp;^~|\n{bar}&apos; as &quot;junk&quot;,          &apos;   &apos; as &quot;empty&quot;, n as int   from generate_series(1,2) as n;" db-types="PostgreSQL" />
     <sql-case id="low_prepare_by_postgresql_source_test_case4" value="prepare q as   select &apos;some\text&apos; as &quot;a\title&quot;, E&apos;  &lt;foo&gt;\n&lt;bar&gt;&apos; as &quot;junk&quot;,          &apos;   &apos; as &quot;empty&quot;, n as int   from generate_series(1,2) as n;" db-types="PostgreSQL" />
     <sql-case id="low_prepare_by_postgresql_source_test_case5" value="prepare q as   select &apos;some|text&apos; as &quot;a|title&quot;, &apos;        &apos; as &quot;empty &quot;, n as int   from generate_series(1,2) as n;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case6" value="prepare transaction &apos;twophase_func&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case7" value="prepare transaction &apos;twophase_func&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case8" value="prepare transaction &apos;twophase_operator&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case9" value="prepare transaction &apos;twophase_sequence&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case10" value="prepare transaction &apos;twophase_tab&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case11" value="prepare transaction &apos;twophase_tab&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case12" value="prepare transaction &apos;twophase_tab&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case13" value="prepare transaction &apos;twophase_tab&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case14" value="prepare transaction &apos;twophase_type&apos;;" db-types="PostgreSQL" />
-    <sql-case id="low_prepare_by_postgresql_source_test_case15" value="prepare transaction &apos;twophase_view&apos;;" db-types="PostgreSQL" />
     <sql-case id="low_refresh_by_postgresql_source_test_case1" value="refresh materialized view concurrently parallel_mat_view;" db-types="PostgreSQL" />
     <sql-case id="low_refresh_by_postgresql_source_test_case2" value="refresh materialized view mvtest_error;" db-types="PostgreSQL" />
     <sql-case id="low_refresh_by_postgresql_source_test_case3" value="refresh materialized view parallel_mat_view;" db-types="PostgreSQL" />