You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/05/19 13:17:17 UTC

[GitHub] [iotdb] SteveYurongSu opened a new pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

SteveYurongSu opened a new pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] qiaojialin commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r637907886



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -523,14 +482,6 @@ nodeName
     | COUNT

Review comment:
       remove this also

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -146,82 +82,42 @@ public PhysicalPlan rawDataQueryReqToPhysicalPlan(
    * @return optimized logical operator
    * @throws LogicalOptimizeException exception in logical optimizing
    */
-  protected Operator logicalOptimize(Operator operator, int maxDeduplicatedPathNum)
+  protected Operator logicalOptimize(Operator operator, int fetchSize)
       throws LogicalOperatorException, PathNumOverLimitException {
-    switch (operator.getType()) {
-      case AUTHOR:
-      case METADATA:
-      case SET_STORAGE_GROUP:
-      case DELETE_STORAGE_GROUP:
-      case CREATE_TIMESERIES:
-      case DELETE_TIMESERIES:
-      case ALTER_TIMESERIES:
-      case LOADDATA:
-      case INSERT:
-      case GRANT_WATERMARK_EMBEDDING:
-      case REVOKE_WATERMARK_EMBEDDING:
-      case TTL:
-      case LOAD_CONFIGURATION:
-      case SHOW:
-      case LOAD_FILES:
-      case REMOVE_FILE:
-      case MOVE_FILE:
-      case FLUSH:
-      case MERGE:
-      case TRACING:
-      case CLEAR_CACHE:
-      case NULL:
-      case SHOW_MERGE_STATUS:
-      case DELETE_PARTITION:
-      case CREATE_SCHEMA_SNAPSHOT:
-      case KILL:
-      case CREATE_FUNCTION:
-      case DROP_FUNCTION:
-      case CREATE_TRIGGER:
-      case DROP_TRIGGER:
-      case START_TRIGGER:
-      case STOP_TRIGGER:
-        return operator;
-      case QUERY:
-      case DELETE:
-      case CREATE_INDEX:
-      case DROP_INDEX:
-      case QUERY_INDEX:
-        SFWOperator root = (SFWOperator) operator;
-        return optimizeSFWOperator(root, maxDeduplicatedPathNum);
-      default:
-        throw new LogicalOperatorException(operator.getType().toString(), "");
-    }
+    return operator.getType().equals(QUERY) || operator.getType().equals(QUERY_INDEX)
+        ? optimizeQueryOperator((QueryOperator) operator, fetchSize)
+        : operator;
   }
 
   /**
-   * given an unoptimized select-from-where operator and return an optimized result.
+   * given an unoptimized query operator and return an optimized result.
    *
-   * @param root unoptimized select-from-where operator
-   * @return optimized select-from-where operator
-   * @throws LogicalOptimizeException exception in SFW optimizing
+   * @param root unoptimized query operator
+   * @return optimized query operator
+   * @throws LogicalOptimizeException exception in query optimizing
    */
-  private SFWOperator optimizeSFWOperator(SFWOperator root, int maxDeduplicatedPathNum)
+  private QueryOperator optimizeQueryOperator(QueryOperator root, int fetchSize)
       throws LogicalOperatorException, PathNumOverLimitException {
-    ConcatPathOptimizer concatPathOptimizer = getConcatPathOptimizer();
-    root = (SFWOperator) concatPathOptimizer.transform(root, maxDeduplicatedPathNum);
+    root = (QueryOperator) getConcatPathOptimizer().transform(root, fetchSize);
+
     FilterOperator filter = root.getFilterOperator();
     if (filter == null) {
       return root;
     }
-    Set<PartialPath> pathSet = filter.getPathSet();
-    RemoveNotOptimizer removeNot = new RemoveNotOptimizer();
-    filter = removeNot.optimize(filter);
-    DnfFilterOptimizer dnf = new DnfFilterOptimizer();
-    filter = dnf.optimize(filter);
-    MergeSingleFilterOptimizer merge = new MergeSingleFilterOptimizer();
-    filter = merge.optimize(filter);
+    filter = new RemoveNotOptimizer().optimize(filter);
+    filter = new DnfFilterOptimizer().optimize(filter);
+    filter = new MergeSingleFilterOptimizer().optimize(filter);

Review comment:
       could these be changed to static method?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636745297



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {

Review comment:
       Actually all methods in ConcatPathOptimizer can be static... Let me remove it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636769938



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {
+    FromOperator from = queryOperator.getFromOperator();
+    if (from == null || from.getPrefixPaths().isEmpty()) {
+      LOGGER.warn(WARNING_NO_PREFIX_PATHS);
+      return false;
     }
 
-    // concat filter
-    FilterOperator filter = sfwOperator.getFilterOperator();
-    Set<PartialPath> filterPaths = new HashSet<>();
-    if (filter == null) {
-      return operator;
-    }
-    if (!isAlignByDevice) {
-      sfwOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
+    SelectOperator select = queryOperator.getSelectOperator();
+    if (select == null || select.getResultColumns().isEmpty()) {
+      LOGGER.warn(WARNING_NO_SUFFIX_PATHS);
+      return false;
     }
-    sfwOperator.getFilterOperator().setPathSet(filterPaths);
-    // GROUP_BY_DEVICE leaves the concatFilter to PhysicalGenerator to optimize filter without
-    // prefix first
 
-    return sfwOperator;
+    return true;
   }
 
-  private List<PartialPath> judgeSelectOperator(SelectOperator selectOperator)
-      throws LogicalOptimizeException {
-    List<PartialPath> suffixPaths;
-    if (selectOperator == null) {
-      throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-    } else {
-      suffixPaths = selectOperator.getSuffixPaths();
-      if (suffixPaths.isEmpty()) {
-        throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-      }
+  private void concatSelect(QueryOperator queryOperator) throws LogicalOptimizeException {
+    if (queryOperator.isAlignByDevice() && !queryOperator.isLastQuery()) {
+      return;

Review comment:
       Good suggestion! FIXED~!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r637783157



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;
+
+  private final int maxDeduplicatedPathNum;
+  private final int soffset;
+
+  private int offset;
+  private int limit;
+  private int consumed;
+
+  public WildcardsRemover(
+      ConcatPathOptimizer concatPathOptimizer, QueryOperator queryOperator, int fetchSize) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    // Dataset of last query actually has only three columns, so we shouldn't limit the path num
+    // while constructing logical plan
+    // To avoid overflowing because logicalOptimize function may do maxDeduplicatedPathNum + 1, we
+    // set it to Integer.MAX_VALUE - 1
+    maxDeduplicatedPathNum =
+        queryOperator.isLastQuery()
+            ? Integer.MAX_VALUE - 1
+            : QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
+    soffset = queryOperator.getSeriesOffset();
+    offset = soffset;
+
+    final int slimit = queryOperator.getSeriesLimit();
+    limit = slimit == 0 || maxDeduplicatedPathNum < slimit ? maxDeduplicatedPathNum + 1 : slimit;
+
+    consumed = 0;
+  }
+
+  public WildcardsRemover(ConcatPathOptimizer concatPathOptimizer) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
+    soffset = 0;
+    limit = maxDeduplicatedPathNum + 1;
+    consumed = 0;
+  }
+
+  public List<PartialPath> removeWildcardFrom(PartialPath path) throws LogicalOptimizeException {
+    try {
+      Pair<List<PartialPath>, Integer> pair =
+          concatPathOptimizer.removeWildcard(path, limit, offset);
+
+      consumed += pair.right;
+      if (offset != 0) {
+        int delta = offset - pair.right;
+        offset = Math.max(delta, 0);
+        if (delta < 0) {
+          limit += delta;
+        }
+      } else {
+        limit -= pair.right;
+      }
+
+      return pair.left;
+    } catch (MetadataException e) {
+      throw new LogicalOptimizeException("error occurred when removing star: " + e.getMessage());
+    }
+  }
+
+  public List<List<Expression>> removeWildcardsFrom(List<Expression> expressions)
+      throws LogicalOptimizeException {
+    List<List<Expression>> extendedExpressions = new ArrayList<>();
+
+    boolean atLeastOneSeriesNotExisted = false;
+    for (Expression originExpression : expressions) {
+      List<Expression> actualExpressions = new ArrayList<>();
+      originExpression.removeWildcards(
+          new WildcardsRemover(concatPathOptimizer), actualExpressions);
+      if (actualExpressions.isEmpty()) {
+        atLeastOneSeriesNotExisted = true;
+        break;
+      }
+      extendedExpressions.add(actualExpressions);
+    }
+    if (atLeastOneSeriesNotExisted) {
+      return Collections.emptyList();
+    }
+
+    List<List<Expression>> actualExpressions = new ArrayList<>();
+    ConcatPathOptimizer.cartesianProduct(
+        extendedExpressions, actualExpressions, 0, new ArrayList<>());
+
+    List<List<Expression>> splitExpressions = new ArrayList<>();
+    for (List<Expression> actualExpression : actualExpressions) {
+      if (offset != 0) {
+        --offset;
+        continue;
+      } else if (limit != 0) {
+        --limit;
+      } else {
+        break;
+      }
+      splitExpressions.add(actualExpression);
+    }
+    consumed += actualExpressions.size();
+    return splitExpressions;

Review comment:
       More comments are added. Thanks for reviewing!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] Alima777 commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
Alima777 commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636574522



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalChecker.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.iotdb.db.qp.strategy;
+
+import org.apache.iotdb.db.exception.query.LogicalOperatorException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.db.query.expression.unary.TimeSeriesOperand;
+
+public class LogicalChecker {
+
+  /**
+   * TODO: make check() an abstract method and call check() in this method or outside the
+   * LogicalChecker.
+   */
+  public static void check(Operator operator) throws LogicalOperatorException {
+    if (operator instanceof QueryOperator) {
+      checkQueryOperator((QueryOperator) operator);
+    }
+  }

Review comment:
       Later, I will move these check() to each subclass of `QueryOperator`, so don't worry about here.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.iotdb.db.query.expression;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.utils.WildcardsRemover;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.util.List;
+import java.util.Set;
+
+public abstract class Expression {
+
+  protected boolean isAggregationFunctionExpression = false;
+  protected boolean isTimeSeriesGeneratingFunctionExpression = false;
+

Review comment:
       Move these two flags to `FunctionExpression`, and i think one flag is enough. See the comment in `selectComponent`

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java
##########
@@ -30,87 +33,98 @@
 public final class SelectOperator extends Operator {
 
   private final ZoneId zoneId;
-  private List<PartialPath> suffixList;
-  private List<String> aggregations;
-  private List<UDFContext> udfList;
 
-  private boolean lastQuery;
-  private boolean udfQuery;
-  private boolean hasBuiltinAggregation;
+  private boolean isLastQuery = false;
+  private boolean hasAggregationFunction = false;
+  private boolean hasTimeSeriesGeneratingFunction = false;
+
+  private List<ResultColumn> resultColumns = new ArrayList<>();
+
+  private List<PartialPath> pathsCache;
+  private List<String> aggregationFunctionsCache;
 
   /** init with tokenIntType, default operatorType is <code>OperatorType.SELECT</code>. */
   public SelectOperator(int tokenIntType, ZoneId zoneId) {
     super(tokenIntType);
-    this.zoneId = zoneId;
     operatorType = OperatorType.SELECT;
-    suffixList = new ArrayList<>();
-    aggregations = new ArrayList<>();
-    udfList = new ArrayList<>();
-    lastQuery = false;
-    udfQuery = false;
-    hasBuiltinAggregation = false;
+    this.zoneId = zoneId;
   }
 
   public ZoneId getZoneId() {
     return zoneId;
   }
 
-  public void addSelectPath(PartialPath suffixPath) {
-    suffixList.add(suffixPath);
-  }
-
-  public void addClusterPath(PartialPath suffixPath, String aggregation) {
-    suffixList.add(suffixPath);
-    aggregations.add(aggregation);
-    if (aggregation != null) {
-      hasBuiltinAggregation = true;
-    }
+  public void markAsLastQuery() {
+    isLastQuery = true;
   }
 
   public boolean isLastQuery() {
-    return this.lastQuery;
+    return isLastQuery;
   }
 
-  public void setLastQuery() {
-    lastQuery = true;
+  public boolean hasAggregationFunction() {
+    return hasAggregationFunction;
   }
 
-  public List<String> getAggregations() {
-    return this.aggregations;
+  public boolean hasTimeSeriesGeneratingFunction() {
+    return hasTimeSeriesGeneratingFunction;
   }
 
-  public void setAggregations(List<String> aggregations) {
-    this.aggregations = aggregations;
+  public void addResultColumn(ResultColumn resultColumn) {
+    resultColumns.add(resultColumn);
+    if (resultColumn.getExpression().isAggregationFunctionExpression()) {
+      hasAggregationFunction = true;
+    }
+    if (resultColumn.getExpression().isTimeSeriesGeneratingFunctionExpression()) {
+      hasTimeSeriesGeneratingFunction = true;
+    }

Review comment:
       ```suggestion
       if (resultColumn.getExpression() instance of FunctionExpression) {
         Expression expression = (FunctionExpression) resultColumn.getExpression();
         if (expression.isAggregationFunctionExpression) {  
           hasAggregationFunction = true;
         } else {
           hasTimeSeriesGeneratingFunction = true;
         }
       }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-845643723


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-844190975


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r637770091



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;
+
+  private final int maxDeduplicatedPathNum;
+  private final int soffset;
+
+  private int offset;
+  private int limit;
+  private int consumed;
+
+  public WildcardsRemover(
+      ConcatPathOptimizer concatPathOptimizer, QueryOperator queryOperator, int fetchSize) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    // Dataset of last query actually has only three columns, so we shouldn't limit the path num
+    // while constructing logical plan
+    // To avoid overflowing because logicalOptimize function may do maxDeduplicatedPathNum + 1, we
+    // set it to Integer.MAX_VALUE - 1
+    maxDeduplicatedPathNum =
+        queryOperator.isLastQuery()
+            ? Integer.MAX_VALUE - 1
+            : QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
+    soffset = queryOperator.getSeriesOffset();
+    offset = soffset;
+
+    final int slimit = queryOperator.getSeriesLimit();
+    limit = slimit == 0 || maxDeduplicatedPathNum < slimit ? maxDeduplicatedPathNum + 1 : slimit;
+
+    consumed = 0;
+  }
+
+  public WildcardsRemover(ConcatPathOptimizer concatPathOptimizer) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
+    soffset = 0;
+    limit = maxDeduplicatedPathNum + 1;
+    consumed = 0;
+  }
+
+  public List<PartialPath> removeWildcardFrom(PartialPath path) throws LogicalOptimizeException {
+    try {
+      Pair<List<PartialPath>, Integer> pair =
+          concatPathOptimizer.removeWildcard(path, limit, offset);
+
+      consumed += pair.right;
+      if (offset != 0) {
+        int delta = offset - pair.right;
+        offset = Math.max(delta, 0);
+        if (delta < 0) {
+          limit += delta;
+        }
+      } else {
+        limit -= pair.right;
+      }
+
+      return pair.left;
+    } catch (MetadataException e) {
+      throw new LogicalOptimizeException("error occurred when removing star: " + e.getMessage());
+    }
+  }
+
+  public List<List<Expression>> removeWildcardsFrom(List<Expression> expressions)
+      throws LogicalOptimizeException {
+    List<List<Expression>> extendedExpressions = new ArrayList<>();
+
+    boolean atLeastOneSeriesNotExisted = false;
+    for (Expression originExpression : expressions) {
+      List<Expression> actualExpressions = new ArrayList<>();
+      originExpression.removeWildcards(
+          new WildcardsRemover(concatPathOptimizer), actualExpressions);
+      if (actualExpressions.isEmpty()) {
+        atLeastOneSeriesNotExisted = true;
+        break;
+      }
+      extendedExpressions.add(actualExpressions);
+    }
+    if (atLeastOneSeriesNotExisted) {
+      return Collections.emptyList();
+    }

Review comment:
       > Now that once one series is not existed, break the loop and return, I think we don't need the flag.
   
   Very GOOD finding! I fixed it.
   
   >  And I doubt that whether we should return a empty list or throw an exception.
   
   We ignore the eval of the function which has at least one non-existence series as input. 
   
   See IOTDB-1212: https://github.com/apache/iotdb/pull/3101




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636749472



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -103,71 +103,30 @@ statement
     | START TRIGGER triggerName=ID #startTrigger
     | STOP TRIGGER triggerName=ID #stopTrigger
     | SHOW TRIGGERS #showTriggers
-    | SELECT topClause? selectElements
-    fromClause
-    whereClause?
-    specialClause? #selectStatement
+    | selectClause fromClause whereClause? specialClause? #selectStatement
     ;
 
-selectElements
-    : aggregationCall (COMMA aggregationCall)* #aggregationElement
-    | tableCall (COMMA tableCall)* #tableElement
-    | lastClause #lastElement
-    | asClause (COMMA asClause)* #asElement
-    | functionAsClause (COMMA functionAsClause)* #functionAsElement
-    ;
-
-aggregationCall
-    : builtInFunctionCall
-    | udfCall
-    ;
-
-tableCall
-    : suffixPath
-    | udfCall
-    | SINGLE_QUOTE_STRING_LITERAL
-    ;
-
-udfCall
-    : udfName=ID LR_BRACKET udfSuffixPaths udfAttribute* RR_BRACKET
-    ;
-
-udfSuffixPaths
-    : suffixPath (COMMA suffixPath)*
-    ;
-
-udfAttribute
-    : COMMA udfAttributeKey=stringLiteral OPERATOR_EQ udfAttributeValue=stringLiteral
-    ;
-
-builtInFunctionCall
-    : functionName LR_BRACKET suffixPath RR_BRACKET
-    ;
-
-functionName
-    : MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | COUNT
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
-    ;
+selectClause
+   : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
+   ;
 
-functionAsClause
-    : builtInFunctionCall (AS ID)?
-    ;
+resultColumn
+   : expression (AS ID)?
+   ;
 
-lastClause
-    : LAST suffixPath (COMMA suffixPath)*
-    | LAST asClause (COMMA asClause)*
-    ;
+expression
+   : LR_BRACKET unary=expression RR_BRACKET
+   | (PLUS | MINUS) unary=expression
+   | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
+   | leftExpression=expression (PLUS | MINUS) rightExpression=expression
+   | functionName=suffixPath LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
+   | suffixPath

Review comment:
       The order defined in SqlBase.g4 cannot be changed. The order is meaningful, it implicitly defines the priority of antlr to parse the expression.
   
   On the other hand, the order of expreesion parsing in parseExpression() is classified according to whether the expression is binary or unary. I think it is clearer.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r638618442



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;
+
+  private final int maxDeduplicatedPathNum;
+  private final int soffset;
+
+  private int offset;
+  private int limit;
+  private int consumed;
+
+  public WildcardsRemover(
+      ConcatPathOptimizer concatPathOptimizer, QueryOperator queryOperator, int fetchSize) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    // Dataset of last query actually has only three columns, so we shouldn't limit the path num
+    // while constructing logical plan
+    // To avoid overflowing because logicalOptimize function may do maxDeduplicatedPathNum + 1, we
+    // set it to Integer.MAX_VALUE - 1
+    maxDeduplicatedPathNum =
+        queryOperator.isLastQuery()
+            ? Integer.MAX_VALUE - 1
+            : QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
+    soffset = queryOperator.getSeriesOffset();
+    offset = soffset;
+
+    final int slimit = queryOperator.getSeriesLimit();
+    limit = slimit == 0 || maxDeduplicatedPathNum < slimit ? maxDeduplicatedPathNum + 1 : slimit;
+
+    consumed = 0;
+  }
+
+  public WildcardsRemover(ConcatPathOptimizer concatPathOptimizer) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
+    soffset = 0;
+    limit = maxDeduplicatedPathNum + 1;
+    consumed = 0;
+  }
+
+  public List<PartialPath> removeWildcardFrom(PartialPath path) throws LogicalOptimizeException {
+    try {
+      Pair<List<PartialPath>, Integer> pair =
+          concatPathOptimizer.removeWildcard(path, limit, offset);
+
+      consumed += pair.right;

Review comment:
       In WildcardsRemover, we use 
   ```
   WildcardsRemover#checkIfPathNumberIsOverLimit
   WildcardsRemover#checkIfSoffsetIsExceeded
   ```
   to check.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636752769



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/RemoveNotOptimizer.java
##########
@@ -41,7 +43,10 @@
    */
   @Override
   public FilterOperator optimize(FilterOperator filter) throws LogicalOperatorException {
-    return removeNot(filter);
+    Set<PartialPath> pathSet = filter.getPathSet();
+    FilterOperator optimizedFilterOperator = removeNot(filter);
+    optimizedFilterOperator.setPathSet(pathSet);
+    return optimizedFilterOperator;

Review comment:
       Previously this code actually appeared in `optimizeSFWOperator`... Removing the code would cause tests failed... The reason is that the `removeNot` may not return the original `OptimizedFilterOperator`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-845643723


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] liutaohua commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r638491012



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -306,20 +187,14 @@ private FilterOperator constructBinaryFilterTreeWithAnd(
     return filterBinaryTree;
   }
 
-  /**
-   * replace "*" by actual paths.
-   *
-   * @param paths list of paths which may contain stars
-   * @return a unique seriesPath list
-   */
-  private List<PartialPath> removeStarsInPathWithUnique(List<PartialPath> paths)
+  private List<PartialPath> removeWildcardsInConcatPaths(List<PartialPath> paths)
       throws LogicalOptimizeException {
     List<PartialPath> retPaths = new ArrayList<>();
     HashSet<PartialPath> pathSet = new HashSet<>();

Review comment:
       What's the difference between a `pathSet` and `retPaths`

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;
+
+  private final int maxDeduplicatedPathNum;
+  private final int soffset;
+
+  private int offset;
+  private int limit;
+  private int consumed;
+
+  public WildcardsRemover(
+      ConcatPathOptimizer concatPathOptimizer, QueryOperator queryOperator, int fetchSize) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    // Dataset of last query actually has only three columns, so we shouldn't limit the path num
+    // while constructing logical plan
+    // To avoid overflowing because logicalOptimize function may do maxDeduplicatedPathNum + 1, we
+    // set it to Integer.MAX_VALUE - 1
+    maxDeduplicatedPathNum =
+        queryOperator.isLastQuery()
+            ? Integer.MAX_VALUE - 1
+            : QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
+    soffset = queryOperator.getSeriesOffset();
+    offset = soffset;
+
+    final int slimit = queryOperator.getSeriesLimit();
+    limit = slimit == 0 || maxDeduplicatedPathNum < slimit ? maxDeduplicatedPathNum + 1 : slimit;
+
+    consumed = 0;
+  }
+
+  public WildcardsRemover(ConcatPathOptimizer concatPathOptimizer) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
+    soffset = 0;
+    limit = maxDeduplicatedPathNum + 1;
+    consumed = 0;
+  }
+
+  public List<PartialPath> removeWildcardFrom(PartialPath path) throws LogicalOptimizeException {
+    try {
+      Pair<List<PartialPath>, Integer> pair =
+          concatPathOptimizer.removeWildcard(path, limit, offset);
+
+      consumed += pair.right;

Review comment:
       There is less checking of the `path` compared to Filter's `RemoveWildCard` method.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;

Review comment:
       Can we reduce circular dependencies in `Remover` and `Optimizer` through high-level abstraction?

##########
File path: server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.iotdb.db.query.expression;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.utils.WildcardsRemover;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.util.List;
+import java.util.Set;
+
+public interface Expression {
+
+  default boolean isAggregationFunctionExpression() {
+    return false;
+  }
+
+  default boolean isTimeSeriesGeneratingFunctionExpression() {
+    return false;
+  }
+
+  // TODO: implement this method
+  TSDataType dataType() throws MetadataException;
+
+  void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions);
+
+  void removeWildcards(WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)

Review comment:
       The act of removing the asterisk is not only the Expression in the `select`, but also belongs to the path in the `where`. 
   Is it possible to abstract both `Expression` and `where`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-845854193


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] liutaohua commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636603307



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -103,71 +103,30 @@ statement
     | START TRIGGER triggerName=ID #startTrigger
     | STOP TRIGGER triggerName=ID #stopTrigger
     | SHOW TRIGGERS #showTriggers
-    | SELECT topClause? selectElements
-    fromClause
-    whereClause?
-    specialClause? #selectStatement
+    | selectClause fromClause whereClause? specialClause? #selectStatement
     ;
 
-selectElements
-    : aggregationCall (COMMA aggregationCall)* #aggregationElement
-    | tableCall (COMMA tableCall)* #tableElement
-    | lastClause #lastElement
-    | asClause (COMMA asClause)* #asElement
-    | functionAsClause (COMMA functionAsClause)* #functionAsElement
-    ;
-
-aggregationCall
-    : builtInFunctionCall
-    | udfCall
-    ;
-
-tableCall
-    : suffixPath
-    | udfCall
-    | SINGLE_QUOTE_STRING_LITERAL
-    ;
-
-udfCall
-    : udfName=ID LR_BRACKET udfSuffixPaths udfAttribute* RR_BRACKET
-    ;
-
-udfSuffixPaths
-    : suffixPath (COMMA suffixPath)*
-    ;
-
-udfAttribute
-    : COMMA udfAttributeKey=stringLiteral OPERATOR_EQ udfAttributeValue=stringLiteral
-    ;
-
-builtInFunctionCall
-    : functionName LR_BRACKET suffixPath RR_BRACKET
-    ;
-
-functionName
-    : MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | COUNT
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
-    ;
+selectClause
+   : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
+   ;
 
-functionAsClause
-    : builtInFunctionCall (AS ID)?
-    ;
+resultColumn
+   : expression (AS ID)?
+   ;
 
-lastClause
-    : LAST suffixPath (COMMA suffixPath)*
-    | LAST asClause (COMMA asClause)*
-    ;
+expression
+   : LR_BRACKET unary=expression RR_BRACKET
+   | (PLUS | MINUS) unary=expression
+   | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
+   | leftExpression=expression (PLUS | MINUS) rightExpression=expression
+   | functionName=suffixPath LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
+   | suffixPath

Review comment:
       `SuffixPath` is so open that it supports almost everything. We should restrict it to `+ - * / % mod `and other special characters

##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -103,71 +103,30 @@ statement
     | START TRIGGER triggerName=ID #startTrigger
     | STOP TRIGGER triggerName=ID #stopTrigger
     | SHOW TRIGGERS #showTriggers
-    | SELECT topClause? selectElements
-    fromClause
-    whereClause?
-    specialClause? #selectStatement
+    | selectClause fromClause whereClause? specialClause? #selectStatement
     ;
 
-selectElements
-    : aggregationCall (COMMA aggregationCall)* #aggregationElement
-    | tableCall (COMMA tableCall)* #tableElement
-    | lastClause #lastElement
-    | asClause (COMMA asClause)* #asElement
-    | functionAsClause (COMMA functionAsClause)* #functionAsElement
-    ;
-
-aggregationCall
-    : builtInFunctionCall
-    | udfCall
-    ;
-
-tableCall
-    : suffixPath
-    | udfCall
-    | SINGLE_QUOTE_STRING_LITERAL
-    ;
-
-udfCall
-    : udfName=ID LR_BRACKET udfSuffixPaths udfAttribute* RR_BRACKET
-    ;
-
-udfSuffixPaths
-    : suffixPath (COMMA suffixPath)*
-    ;
-
-udfAttribute
-    : COMMA udfAttributeKey=stringLiteral OPERATOR_EQ udfAttributeValue=stringLiteral
-    ;
-
-builtInFunctionCall
-    : functionName LR_BRACKET suffixPath RR_BRACKET
-    ;
-
-functionName
-    : MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | COUNT
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
-    ;
+selectClause
+   : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
+   ;
 
-functionAsClause
-    : builtInFunctionCall (AS ID)?
-    ;
+resultColumn
+   : expression (AS ID)?
+   ;
 
-lastClause
-    : LAST suffixPath (COMMA suffixPath)*
-    | LAST asClause (COMMA asClause)*
-    ;
+expression
+   : LR_BRACKET unary=expression RR_BRACKET
+   | (PLUS | MINUS) unary=expression
+   | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
+   | leftExpression=expression (PLUS | MINUS) rightExpression=expression
+   | functionName=suffixPath LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
+   | suffixPath
+   | literal=SINGLE_QUOTE_STRING_LITERAL

Review comment:
       Special branches that are only used as `align by device` are not recommended to be retained in recursion. 
   I think it will affect the speed of parsing.

##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -1370,6 +1281,10 @@ MINUS : '-';
 
 PLUS : '+';
 
+DIV : '/';
+
+MOD : M O D | '%';

Review comment:
       `MOD` keywords are not recommended, which makes it necessary to reserve more keywords




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-845854193


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-846885410


   > LGTM now. Thank you for the great contribution.
   
   Oh, Thanks for your great comments!!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r638615504



##########
File path: server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.iotdb.db.query.expression;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.utils.WildcardsRemover;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.util.List;
+import java.util.Set;
+
+public interface Expression {
+
+  default boolean isAggregationFunctionExpression() {
+    return false;
+  }
+
+  default boolean isTimeSeriesGeneratingFunctionExpression() {
+    return false;
+  }
+
+  // TODO: implement this method
+  TSDataType dataType() throws MetadataException;
+
+  void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions);
+
+  void removeWildcards(WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)

Review comment:
       Yes, this is a good idea! 
   
   At present, the Expression used by select clauses and the Expression used by where clauses are completely different. I think we can abstract them in the future. 
   
   I have recorded a JIRA: https://issues.apache.org/jira/browse/IOTDB-1395




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r638601681



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -306,20 +187,14 @@ private FilterOperator constructBinaryFilterTreeWithAnd(
     return filterBinaryTree;
   }
 
-  /**
-   * replace "*" by actual paths.
-   *
-   * @param paths list of paths which may contain stars
-   * @return a unique seriesPath list
-   */
-  private List<PartialPath> removeStarsInPathWithUnique(List<PartialPath> paths)
+  private List<PartialPath> removeWildcardsInConcatPaths(List<PartialPath> paths)
       throws LogicalOptimizeException {
     List<PartialPath> retPaths = new ArrayList<>();
     HashSet<PartialPath> pathSet = new HashSet<>();

Review comment:
       Good discovery! This is code from ancient times. I've made some changes!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r638392664



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -523,14 +482,6 @@ nodeName
     | COUNT

Review comment:
       This `COUNT` is still used in 
   ``` antlr
   COUNT TIMESERIES prefixPath? (GROUP BY LEVEL OPERATOR_EQ INT)? #countTimeseries
   COUNT DEVICES prefixPath? #countDevices
   COUNT STORAGE GROUP prefixPath? #countStorageGroup
   COUNT NODES prefixPath LEVEL OPERATOR_EQ INT #countNodes
   ```
   SO it cannot be removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-844190975


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.3%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-846901769


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [37 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636754169



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -1370,6 +1281,10 @@ MINUS : '-';
 
 PLUS : '+';
 
+DIV : '/';
+
+MOD : M O D | '%';

Review comment:
       I removed the `M O D`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] liutaohua merged pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
liutaohua merged pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636792221



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -103,71 +103,30 @@ statement
     | START TRIGGER triggerName=ID #startTrigger
     | STOP TRIGGER triggerName=ID #stopTrigger
     | SHOW TRIGGERS #showTriggers
-    | SELECT topClause? selectElements
-    fromClause
-    whereClause?
-    specialClause? #selectStatement
+    | selectClause fromClause whereClause? specialClause? #selectStatement
     ;
 
-selectElements
-    : aggregationCall (COMMA aggregationCall)* #aggregationElement
-    | tableCall (COMMA tableCall)* #tableElement
-    | lastClause #lastElement
-    | asClause (COMMA asClause)* #asElement
-    | functionAsClause (COMMA functionAsClause)* #functionAsElement
-    ;
-
-aggregationCall
-    : builtInFunctionCall
-    | udfCall
-    ;
-
-tableCall
-    : suffixPath
-    | udfCall
-    | SINGLE_QUOTE_STRING_LITERAL
-    ;
-
-udfCall
-    : udfName=ID LR_BRACKET udfSuffixPaths udfAttribute* RR_BRACKET
-    ;
-
-udfSuffixPaths
-    : suffixPath (COMMA suffixPath)*
-    ;
-
-udfAttribute
-    : COMMA udfAttributeKey=stringLiteral OPERATOR_EQ udfAttributeValue=stringLiteral
-    ;
-
-builtInFunctionCall
-    : functionName LR_BRACKET suffixPath RR_BRACKET
-    ;
-
-functionName
-    : MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | COUNT
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
-    ;
+selectClause
+   : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
+   ;
 
-functionAsClause
-    : builtInFunctionCall (AS ID)?
-    ;
+resultColumn
+   : expression (AS ID)?
+   ;
 
-lastClause
-    : LAST suffixPath (COMMA suffixPath)*
-    | LAST asClause (COMMA asClause)*
-    ;
+expression
+   : LR_BRACKET unary=expression RR_BRACKET
+   | (PLUS | MINUS) unary=expression
+   | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
+   | leftExpression=expression (PLUS | MINUS) rightExpression=expression
+   | functionName=suffixPath LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
+   | suffixPath
+   | literal=SINGLE_QUOTE_STRING_LITERAL

Review comment:
       I think `literal` should still be under `expression` for the following reasons:
   1. We are likely to support `literal` in expressions in the future, such as select a + 1 from xxx. `literal` cannot be counted as a special field.
   2. Putting `literal` in `resultColumn` still cannot avoid checking the logical operator after it is generated.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-846683778


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.6%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.6% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r638602034



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;

Review comment:
       Fixed 👍🏼 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-847745307


   @liutaohua Thanks for reviewing this PR! I will merge this PR to master. If you still find any issues, please continue to raise them here, I will solve them in next PR. 😊
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] Alima777 commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
Alima777 commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636651590



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {

Review comment:
       Why a private static method?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {
+    FromOperator from = queryOperator.getFromOperator();
+    if (from == null || from.getPrefixPaths().isEmpty()) {
+      LOGGER.warn(WARNING_NO_PREFIX_PATHS);
+      return false;
     }
 
-    // concat filter
-    FilterOperator filter = sfwOperator.getFilterOperator();
-    Set<PartialPath> filterPaths = new HashSet<>();
-    if (filter == null) {
-      return operator;
-    }
-    if (!isAlignByDevice) {
-      sfwOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
+    SelectOperator select = queryOperator.getSelectOperator();
+    if (select == null || select.getResultColumns().isEmpty()) {
+      LOGGER.warn(WARNING_NO_SUFFIX_PATHS);
+      return false;

Review comment:
       It's ok but i'd like to put `select` in front of `from`

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);

Review comment:
       Since `alignbydevice` is judged in all three methods, you can put it outside or check it in optimizable() method.

##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -103,71 +103,30 @@ statement
     | START TRIGGER triggerName=ID #startTrigger
     | STOP TRIGGER triggerName=ID #stopTrigger
     | SHOW TRIGGERS #showTriggers
-    | SELECT topClause? selectElements
-    fromClause
-    whereClause?
-    specialClause? #selectStatement
+    | selectClause fromClause whereClause? specialClause? #selectStatement
     ;
 
-selectElements
-    : aggregationCall (COMMA aggregationCall)* #aggregationElement
-    | tableCall (COMMA tableCall)* #tableElement
-    | lastClause #lastElement
-    | asClause (COMMA asClause)* #asElement
-    | functionAsClause (COMMA functionAsClause)* #functionAsElement
-    ;
-
-aggregationCall
-    : builtInFunctionCall
-    | udfCall
-    ;
-
-tableCall
-    : suffixPath
-    | udfCall
-    | SINGLE_QUOTE_STRING_LITERAL
-    ;
-
-udfCall
-    : udfName=ID LR_BRACKET udfSuffixPaths udfAttribute* RR_BRACKET
-    ;
-
-udfSuffixPaths
-    : suffixPath (COMMA suffixPath)*
-    ;
-
-udfAttribute
-    : COMMA udfAttributeKey=stringLiteral OPERATOR_EQ udfAttributeValue=stringLiteral
-    ;
-
-builtInFunctionCall
-    : functionName LR_BRACKET suffixPath RR_BRACKET
-    ;
-
-functionName
-    : MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | COUNT
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
-    ;
+selectClause
+   : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
+   ;
 
-functionAsClause
-    : builtInFunctionCall (AS ID)?
-    ;
+resultColumn
+   : expression (AS ID)?
+   ;
 
-lastClause
-    : LAST suffixPath (COMMA suffixPath)*
-    | LAST asClause (COMMA asClause)*
-    ;
+expression
+   : LR_BRACKET unary=expression RR_BRACKET
+   | (PLUS | MINUS) unary=expression
+   | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
+   | leftExpression=expression (PLUS | MINUS) rightExpression=expression
+   | functionName=suffixPath LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
+   | suffixPath

Review comment:
       It's ok now. But I think you'd better keep the sequence the same as the parseExpression() in IoTDBSQLVisitor. It's easier to understand and maintain.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {
+    FromOperator from = queryOperator.getFromOperator();
+    if (from == null || from.getPrefixPaths().isEmpty()) {
+      LOGGER.warn(WARNING_NO_PREFIX_PATHS);
+      return false;
     }
 
-    // concat filter
-    FilterOperator filter = sfwOperator.getFilterOperator();
-    Set<PartialPath> filterPaths = new HashSet<>();
-    if (filter == null) {
-      return operator;
-    }
-    if (!isAlignByDevice) {
-      sfwOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
+    SelectOperator select = queryOperator.getSelectOperator();
+    if (select == null || select.getResultColumns().isEmpty()) {
+      LOGGER.warn(WARNING_NO_SUFFIX_PATHS);
+      return false;
     }
-    sfwOperator.getFilterOperator().setPathSet(filterPaths);
-    // GROUP_BY_DEVICE leaves the concatFilter to PhysicalGenerator to optimize filter without
-    // prefix first
 
-    return sfwOperator;
+    return true;
   }
 
-  private List<PartialPath> judgeSelectOperator(SelectOperator selectOperator)
-      throws LogicalOptimizeException {
-    List<PartialPath> suffixPaths;
-    if (selectOperator == null) {
-      throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-    } else {
-      suffixPaths = selectOperator.getSuffixPaths();
-      if (suffixPaths.isEmpty()) {
-        throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-      }
+  private void concatSelect(QueryOperator queryOperator) throws LogicalOptimizeException {
+    if (queryOperator.isAlignByDevice() && !queryOperator.isLastQuery()) {
+      return;

Review comment:
       Actually, last query is not supposed to support align by device, so you just need to judge `queryOperator.isAlignByDevice()`. 
   Leave the check() to me or you can add it in checkLast() if you can not pass tests.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/RemoveNotOptimizer.java
##########
@@ -41,7 +43,10 @@
    */
   @Override
   public FilterOperator optimize(FilterOperator filter) throws LogicalOperatorException {
-    return removeNot(filter);
+    Set<PartialPath> pathSet = filter.getPathSet();
+    FilterOperator optimizedFilterOperator = removeNot(filter);
+    optimizedFilterOperator.setPathSet(pathSet);
+    return optimizedFilterOperator;

Review comment:
       It seems `pathSet` is not changed in removeNot() method, why reseting it?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {
+    FromOperator from = queryOperator.getFromOperator();
+    if (from == null || from.getPrefixPaths().isEmpty()) {
+      LOGGER.warn(WARNING_NO_PREFIX_PATHS);
+      return false;
     }
 
-    // concat filter
-    FilterOperator filter = sfwOperator.getFilterOperator();
-    Set<PartialPath> filterPaths = new HashSet<>();
-    if (filter == null) {
-      return operator;
-    }
-    if (!isAlignByDevice) {
-      sfwOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
+    SelectOperator select = queryOperator.getSelectOperator();
+    if (select == null || select.getResultColumns().isEmpty()) {
+      LOGGER.warn(WARNING_NO_SUFFIX_PATHS);
+      return false;
     }
-    sfwOperator.getFilterOperator().setPathSet(filterPaths);
-    // GROUP_BY_DEVICE leaves the concatFilter to PhysicalGenerator to optimize filter without
-    // prefix first
 
-    return sfwOperator;
+    return true;
   }
 
-  private List<PartialPath> judgeSelectOperator(SelectOperator selectOperator)
-      throws LogicalOptimizeException {
-    List<PartialPath> suffixPaths;
-    if (selectOperator == null) {
-      throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-    } else {
-      suffixPaths = selectOperator.getSuffixPaths();
-      if (suffixPaths.isEmpty()) {
-        throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-      }
+  private void concatSelect(QueryOperator queryOperator) throws LogicalOptimizeException {
+    if (queryOperator.isAlignByDevice() && !queryOperator.isLastQuery()) {
+      return;
     }
-    return suffixPaths;
-  }
 
-  private void checkAggrOfSelectOperator(SelectOperator selectOperator)
-      throws LogicalOptimizeException {
-    if (!selectOperator.getAggregations().isEmpty()
-        && selectOperator.getSuffixPaths().size() != selectOperator.getAggregations().size()) {
-      throw new LogicalOptimizeException(
-          "Common queries and aggregated queries are not allowed to appear at the same time");
+    List<PartialPath> prefixPaths = queryOperator.getFromOperator().getPrefixPaths();
+    List<ResultColumn> resultColumns = new ArrayList<>();
+    for (ResultColumn suffixColumn : queryOperator.getSelectOperator().getResultColumns()) {
+      suffixColumn.concat(prefixPaths, resultColumns);
     }
+    queryOperator.getSelectOperator().setResultColumns(resultColumns);
   }
 
-  private void extendListSafely(List<String> source, int index, List<String> target) {
-    if (source != null && !source.isEmpty()) {
-      target.add(source.get(index));
-    }
-  }
-
-  /**
-   * Extract paths from select&from cql, expand them into complete versions, and reassign them to
-   * selectOperator's suffixPathList. Treat aggregations similarly.
-   */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void concatSelect(
-      List<PartialPath> fromPaths,
-      SelectOperator selectOperator,
-      int limit,
-      int offset,
-      int maxDeduplicatedPathNum,
-      boolean needRemoveStar)
+  private void removeWildcardsInSelectPaths(QueryOperator queryOperator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    List<PartialPath> suffixPaths = judgeSelectOperator(selectOperator);
-    List<PartialPath> afterConcatPaths = new ArrayList<>(); // null elements are for the UDFs
-
-    List<String> originAggregations = selectOperator.getAggregations();
-    List<String> afterConcatAggregations = new ArrayList<>(); // null elements are for the UDFs
-
-    List<UDFContext> originUdfList = selectOperator.getUdfList();
-    List<UDFContext> afterConcatUdfList = new ArrayList<>();
-
-    for (int i = 0; i < suffixPaths.size(); i++) {
-      // selectPath cannot start with ROOT, which is guaranteed by TSParser
-      PartialPath selectPath = suffixPaths.get(i);
-
-      if (selectPath == null) { // udf
-        UDFContext originUdf = originUdfList.get(i);
-        List<PartialPath> originUdfSuffixPaths = originUdf.getPaths();
-
-        List<List<PartialPath>> afterConcatUdfPathsList = new ArrayList<>();
-        for (PartialPath originUdfSuffixPath : originUdfSuffixPaths) {
-          List<PartialPath> afterConcatUdfPaths = new ArrayList<>();
-          for (PartialPath fromPath : fromPaths) {
-            afterConcatUdfPaths.add(fromPath.concatPath(originUdfSuffixPath));
-          }
-          afterConcatUdfPathsList.add(afterConcatUdfPaths);
-        }
-        List<List<PartialPath>> extendedAfterConcatUdfPathsList = new ArrayList<>();
-        cartesianProduct(
-            afterConcatUdfPathsList, extendedAfterConcatUdfPathsList, 0, new ArrayList<>());
-
-        for (List<PartialPath> afterConcatUdfPaths : extendedAfterConcatUdfPathsList) {
-          afterConcatPaths.add(null);
-          extendListSafely(originAggregations, i, afterConcatAggregations);
-
-          afterConcatUdfList.add(
-              new UDFContext(originUdf.getName(), originUdf.getAttributes(), afterConcatUdfPaths));
-        }
-      } else { // non-udf
-        for (PartialPath fromPath : fromPaths) {
-          PartialPath fullPath = fromPath.concatPath(selectPath);
-          if (selectPath.isTsAliasExists()) {
-            fullPath.setTsAlias(selectPath.getTsAlias());
-          }
-          afterConcatPaths.add(fullPath);
-          extendListSafely(originAggregations, i, afterConcatAggregations);
+    if (queryOperator.isAlignByDevice() && !queryOperator.isLastQuery()
+        || queryOperator.getIndexType() != null) {
+      return;
+    }
 
-          afterConcatUdfList.add(null);
-        }
+    WildcardsRemover wildcardsRemover = new WildcardsRemover(this, queryOperator, fetchSize);
+    List<ResultColumn> resultColumns = new ArrayList<>();
+    for (ResultColumn resultColumn : queryOperator.getSelectOperator().getResultColumns()) {
+      resultColumn.removeWildcards(wildcardsRemover, resultColumns);
+      if (wildcardsRemover.checkIfPathNumberIsOverLimit(resultColumns)) {
+        break;
       }
     }
+    wildcardsRemover.checkIfSoffsetIsExceeded(resultColumns);
+    queryOperator.getSelectOperator().setResultColumns(resultColumns);
+  }
 
-    if (needRemoveStar) {
-      removeStarsInPath(
-          afterConcatPaths,
-          afterConcatAggregations,
-          afterConcatUdfList,
-          selectOperator,
-          limit,
-          offset,
-          maxDeduplicatedPathNum);
-    } else {
-      selectOperator.setSuffixPathList(afterConcatPaths);
+  private void concatFilter(QueryOperator queryOperator) throws LogicalOptimizeException {
+    FilterOperator filter = queryOperator.getFilterOperator();
+    if (filter == null) {
+      return;
     }
+
+    Set<PartialPath> filterPaths = new HashSet<>();
+    if (!queryOperator.isAlignByDevice() || queryOperator.isLastQuery()) {
+      // GROUP_BY_DEVICE leaves the concatFilter to PhysicalGenerator to optimize filter without
+      // prefix first
+      queryOperator.setFilterOperator(
+          concatFilter(queryOperator.getFromOperator().getPrefixPaths(), filter, filterPaths));
+    }
+    queryOperator.getFilterOperator().setPathSet(filterPaths);

Review comment:
       The logic is confusing here. Actually it's the same as `concatSelect`. If `alignbydevice`, return, otherwise concat filter.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {
+    FromOperator from = queryOperator.getFromOperator();
+    if (from == null || from.getPrefixPaths().isEmpty()) {
+      LOGGER.warn(WARNING_NO_PREFIX_PATHS);
+      return false;
     }
 
-    // concat filter
-    FilterOperator filter = sfwOperator.getFilterOperator();
-    Set<PartialPath> filterPaths = new HashSet<>();
-    if (filter == null) {
-      return operator;
-    }
-    if (!isAlignByDevice) {
-      sfwOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
+    SelectOperator select = queryOperator.getSelectOperator();
+    if (select == null || select.getResultColumns().isEmpty()) {
+      LOGGER.warn(WARNING_NO_SUFFIX_PATHS);
+      return false;
     }
-    sfwOperator.getFilterOperator().setPathSet(filterPaths);
-    // GROUP_BY_DEVICE leaves the concatFilter to PhysicalGenerator to optimize filter without
-    // prefix first
 
-    return sfwOperator;
+    return true;
   }
 
-  private List<PartialPath> judgeSelectOperator(SelectOperator selectOperator)
-      throws LogicalOptimizeException {
-    List<PartialPath> suffixPaths;
-    if (selectOperator == null) {
-      throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-    } else {
-      suffixPaths = selectOperator.getSuffixPaths();
-      if (suffixPaths.isEmpty()) {
-        throw new LogicalOptimizeException(WARNING_NO_SUFFIX_PATHS);
-      }
+  private void concatSelect(QueryOperator queryOperator) throws LogicalOptimizeException {
+    if (queryOperator.isAlignByDevice() && !queryOperator.isLastQuery()) {
+      return;
     }
-    return suffixPaths;
-  }
 
-  private void checkAggrOfSelectOperator(SelectOperator selectOperator)
-      throws LogicalOptimizeException {
-    if (!selectOperator.getAggregations().isEmpty()
-        && selectOperator.getSuffixPaths().size() != selectOperator.getAggregations().size()) {
-      throw new LogicalOptimizeException(
-          "Common queries and aggregated queries are not allowed to appear at the same time");
+    List<PartialPath> prefixPaths = queryOperator.getFromOperator().getPrefixPaths();
+    List<ResultColumn> resultColumns = new ArrayList<>();
+    for (ResultColumn suffixColumn : queryOperator.getSelectOperator().getResultColumns()) {
+      suffixColumn.concat(prefixPaths, resultColumns);
     }
+    queryOperator.getSelectOperator().setResultColumns(resultColumns);
   }
 
-  private void extendListSafely(List<String> source, int index, List<String> target) {
-    if (source != null && !source.isEmpty()) {
-      target.add(source.get(index));
-    }
-  }
-
-  /**
-   * Extract paths from select&from cql, expand them into complete versions, and reassign them to
-   * selectOperator's suffixPathList. Treat aggregations similarly.
-   */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void concatSelect(
-      List<PartialPath> fromPaths,
-      SelectOperator selectOperator,
-      int limit,
-      int offset,
-      int maxDeduplicatedPathNum,
-      boolean needRemoveStar)
+  private void removeWildcardsInSelectPaths(QueryOperator queryOperator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    List<PartialPath> suffixPaths = judgeSelectOperator(selectOperator);
-    List<PartialPath> afterConcatPaths = new ArrayList<>(); // null elements are for the UDFs
-
-    List<String> originAggregations = selectOperator.getAggregations();
-    List<String> afterConcatAggregations = new ArrayList<>(); // null elements are for the UDFs
-
-    List<UDFContext> originUdfList = selectOperator.getUdfList();
-    List<UDFContext> afterConcatUdfList = new ArrayList<>();
-
-    for (int i = 0; i < suffixPaths.size(); i++) {
-      // selectPath cannot start with ROOT, which is guaranteed by TSParser
-      PartialPath selectPath = suffixPaths.get(i);
-
-      if (selectPath == null) { // udf
-        UDFContext originUdf = originUdfList.get(i);
-        List<PartialPath> originUdfSuffixPaths = originUdf.getPaths();
-
-        List<List<PartialPath>> afterConcatUdfPathsList = new ArrayList<>();
-        for (PartialPath originUdfSuffixPath : originUdfSuffixPaths) {
-          List<PartialPath> afterConcatUdfPaths = new ArrayList<>();
-          for (PartialPath fromPath : fromPaths) {
-            afterConcatUdfPaths.add(fromPath.concatPath(originUdfSuffixPath));
-          }
-          afterConcatUdfPathsList.add(afterConcatUdfPaths);
-        }
-        List<List<PartialPath>> extendedAfterConcatUdfPathsList = new ArrayList<>();
-        cartesianProduct(
-            afterConcatUdfPathsList, extendedAfterConcatUdfPathsList, 0, new ArrayList<>());
-
-        for (List<PartialPath> afterConcatUdfPaths : extendedAfterConcatUdfPathsList) {
-          afterConcatPaths.add(null);
-          extendListSafely(originAggregations, i, afterConcatAggregations);
-
-          afterConcatUdfList.add(
-              new UDFContext(originUdf.getName(), originUdf.getAttributes(), afterConcatUdfPaths));
-        }
-      } else { // non-udf
-        for (PartialPath fromPath : fromPaths) {
-          PartialPath fullPath = fromPath.concatPath(selectPath);
-          if (selectPath.isTsAliasExists()) {
-            fullPath.setTsAlias(selectPath.getTsAlias());
-          }
-          afterConcatPaths.add(fullPath);
-          extendListSafely(originAggregations, i, afterConcatAggregations);
+    if (queryOperator.isAlignByDevice() && !queryOperator.isLastQuery()
+        || queryOperator.getIndexType() != null) {
+      return;
+    }

Review comment:
       The same.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636782599



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
##########
@@ -103,71 +103,30 @@ statement
     | START TRIGGER triggerName=ID #startTrigger
     | STOP TRIGGER triggerName=ID #stopTrigger
     | SHOW TRIGGERS #showTriggers
-    | SELECT topClause? selectElements
-    fromClause
-    whereClause?
-    specialClause? #selectStatement
+    | selectClause fromClause whereClause? specialClause? #selectStatement
     ;
 
-selectElements
-    : aggregationCall (COMMA aggregationCall)* #aggregationElement
-    | tableCall (COMMA tableCall)* #tableElement
-    | lastClause #lastElement
-    | asClause (COMMA asClause)* #asElement
-    | functionAsClause (COMMA functionAsClause)* #functionAsElement
-    ;
-
-aggregationCall
-    : builtInFunctionCall
-    | udfCall
-    ;
-
-tableCall
-    : suffixPath
-    | udfCall
-    | SINGLE_QUOTE_STRING_LITERAL
-    ;
-
-udfCall
-    : udfName=ID LR_BRACKET udfSuffixPaths udfAttribute* RR_BRACKET
-    ;
-
-udfSuffixPaths
-    : suffixPath (COMMA suffixPath)*
-    ;
-
-udfAttribute
-    : COMMA udfAttributeKey=stringLiteral OPERATOR_EQ udfAttributeValue=stringLiteral
-    ;
-
-builtInFunctionCall
-    : functionName LR_BRACKET suffixPath RR_BRACKET
-    ;
-
-functionName
-    : MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | COUNT
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
-    ;
+selectClause
+   : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
+   ;
 
-functionAsClause
-    : builtInFunctionCall (AS ID)?
-    ;
+resultColumn
+   : expression (AS ID)?
+   ;
 
-lastClause
-    : LAST suffixPath (COMMA suffixPath)*
-    | LAST asClause (COMMA asClause)*
-    ;
+expression
+   : LR_BRACKET unary=expression RR_BRACKET
+   | (PLUS | MINUS) unary=expression
+   | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
+   | leftExpression=expression (PLUS | MINUS) rightExpression=expression
+   | functionName=suffixPath LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
+   | suffixPath

Review comment:
       +1 from my side!
   I've created a JIRA for this, plz have a look: https://issues.apache.org/jira/browse/IOTDB-1388




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636745716



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
##########
@@ -47,202 +47,88 @@
 /** concat paths in select and from clause. */
 public class ConcatPathOptimizer implements ILogicalOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(ConcatPathOptimizer.class);
+
   private static final String WARNING_NO_SUFFIX_PATHS =
-      "given SFWOperator doesn't have suffix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have suffix paths";
   private static final String WARNING_NO_PREFIX_PATHS =
-      "given SFWOperator doesn't have prefix paths, cannot concat seriesPath";
+      "failed to concat series paths because the given query operator didn't have prefix paths";
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
-  public Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  public Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException {
-    if (!(operator instanceof SFWOperator)) {
-      logger.warn("given operator isn't SFWOperator, cannot concat seriesPath");
-      return operator;
-    }
-    SFWOperator sfwOperator = (SFWOperator) operator;
-    FromOperator from = sfwOperator.getFromOperator();
-    List<PartialPath> prefixPaths;
-    if (from == null) {
-      logger.warn(WARNING_NO_PREFIX_PATHS);
-      return operator;
-    } else {
-      prefixPaths = from.getPrefixPaths();
-      if (prefixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_PREFIX_PATHS);
-        return operator;
-      }
-    }
-    SelectOperator select = sfwOperator.getSelectOperator();
-    List<PartialPath> initialSuffixPaths;
-    if (select == null) {
-      logger.warn(WARNING_NO_SUFFIX_PATHS);
-      return operator;
-    } else {
-      initialSuffixPaths = select.getSuffixPaths();
-      if (initialSuffixPaths.isEmpty()) {
-        logger.warn(WARNING_NO_SUFFIX_PATHS);
-        return operator;
-      }
-    }
-
-    checkAggrOfSelectOperator(select);
+    QueryOperator queryOperator = (QueryOperator) operator;
+    if (!optimizable(queryOperator)) {
+      return queryOperator;
+    }
+    concatSelect(queryOperator);
+    removeWildcardsInSelectPaths(queryOperator, fetchSize);
+    concatFilter(queryOperator);
+    return queryOperator;
+  }
 
-    boolean isAlignByDevice = false;
-    if (operator instanceof QueryOperator) {
-      if (!((QueryOperator) operator).isAlignByDevice()
-          || ((QueryOperator) operator).isLastQuery()) {
-        // concat paths and remove stars
-        int seriesLimit = ((QueryOperator) operator).getSeriesLimit();
-        int seriesOffset = ((QueryOperator) operator).getSeriesOffset();
-        concatSelect(
-            prefixPaths,
-            select,
-            seriesLimit,
-            seriesOffset,
-            maxDeduplicatedPathNum,
-            ((QueryOperator) operator).getIndexType() == null);
-      } else {
-        isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
-          throw new LogicalOptimizeException(
-              "ALIGN BY DEVICE clause is not supported in UDF queries.");
-        }
-        for (PartialPath path : initialSuffixPaths) {
-          String device = path.getDevice();
-          if (!device.isEmpty()) {
-            throw new LogicalOptimizeException(
-                "The paths of the SELECT clause can only be single level. In other words, "
-                    + "the paths of the SELECT clause can only be measurements or STAR, without DOT."
-                    + " For more details please refer to the SQL document.");
-          }
-        }
-        // ALIGN_BY_DEVICE leaves the 1) concat, 2) remove star, 3) slimit tasks to the next phase,
-        // i.e., PhysicalGenerator.transformQuery
-      }
+  private static boolean optimizable(QueryOperator queryOperator) {
+    FromOperator from = queryOperator.getFromOperator();
+    if (from == null || from.getPrefixPaths().isEmpty()) {
+      LOGGER.warn(WARNING_NO_PREFIX_PATHS);
+      return false;
     }
 
-    // concat filter
-    FilterOperator filter = sfwOperator.getFilterOperator();
-    Set<PartialPath> filterPaths = new HashSet<>();
-    if (filter == null) {
-      return operator;
-    }
-    if (!isAlignByDevice) {
-      sfwOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
+    SelectOperator select = queryOperator.getSelectOperator();
+    if (select == null || select.getResultColumns().isEmpty()) {
+      LOGGER.warn(WARNING_NO_SUFFIX_PATHS);
+      return false;

Review comment:
       Moved.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r638393749



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -146,82 +82,42 @@ public PhysicalPlan rawDataQueryReqToPhysicalPlan(
    * @return optimized logical operator
    * @throws LogicalOptimizeException exception in logical optimizing
    */
-  protected Operator logicalOptimize(Operator operator, int maxDeduplicatedPathNum)
+  protected Operator logicalOptimize(Operator operator, int fetchSize)
       throws LogicalOperatorException, PathNumOverLimitException {
-    switch (operator.getType()) {
-      case AUTHOR:
-      case METADATA:
-      case SET_STORAGE_GROUP:
-      case DELETE_STORAGE_GROUP:
-      case CREATE_TIMESERIES:
-      case DELETE_TIMESERIES:
-      case ALTER_TIMESERIES:
-      case LOADDATA:
-      case INSERT:
-      case GRANT_WATERMARK_EMBEDDING:
-      case REVOKE_WATERMARK_EMBEDDING:
-      case TTL:
-      case LOAD_CONFIGURATION:
-      case SHOW:
-      case LOAD_FILES:
-      case REMOVE_FILE:
-      case MOVE_FILE:
-      case FLUSH:
-      case MERGE:
-      case TRACING:
-      case CLEAR_CACHE:
-      case NULL:
-      case SHOW_MERGE_STATUS:
-      case DELETE_PARTITION:
-      case CREATE_SCHEMA_SNAPSHOT:
-      case KILL:
-      case CREATE_FUNCTION:
-      case DROP_FUNCTION:
-      case CREATE_TRIGGER:
-      case DROP_TRIGGER:
-      case START_TRIGGER:
-      case STOP_TRIGGER:
-        return operator;
-      case QUERY:
-      case DELETE:
-      case CREATE_INDEX:
-      case DROP_INDEX:
-      case QUERY_INDEX:
-        SFWOperator root = (SFWOperator) operator;
-        return optimizeSFWOperator(root, maxDeduplicatedPathNum);
-      default:
-        throw new LogicalOperatorException(operator.getType().toString(), "");
-    }
+    return operator.getType().equals(QUERY) || operator.getType().equals(QUERY_INDEX)
+        ? optimizeQueryOperator((QueryOperator) operator, fetchSize)
+        : operator;
   }
 
   /**
-   * given an unoptimized select-from-where operator and return an optimized result.
+   * given an unoptimized query operator and return an optimized result.
    *
-   * @param root unoptimized select-from-where operator
-   * @return optimized select-from-where operator
-   * @throws LogicalOptimizeException exception in SFW optimizing
+   * @param root unoptimized query operator
+   * @return optimized query operator
+   * @throws LogicalOptimizeException exception in query optimizing
    */
-  private SFWOperator optimizeSFWOperator(SFWOperator root, int maxDeduplicatedPathNum)
+  private QueryOperator optimizeQueryOperator(QueryOperator root, int fetchSize)
       throws LogicalOperatorException, PathNumOverLimitException {
-    ConcatPathOptimizer concatPathOptimizer = getConcatPathOptimizer();
-    root = (SFWOperator) concatPathOptimizer.transform(root, maxDeduplicatedPathNum);
+    root = (QueryOperator) getConcatPathOptimizer().transform(root, fetchSize);
+
     FilterOperator filter = root.getFilterOperator();
     if (filter == null) {
       return root;
     }
-    Set<PartialPath> pathSet = filter.getPathSet();
-    RemoveNotOptimizer removeNot = new RemoveNotOptimizer();
-    filter = removeNot.optimize(filter);
-    DnfFilterOptimizer dnf = new DnfFilterOptimizer();
-    filter = dnf.optimize(filter);
-    MergeSingleFilterOptimizer merge = new MergeSingleFilterOptimizer();
-    filter = merge.optimize(filter);
+    filter = new RemoveNotOptimizer().optimize(filter);
+    filter = new DnfFilterOptimizer().optimize(filter);
+    filter = new MergeSingleFilterOptimizer().optimize(filter);

Review comment:
       Yes, they could be changed to static methods. But I think it's ok now because all optimizer are implemented from `IFilterOptimizer`, they share the same interface.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-847737582


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [40 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.7%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.7% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-846683778


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.6%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.6% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-847676021


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [40 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] Alima777 commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
Alima777 commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r637753820



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;
+
+  private final int maxDeduplicatedPathNum;
+  private final int soffset;
+
+  private int offset;
+  private int limit;
+  private int consumed;
+
+  public WildcardsRemover(
+      ConcatPathOptimizer concatPathOptimizer, QueryOperator queryOperator, int fetchSize) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    // Dataset of last query actually has only three columns, so we shouldn't limit the path num
+    // while constructing logical plan
+    // To avoid overflowing because logicalOptimize function may do maxDeduplicatedPathNum + 1, we
+    // set it to Integer.MAX_VALUE - 1
+    maxDeduplicatedPathNum =
+        queryOperator.isLastQuery()
+            ? Integer.MAX_VALUE - 1
+            : QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
+    soffset = queryOperator.getSeriesOffset();
+    offset = soffset;
+
+    final int slimit = queryOperator.getSeriesLimit();
+    limit = slimit == 0 || maxDeduplicatedPathNum < slimit ? maxDeduplicatedPathNum + 1 : slimit;
+
+    consumed = 0;
+  }
+
+  public WildcardsRemover(ConcatPathOptimizer concatPathOptimizer) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
+    soffset = 0;
+    limit = maxDeduplicatedPathNum + 1;
+    consumed = 0;
+  }
+
+  public List<PartialPath> removeWildcardFrom(PartialPath path) throws LogicalOptimizeException {
+    try {
+      Pair<List<PartialPath>, Integer> pair =
+          concatPathOptimizer.removeWildcard(path, limit, offset);
+
+      consumed += pair.right;
+      if (offset != 0) {
+        int delta = offset - pair.right;
+        offset = Math.max(delta, 0);
+        if (delta < 0) {
+          limit += delta;
+        }
+      } else {
+        limit -= pair.right;
+      }
+
+      return pair.left;
+    } catch (MetadataException e) {
+      throw new LogicalOptimizeException("error occurred when removing star: " + e.getMessage());
+    }
+  }
+
+  public List<List<Expression>> removeWildcardsFrom(List<Expression> expressions)
+      throws LogicalOptimizeException {
+    List<List<Expression>> extendedExpressions = new ArrayList<>();
+
+    boolean atLeastOneSeriesNotExisted = false;
+    for (Expression originExpression : expressions) {
+      List<Expression> actualExpressions = new ArrayList<>();
+      originExpression.removeWildcards(
+          new WildcardsRemover(concatPathOptimizer), actualExpressions);
+      if (actualExpressions.isEmpty()) {
+        atLeastOneSeriesNotExisted = true;
+        break;
+      }
+      extendedExpressions.add(actualExpressions);
+    }
+    if (atLeastOneSeriesNotExisted) {
+      return Collections.emptyList();
+    }

Review comment:
       Now that once one series is not existed, break the loop and return, I think we don't need the flag.
   ```suggestion
       for (Expression originExpression : expressions) {
         List<Expression> actualExpressions = new ArrayList<>();
         originExpression.removeWildcards(
             new WildcardsRemover(concatPathOptimizer), actualExpressions);
         if (actualExpressions.isEmpty()) {
           return Collections.emptyList();
         }
         extendedExpressions.add(actualExpressions);
       }
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;
+
+  private final int maxDeduplicatedPathNum;
+  private final int soffset;
+
+  private int offset;
+  private int limit;
+  private int consumed;
+
+  public WildcardsRemover(
+      ConcatPathOptimizer concatPathOptimizer, QueryOperator queryOperator, int fetchSize) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    // Dataset of last query actually has only three columns, so we shouldn't limit the path num
+    // while constructing logical plan
+    // To avoid overflowing because logicalOptimize function may do maxDeduplicatedPathNum + 1, we
+    // set it to Integer.MAX_VALUE - 1
+    maxDeduplicatedPathNum =
+        queryOperator.isLastQuery()
+            ? Integer.MAX_VALUE - 1
+            : QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
+    soffset = queryOperator.getSeriesOffset();
+    offset = soffset;
+
+    final int slimit = queryOperator.getSeriesLimit();
+    limit = slimit == 0 || maxDeduplicatedPathNum < slimit ? maxDeduplicatedPathNum + 1 : slimit;
+
+    consumed = 0;
+  }
+
+  public WildcardsRemover(ConcatPathOptimizer concatPathOptimizer) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
+    soffset = 0;
+    limit = maxDeduplicatedPathNum + 1;
+    consumed = 0;
+  }
+
+  public List<PartialPath> removeWildcardFrom(PartialPath path) throws LogicalOptimizeException {
+    try {
+      Pair<List<PartialPath>, Integer> pair =
+          concatPathOptimizer.removeWildcard(path, limit, offset);
+
+      consumed += pair.right;
+      if (offset != 0) {
+        int delta = offset - pair.right;
+        offset = Math.max(delta, 0);
+        if (delta < 0) {
+          limit += delta;
+        }
+      } else {
+        limit -= pair.right;
+      }
+
+      return pair.left;
+    } catch (MetadataException e) {
+      throw new LogicalOptimizeException("error occurred when removing star: " + e.getMessage());
+    }
+  }
+
+  public List<List<Expression>> removeWildcardsFrom(List<Expression> expressions)
+      throws LogicalOptimizeException {
+    List<List<Expression>> extendedExpressions = new ArrayList<>();
+
+    boolean atLeastOneSeriesNotExisted = false;
+    for (Expression originExpression : expressions) {
+      List<Expression> actualExpressions = new ArrayList<>();
+      originExpression.removeWildcards(
+          new WildcardsRemover(concatPathOptimizer), actualExpressions);
+      if (actualExpressions.isEmpty()) {
+        atLeastOneSeriesNotExisted = true;
+        break;
+      }
+      extendedExpressions.add(actualExpressions);
+    }
+    if (atLeastOneSeriesNotExisted) {
+      return Collections.emptyList();
+    }
+
+    List<List<Expression>> actualExpressions = new ArrayList<>();
+    ConcatPathOptimizer.cartesianProduct(
+        extendedExpressions, actualExpressions, 0, new ArrayList<>());
+
+    List<List<Expression>> splitExpressions = new ArrayList<>();
+    for (List<Expression> actualExpression : actualExpressions) {
+      if (offset != 0) {
+        --offset;
+        continue;
+      } else if (limit != 0) {
+        --limit;
+      } else {
+        break;
+      }
+      splitExpressions.add(actualExpression);
+    }
+    consumed += actualExpressions.size();
+    return splitExpressions;

Review comment:
       From my point of view, `extendedExpressions`, `actualExpressions`, `splitExpressions`,I can't get the meaning of each name directly.
   Therefore, we need add more comments or abstract a private method to do that with a clear method name.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/utils/WildcardsRemover.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.iotdb.db.qp.utils;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class WildcardsRemover {
+
+  private final ConcatPathOptimizer concatPathOptimizer;
+
+  private final int maxDeduplicatedPathNum;
+  private final int soffset;
+
+  private int offset;
+  private int limit;
+  private int consumed;
+
+  public WildcardsRemover(
+      ConcatPathOptimizer concatPathOptimizer, QueryOperator queryOperator, int fetchSize) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    // Dataset of last query actually has only three columns, so we shouldn't limit the path num
+    // while constructing logical plan
+    // To avoid overflowing because logicalOptimize function may do maxDeduplicatedPathNum + 1, we
+    // set it to Integer.MAX_VALUE - 1
+    maxDeduplicatedPathNum =
+        queryOperator.isLastQuery()
+            ? Integer.MAX_VALUE - 1
+            : QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
+    soffset = queryOperator.getSeriesOffset();
+    offset = soffset;
+
+    final int slimit = queryOperator.getSeriesLimit();
+    limit = slimit == 0 || maxDeduplicatedPathNum < slimit ? maxDeduplicatedPathNum + 1 : slimit;
+
+    consumed = 0;
+  }
+
+  public WildcardsRemover(ConcatPathOptimizer concatPathOptimizer) {
+    this.concatPathOptimizer = concatPathOptimizer;
+
+    maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
+    soffset = 0;
+    limit = maxDeduplicatedPathNum + 1;
+    consumed = 0;
+  }
+
+  public List<PartialPath> removeWildcardFrom(PartialPath path) throws LogicalOptimizeException {
+    try {
+      Pair<List<PartialPath>, Integer> pair =
+          concatPathOptimizer.removeWildcard(path, limit, offset);
+
+      consumed += pair.right;
+      if (offset != 0) {
+        int delta = offset - pair.right;
+        offset = Math.max(delta, 0);
+        if (delta < 0) {
+          limit += delta;
+        }
+      } else {
+        limit -= pair.right;
+      }
+
+      return pair.left;
+    } catch (MetadataException e) {
+      throw new LogicalOptimizeException("error occurred when removing star: " + e.getMessage());
+    }
+  }
+
+  public List<List<Expression>> removeWildcardsFrom(List<Expression> expressions)
+      throws LogicalOptimizeException {
+    List<List<Expression>> extendedExpressions = new ArrayList<>();
+
+    boolean atLeastOneSeriesNotExisted = false;
+    for (Expression originExpression : expressions) {
+      List<Expression> actualExpressions = new ArrayList<>();
+      originExpression.removeWildcards(
+          new WildcardsRemover(concatPathOptimizer), actualExpressions);
+      if (actualExpressions.isEmpty()) {
+        atLeastOneSeriesNotExisted = true;
+        break;
+      }
+      extendedExpressions.add(actualExpressions);
+    }
+    if (atLeastOneSeriesNotExisted) {
+      return Collections.emptyList();
+    }

Review comment:
       And I doubt that whether we should return a empty list or throw an exception.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-847676021


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [40 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#discussion_r636596218



##########
File path: server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.iotdb.db.query.expression;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.utils.WildcardsRemover;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.util.List;
+import java.util.Set;
+
+public abstract class Expression {
+
+  protected boolean isAggregationFunctionExpression = false;
+  protected boolean isTimeSeriesGeneratingFunctionExpression = false;
+

Review comment:
       Good comments! I have updated then :D




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3236: [IOTDB-1022] ANTLR Grammar & Logical Operator for Arithmetic Operations and Nested Operations in SELECT Clauses

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3236:
URL: https://github.com/apache/iotdb/pull/3236#issuecomment-846901769


   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=VULNERABILITY)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=SECURITY_HOTSPOT)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL) [37 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3236&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60.png' alt='78.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list) [78.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_coverage&view=list)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3236&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org