You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by tu...@apache.org on 2022/07/08 01:21:59 UTC

[shardingsphere] branch master updated: support LOCK TABLE in Oracle (#18946)

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

tuichenchuxin 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 ac6eacb9b1d support LOCK TABLE in Oracle (#18946)
ac6eacb9b1d is described below

commit ac6eacb9b1d275252d5aec336e34fbcd12d07e1a
Author: Trydamere <30...@qq.com>
AuthorDate: Fri Jul 8 09:21:52 2022 +0800

    support LOCK TABLE in Oracle (#18946)
---
 .../src/main/antlr4/imports/oracle/DMLStatement.g4 | 17 +++++++++++++
 .../main/antlr4/imports/oracle/OracleKeyword.g4    |  8 ++++++
 .../sql/parser/autogen/OracleStatement.g4          |  1 +
 .../impl/OracleDMLStatementSQLVisitor.java         |  7 ++++++
 .../core/database/visitor/SQLVisitorRule.java      |  2 ++
 .../common/statement/dml/LockTableStatement.java   | 26 +++++++++++++++++++
 .../oracle/dml/OracleLockTableStatement.java       | 29 ++++++++++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  5 ++++
 .../statement/dml/LockTableStatementTestCase.java  | 26 +++++++++++++++++++
 .../src/main/resources/case/dml/lock-table.xml     | 23 +++++++++++++++++
 .../resources/sql/supported/dml/lock-table.xml     | 22 ++++++++++++++++
 11 files changed, 166 insertions(+)

diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/DMLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/DMLStatement.g4
index 8c0ad4c523a..508976d5048 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/DMLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/DMLStatement.g4
@@ -869,3 +869,20 @@ rowPatternNavCompound
 rowPatternAggregateFunc
     : (RUNNING | FINAL)? aggregationFunction
     ;
+
+lockTable
+    : LOCK TABLE (tableName | viewName) (partitionExtensionClause | AT_ dbLink)? (COMMA_ (tableName | viewName) (partitionExtensionClause | AT_ dbLink)? )* IN lockmodeClause MODE ( NOWAIT | WAIT INTEGER_)?
+    ;
+
+partitionExtensionClause
+    : PARTITION LP_ partitionName RP_
+    | PARTITION FOR LP_ partitionKeyValue (COMMA_ partitionKeyValue)* RP_
+    | SUBPARTITION LP_ subpartitionName RP_
+    | SUBPARTITION FOR LP_ subpartitionKeyValue (COMMA_ subpartitionKeyValue)* RP_
+    ;
+
+lockmodeClause
+    : ROW (SHARE | EXCLUSIVE)
+    | SHARE (UPDATE | ROW EXCLUSIVE)?
+    | EXCLUSIVE
+    ;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/OracleKeyword.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/OracleKeyword.g4
index dfcd75c6787..11b933cb7c9 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/OracleKeyword.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/imports/oracle/OracleKeyword.g4
@@ -2850,3 +2850,11 @@ RESOLVE
 RESOLVER
    : R E S O L V E R
    ;
+
+SHARE
+    : S H A R E
+    ;
+
+EXCLUSIVE
+    : E X C L U S I V E
+    ;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/OracleStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/OracleStatement.g4
index 626cf3f8dc1..1fac0f4a114 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/OracleStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/antlr4/org/apache/shardingsphere/sql/parser/autogen/OracleStatement.g4
@@ -28,6 +28,7 @@ execute
     | alterTable
     | dropTable
     | truncateTable
+    | lockTable
     | createIndex
     | dropIndex
     | alterIndex
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/java/org/apache/shardingsphere/sql/parser/oracle/visitor/statement/impl/OracleDMLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/java/org/apache/shardingsphere/sql/parser/oracle/visitor/statement/impl/OracleDMLStatementSQLVisitor.java
index 1cbc11e0268..05ffb7207a7 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/java/org/apache/shardingsphere/sql/parser/oracle/visitor/statement/impl/OracleDMLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-oracle/src/main/java/org/apache/shardingsphere/sql/parser/oracle/visitor/statement/impl/OracleDMLStatementSQLVisitor.java
@@ -100,6 +100,7 @@ import org.apache.shardingsphere.sql.parser.autogen.OracleStatementParser.Update
 import org.apache.shardingsphere.sql.parser.autogen.OracleStatementParser.UsingClauseContext;
 import org.apache.shardingsphere.sql.parser.autogen.OracleStatementParser.WhereClauseContext;
 import org.apache.shardingsphere.sql.parser.autogen.OracleStatementParser.WithClauseContext;
+import org.apache.shardingsphere.sql.parser.autogen.OracleStatementParser.LockTableContext;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.OrderDirection;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.ColumnAssignmentSegment;
@@ -149,6 +150,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.oracle.dml.Ora
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.oracle.dml.OracleMergeStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.oracle.dml.OracleSelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.oracle.dml.OracleUpdateStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.oracle.dml.OracleLockTableStatement;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -1098,4 +1100,9 @@ public final class OracleDMLStatementSQLVisitor extends OracleStatementSQLVisito
         ASTNode segment = visit(ctx.whereClause().expr());
         return new WhereSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), (ExpressionSegment) segment);
     }
