You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/10/29 05:17:23 UTC

[shardingsphere] branch master updated: Determine SQL parser API (#7954)

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

zhangyonglun 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 1bc809c  Determine SQL parser API (#7954)
1bc809c is described below

commit 1bc809c7f069165d1aa521fa00edf012f38f4df2
Author: Liang Zhang <te...@163.com>
AuthorDate: Thu Oct 29 13:16:47 2020 +0800

    Determine SQL parser API (#7954)
    
    * Refactor SQLParserEngine
    
    * Determine SQL parser API
---
 .../sql/parser/api/SQLParserEngine.java            | 80 ++++++++++++++++++++++
 .../parser/api/parser/SQLParserEngineFactory.java  | 53 --------------
 .../sql/parser/api/visitor/SQLVisitorEngine.java   | 48 -------------
 .../api/visitor/SQLVisitorEngineFactory.java       | 40 -----------
 .../parser/SQLParserExecutor.java}                 |  9 ++-
 .../standard/StandardSQLStatementParserEngine.java | 16 +----
 .../sql/parser/api/SQLParserEngineFactoryTest.java | 45 ------------
 7 files changed, 87 insertions(+), 204 deletions(-)

diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/SQLParserEngine.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/SQLParserEngine.java
new file mode 100644
index 0000000..12a1647
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/SQLParserEngine.java
@@ -0,0 +1,80 @@
+/*
+ * 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.api;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.antlr.v4.runtime.tree.ParseTreeVisitor;
+import org.apache.shardingsphere.sql.parser.core.parser.SQLParserExecutor;
+import org.apache.shardingsphere.sql.parser.core.visitor.SQLVisitorFactory;
+import org.apache.shardingsphere.sql.parser.core.visitor.SQLVisitorRule;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * SQL parser engine.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class SQLParserEngine {
+    
+    private static final Map<String, SQLParserExecutor> ENGINES = new ConcurrentHashMap<>();
+    
+    /**
+     * Parse SQL.
+     *
+     * @param databaseType database type
+     * @param sql SQL to be parsed
+     * @param useCache whether use cache
+     * @param visitorType SQL visitor type
+     * @param <T> type of SQL visitor result
+     * @return SQL visitor result
+     */
+    public static <T> T parse(final String databaseType, final String sql, final boolean useCache, final String visitorType) {
+        ParseTree parseTree = parse(databaseType, sql, useCache);
+        ParseTreeVisitor<T> visitor = SQLVisitorFactory.newInstance(databaseType, visitorType, SQLVisitorRule.valueOf(parseTree.getClass()));
+        return parseTree.accept(visitor);
+    }
+    
+    /**
+     * Parse SQL.
+     *
+     * @param databaseType database type
+     * @param sql SQL to be parsed
+     * @param useCache whether use cache
+     * @return parse tree
+     */
+    public static ParseTree parse(final String databaseType, final String sql, final boolean useCache) {
+        return getSQLParserExecutor(databaseType).parse(sql, useCache);
+    }
+    
+    private static SQLParserExecutor getSQLParserExecutor(final String databaseType) {
+        if (ENGINES.containsKey(databaseType)) {
+            return ENGINES.get(databaseType);
+        }
+        synchronized (ENGINES) {
+            if (ENGINES.containsKey(databaseType)) {
+                return ENGINES.get(databaseType);
+            }
+            SQLParserExecutor result = new SQLParserExecutor(databaseType);
+            ENGINES.put(databaseType, result);
+            return result;
+        }
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/parser/SQLParserEngineFactory.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/parser/SQLParserEngineFactory.java
deleted file mode 100644
index 70e2203..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/parser/SQLParserEngineFactory.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.api.parser;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * SQL parser engine factory.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class SQLParserEngineFactory {
-    
-    private static final Map<String, SQLParserEngine> ENGINES = new ConcurrentHashMap<>();
-    
-    /**
-     * Get SQL parser engine.
-     *
-     * @param databaseType database type
-     * @return SQL parser engine
-     */
-    public static SQLParserEngine getSQLParserEngine(final String databaseType) {
-        if (ENGINES.containsKey(databaseType)) {
-            return ENGINES.get(databaseType);
-        }
-        synchronized (ENGINES) {
-            if (ENGINES.containsKey(databaseType)) {
-                return ENGINES.get(databaseType);
-            }
-            SQLParserEngine result = new SQLParserEngine(databaseType);
-            ENGINES.put(databaseType, result);
-            return result;
-        }
-    }
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/visitor/SQLVisitorEngine.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/visitor/SQLVisitorEngine.java
deleted file mode 100644
index b488522..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/visitor/SQLVisitorEngine.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.api.visitor;
-
-import lombok.RequiredArgsConstructor;
-import org.antlr.v4.runtime.tree.ParseTree;
-import org.antlr.v4.runtime.tree.ParseTreeVisitor;
-import org.apache.shardingsphere.sql.parser.core.visitor.SQLVisitorFactory;
-import org.apache.shardingsphere.sql.parser.core.visitor.SQLVisitorRule;
-
-/**
- * SQL visitor engnie.
- * 
- * @param <T> type of return value
- */
-@RequiredArgsConstructor
-public final class SQLVisitorEngine<T> {
-    
-    private final String databaseType;
-    
-    private final String visitorType;
-    
-    /**
-     * Visit parse tree.
-     *
-     * @param parseTree parse tree
-     * @return visit result
-     */
-    public T visit(final ParseTree parseTree) {
-        ParseTreeVisitor<T> visitor = SQLVisitorFactory.newInstance(databaseType, visitorType, SQLVisitorRule.valueOf(parseTree.getClass()));
-        return parseTree.accept(visitor);
-    }
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/visitor/SQLVisitorEngineFactory.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/visitor/SQLVisitorEngineFactory.java
deleted file mode 100644
index c30f3df..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/visitor/SQLVisitorEngineFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.api.visitor;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-
-/**
- * SQL visitor engine factory.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class SQLVisitorEngineFactory {
-    
-    /**
-     * Get SQL visitor engine.
-     *
-     * @param databaseType database type
-     * @param visitorType visitor type
-     * @param <T> type of visitor result
-     * @return SQL visitor engine
-     */
-    public static <T> SQLVisitorEngine<T> getSQLVisitorEngine(final String databaseType, final String visitorType) {
-        return new SQLVisitorEngine<>(databaseType, visitorType);
-    }
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/parser/SQLParserEngine.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
similarity index 92%
rename from shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/parser/SQLParserEngine.java
rename to shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
index 9a159fc..35f2ce7 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/parser/SQLParserEngine.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sql.parser.api.parser;
+package org.apache.shardingsphere.sql.parser.core.parser;
 
 import lombok.RequiredArgsConstructor;
 import org.antlr.v4.runtime.BailErrorStrategy;
@@ -24,18 +24,17 @@ import org.antlr.v4.runtime.atn.PredictionMode;
 import org.antlr.v4.runtime.misc.ParseCancellationException;
 import org.antlr.v4.runtime.tree.ErrorNode;
 import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
 import org.apache.shardingsphere.sql.parser.cache.SQLParsedResultCache;
-import org.apache.shardingsphere.sql.parser.core.parser.ParseASTNode;
-import org.apache.shardingsphere.sql.parser.core.parser.SQLParserFactory;
 import org.apache.shardingsphere.sql.parser.exception.SQLParsingException;
 
 import java.util.Optional;
 
 /**
- * SQL parser engine.
+ * SQL parser executor.
  */
 @RequiredArgsConstructor
-public final class SQLParserEngine {
+public final class SQLParserExecutor {
     
     private final String databaseType;
     
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/statement/standard/StandardSQLStatementParserEngine.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/statement/standard/StandardSQLStatementParserEngine.java
index 664ad42..2a019c6 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/statement/standard/StandardSQLStatementParserEngine.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/statement/standard/StandardSQLStatementParserEngine.java
@@ -18,12 +18,8 @@
 package org.apache.shardingsphere.sql.parser.statement.standard;
 
 import lombok.RequiredArgsConstructor;
-import org.antlr.v4.runtime.tree.ParseTree;
-import org.antlr.v4.runtime.tree.ParseTreeVisitor;
+import org.apache.shardingsphere.sql.parser.api.SQLParserEngine;
 import org.apache.shardingsphere.sql.parser.cache.SQLParsedResultCache;
-import org.apache.shardingsphere.sql.parser.core.visitor.SQLVisitorFactory;
-import org.apache.shardingsphere.sql.parser.core.visitor.SQLVisitorRule;
-import org.apache.shardingsphere.sql.parser.api.parser.SQLParserEngineFactory;
 import org.apache.shardingsphere.sql.parser.hook.ParsingHookRegistry;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.statement.SQLStatementParserEngine;
@@ -65,20 +61,14 @@ public final class StandardSQLStatementParserEngine implements SQLStatementParse
     
     private SQLStatement parse0(final String sql, final boolean useCache) {
         if (!useCache) {
-            return parseSQLStatement(sql);
+            return SQLParserEngine.parse(databaseTypeName, sql, false, "STATEMENT");
         }
         Optional<SQLStatement> statement = cache.get(sql);
         if (statement.isPresent()) {
             return statement.get();
         }
-        SQLStatement result = parseSQLStatement(sql);
+        SQLStatement result = SQLParserEngine.parse(databaseTypeName, sql, false, "STATEMENT");
         cache.put(sql, result);
         return result;
     }
-    
-    private SQLStatement parseSQLStatement(final String sql) {
-        ParseTree parseTree = SQLParserEngineFactory.getSQLParserEngine(databaseTypeName).parse(sql, false);
-        ParseTreeVisitor<SQLStatement> visitor = SQLVisitorFactory.newInstance(databaseTypeName, "STATEMENT", SQLVisitorRule.valueOf(parseTree.getClass()));
-        return parseTree.accept(visitor);
-    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/test/java/org/apache/shardingsphere/sql/parser/api/SQLParserEngineFactoryTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/test/java/org/apache/shardingsphere/sql/parser/api/SQLParserEngineFactoryTest.java
deleted file mode 100644
index 1887e21..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/test/java/org/apache/shardingsphere/sql/parser/api/SQLParserEngineFactoryTest.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.api;
-
-import org.apache.shardingsphere.sql.parser.api.parser.SQLParserEngineFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.Map;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-public final class SQLParserEngineFactoryTest {
-    
-    @Before
-    @After
-    public void reset() throws NoSuchFieldException, IllegalAccessException {
-        Field field = SQLParserEngineFactory.class.getDeclaredField("ENGINES");
-        field.setAccessible(true);
-        ((Map) field.get(SQLParserEngineFactory.class)).clear();
-    }
-    
-    @Test
-    public void assertGetSQLParserEngine() {
-        assertThat(SQLParserEngineFactory.getSQLParserEngine("MySQL"), is(SQLParserEngineFactory.getSQLParserEngine("MySQL")));
-    }
-}