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/19 11:45:02 UTC

[shardingsphere] branch master updated: Move static cache into parser engine instance (#8231)

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 894c3ed  Move static cache into parser engine instance (#8231)
894c3ed is described below

commit 894c3edb87ee8e091bbbe7fecc9e4be4e6804c3c
Author: Liang Zhang <te...@163.com>
AuthorDate: Thu Nov 19 19:44:22 2020 +0800

    Move static cache into parser engine instance (#8231)
---
 .../sql/parser/api/SQLParserEngine.java            | 60 ++++++++++++--
 .../sql/parser/core/parser/SQLParserExecutor.java  | 92 ----------------------
 2 files changed, 53 insertions(+), 99 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 e5fd117..cc924aa 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,12 +17,21 @@
 
 package org.apache.shardingsphere.sql.parser.api;
 
+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;
+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.core.parser.SQLParserExecutor;
+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.Map;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.Optional;
 
 /**
  * SQL parser engine.
@@ -30,10 +39,10 @@ import java.util.concurrent.ConcurrentHashMap;
 @RequiredArgsConstructor
 public final class SQLParserEngine {
     
-    private static final Map<String, SQLParserExecutor> EXECUTORS = new ConcurrentHashMap<>();
-    
     private final String databaseType;
     
+    private final Cache<String, ParseTree> cache = CacheBuilder.newBuilder().softValues().initialCapacity(2000).maximumSize(65535).build();
+    
     /**
      * Parse SQL.
      *
@@ -42,7 +51,44 @@ public final class SQLParserEngine {
      * @return parse tree
      */
     public ParseTree parse(final String sql, final boolean useCache) {
-        SQLParserExecutor executor = EXECUTORS.containsKey(databaseType) ? EXECUTORS.get(databaseType) : EXECUTORS.computeIfAbsent(databaseType, SQLParserExecutor::new);
-        return executor.parse(sql, 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);
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
deleted file mode 100644
index a892f89..0000000
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
+++ /dev/null
@@ -1,92 +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.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;
-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.exception.SQLParsingException;
-
-import java.util.Optional;
-
-/**
- * SQL parser executor.
- */
-@RequiredArgsConstructor
-public final class SQLParserExecutor {
-    
-    private final String databaseType;
-    
-    private final Cache<String, ParseTree> cache = CacheBuilder.newBuilder().softValues().initialCapacity(2000).maximumSize(65535).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) {
-        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);
-    }
-}