+    
+    @Override
+    public ASTNode visitLockTable(final LockTableContext ctx) {
+        return new OracleLockTableStatement();
+    }
 }
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 56c3868d090..5cd65928b5e 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
@@ -44,6 +44,8 @@ public enum SQLVisitorRule {
     
     COPY("Copy", SQLStatementType.DML),
     
+    LOCKTABLE("LockTable", SQLStatementType.DML),
+    
     CREATE_TABLE("CreateTable", SQLStatementType.DDL),
     
     CREATE_AGGREGATE("CreateAggregate", SQLStatementType.DDL),
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/dml/LockTableStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/dml/LockTableStatement.java
new file mode 100644
index 00000000000..801c554472a
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/dml/LockTableStatement.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.dml;
+
+import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
+
+/**
+ * Lock table statement.
+ */
+public abstract class LockTableStatement extends AbstractSQLStatement implements DMLStatement {
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/oracle/dml/OracleLockTableStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/oracle/dml/OracleLockTableStatement.java
new file mode 100644
index 00000000000..32642232fb8
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/oracle/dml/OracleLockTableStatement.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.oracle.dml;
+
+import lombok.ToString;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.LockTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.oracle.OracleStatement;
+
+/**
+ * Oracle lock table statement.
+ */
+@ToString
+public final class OracleLockTableStatement extends LockTableStatement implements OracleStatement {
+}
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 cf481a4cf1c..3ae83247077 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
@@ -382,6 +382,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.MergeStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.SelectStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.UpdateStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.LockTableStatementTestCase;
 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;
@@ -438,6 +439,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "drop-table")
     private final List<DropTableStatementTestCase> dropTableTestCases = new LinkedList<>();
     
+    @XmlElement(name = "lock-table")
+    private final List<LockTableStatementTestCase> lockTableTestCases = new LinkedList<>();
+    
     @XmlElement(name = "drop-text-search")
     private final List<DropTextSearchStatementTestCase> dropTextSearchTestCases = new LinkedList<>();
     
@@ -1552,6 +1556,7 @@ public final class SQLParserTestCases {
         putAll(createEditionTestCases, result);
         putAll(alterTableTestCases, result);
         putAll(dropTableTestCases, result);
+        putAll(lockTableTestCases, result);
         putAll(dropTextSearchTestCases, result);
         putAll(truncateTestCases, result);
         putAll(dropEditionTestCases, result);
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/LockTableStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/LockTableStatementTestCase.java
new file mode 100644
index 00000000000..aeadbcb742d
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/LockTableStatementTestCase.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.dml;
+
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+
+/**
+ * Lock table statement test case.
+ */
+public final class LockTableStatementTestCase extends SQLParserTestCase {
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/lock-table.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/lock-table.xml
new file mode 100644
index 00000000000..9f447f81841
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/lock-table.xml
@@ -0,0 +1,23 @@
+<?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>
+    <lock-table sql-case-id="lock-table" />
+    <lock-table sql-case-id="lock-table-remote" />
+</sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/lock-table.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/lock-table.xml
new file mode 100644
index 00000000000..117aa1a305f
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/lock-table.xml
@@ -0,0 +1,22 @@
+<?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="lock-table" value="LOCK TABLE employees IN EXCLUSIVE MODE NOWAIT" db-types="Oracle" />
+    <sql-case id="lock-table-remote" value="LOCK TABLE employees@remote IN SHARE MODE" db-types="Oracle" />
+</sql-cases>