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/04 01:55:19 UTC

[shardingsphere] branch master updated: support lock statement in PostgreSQL (#18789)

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 7c560888d0d support lock statement in PostgreSQL (#18789)
7c560888d0d is described below

commit 7c560888d0d7711709f2e6fdc8118b796bf85476
Author: RunQi Zhao <40...@users.noreply.github.com>
AuthorDate: Mon Jul 4 09:55:12 2022 +0800

    support lock statement in PostgreSQL (#18789)
    
    * support lock statement
    
    * support lock statement
    
    * support lock statement
---
 .../main/antlr4/imports/postgresql/DMLStatement.g4 | 15 -------
 .../main/antlr4/imports/postgresql/TCLStatement.g4 | 15 +++++++
 .../parser/autogen/PostgreSQLStatementParser.g4    |  1 +
 .../impl/PostgreSQLTCLStatementSQLVisitor.java     | 25 +++++++++++
 .../postgresql/tcl/PostgreSQLLockStatement.java    | 39 +++++++++++++++++
 .../statement/tcl/impl/LockStatementAssert.java    | 17 +++++++-
 .../src/main/resources/case/tcl/lock.xml           | 51 ++++++++++++++++++++++
 .../src/main/resources/sql/supported/tcl/lock.xml  | 17 ++++++++
 .../main/resources/sql/unsupported/unsupported.xml | 44 -------------------
 9 files changed, 163 insertions(+), 61 deletions(-)

diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
index 7f810615b73..f762d1fe3ba 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
@@ -443,21 +443,6 @@ dostmtOptItem
     : STRING_ | LANGUAGE nonReservedWordOrSconst
     ;
 
-lock
-    : LOCK TABLE? relationExprList (IN lockType MODE)? NOWAIT?
-    ;
-
-lockType
-    : ACCESS SHARE
-    | ROW SHARE
-    | ROW EXCLUSIVE
-    | SHARE UPDATE EXCLUSIVE
-    | SHARE
-    | SHARE ROW EXCLUSIVE
-    | EXCLUSIVE
-    | ACCESS EXCLUSIVE
-    ;
-
 checkpoint
     : CHECKPOINT
     ;
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 7d3b6a19555..10155a45e0f 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
@@ -87,3 +87,18 @@ constraintsSetMode
 constraintsSetList
     : ALL | qualifiedNameList
     ;
+
+lock
+    : LOCK TABLE? relationExprList (IN lockType MODE)? NOWAIT?
+    ;
+
+lockType
+    : ACCESS SHARE
+    | ROW SHARE
+    | ROW EXCLUSIVE
+    | SHARE UPDATE EXCLUSIVE
+    | SHARE
+    | SHARE ROW EXCLUSIVE
+    | EXCLUSIVE
+    | ACCESS EXCLUSIVE
+    ;
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 a0ff30a9d94..29c1aaf9427 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
@@ -108,6 +108,7 @@ execute
     | notifyStmt
     | unlisten
     | load
+    | lock
     | createTablespace
     | alterTablespace
     | dropTablespace
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 b48bf004192..ffbfc56ae02 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
@@ -26,6 +26,8 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Be
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CommitContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CommitPreparedContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.EndContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.LockContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.RelationExprContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.ReleaseSavepointContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.RollbackContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.RollbackPreparedContext;
@@ -34,9 +36,11 @@ 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.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.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;
@@ -45,6 +49,9 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLSetTransactionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLStartTransactionStatement;
 
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Properties;
 
 /**
@@ -129,4 +136,22 @@ public final class PostgreSQLTCLStatementSQLVisitor extends PostgreSQLStatementS
     public ASTNode visitRollbackPrepared(final RollbackPreparedContext ctx) {
         return new PostgreSQLRollbackPreparedStatement();
     }
+    
+    @Override
+    public ASTNode visitLock(final LockContext ctx) {
+        PostgreSQLLockStatement result = new PostgreSQLLockStatement();
+        if (null != ctx.relationExprList()) {
+            result.setTables(getLockTables(ctx.relationExprList().relationExpr()));
+        }
+        return result;
+    }
+    
+    private Collection<SimpleTableSegment> getLockTables(final List<RelationExprContext> relationExprContexts) {
+        Collection<SimpleTableSegment> result = new LinkedList<>();
+        for (RelationExprContext each : relationExprContexts) {
+            SimpleTableSegment tableSegment = (SimpleTableSegment) visit(each.qualifiedName());
+            result.add(tableSegment);
+        }
+        return result;
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/tcl/PostgreSQLLockStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/tcl/PostgreSQLLockStatement.java
new file mode 100644
index 00000000000..780cbdd0aa5
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/tcl/PostgreSQLLockStatement.java
@@ -0,0 +1,39 @@
+/*
+ * 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.Getter;
+import lombok.Setter;
+import lombok.ToString;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.LockStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.PostgreSQLStatement;
+
+import java.util.Collection;
+import java.util.LinkedList;
+
+/**
+ * PostgreSQL lock statement.
+ */
+@Getter
+@Setter
+@ToString
+public final class PostgreSQLLockStatement extends LockStatement implements PostgreSQLStatement {
+    
+    private Collection<SimpleTableSegment> tables = new LinkedList<>();
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/impl/LockStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/impl/LockStatementAssert.java
index f9f91737797..b590d3146ca 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/impl/LockStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/tcl/impl/LockStatementAssert.java
@@ -22,6 +22,7 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.tcl.LockStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.tcl.MySQLLockStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.tcl.PostgreSQLLockStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.table.TableAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.tcl.LockStatementTestCase;
@@ -34,10 +35,10 @@ import static org.junit.Assert.assertTrue;
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class LockStatementAssert {
-    
+
     /**
      * Assert lock statement is correct with expected parser result.
-     * 
+     *
      * @param assertContext assert context
      * @param actual actual lock statement
      * @param expected expected lock statement test case
@@ -55,6 +56,18 @@ public final class LockStatementAssert {
             } else {
                 assertTrue(assertContext.getText("Actual lock statement should not exist."), lockStatement.getTables().isEmpty());
             }
+        } else if (actual instanceof PostgreSQLLockStatement) {
+            PostgreSQLLockStatement lockStatement = (PostgreSQLLockStatement) actual;
+            if (null != expected.getTables() && !expected.getTables().isEmpty()) {
+                assertFalse(assertContext.getText("Actual lock statement should exist."), lockStatement.getTables().isEmpty());
+                int count = 0;
+                for (SimpleTableSegment each : lockStatement.getTables()) {
+                    TableAssert.assertIs(assertContext, each, expected.getTables().get(count));
+                    count++;
+                }
+            } else {
+                assertTrue(assertContext.getText("Actual lock statement should not exist."), lockStatement.getTables().isEmpty());
+            }
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/tcl/lock.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/tcl/lock.xml
index fd9e4db79cc..c00816b9a44 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/tcl/lock.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/tcl/lock.xml
@@ -46,4 +46,55 @@
         <table name="t1" start-index="12" stop-index="13" />
         <table name="t1" alias="TableAlias" start-index="21" stop-index="36" />
     </lock>
+    <lock sql-case-id="lock_table_with_only">
+        <table name="lock_tbl1" start-index="16" stop-index="24"/>
+    </lock>
+    <lock sql-case-id="lock_table">
+        <table name="fast_emp4000" start-index="11" stop-index="22"/>
+    </lock>
+    <lock sql-case-id="lock_table_access_exclusive">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_access_exclusive_nowait">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_access_share_nowait">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_access_share">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_exclusive_nowait">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_exclusive">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_row_exclusive_nowait">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_row_exclusive">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_row_share_nowait">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_share_nowait">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_share">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_share_row_exclusive">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_share_update_exclusive_nowait">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_table_share_update_exclusive">
+        <table name="lock_tbl1" start-index="11" stop-index="19"/>
+    </lock>
+    <lock sql-case-id="lock_hs1">
+        <table name="hs1" start-index="5" stop-index="7"/>
+    </lock>
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/tcl/lock.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/tcl/lock.xml
index 4a9a81b5997..07afe684cfa 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/tcl/lock.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/tcl/lock.xml
@@ -25,4 +25,21 @@
     <sql-case id="lock_instance_for_backup" value="LOCK INSTANCE FOR BACKUP" db-types="MySQL" />
     <sql-case id="lock_table_with_read_local" value="LOCK TABLES t1 READ LOCAL" db-types="MySQL" />
     <sql-case id="lock_table_with_alias" value="LOCK TABLES t1 READ, t1 as TableAlias READ" db-types="MySQL" />
+    <sql-case id="lock_table_with_only" value="LOCK TABLE ONLY lock_tbl1;" db-types="PostgreSQL" />
+    <sql-case id="lock_table" value="LOCK TABLE fast_emp4000;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_access_exclusive" value="LOCK TABLE lock_tbl1 * IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_access_exclusive_nowait" value="LOCK TABLE lock_tbl1 IN ACCESS EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_access_share_nowait" value="LOCK TABLE lock_tbl1 IN ACCESS SHARE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_access_share" value="LOCK TABLE lock_tbl1 IN ACCESS SHARE MODE;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_exclusive_nowait" value="LOCK TABLE lock_tbl1 IN EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_exclusive" value="LOCK TABLE lock_tbl1 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_row_exclusive_nowait" value="LOCK TABLE lock_tbl1 IN ROW EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_row_exclusive" value="LOCK TABLE lock_tbl1 IN ROW EXCLUSIVE MODE;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_row_share_nowait" value="LOCK TABLE lock_tbl1 IN ROW SHARE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_share_nowait" value="LOCK TABLE lock_tbl1 IN SHARE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_share" value="LOCK TABLE lock_tbl1 IN SHARE MODE;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_share_row_exclusive" value="LOCK TABLE lock_tbl1 IN SHARE ROW EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_share_update_exclusive_nowait" value="LOCK TABLE lock_tbl1 IN SHARE UPDATE EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
+    <sql-case id="lock_table_share_update_exclusive" value="LOCK TABLE lock_tbl1 IN SHARE UPDATE EXCLUSIVE MODE;" db-types="PostgreSQL" />
+    <sql-case id="lock_hs1" value="LOCK hs1;" 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 c7a059218fe..854fb9f06c0 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,47 +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="lock_by_postgresql_source_test_case1" value="LOCK TABLE ONLY lock_tbl1;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case2" value="LOCK TABLE fast_emp4000;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case3" value="LOCK TABLE lock_tbl1 * IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case4" value="LOCK TABLE lock_tbl1 * IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case5" value="LOCK TABLE lock_tbl1 IN ACCESS EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case6" value="LOCK TABLE lock_tbl1 IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case7" value="LOCK TABLE lock_tbl1 IN ACCESS SHARE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case8" value="LOCK TABLE lock_tbl1 IN ACCESS SHARE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case9" value="LOCK TABLE lock_tbl1 IN EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case10" value="LOCK TABLE lock_tbl1 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case11" value="LOCK TABLE lock_tbl1 IN ROW EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case12" value="LOCK TABLE lock_tbl1 IN ROW EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case13" value="LOCK TABLE lock_tbl1 IN ROW SHARE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case14" value="LOCK TABLE lock_tbl1 IN SHARE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case15" value="LOCK TABLE lock_tbl1 IN SHARE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case16" value="LOCK TABLE lock_tbl1 IN SHARE ROW EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case17" value="LOCK TABLE lock_tbl1 IN SHARE UPDATE EXCLUSIVE MODE NOWAIT;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case18" value="LOCK TABLE lock_tbl1 IN SHARE UPDATE EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case19" value="LOCK TABLE lock_tbl2;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case20" value="LOCK TABLE lock_view1 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case21" value="LOCK TABLE lock_view2 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case22" value="LOCK TABLE lock_view2 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case23" value="LOCK TABLE lock_view3 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case24" value="LOCK TABLE lock_view4 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case25" value="LOCK TABLE lock_view5 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case26" value="LOCK TABLE lock_view6 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case27" value="LOCK TABLE lock_view7 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case28" value="LOCK atest1 IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case29" value="LOCK atest2 IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case30" value="LOCK atest2 IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case31" value="LOCK hs1 IN ACCESS EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case32" value="LOCK hs1 IN ACCESS SHARE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case33" value="LOCK hs1 IN EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case34" value="LOCK hs1 IN ROW EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case35" value="LOCK hs1 IN ROW SHARE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case36" value="LOCK hs1 IN SHARE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case37" value="LOCK hs1 IN SHARE ROW EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case38" value="LOCK hs1 IN SHARE UPDATE EXCLUSIVE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case39" value="LOCK hs1;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case40" value="LOCK lock_tbl1 IN ROW SHARE MODE;" db-types="PostgreSQL" />
-    <sql-case id="lock_by_postgresql_source_test_case41" value="LOCK lock_tbl1 IN SHARE ROW EXCLUSIVE MODE;" 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" />
@@ -6908,9 +6867,6 @@
     <sql-case id="low_insert_by_postgresql_source_test_case136" value="insert into trigtest default values;" db-types="PostgreSQL" />
     <sql-case id="low_insert_by_postgresql_source_test_case137" value="insert into trigtest default values;" db-types="PostgreSQL" />
     <sql-case id="low_insert_by_postgresql_source_test_case138" value="insert into trigtest default values;" db-types="PostgreSQL" />
-    <sql-case id="low_lock_by_postgresql_source_test_case1" value="lock table pxtest3 in access share mode nowait;" db-types="PostgreSQL" />
-    <sql-case id="low_lock_by_postgresql_source_test_case2" value="lock table pxtest3 in access share mode nowait;" db-types="PostgreSQL" />
-    <sql-case id="low_lock_by_postgresql_source_test_case3" value="lock twophase_tab in access exclusive mode;" db-types="PostgreSQL" />
     <sql-case id="low_prepare_by_postgresql_source_test_case1" value="prepare q as   select &apos;some&quot;text&apos; as &quot;a&amp;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_case2" 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_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" />