You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2020/11/20 05:25:45 UTC

[shardingsphere] branch master updated: Use LoadingCache to simplify SQL parser process (#8236)

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

panjuan 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 841de7d  Use LoadingCache to simplify SQL parser process (#8236)
841de7d is described below

commit 841de7d479172ea194ef9e4714b3ebba85052847
Author: Liang Zhang <te...@163.com>
AuthorDate: Fri Nov 20 13:25:10 2020 +0800

    Use LoadingCache to simplify SQL parser process (#8236)
---
 .../sql/parser/api/SQLParserEngine.java            | 65 +++-------------------
 .../sql/parser/cache/ParseTreeCacheBuilder.java    | 44 +++++++++++++++
 .../sql/parser/cache/ParseTreeCacheLoader.java     | 42 ++++++++++++++
 .../parser/SQLParserExecutor.java}                 | 48 +++-------------
 4 files changed, 101 insertions(+), 98 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
index 6753fe1..90d2ae2 100644
--- 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
@@ -17,38 +17,27 @@
 
 package org.apache.shardingsphere.sql.parser.api;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import org.antlr.v4.runtime.BailErrorStrategy;
-import org.antlr.v4.runtime.Parser;
-import org.antlr.v4.runtime.atn.PredictionMode;
-import org.antlr.v4.runtime.misc.ParseCancellationException;
-import org.antlr.v4.runtime.tree.ErrorNode;
+import com.google.common.cache.LoadingCache;
 import org.antlr.v4.runtime.tree.ParseTree;
-import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
-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;
+import org.apache.shardingsphere.sql.parser.cache.ParseTreeCacheBuilder;
+import org.apache.shardingsphere.sql.parser.core.parser.SQLParserExecutor;
 
 /**
  * SQL parser engine.
  */
 public final class SQLParserEngine {
     
-    private final String databaseType;
+    private final SQLParserExecutor sqlParserExecutor;
     
-    private final Cache<String, ParseTree> cache;
+    private final LoadingCache<String, ParseTree> parseTreeCache;
     
     public SQLParserEngine(final String databaseType) {
         this(databaseType, new CacheOption(128, 1024L, 4));
     }
     
     public SQLParserEngine(final String databaseType, final CacheOption cacheOption) {
-        this.databaseType = databaseType;
-        cache = CacheBuilder.newBuilder().softValues()
-                .initialCapacity(cacheOption.getInitialCapacity()).maximumSize(cacheOption.getMaximumSize()).concurrencyLevel(cacheOption.getConcurrencyLevel()).build();
+        sqlParserExecutor = new SQLParserExecutor(databaseType);
+        parseTreeCache = ParseTreeCacheBuilder.build(cacheOption, databaseType);
     }
     
     /**
@@ -59,44 +48,6 @@ public final class SQLParserEngine {
      * @return parse tree
      */
     public ParseTree parse(final String sql, final boolean useCache) {
-        if (!useCache) {
-            return parse(sql);
-        }
-        return parseAndCacheParseTree(sql);
-    }
-    
-    private ParseTree parse(final String sql) {
-        ParseASTNode result = twoPhaseParse(sql);
-        if (result.getRootNode() instanceof ErrorNode) {
-            throw new SQLParsingException("Unsupported SQL of `%s`", sql);
-        }
-        return result.getRootNode();
-    }
-    
-    private ParseTree parseAndCacheParseTree(final String sql) {
-        Optional<ParseTree> parseTree = Optional.ofNullable(cache.getIfPresent(sql));
-        if (parseTree.isPresent()) {
-            return parseTree.get();
-        }
-        ParseTree result = parse(sql);
-        cache.put(sql, result);
-        return result;
-    }
-    
-    private ParseASTNode twoPhaseParse(final String sql) {
-        SQLParser sqlParser = SQLParserFactory.newInstance(databaseType, sql);
-        try {
-            setPredictionMode((Parser) sqlParser, PredictionMode.SLL);
-            return (ParseASTNode) sqlParser.parse();
-        } catch (final ParseCancellationException ex) {
-            ((Parser) sqlParser).reset();
-            setPredictionMode((Parser) sqlParser, PredictionMode.LL);
-            return (ParseASTNode) sqlParser.parse();
-        }
-    }
-    
-    private void setPredictionMode(final Parser sqlParser, final PredictionMode mode) {
-        sqlParser.setErrorHandler(new BailErrorStrategy());
-        sqlParser.getInterpreter().setPredictionMode(mode);
+        return useCache ? parseTreeCache.getUnchecked(sql) : sqlParserExecutor.parse(sql);
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/cache/ParseTreeCacheBuilder.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/cache/ParseTreeCacheBuilder.java
new file mode 100644
index 0000000..4cabef2
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/cache/ParseTreeCacheBuilder.java
@@ -0,0 +1,44 @@
+/*
+ * 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.cache;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.LoadingCache;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+
+/**
+ * Parse tree cache builder.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ParseTreeCacheBuilder {
+    
+    /**
+     * Build parse tree cache.
+     * 
+     * @param option cache option
+     * @param databaseType database type
+     * @return built parse tree cache
+     */
+    public static LoadingCache<String, ParseTree> build(final CacheOption option, final String databaseType) {
+        return CacheBuilder.newBuilder().softValues()
+                .initialCapacity(option.getInitialCapacity()).maximumSize(option.getMaximumSize()).concurrencyLevel(option.getConcurrencyLevel()).build(new ParseTreeCacheLoader(databaseType));
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/cache/ParseTreeCacheLoader.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/cache/ParseTreeCacheLoader.java
new file mode 100644
index 0000000..612ba42
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/cache/ParseTreeCacheLoader.java
@@ -0,0 +1,42 @@
+/*
+ * 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.cache;
+
+import com.google.common.cache.CacheLoader;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.shardingsphere.sql.parser.core.parser.SQLParserExecutor;
+
+import javax.annotation.ParametersAreNonnullByDefault;
+
+/**
+ * Parse tree cache loader.
+ */
+public final class ParseTreeCacheLoader extends CacheLoader<String, ParseTree> {
+    
+    private final SQLParserExecutor sqlParserExecutor;
+    
+    public ParseTreeCacheLoader(final String databaseType) {
+        sqlParserExecutor = new SQLParserExecutor(databaseType);
+    }
+    
+    @ParametersAreNonnullByDefault
+    @Override
+    public ParseTree load(final String key) {
+        return sqlParserExecutor.parse(key);
+    }
+}
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/core/parser/SQLParserExecutor.java
similarity index 60%
copy from shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/api/SQLParserEngine.java
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
index 6753fe1..be85a65 100644
--- 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/core/parser/SQLParserExecutor.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sql.parser.api;
+package org.apache.shardingsphere.sql.parser.core.parser;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import lombok.RequiredArgsConstructor;
 import org.antlr.v4.runtime.BailErrorStrategy;
 import org.antlr.v4.runtime.Parser;
 import org.antlr.v4.runtime.atn.PredictionMode;
@@ -26,46 +25,23 @@ 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.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.
  */
-public final class SQLParserEngine {
+@RequiredArgsConstructor
+public final class SQLParserExecutor {
     
     private final String databaseType;
     
-    private final Cache<String, ParseTree> cache;
-    
-    public SQLParserEngine(final String databaseType) {
-        this(databaseType, new CacheOption(128, 1024L, 4));
-    }
-    
-    public SQLParserEngine(final String databaseType, final CacheOption cacheOption) {
-        this.databaseType = databaseType;
-        cache = CacheBuilder.newBuilder().softValues()
-                .initialCapacity(cacheOption.getInitialCapacity()).maximumSize(cacheOption.getMaximumSize()).concurrencyLevel(cacheOption.getConcurrencyLevel()).build();
-    }
-    
     /**
      * Parse SQL.
-     *
+     * 
      * @param sql SQL to be parsed
-     * @param useCache whether use cache
      * @return parse tree
      */
-    public ParseTree parse(final String sql, final boolean useCache) {
-        if (!useCache) {
-            return parse(sql);
-        }
-        return parseAndCacheParseTree(sql);
-    }
-    
-    private ParseTree parse(final String sql) {
+    public ParseTree parse(final String sql) {
         ParseASTNode result = twoPhaseParse(sql);
         if (result.getRootNode() instanceof ErrorNode) {
             throw new SQLParsingException("Unsupported SQL of `%s`", sql);
@@ -73,16 +49,6 @@ public final class SQLParserEngine {
         return result.getRootNode();
     }
     
-    private ParseTree parseAndCacheParseTree(final String sql) {
-        Optional<ParseTree> parseTree = Optional.ofNullable(cache.getIfPresent(sql));
-        if (parseTree.isPresent()) {
-            return parseTree.get();
-        }
-        ParseTree result = parse(sql);
-        cache.put(sql, result);
-        return result;
-    }
-    
     private ParseASTNode twoPhaseParse(final String sql) {
         SQLParser sqlParser = SQLParserFactory.newInstance(databaseType, sql);
         try {