You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2021/11/24 10:41:21 UTC

[shardingsphere] branch master updated: support distinct aggregation and Optimized converterContext use threadLocal holder (#13763)

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

duanzhengqiang 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 d644dfa  support distinct aggregation and Optimized converterContext use threadLocal holder (#13763)
d644dfa is described below

commit d644dfa8b1fbdf1b617a95c0ee4cbee7c3e2d5fe
Author: liguoping <xd...@163.com>
AuthorDate: Wed Nov 24 18:40:37 2021 +0800

    support distinct aggregation and Optimized converterContext use threadLocal holder (#13763)
    
    * support select_avg
    
    * Optimized calculateParamCount use threadLocal holder
    
    * support distinct aggregation
    
    * AggregationProjectionConverter support bit_xor
    
    * AggregationProjectionConverter support bit_xor
    
    * support position function
    
    * code style
    
    * adjust position function unit test
    
    * binary operation registerAlias
    
    * error commit rollback
    
    * check style overload method
    
    * resolve conversation
    
    * resolve conversation2
---
 .../converter/context/ConverterContextHolder.java  | 55 ++++++++++++++++
 .../segment/expression/ExpressionConverter.java    |  8 +++
 .../impl/BinaryOperationExpressionConverter.java   |  9 +++
 .../segment/expression/impl/FunctionConverter.java | 73 ++++++++++++++++++++++
 .../impl/ParameterMarkerExpressionConverter.java   |  2 +
 .../converter/segment/from/TableConverter.java     |  6 ++
 .../segment/projection/ProjectionsConverter.java   |  2 +-
 .../impl/AggregationProjectionConverter.java       | 64 ++++++++++++++++++-
 .../statement/SelectStatementConverter.java        | 58 +++++++----------
 .../statement/impl/MySQLStatementSQLVisitor.java   |  5 +-
 .../SQLNodeConvertEngineParameterizedTest.java     |  5 ++
 .../resources/case/dml/select-special-function.xml |  9 ++-
 12 files changed, 255 insertions(+), 41 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/context/ConverterContextHolder.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/context/ConverterContextHolder.java
new file mode 100644
index 0000000..e2e0930
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/context/ConverterContextHolder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.infra.optimize.converter.context;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+
+/**
+ * Hold convert context for current thread.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ConverterContextHolder {
+    
+    private static final ThreadLocal<ConverterContext> CONVERTER_CONTEXT = new ThreadLocal<>();
+    
+    /**
+     * Set convert context.
+     *
+     * @param converterContext convert context
+     */
+    public static void set(final ConverterContext converterContext) {
+        CONVERTER_CONTEXT.set(converterContext);
+    }
+    
+    /**
+     * Get convert context.
+     *
+     * @return convert context
+     */
+    public static ConverterContext get() {
+        return CONVERTER_CONTEXT.get();
+    }
+    
+    /**
+     * Remove convert context.
+     */
+    public static void remove() {
+        CONVERTER_CONTEXT.remove();
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/ExpressionConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/ExpressionConverter.java
index 51e0ea6..9711dda 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/ExpressionConverter.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/ExpressionConverter.java
@@ -27,12 +27,14 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.fun.SqlBetweenOperator;
 import org.apache.calcite.sql.fun.SqlInOperator;
+import org.apache.calcite.sql.fun.SqlPositionFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.shardingsphere.infra.optimize.converter.segment.SQLSegmentConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.BetweenExpressionConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.BinaryOperationExpressionConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.ColumnConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.ExistsSubqueryExpressionConverter;
+import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.FunctionConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.InExpressionConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.ListExpressionConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.expression.impl.LiteralExpressionConverter;
@@ -43,6 +45,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenE
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExistsSubqueryExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.FunctionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ListExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
@@ -83,6 +86,8 @@ public final class ExpressionConverter implements SQLSegmentConverter<Expression
             return new BetweenExpressionConverter().convertToSQLNode((BetweenExpression) segment).map(optional -> optional);
         } else if (segment instanceof ParameterMarkerExpressionSegment) {
             return new ParameterMarkerExpressionConverter().convertToSQLNode((ParameterMarkerExpressionSegment) segment).map(optional -> optional);
+        } else if (segment instanceof FunctionSegment) {
+            return new FunctionConverter().convertToSQLNode((FunctionSegment) segment).map(optional -> optional);
         }
         throw new UnsupportedOperationException("unsupported TableSegment type: " + segment.getClass());
     }
@@ -130,6 +135,9 @@ public final class ExpressionConverter implements SQLSegmentConverter<Expression
         if (operator instanceof SqlBinaryOperator) {
             return new BinaryOperationExpressionConverter().convertToSQLSegment(sqlBasicCall).map(optional -> optional);
         }
+        if (operator instanceof SqlPositionFunction) {
+            return new FunctionConverter().convertToSQLSegment(sqlBasicCall).map(optional -> optional);
+        }
         return Optional.empty();
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/BinaryOperationExpressionConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/BinaryOperationExpressionConverter.java
index 459c967..358d500 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/BinaryOperationExpressionConverter.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/BinaryOperationExpressionConverter.java
@@ -40,6 +40,11 @@ public final class BinaryOperationExpressionConverter implements SQLSegmentConve
     private static final Map<String, SqlBinaryOperator> REGISTRY = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
     
     static {
+        register();
+        registerAlias();
+    }
+    
+    private static void register() {
         register(SqlStdOperatorTable.EQUALS);
         register(SqlStdOperatorTable.NOT_EQUALS);
         register(SqlStdOperatorTable.GREATER_THAN);
@@ -58,6 +63,10 @@ public final class BinaryOperationExpressionConverter implements SQLSegmentConve
         REGISTRY.put(sqlBinaryOperator.getName(), sqlBinaryOperator);
     }
     
+    private static void registerAlias() {
+        REGISTRY.put("!=", SqlStdOperatorTable.NOT_EQUALS);
+    }
+    
     @Override
     public Optional<SqlBasicCall> convertToSQLNode(final BinaryOperationExpression segment) {
         SqlBinaryOperator operator = convertOperator(segment.getOperator());
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/FunctionConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/FunctionConverter.java
new file mode 100644
index 0000000..be49fd3
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/FunctionConverter.java
@@ -0,0 +1,73 @@
+/*
+ * 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.infra.optimize.converter.segment.expression.impl;
+
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlPositionFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.shardingsphere.infra.optimize.converter.segment.SQLSegmentConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.FunctionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Function converter.
+ */
+public final class FunctionConverter implements SQLSegmentConverter<FunctionSegment, SqlBasicCall> {
+    
+    @Override
+    public Optional<SqlBasicCall> convertToSQLNode(final FunctionSegment segment) {
+        if ("POSITION".equalsIgnoreCase(segment.getFunctionName())) {
+            return Optional.of(new SqlBasicCall(new SqlPositionFunction(), getPositionSqlNodes(segment.getParameters()), SqlParserPos.ZERO));
+        }
+        return Optional.empty();
+    }
+    
+    @Override
+    public Optional<FunctionSegment> convertToSQLSegment(final SqlBasicCall sqlBasicCall) {
+        if (null == sqlBasicCall) {
+            return Optional.empty();
+        }
+        FunctionSegment functionSegment = new FunctionSegment(getStartIndex(sqlBasicCall), getStopIndex(sqlBasicCall), sqlBasicCall.getOperator().getName(), sqlBasicCall.toString());
+        functionSegment.getParameters().addAll(getParameters(sqlBasicCall));
+        return Optional.of(functionSegment);
+    }
+    
+    private List<ExpressionSegment> getParameters(final SqlBasicCall sqlBasicCall) {
+        return sqlBasicCall.getOperandList().stream()
+                .map(operand -> new LiteralExpressionSegment(getStartIndex(operand), getStopIndex(operand), operand.toString().replace("'", ""))).collect(Collectors.toList());
+    }
+    
+    private SqlNode[] getPositionSqlNodes(final Collection<ExpressionSegment> expressionSegments) {
+        List<SqlNode> sqlNodes = new ArrayList<>();
+        expressionSegments.forEach(expressionSegment -> {
+            if (expressionSegment instanceof LiteralExpressionSegment) {
+                sqlNodes.add(SqlLiteral.createCharString(((LiteralExpressionSegment) expressionSegment).getLiterals().toString(), SqlParserPos.ZERO));
+            }
+        });
+        return sqlNodes.toArray(new SqlNode[0]);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/ParameterMarkerExpressionConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/ParameterMarkerExpressionConverter.java
index 33ba472..36344fd 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/ParameterMarkerExpressionConverter.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/expression/impl/ParameterMarkerExpressionConverter.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.infra.optimize.converter.segment.expression.im
 import org.apache.calcite.sql.SqlDynamicParam;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.shardingsphere.infra.optimize.converter.context.ConverterContextHolder;
 import org.apache.shardingsphere.infra.optimize.converter.segment.SQLSegmentConverter;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.ParameterMarkerExpressionSegment;
 
@@ -37,6 +38,7 @@ public final class ParameterMarkerExpressionConverter implements SQLSegmentConve
     
     @Override
     public Optional<ParameterMarkerExpressionSegment> convertToSQLSegment(final SqlNode sqlNode) {
+        ConverterContextHolder.get().getParameterCount().getAndIncrement();
         return Optional.of(new ParameterMarkerExpressionSegment(getStartIndex(sqlNode), getStopIndex(sqlNode), ((SqlDynamicParam) sqlNode).getIndex()));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/from/TableConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/from/TableConverter.java
index c7421e0..8d7b713 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/from/TableConverter.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/from/TableConverter.java
@@ -41,6 +41,9 @@ public final class TableConverter implements SQLSegmentConverter<TableSegment, S
     
     @Override
     public Optional<SqlNode> convertToSQLNode(final TableSegment segment) {
+        if (null == segment) {
+            return Optional.empty();
+        }
         if (segment instanceof SimpleTableSegment) {
             return new SimpleTableConverter().convertToSQLNode((SimpleTableSegment) segment);
         } else if (segment instanceof JoinTableSegment) {
@@ -53,6 +56,9 @@ public final class TableConverter implements SQLSegmentConverter<TableSegment, S
     
     @Override
     public Optional<TableSegment> convertToSQLSegment(final SqlNode sqlNode) {
+        if (null == sqlNode) {
+            return Optional.empty();
+        }
         if (sqlNode instanceof SqlIdentifier) {
             return new SimpleTableConverter().convertToSQLSegment(sqlNode).map(optional -> optional);
         } else if (sqlNode instanceof SqlJoin) {
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/ProjectionsConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/ProjectionsConverter.java
index dbf0afd..ba0c124 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/ProjectionsConverter.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/segment/projection/ProjectionsConverter.java
@@ -105,7 +105,7 @@ public final class ProjectionsConverter implements SQLSegmentConverter<Projectio
             return new ColumnProjectionConverter().convertToSQLSegment(sqlIdentifier).map(optional -> optional);
         } else if (sqlNode instanceof SqlBasicCall) {
             SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode;
-            if (AggregationType.isAggregationType(sqlBasicCall.getOperator().getName())) {
+            if (AggregationType.isAggregationType(sqlBasicCall.getOperator().getName()) || AggregationProjectionConverter.isAsOperatorAggregationType(sqlBasicCall)) {
                 return new AggregationProjectionConverter().convertToSQLSegment(sqlBasicCall).map(optional -> optional);
             }
             if (null != sqlBasicCall.getOperator() && SqlKind.AS == sqlBasicCall.getOperator().getKind()) {
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 570045a..c154efb 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
@@ -22,14 +22,20 @@ import com.google.common.base.Splitter;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelectKeyword;
 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.AggregationDistinctProjectionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.AggregationProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.AliasSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.util.SQLUtil;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 
 import java.util.Collections;
 import java.util.List;
@@ -51,6 +57,7 @@ public final class AggregationProjectionConverter implements SQLSegmentConverter
         register(SqlStdOperatorTable.SUM);
         register(SqlStdOperatorTable.COUNT);
         register(SqlStdOperatorTable.AVG);
+        register(SqlStdOperatorTable.BIT_XOR);
     }
     
     private static void register(final SqlAggFunction sqlAggFunction) {
@@ -62,9 +69,19 @@ public final class AggregationProjectionConverter implements SQLSegmentConverter
         if (null == segment) {
             return Optional.empty();
         }
+        SqlLiteral functionQuantifier = null;
         List<String> parameters = Splitter.on(",").trimResults().splitToList(SQLUtil.getExpressionWithoutOutsideParentheses(segment.getInnerExpression()));
+        if (segment instanceof AggregationDistinctProjectionSegment) {
+            parameters = Collections.singletonList(((AggregationDistinctProjectionSegment) segment).getDistinctExpression());
+            functionQuantifier = SqlLiteral.createSymbol(SqlSelectKeyword.DISTINCT, SqlParserPos.ZERO);
+        }
+        if (segment.getAlias().isPresent()) {
+            return Optional.of(new SqlBasicCall(SqlStdOperatorTable.AS, new SqlNode[]{new SqlBasicCall(convertOperator(segment.getType().name()), 
+                    new SqlNode[]{SqlIdentifier.star(parameters, SqlParserPos.ZERO, Collections.singletonList(SqlParserPos.ZERO))}, SqlParserPos.ZERO, false, functionQuantifier),
+                    SqlIdentifier.star(Collections.singletonList(segment.getAlias().get()), SqlParserPos.ZERO, Collections.singletonList(SqlParserPos.ZERO))}, SqlParserPos.ZERO));
+        }
         return Optional.of(new SqlBasicCall(convertOperator(segment.getType().name()), 
-                new SqlNode[]{SqlIdentifier.star(parameters, SqlParserPos.ZERO, Collections.singletonList(SqlParserPos.ZERO))}, SqlParserPos.ZERO));
+                new SqlNode[]{SqlIdentifier.star(parameters, SqlParserPos.ZERO, Collections.singletonList(SqlParserPos.ZERO))}, SqlParserPos.ZERO, false, functionQuantifier));
     }
     
     @Override
@@ -72,11 +89,45 @@ public final class AggregationProjectionConverter implements SQLSegmentConverter
         if (null == sqlBasicCall) {
             return Optional.empty();
         }
+        if (isAsOperatorAggregationType(sqlBasicCall)) {
+            SqlBasicCall subSqlBasicCall = (SqlBasicCall) sqlBasicCall.getOperandList().get(0);
+            AggregationType aggregationType = AggregationType.valueOf(subSqlBasicCall.getOperator().getName());
+            String innerExpression = getInnerExpression(subSqlBasicCall);
+            AliasSegment aliasSegment = new AliasSegment(getStartIndex(sqlBasicCall.getOperandList().get(1)), getStopIndex(sqlBasicCall.getOperandList().get(1)), 
+                    new IdentifierValue(((SqlIdentifier) sqlBasicCall.getOperandList().get(1)).names.get(0)));
+            if (null != subSqlBasicCall.getFunctionQuantifier() && SqlSelectKeyword.DISTINCT == subSqlBasicCall.getFunctionQuantifier().getValue()) {
+                return Optional.of(getAggregationDistinctProjectionSegment(subSqlBasicCall, aggregationType, aliasSegment));
+            }
+            AggregationProjectionSegment aggregationProjectionSegment = new AggregationProjectionSegment(getStartIndex(subSqlBasicCall), getStopIndex(subSqlBasicCall),
+                    aggregationType, innerExpression);
+            aggregationProjectionSegment.setAlias(aliasSegment);
+            return Optional.of(aggregationProjectionSegment);
+        }
         AggregationType aggregationType = AggregationType.valueOf(sqlBasicCall.getOperator().getName());
+        if (null != sqlBasicCall.getFunctionQuantifier() && SqlSelectKeyword.DISTINCT == sqlBasicCall.getFunctionQuantifier().getValue()) {
+            return Optional.of(getAggregationDistinctProjectionSegment(sqlBasicCall, aggregationType, null));
+        }
         String innerExpression = getInnerExpression(sqlBasicCall);
         return Optional.of(new AggregationProjectionSegment(getStartIndex(sqlBasicCall), getStopIndex(sqlBasicCall), aggregationType, innerExpression));
     }
     
+    private AggregationDistinctProjectionSegment getAggregationDistinctProjectionSegment(final SqlBasicCall sqlBasicCall, final AggregationType aggregationType, final AliasSegment aliasSegment) {
+        String innerExpression = getInnerExpression(sqlBasicCall, SqlSelectKeyword.DISTINCT);
+        String distinctParams = sqlBasicCall.getOperandList().stream().map(SqlNode::toString).collect(Collectors.joining(", "));
+        AggregationDistinctProjectionSegment aggregationDistinctProjectionSegment = new AggregationDistinctProjectionSegment(getStartIndex(sqlBasicCall), getStopIndex(sqlBasicCall), 
+                aggregationType, innerExpression, distinctParams);
+        aggregationDistinctProjectionSegment.setAlias(aliasSegment);
+        return aggregationDistinctProjectionSegment;
+    }
+    
+    private String getInnerExpression(final SqlBasicCall sqlBasicCall, final SqlSelectKeyword selectKeyword) {
+        if (selectKeyword == null) {
+            return getInnerExpression(sqlBasicCall);
+        }
+        String params = sqlBasicCall.getOperandList().stream().map(SqlNode::toString).collect(Collectors.joining(", "));
+        return QuoteCharacter.PARENTHESES.wrap(selectKeyword.toString() + " " + params);
+    }
+    
     private String getInnerExpression(final SqlBasicCall sqlBasicCall) {
         String params = sqlBasicCall.getOperandList().stream().map(SqlNode::toString).collect(Collectors.joining(", "));
         return QuoteCharacter.PARENTHESES.wrap(params);
@@ -86,4 +137,15 @@ public final class AggregationProjectionConverter implements SQLSegmentConverter
         Preconditions.checkState(REGISTRY.containsKey(operator), "Unsupported SQL operator: `%s`", operator);
         return REGISTRY.get(operator);
     }
+    
+    /**
+     * Judge whether sqlBasicCall is as operator aggregation type or not.
+     * @param sqlBasicCall sqlBasicCall 
+     * @return whether sqlBasicCall is as operator aggregation type or not
+     */
+    public static boolean isAsOperatorAggregationType(final SqlBasicCall sqlBasicCall) {
+        return null != sqlBasicCall.getOperator() && SqlKind.AS == sqlBasicCall.getOperator().getKind()
+                && sqlBasicCall.getOperandList().get(0) instanceof SqlBasicCall
+                && AggregationType.isAggregationType(((SqlBasicCall) sqlBasicCall.getOperandList().get(0)).getOperator().getName());
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/statement/SelectStatementConverter.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/statement/SelectStatementConverter.java
index e3e2498..1d6744f 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/statement/SelectStatementConverter.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/converter/statement/SelectStatementConverter.java
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.SqlOrderBy;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.shardingsphere.infra.optimize.converter.context.ConverterContext;
+import org.apache.shardingsphere.infra.optimize.converter.context.ConverterContextHolder;
 import org.apache.shardingsphere.infra.optimize.converter.segment.from.TableConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.groupby.GroupByConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.groupby.HavingConverter;
@@ -32,8 +33,6 @@ import org.apache.shardingsphere.infra.optimize.converter.segment.projection.Dis
 import org.apache.shardingsphere.infra.optimize.converter.segment.projection.ProjectionsConverter;
 import org.apache.shardingsphere.infra.optimize.converter.segment.where.WhereConverter;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.ParameterMarkerExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.LimitSegment;
@@ -76,46 +75,31 @@ public final class SelectStatementConverter implements SQLStatementConverter<Sel
     
     @Override
     public SelectStatement convertToSQLStatement(final SqlNode sqlNode) {
-        SqlSelect sqlSelect = sqlNode instanceof SqlOrderBy ? (SqlSelect) ((SqlOrderBy) sqlNode).query : (SqlSelect) sqlNode;
-        ProjectionsSegment projections = new ProjectionsConverter().convertToSQLSegment(sqlSelect.getSelectList()).orElseThrow(IllegalStateException::new);
-        projections.setDistinctRow(sqlSelect.isDistinct());
-        // TODO create select statement for different dialect 
         MySQLSelectStatement result = new MySQLSelectStatement();
-        result.setProjections(projections);
-        new TableConverter().convertToSQLSegment(sqlSelect.getFrom()).ifPresent(result::setFrom);
-        new WhereConverter().convertToSQLSegment(sqlSelect.getWhere()).ifPresent(result::setWhere);
-        new GroupByConverter().convertToSQLSegment(sqlSelect.getGroup()).ifPresent(result::setGroupBy);
-        new HavingConverter().convertToSQLSegment(sqlSelect.getHaving()).ifPresent(result::setHaving);
-        ConverterContext context = new ConverterContext();
-        if (sqlNode instanceof SqlOrderBy) {
-            SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode;
-            new OrderByConverter().convertToSQLSegment(sqlOrderBy.orderList).ifPresent(result::setOrderBy);
-            createLimitSegment(sqlOrderBy, context).ifPresent(result::setLimit);
+        try {
+            SqlSelect sqlSelect = sqlNode instanceof SqlOrderBy ? (SqlSelect) ((SqlOrderBy) sqlNode).query : (SqlSelect) sqlNode;
+            ConverterContext context = new ConverterContext();
+            ConverterContextHolder.set(context);
+            ProjectionsSegment projections = new ProjectionsConverter().convertToSQLSegment(sqlSelect.getSelectList()).orElseThrow(IllegalStateException::new);
+            projections.setDistinctRow(sqlSelect.isDistinct());
+            // TODO create select statement for different dialect 
+            result.setProjections(projections);
+            new TableConverter().convertToSQLSegment(sqlSelect.getFrom()).ifPresent(result::setFrom);
+            new WhereConverter().convertToSQLSegment(sqlSelect.getWhere()).ifPresent(result::setWhere);
+            new GroupByConverter().convertToSQLSegment(sqlSelect.getGroup()).ifPresent(result::setGroupBy);
+            new HavingConverter().convertToSQLSegment(sqlSelect.getHaving()).ifPresent(result::setHaving);
+            if (sqlNode instanceof SqlOrderBy) {
+                SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode;
+                new OrderByConverter().convertToSQLSegment(sqlOrderBy.orderList).ifPresent(result::setOrderBy);
+                createLimitSegment(sqlOrderBy, context).ifPresent(result::setLimit);
+            }
+            result.setParameterCount(context.getParameterCount().get());
+        } finally {
+            ConverterContextHolder.remove();
         }
-        calculateParamCount(result, context);
-        result.setParameterCount(context.getParameterCount().get());
         return result;
     }
     
-    private void calculateParamCount(final MySQLSelectStatement result, final ConverterContext context) {
-        result.getWhere().ifPresent(whereSegment -> {
-            if (whereSegment.getExpr() instanceof BinaryOperationExpression) {
-                if (((BinaryOperationExpression) whereSegment.getExpr()).getLeft() instanceof ParameterMarkerExpressionSegment 
-                        || ((BinaryOperationExpression) whereSegment.getExpr()).getRight() instanceof ParameterMarkerExpressionSegment) {
-                    context.getParameterCount().incrementAndGet();
-                }
-            }
-        });
-        result.getHaving().ifPresent(havingSegment -> {
-            if (havingSegment.getExpr() instanceof BinaryOperationExpression) {
-                if (((BinaryOperationExpression) havingSegment.getExpr()).getLeft() instanceof ParameterMarkerExpressionSegment
-                        || ((BinaryOperationExpression) havingSegment.getExpr()).getRight() instanceof ParameterMarkerExpressionSegment) {
-                    context.getParameterCount().incrementAndGet();
-                }
-            }
-        });
-    }
-    
     private Optional<LimitSegment> createLimitSegment(final SqlOrderBy sqlOrderBy, final ConverterContext context) {
         if (null == sqlOrderBy.offset && null == sqlOrderBy.fetch) {
             return Optional.empty();
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 892b399c..d11a65d 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
@@ -809,7 +809,10 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
     @Override
     public final ASTNode visitPositionFunction(final PositionFunctionContext ctx) {
         calculateParameterCount(ctx.expr());
-        return new FunctionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.POSITION().getText(), getOriginalText(ctx));
+        FunctionSegment result = new FunctionSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), ctx.POSITION().getText(), getOriginalText(ctx));
+        result.getParameters().add((LiteralExpressionSegment) visit(ctx.expr(0)));
+        result.getParameters().add((LiteralExpressionSegment) visit(ctx.expr(1)));
+        return result;
     }
     
     @Override
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 c46407b..0b3a12c 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
@@ -84,6 +84,11 @@ public final class SQLNodeConvertEngineParameterizedTest {
         SUPPORTED_SQL_CASE_IDS.add("select_left_outer_join_related_with_alias");
         SUPPORTED_SQL_CASE_IDS.add("select_right_outer_join_related_with_alias");
         SUPPORTED_SQL_CASE_IDS.add("select_alias_as_keyword");
+        SUPPORTED_SQL_CASE_IDS.add("select_avg");
+        SUPPORTED_SQL_CASE_IDS.add("select_between_with_single_table");
+        SUPPORTED_SQL_CASE_IDS.add("select_distinct_with_single_count_group_by");
+        SUPPORTED_SQL_CASE_IDS.add("select_bit_xor");
+        SUPPORTED_SQL_CASE_IDS.add("select_position");
     }
     
     private final String sqlCaseId;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-special-function.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-special-function.xml
index c633efb..a5387f5 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-special-function.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select-special-function.xml
@@ -64,7 +64,14 @@
         <projections start-index="7" stop-index="36">
             <expression-projection text="POSITION('bar' IN 'foobarbar')" start-index="7" stop-index="36">
                 <expr>
-                    <function function-name="POSITION" start-index="7" stop-index="36" text="POSITION('bar' IN 'foobarbar')" />
+                    <function function-name="POSITION" start-index="7" stop-index="36" text="POSITION('bar' IN 'foobarbar')" >
+                        <parameter>
+                            <literal-expression value="bar" start-index="16" stop-index="20" />
+                        </parameter>
+                        <parameter>
+                            <literal-expression value="foobarbar" start-index="25" stop-index="35" />
+                        </parameter>
+                    </function>
                 </expr>
             </expression-projection>
         </projections>