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 2021/10/28 07:05:20 UTC

[shardingsphere] branch master updated: fix aggregation function convert to sql node wrong result (#13324)

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 cee6e62  fix aggregation function convert to sql node wrong result (#13324)
cee6e62 is described below

commit cee6e628ca8faad72012d9381bcfa8f38de27fe4
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Thu Oct 28 15:03:42 2021 +0800

    fix aggregation function convert to sql node wrong result (#13324)
    
    * fix aggregation function convert to sql node wrong result
    
    * optimize method declaration
---
 .../impl/AggregationProjectionConverter.java       | 19 ++++++--
 .../statement/impl/MySQLStatementSQLVisitor.java   | 51 +++++++++++-----------
 .../statement/impl/OracleStatementSQLVisitor.java  | 21 ++++++---
 .../impl/PostgreSQLStatementSQLVisitor.java        | 20 ++++++---
 .../impl/SQL92DMLStatementSQLVisitor.java          |  2 +-
 .../statement/impl/SQL92StatementSQLVisitor.java   | 24 ++++++----
 .../impl/SQLServerStatementSQLVisitor.java         | 21 ++++++---
 .../parser/sql/common/constant/QuoteCharacter.java |  2 +
 .../SQLNodeConvertEngineParameterizedTest.java     |  3 +-
 .../main/resources/case/dml/select-expression.xml  |  4 +-
 .../main/resources/case/dml/select-sub-query.xml   | 42 ++++++++++++++----
 .../src/main/resources/case/dml/select.xml         |  2 +-
 .../sql/supported/dml/select-sub-query.xml         |  3 +-
 13 files changed, 145 insertions(+), 69 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/impl/AggregationProjectionConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/impl/AggregationProjectionConverter.java
index 368d5cb..570045a 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/impl/AggregationProjectionConverter.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/impl/AggregationProjectionConverter.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.infra.optimize.converter.segment.projection.impl;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlIdentifier;
@@ -26,11 +27,16 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.shardingsphere.infra.optimize.converter.segment.SQLSegmentConverter;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.AggregationType;
+import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.AggregationProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.util.SQLUtil;
 
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.TreeMap;
+import java.util.stream.Collectors;
 
 /**
  * Aggregation projection converter. 
@@ -56,19 +62,26 @@ public final class AggregationProjectionConverter implements SQLSegmentConverter
         if (null == segment) {
             return Optional.empty();
         }
-        return Optional.of(new SqlBasicCall(convertOperator(segment.getType().name()), new SqlNode[]{SqlIdentifier.star(SqlParserPos.ZERO)}, SqlParserPos.ZERO));
+        List<String> parameters = Splitter.on(",").trimResults().splitToList(SQLUtil.getExpressionWithoutOutsideParentheses(segment.getInnerExpression()));
+        return Optional.of(new SqlBasicCall(convertOperator(segment.getType().name()), 
+                new SqlNode[]{SqlIdentifier.star(parameters, SqlParserPos.ZERO, Collections.singletonList(SqlParserPos.ZERO))}, SqlParserPos.ZERO));
     }
     
     @Override
     public Optional<AggregationProjectionSegment> convertToSQLSegment(final SqlBasicCall sqlBasicCall) {
         if (null == sqlBasicCall) {
-            return Optional.empty();    
+            return Optional.empty();
         }
         AggregationType aggregationType = AggregationType.valueOf(sqlBasicCall.getOperator().getName());
-        String innerExpression = sqlBasicCall.toString().replace(sqlBasicCall.getOperator().getName(), "");
+        String innerExpression = getInnerExpression(sqlBasicCall);
         return Optional.of(new AggregationProjectionSegment(getStartIndex(sqlBasicCall), getStopIndex(sqlBasicCall), aggregationType, innerExpression));
     }
     
+    private String getInnerExpression(final SqlBasicCall sqlBasicCall) {
+        String params = sqlBasicCall.getOperandList().stream().map(SqlNode::toString).collect(Collectors.joining(", "));
+        return QuoteCharacter.PARENTHESES.wrap(params);
+    }
+    
     private SqlAggFunction convertOperator(final String operator) {
         Preconditions.checkState(REGISTRY.containsKey(operator), "Unsupported SQL operator: `%s`", operator);
         return REGISTRY.get(operator);
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
index 351e26f..622f191 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.sql.parser.mysql.visitor.statement.impl;
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.NoArgsConstructor;
+import org.antlr.v4.runtime.ParserRuleContext;
 import org.antlr.v4.runtime.misc.Interval;
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
@@ -676,7 +677,7 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
     @Override
     public final ASTNode visitIntervalExpression(final IntervalExpressionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.intervalValue().expr()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
@@ -696,9 +697,8 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
     @Override
     public final ASTNode visitAggregationFunction(final AggregationFunctionContext ctx) {
         String aggregationType = ctx.aggregationFunctionName().getText();
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
         return AggregationType.isAggregationType(aggregationType)
-                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private ASTNode createAggregationSegment(final AggregationFunctionContext ctx, final String aggregationType) {
@@ -747,71 +747,61 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
         if (null != ctx.weightStringFunction()) {
             return visit(ctx.weightStringFunction());
         }
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitGroupConcatFunction(final GroupConcatFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitWindowFunction(final WindowFunctionContext ctx) {
         super.visitWindowFunction(ctx);
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitCastFunction(final CastFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitConvertFunction(final ConvertFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitPositionFunction(final PositionFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitSubstringFunction(final SubstringFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitExtractFunction(final ExtractFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitCharFunction(final CharFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitWeightStringFunction(final WeightStringFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
@@ -819,8 +809,7 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
         if (null != ctx.completeRegularFunction()) {
             calculateParameterCount(ctx.completeRegularFunction().expr());
         }
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private ASTNode visitRemainSimpleExpr(final SimpleExprContext ctx) {
@@ -1291,7 +1280,7 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
         }
         // FIXME :For DISTINCT()
         if (projection instanceof ColumnSegment) {
-            ExpressionProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), ctx.getText(), (ColumnSegment) projection);
+            ExpressionProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), getOriginalText(ctx), (ColumnSegment) projection);
             result.setAlias(alias);
             return result;
         }
@@ -1490,4 +1479,14 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
         }
         return new ParameterMarkerLimitValueSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ((ParameterMarkerValue) visit(ctx.parameterMarker())).getValue());
     }
+    
+    /**
+     * Get original text.
+     *
+     * @param ctx context
+     * @return original text
+     */
+    protected String getOriginalText(final ParserRuleContext ctx) {
+        return ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+    }
 }
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/OracleStatementSQLVisitor.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/OracleStatementSQLVisitor.java
index ff3f9d5..03848d7 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/OracleStatementSQLVisitor.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/OracleStatementSQLVisitor.java
@@ -466,7 +466,7 @@ public abstract class OracleStatementSQLVisitor extends OracleStatementBaseVisit
     public final ASTNode visitAggregationFunction(final AggregationFunctionContext ctx) {
         String aggregationType = ctx.aggregationFunctionName().getText();
         return AggregationType.isAggregationType(aggregationType)
-                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private ASTNode createAggregationSegment(final AggregationFunctionContext ctx, final String aggregationType) {
@@ -491,26 +491,25 @@ public abstract class OracleStatementSQLVisitor extends OracleStatementBaseVisit
         if (null != ctx.castFunction()) {
             return visit(ctx.castFunction());
         }
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitCastFunction(final CastFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitCharFunction(final CharFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitRegularFunction(final RegularFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
@@ -580,4 +579,14 @@ public abstract class OracleStatementSQLVisitor extends OracleStatementBaseVisit
         }
         return result;
     }
+    
+    /**
+     * Get original text.
+     *
+     * @param ctx context
+     * @return original text
+     */
+    protected String getOriginalText(final ParserRuleContext ctx) {
+        return ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+    }
 }
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/PostgreSQLStatementSQLVisitor.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/PostgreSQLStatementSQLVisitor.java
index d17aa0a..8c30ea5 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/PostgreSQLStatementSQLVisitor.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/PostgreSQLStatementSQLVisitor.java
@@ -21,6 +21,7 @@ import com.google.common.base.Joiner;
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.NoArgsConstructor;
+import org.antlr.v4.runtime.ParserRuleContext;
 import org.antlr.v4.runtime.misc.Interval;
 import org.antlr.v4.runtime.tree.ParseTree;
 import org.antlr.v4.runtime.tree.TerminalNode;
@@ -342,8 +343,7 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementB
     @Override
     public ASTNode visitFuncExpr(final FuncExprContext ctx) {
         calculateParameterCount(getTargetRuleContextFromParseTree(ctx, CExprContext.class));
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private <T extends ParseTree> Collection<T> getTargetRuleContextFromParseTree(final ParseTree parseTree, final Class<? extends T> clazz) {
@@ -469,11 +469,11 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementB
         if (AggregationType.isAggregationType(aggregationType)) {
             return createAggregationSegment(ctx, aggregationType);
         }
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private ProjectionSegment generateProjectFromFunctionExprCommonSubexpr(final FunctionExprCommonSubexprContext ctx) {
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private ProjectionSegment createAggregationSegment(final FuncApplicationContext ctx, final String aggregationType) {
@@ -872,7 +872,7 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementB
             return shorthandProjection;
         }
         AExprContext expr = ctx.aExpr();
-        ProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), expr.getText());
+        ProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), getOriginalText(expr));
         if (null != expr.cExpr()) {
             ASTNode projection = visit(expr.cExpr());
             if (projection instanceof ColumnSegment) {
@@ -1084,4 +1084,14 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementB
     public ASTNode visitExecuteStmt(final ExecuteStmtContext ctx) {
         return new PostgreSQLExecuteStatement();
     }
+    
+    /**
+     * Get original text.
+     *
+     * @param ctx context
+     * @return original text
+     */
+    protected String getOriginalText(final ParserRuleContext ctx) {
+        return ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92DMLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92DMLStatementSQLVisitor.java
index c2f3951f..cb18b76 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92DMLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92DMLStatementSQLVisitor.java
@@ -326,7 +326,7 @@ public final class SQL92DMLStatementSQLVisitor extends SQL92StatementSQLVisitor
         }
         // FIXME :For DISTINCT()
         if (projection instanceof ColumnSegment) {
-            ExpressionProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), ctx.getText());
+            ExpressionProjectionSegment result = new ExpressionProjectionSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), getOriginalText(ctx));
             result.setAlias(alias);
             return result;
         }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92StatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92StatementSQLVisitor.java
index 661dc56..0162d2f 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92StatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sql92/src/main/java/org/apache/shardingsphere/sql/parser/sql92/visitor/statement/impl/SQL92StatementSQLVisitor.java
@@ -389,7 +389,7 @@ public abstract class SQL92StatementSQLVisitor extends SQL92StatementBaseVisitor
     @Override
     public final ASTNode visitIntervalExpression(final IntervalExpressionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
@@ -409,9 +409,8 @@ public abstract class SQL92StatementSQLVisitor extends SQL92StatementBaseVisitor
     @Override
     public final ASTNode visitAggregationFunction(final AggregationFunctionContext ctx) {
         String aggregationType = ctx.aggregationFunctionName().getText();
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
         return AggregationType.isAggregationType(aggregationType)
-                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private ASTNode createAggregationSegment(final AggregationFunctionContext ctx, final String aggregationType) {
@@ -436,22 +435,19 @@ public abstract class SQL92StatementSQLVisitor extends SQL92StatementBaseVisitor
         if (null != ctx.castFunction()) {
             return visit(ctx.castFunction());
         }
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitCastFunction(final CastFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitRegularFunction(final RegularFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
@@ -518,4 +514,14 @@ public abstract class SQL92StatementSQLVisitor extends SQL92StatementBaseVisitor
         }
         return result;
     }
+    
+    /**
+     * Get original text.
+     * 
+     * @param ctx context
+     * @return original text
+     */
+    protected String getOriginalText(final ParserRuleContext ctx) {
+        return ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sqlserver/src/main/java/org/apache/shardingsphere/sql/parser/sqlserver/visitor/statement/impl/SQLServerStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sqlserver/src/main/java/org/apache/shardingsphere/sql/parser/sqlserver/visitor/statement/impl/SQLServerStatementSQLVisitor.java
index 49a66ce..27409bf 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sqlserver/src/main/java/org/apache/shardingsphere/sql/parser/sqlserver/visitor/statement/impl/SQLServerStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-sqlserver/src/main/java/org/apache/shardingsphere/sql/parser/sqlserver/visitor/statement/impl/SQLServerStatementSQLVisitor.java
@@ -458,7 +458,7 @@ public abstract class SQLServerStatementSQLVisitor extends SQLServerStatementBas
     public final ASTNode visitAggregationFunction(final AggregationFunctionContext ctx) {
         String aggregationType = ctx.aggregationFunctionName().getText();
         return AggregationType.isAggregationType(aggregationType)
-                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+                ? createAggregationSegment(ctx, aggregationType) : new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     private ASTNode createAggregationSegment(final AggregationFunctionContext ctx, final String aggregationType) {
@@ -483,26 +483,25 @@ public abstract class SQLServerStatementSQLVisitor extends SQLServerStatementBas
         if (null != ctx.castFunction()) {
             return visit(ctx.castFunction());
         }
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitCastFunction(final CastFunctionContext ctx) {
         calculateParameterCount(Collections.singleton(ctx.expr()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitCharFunction(final CharFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.getText());
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
     public final ASTNode visitRegularFunction(final RegularFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        String text = ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
-        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), text);
+        return new ExpressionProjectionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getOriginalText(ctx));
     }
     
     @Override
@@ -706,4 +705,14 @@ public abstract class SQLServerStatementSQLVisitor extends SQLServerStatementBas
         }
         return new GroupBySegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), items);
     }
+    
+    /**
+     * Get original text.
+     *
+     * @param ctx context
+     * @return original text
+     */
+    protected String getOriginalText(final ParserRuleContext ctx) {
+        return ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+    }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/constant/QuoteCharacter.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/constant/QuoteCharacter.java
index aa53f0f..72efbcc 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/constant/QuoteCharacter.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/constant/QuoteCharacter.java
@@ -38,6 +38,8 @@ public enum QuoteCharacter {
     
     BRACKETS("[", "]"),
     
+    PARENTHESES("(", ")"),
+    
     NONE("", "");
     
     private final String startDelimiter;
diff --git a/shardingsphere-test/shardingsphere-optimize-test/src/test/java/org/apache/shardingsphere/infra/optimize/converter/parameterized/engine/SQLNodeConvertEngineParameterizedTest.java b/shardingsphere-test/shardingsphere-optimize-test/src/test/java/org/apache/shardingsphere/infra/optimize/converter/parameterized/engine/SQLNodeConvertEngineParameterizedTest.java
index ccf55a3..024042d 100644
--- a/shardingsphere-test/shardingsphere-optimize-test/src/test/java/org/apache/shardingsphere/infra/optimize/converter/parameterized/engine/SQLNodeConvertEngineParameterizedTest.java
+++ b/shardingsphere-test/shardingsphere-optimize-test/src/test/java/org/apache/shardingsphere/infra/optimize/converter/parameterized/engine/SQLNodeConvertEngineParameterizedTest.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.infra.optimize.converter.parameterized.engine;
 
 import lombok.RequiredArgsConstructor;
 import lombok.SneakyThrows;
+import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.config.CalciteConnectionConfigImpl;
 import org.apache.calcite.config.CalciteConnectionProperty;
@@ -141,7 +142,7 @@ public final class SQLNodeConvertEngineParameterizedTest {
     
     private Config createConfig(final DatabaseType databaseType) {
         CalciteConnectionConfig connectionConfig = new CalciteConnectionConfigImpl(createSQLDialectProperties(databaseType));
-        return SqlParser.config().withLex(connectionConfig.lex())
+        return SqlParser.config().withLex(connectionConfig.lex()).withUnquotedCasing(Casing.UNCHANGED)
                 .withIdentifierMaxLength(SqlParser.DEFAULT_IDENTIFIER_MAX_LENGTH).withConformance(connectionConfig.conformance()).withParserFactory(SqlParserImpl.FACTORY);
     }
     
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-expression.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-expression.xml
index 0d7888f..336785f 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-expression.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-expression.xml
@@ -1237,7 +1237,7 @@
                     </left>
                     <operator>+</operator>
                     <right>
-                        <expression-projection text="INTERVAL1SECOND" start-index="47" stop-index="63" />
+                        <expression-projection text="INTERVAL 1 SECOND" start-index="47" stop-index="63" />
                     </right>
                 </binary-operation-expression>
             </expr>
@@ -1261,7 +1261,7 @@
                     </left>
                     <operator>-</operator>
                     <right>
-                        <expression-projection text="INTERVAL1SECOND" start-index="47" stop-index="63" />
+                        <expression-projection text="INTERVAL 1 SECOND" start-index="47" stop-index="63" />
                     </right>
                 </binary-operation-expression>
             </expr>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-sub-query.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-sub-query.xml
index 29e3305..22d6628 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-sub-query.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-sub-query.xml
@@ -292,28 +292,54 @@
     </select>
 
     <select sql-case-id="select_with_projection_subquery">
-        <projections start-index="7" stop-index="93">
+        <projections start-index="7" stop-index="99">
             <column-projection name="order_id" start-index="7" stop-index="31">
                 <owner start-index="7" stop-index="22" name="t_order_federate"/>
             </column-projection>
             <column-projection name="user_id" start-index="34" stop-index="57">
                 <owner start-index="34" stop-index="49" name="t_order_federate"/>
             </column-projection>
-            <subquery-projection start-index="60" stop-index="93" text="(SELECT COUNT(*) FROM t_user_info)">
+            <subquery-projection start-index="60" stop-index="99" text="(SELECT COUNT(user_id) FROM t_user_info)">
                 <subquery>
                     <select>
-                        <projections start-index="68" stop-index="75">
-                            <aggregation-projection type="COUNT" inner-expression="(*)" start-index="68" stop-index="75" />
+                        <projections start-index="68" stop-index="81">
+                            <aggregation-projection type="COUNT" inner-expression="(user_id)" start-index="68" stop-index="81" />
                         </projections>
-                        <from start-index="77" stop-index="80">
-                            <simple-table name="t_user_info" start-index="82" stop-index="92" />
+                        <from start-index="83" stop-index="98">
+                            <simple-table name="t_user_info" start-index="88" stop-index="98" />
                         </from>
                     </select>
                 </subquery>
             </subquery-projection>
         </projections>
-        <from start-index="95" stop-index="115">
-            <simple-table name="t_order_federate" start-index="100" stop-index="115" />
+        <from start-index="101" stop-index="121">
+            <simple-table name="t_order_federate" start-index="106" stop-index="121" />
+        </from>
+    </select>
+
+    <select sql-case-id="select_with_projection_subquery_and_multiple_parameters">
+        <projections start-index="7" stop-index="110">
+            <column-projection name="order_id" start-index="7" stop-index="31">
+                <owner start-index="7" stop-index="22" name="t_order_federate"/>
+            </column-projection>
+            <column-projection name="user_id" start-index="34" stop-index="57">
+                <owner start-index="34" stop-index="49" name="t_order_federate"/>
+            </column-projection>
+            <subquery-projection start-index="60" stop-index="110" text="(SELECT CONCAT(order_id, user_id) FROM t_user_info)">
+                <subquery>
+                    <select>
+                        <projections start-index="68" stop-index="92">
+                            <expression-projection text="CONCAT(order_id, user_id)" start-index="68" stop-index="92" />
+                        </projections>
+                        <from start-index="94" stop-index="109">
+                            <simple-table name="t_user_info" start-index="99" stop-index="109" />
+                        </from>
+                    </select>
+                </subquery>
+            </subquery-projection>
+        </projections>
+        <from start-index="112" stop-index="132">
+            <simple-table name="t_order_federate" start-index="117" stop-index="132" />
         </from>
     </select>
 
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml
index b83c8fc..48aec0d 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml
@@ -2917,7 +2917,7 @@
             <simple-table name="tb_content_json" start-index="59" stop-index="73" />
         </from>
         <projections start-index="7" stop-index="52">
-            <expression-projection text="content_json::jsonb@?'$.keyword[*]?(@==&quot;ss&quot;)'" start-index="7" stop-index="52" />
+            <expression-projection text="content_json::jsonb @?'$.keyword[*]?(@==&quot;ss&quot;)'" start-index="7" stop-index="52" />
         </projections>
     </select>
 
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select-sub-query.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select-sub-query.xml
index 6da3e1c..62c9774 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select-sub-query.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select-sub-query.xml
@@ -24,7 +24,8 @@
     <sql-case id="select_with_between_subquery" value="SELECT * FROM t_order WHERE user_id BETWEEN (SELECT user_id FROM t_order_item WHERE order_id = 10) AND ?" db-types="MySQL, PostgreSQL" />
     <sql-case id="select_with_exists_sub_query_with_project" value="SELECT EXISTS (SELECT 1 FROM t_order)" db-types="MySQL, PostgreSQL" />
     <sql-case id="select_with_join_table_subquery" value="SELECT t_order_federate.order_id, t_order_federate.user_id, u.user_id FROM t_order_federate, (SELECT * FROM t_user_info) as u WHERE t_order_federate.user_id = u.user_id" db-types="MySQL, PostgreSQL, SQLServer, SQL92" />
-    <sql-case id="select_with_projection_subquery" value="SELECT t_order_federate.order_id, t_order_federate.user_id, (SELECT COUNT(*) FROM t_user_info) FROM t_order_federate" />
+    <sql-case id="select_with_projection_subquery" value="SELECT t_order_federate.order_id, t_order_federate.user_id, (SELECT COUNT(user_id) FROM t_user_info) FROM t_order_federate" />
+    <sql-case id="select_with_projection_subquery_and_multiple_parameters" value="SELECT t_order_federate.order_id, t_order_federate.user_id, (SELECT CONCAT(order_id, user_id) FROM t_user_info) FROM t_order_federate" />
     <sql-case id="select_with_in_subquery_condition" value="SELECT t_order_federate.order_id, t_order_federate.user_id FROM t_order_federate WHERE user_id IN (SELECT * FROM t_user_info)" />
     <sql-case id="select_with_between_and_subquery_condition" value="SELECT t_order_federate.order_id, t_order_federate.user_id FROM t_order_federate WHERE user_id BETWEEN (SELECT user_id FROM t_user_info WHERE information = 'before') AND (SELECT user_id FROM t_user_info WHERE information = 'after')" />
     <sql-case id="select_with_exist_subquery_condition" value="SELECT t_order_federate.order_id, t_order_federate.user_id FROM t_order_federate WHERE EXISTS (SELECT * FROM t_user_info WHERE t_order_federate.user_id = t_user_info.user_id)" db-types="MySQL, PostgreSQL" />