You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ro...@apache.org on 2021/05/13 10:03:34 UTC

[iotdb] branch iotdb-1022-v2 created (now 7637825)

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

rong pushed a change to branch iotdb-1022-v2
in repository https://gitbox.apache.org/repos/asf/iotdb.git.


      at 7637825  refactor Planner and LogicalGenerator

This branch includes the following new commits:

     new 2409122  antlr grammar for arithmetic operators and nested operators
     new 63e5ece  expressions for complex operations in select clauses
     new 287c499  new SelectOperator
     new 98790a0  rename
     new 7637825  refactor Planner and LogicalGenerator

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


[iotdb] 05/05: refactor Planner and LogicalGenerator

Posted by ro...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rong pushed a commit to branch iotdb-1022-v2
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 76378252e3989c10ced6e3efa5ef4970f3898939
Author: SteveYurongSu <st...@outlook.com>
AuthorDate: Thu May 13 18:02:35 2021 +0800

    refactor Planner and LogicalGenerator
---
 .../apache/iotdb/cluster/query/ClusterPlanner.java |  20 +--
 .../main/java/org/apache/iotdb/db/qp/Planner.java  | 173 ++++-----------------
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  63 +++++++-
 .../qp/strategy/optimizer/ILogicalOptimizer.java   |   2 +-
 4 files changed, 97 insertions(+), 161 deletions(-)

diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanner.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanner.java
index ecbfb74..f4e9ec4 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanner.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanner.java
@@ -22,11 +22,9 @@ package org.apache.iotdb.cluster.query;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.Planner;
 import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.logical.crud.SFWOperator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
+import org.apache.iotdb.db.qp.strategy.LogicalGenerator;
 import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
-import org.apache.iotdb.db.query.control.QueryResourceManager;
 
 import java.time.ZoneId;
 
@@ -36,19 +34,9 @@ public class ClusterPlanner extends Planner {
   @Override
   public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId, int fetchSize)
       throws QueryProcessException {
-    Operator operator = logicalGenerator.generate(sqlStr, zoneId);
-    int maxDeduplicatedPathNum =
-        QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
-    if (operator instanceof SFWOperator && ((SFWOperator) operator).isLastQuery()) {
-      // 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 = Integer.MAX_VALUE - 1;
-    }
-    operator = logicalOptimize(operator, maxDeduplicatedPathNum);
-    PhysicalGenerator physicalGenerator = new ClusterPhysicalGenerator();
-    return physicalGenerator.transformToPhysicalPlan(operator, fetchSize);
+    Operator operator = LogicalGenerator.generate(sqlStr, zoneId);
+    operator = logicalOptimize(operator, fetchSize);
+    return new ClusterPhysicalGenerator().transformToPhysicalPlan(operator, fetchSize);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
index 840b04f..027468b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
@@ -23,14 +23,9 @@ import org.apache.iotdb.db.exception.query.LogicalOperatorException;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator;
 import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
-import org.apache.iotdb.db.qp.logical.crud.FromOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
-import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.strategy.LogicalGenerator;
 import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
@@ -38,106 +33,41 @@ import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
 import org.apache.iotdb.db.qp.strategy.optimizer.DnfFilterOptimizer;
 import org.apache.iotdb.db.qp.strategy.optimizer.MergeSingleFilterOptimizer;
 import org.apache.iotdb.db.qp.strategy.optimizer.RemoveNotOptimizer;
-import org.apache.iotdb.db.query.control.QueryResourceManager;
-import org.apache.iotdb.db.query.expression.ResultColumn;
-import org.apache.iotdb.db.query.expression.unary.TimeSeriesOperand;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
 
 import java.time.ZoneId;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
 
-import static org.apache.iotdb.db.conf.IoTDBConstant.TIME;
+import static org.apache.iotdb.db.qp.logical.Operator.OperatorType.QUERY;
+import static org.apache.iotdb.db.qp.logical.Operator.OperatorType.QUERY_INDEX;
 
 /** provide a integration method for other user. */
 public class Planner {
 
-  protected LogicalGenerator logicalGenerator;
-
   public Planner() {
-    this.logicalGenerator = new LogicalGenerator();
-  }
-
-  @TestOnly
-  public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr) throws QueryProcessException {
-    return parseSQLToPhysicalPlan(sqlStr, ZoneId.systemDefault(), 1024);
+    // do nothing
   }
 
   /** @param fetchSize this parameter only take effect when it is a query plan */
   public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId, int fetchSize)
       throws QueryProcessException {
-    Operator operator = logicalGenerator.generate(sqlStr, zoneId);
-    int maxDeduplicatedPathNum =
-        QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
-    if (operator instanceof QueryOperator && ((QueryOperator) operator).isLastQuery()) {
-      // 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 = Integer.MAX_VALUE - 1;
-    }
-    operator = logicalOptimize(operator, maxDeduplicatedPathNum);
-    PhysicalGenerator physicalGenerator = new PhysicalGenerator();
-    return physicalGenerator.transformToPhysicalPlan(operator, fetchSize);
+    // from SQL to logical operator
+    Operator operator = LogicalGenerator.generate(sqlStr, zoneId);
+    // optimize the logical operator
+    operator = logicalOptimize(operator, fetchSize);
+    // from logical operator to physical plan
+    return new PhysicalGenerator().transformToPhysicalPlan(operator, fetchSize);
   }
 
-  /** convert raw data query to physical plan directly */
   public PhysicalPlan rawDataQueryReqToPhysicalPlan(
       TSRawDataQueryReq rawDataQueryReq, ZoneId zoneId)
-      throws QueryProcessException, IllegalPathException {
-    List<String> paths = rawDataQueryReq.getPaths();
-    long startTime = rawDataQueryReq.getStartTime();
-    long endTime = rawDataQueryReq.getEndTime();
-
-    // construct query operator and set its global time filter
-    QueryOperator queryOp = new QueryOperator(SQLConstant.TOK_QUERY);
-    FromOperator fromOp = new FromOperator(SQLConstant.TOK_FROM);
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
-
-    // iterate the path list and add it to from operator
-    for (String p : paths) {
-      PartialPath path = new PartialPath(p);
-      fromOp.addPrefixTablePath(path);
-    }
-    selectOp.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath(""))));
-
-    queryOp.setSelectOperator(selectOp);
-    queryOp.setFromOperator(fromOp);
-
-    // set time filter operator
-    FilterOperator filterOp = new FilterOperator(SQLConstant.KW_AND);
-    PartialPath timePath = new PartialPath(TIME);
-    filterOp.setSinglePath(timePath);
-    Set<PartialPath> pathSet = new HashSet<>();
-    pathSet.add(timePath);
-    filterOp.setIsSingle(true);
-    filterOp.setPathSet(pathSet);
-
-    BasicFunctionOperator left =
-        new BasicFunctionOperator(
-            SQLConstant.GREATERTHANOREQUALTO, timePath, Long.toString(startTime));
-    BasicFunctionOperator right =
-        new BasicFunctionOperator(SQLConstant.LESSTHAN, timePath, Long.toString(endTime));
-    filterOp.addChildOperator(left);
-    filterOp.addChildOperator(right);
-
-    queryOp.setFilterOperator(filterOp);
-
-    int maxDeduplicatedPathNum =
-        QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(rawDataQueryReq.fetchSize);
-    if (queryOp.isLastQuery()) {
-      // 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 = Integer.MAX_VALUE - 1;
-    }
-    QueryOperator op = (QueryOperator) logicalOptimize(queryOp, maxDeduplicatedPathNum);
-
-    PhysicalGenerator physicalGenerator = new PhysicalGenerator();
-    return physicalGenerator.transformToPhysicalPlan(op, rawDataQueryReq.fetchSize);
+      throws IllegalPathException, QueryProcessException {
+    // from TSRawDataQueryReq to logical operator
+    Operator operator = LogicalGenerator.generate(rawDataQueryReq, zoneId);
+    // optimize the logical operator
+    operator = logicalOptimize(operator, rawDataQueryReq.fetchSize);
+    // from logical operator to physical plan
+    return new PhysicalGenerator().transformToPhysicalPlan(operator, rawDataQueryReq.fetchSize);
   }
 
   /**
@@ -147,51 +77,11 @@ public class Planner {
    * @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:
-      case QUERY:
-      case DELETE:
-      case CREATE_INDEX:
-      case DROP_INDEX:
-        return operator;
-      case QUERY_INDEX:
-        return optimizeQueryOperator((QueryOperator) operator, maxDeduplicatedPathNum);
-      default:
-        throw new LogicalOperatorException(operator.getType().toString(), "");
-    }
+    return operator.getType().equals(QUERY) || operator.getType().equals(QUERY_INDEX)
+        ? optimizeQueryOperator((QueryOperator) operator, fetchSize)
+        : operator;
   }
 
   /**
@@ -201,27 +91,28 @@ public class Planner {
    * @return optimized query operator
    * @throws LogicalOptimizeException exception in query optimizing
    */
-  private QueryOperator optimizeQueryOperator(QueryOperator root, int maxDeduplicatedPathNum)
+  private QueryOperator optimizeQueryOperator(QueryOperator root, int fetchSize)
       throws LogicalOperatorException, PathNumOverLimitException {
-    ConcatPathOptimizer concatPathOptimizer = getConcatPathOptimizer();
-    root = (QueryOperator) 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);
     root.setFilterOperator(filter);
-    filter.setPathSet(pathSet);
+
     return root;
   }
 
   protected ConcatPathOptimizer getConcatPathOptimizer() {
     return new ConcatPathOptimizer();
   }
+
+  @TestOnly
+  public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr) throws QueryProcessException {
+    return parseSQLToPhysicalPlan(sqlStr, ZoneId.systemDefault(), 1024);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index e4f5a67..51a5e72 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -18,10 +18,21 @@
  */
 package org.apache.iotdb.db.qp.strategy;
 
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator;
+import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
+import org.apache.iotdb.db.qp.logical.crud.FromOperator;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
 import org.apache.iotdb.db.qp.sql.IoTDBSqlVisitor;
 import org.apache.iotdb.db.qp.sql.SqlBaseLexer;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.db.query.expression.unary.TimeSeriesOperand;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
 
 import org.antlr.v4.runtime.CharStream;
 import org.antlr.v4.runtime.CharStreams;
@@ -31,13 +42,15 @@ import org.antlr.v4.runtime.misc.ParseCancellationException;
 import org.antlr.v4.runtime.tree.ParseTree;
 
 import java.time.ZoneId;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.TIME;
 
 /** LogicalGenerator. */
 public class LogicalGenerator {
 
-  public LogicalGenerator() {}
-
-  public Operator generate(String sql, ZoneId zoneId) throws ParseCancellationException {
+  public static Operator generate(String sql, ZoneId zoneId) throws ParseCancellationException {
     IoTDBSqlVisitor ioTDBSqlVisitor = new IoTDBSqlVisitor();
     ioTDBSqlVisitor.setZoneId(zoneId);
     CharStream charStream1 = CharStreams.fromString(sql);
@@ -67,4 +80,48 @@ public class LogicalGenerator {
     }
     return ioTDBSqlVisitor.visit(tree);
   }
+
+  public static Operator generate(TSRawDataQueryReq rawDataQueryReq, ZoneId zoneId)
+      throws IllegalPathException {
+    // construct query operator and set its global time filter
+    QueryOperator queryOp = new QueryOperator(SQLConstant.TOK_QUERY);
+    FromOperator fromOp = new FromOperator(SQLConstant.TOK_FROM);
+    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
+
+    // iterate the path list and add it to from operator
+    for (String p : rawDataQueryReq.getPaths()) {
+      PartialPath path = new PartialPath(p);
+      fromOp.addPrefixTablePath(path);
+    }
+    selectOp.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath(""))));
+
+    queryOp.setSelectOperator(selectOp);
+    queryOp.setFromOperator(fromOp);
+
+    // set time filter operator
+    FilterOperator filterOp = new FilterOperator(SQLConstant.KW_AND);
+    PartialPath timePath = new PartialPath(TIME);
+    filterOp.setSinglePath(timePath);
+    Set<PartialPath> pathSet = new HashSet<>();
+    pathSet.add(timePath);
+    filterOp.setIsSingle(true);
+    filterOp.setPathSet(pathSet);
+
+    BasicFunctionOperator left =
+        new BasicFunctionOperator(
+            SQLConstant.GREATERTHANOREQUALTO,
+            timePath,
+            Long.toString(rawDataQueryReq.getStartTime()));
+    BasicFunctionOperator right =
+        new BasicFunctionOperator(
+            SQLConstant.LESSTHAN, timePath, Long.toString(rawDataQueryReq.getEndTime()));
+    filterOp.addChildOperator(left);
+    filterOp.addChildOperator(right);
+
+    queryOp.setFilterOperator(filterOp);
+
+    return queryOp;
+  }
+
+  private LogicalGenerator() {}
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ILogicalOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ILogicalOptimizer.java
index 26ab0a5..d851f7e 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ILogicalOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ILogicalOptimizer.java
@@ -26,6 +26,6 @@ import org.apache.iotdb.db.qp.logical.Operator;
 @FunctionalInterface
 public interface ILogicalOptimizer {
 
-  Operator transform(Operator operator, int maxDeduplicatedPathNum)
+  Operator transform(Operator operator, int fetchSize)
       throws LogicalOptimizeException, PathNumOverLimitException;
 }

[iotdb] 01/05: antlr grammar for arithmetic operators and nested operators

Posted by ro...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rong pushed a commit to branch iotdb-1022-v2
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 24091226b382570f73df6991ab19c9f2aedb5925
Author: SteveYurongSu <st...@outlook.com>
AuthorDate: Mon May 10 15:40:50 2021 +0800

    antlr grammar for arithmetic operators and nested operators
---
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   | 130 ++++-----------------
 1 file changed, 23 insertions(+), 107 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index dd629dd..5d9d1d3 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -103,71 +103,35 @@ 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
-    ;
+selectClause
+   : SELECT (LAST | topClause)? complexOperationAsClause (COMMA complexOperationAsClause)*
+   ;
 
-functionName
-    : MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | COUNT
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
-    ;
+complexOperationAsClause
+   : complexOperationClause (AS ID)?
+   ;
 
-functionAsClause
-    : builtInFunctionCall (AS ID)?
-    ;
+complexOperationClause
+   : realLiteral
+   | suffixPath
+   | functionClause
+   | '(' complexOperationClause ')'
+   | ('+' | '-') complexOperationClause
+   | complexOperationClause ('*' | '/' | '%') complexOperationClause
+   | complexOperationClause ('+' | '-') complexOperationClause
+   ;
 
-lastClause
-    : LAST suffixPath (COMMA suffixPath)*
-    | LAST asClause (COMMA asClause)*
-    ;
+functionClause
+   : functionName=ID LR_BRACKET complexOperationClause (COMMA complexOperationClause)*
+     functionAttribute* RR_BRACKET
+   ;
 
-asClause
-    : suffixPath (AS ID)?
-    ;
+functionAttribute
+   : COMMA functionAttributeKey=stringLiteral OPERATOR_EQ functionAttributeValue=stringLiteral
+   ;
 
 alias
     : LR_BRACKET ID RR_BRACKET
@@ -518,14 +482,6 @@ nodeName
     | COUNT
     | NODES
     | LEVEL
-    | MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
     | LAST
     | DISABLE
     | ALIGN
@@ -629,14 +585,6 @@ nodeNameWithoutStar
     | COUNT
     | NODES
     | LEVEL
-    | MIN_TIME
-    | MAX_TIME
-    | MIN_VALUE
-    | MAX_VALUE
-    | AVG
-    | FIRST_VALUE
-    | SUM
-    | LAST_VALUE
     | LAST
     | DISABLE
     | ALIGN
@@ -1096,38 +1044,6 @@ LEVEL
     : L E V E L
     ;
 
-MIN_TIME
-    : M I N UNDERLINE T I M E
-    ;
-
-MAX_TIME
-    : M A X UNDERLINE T I M E
-    ;
-
-MIN_VALUE
-    : M I N UNDERLINE V A L U E
-    ;
-
-MAX_VALUE
-    : M A X UNDERLINE V A L U E
-    ;
-
-AVG
-    : A V G
-    ;
-
-FIRST_VALUE
-    : F I R S T UNDERLINE V A L U E
-    ;
-
-SUM
-    : S U M
-    ;
-
-LAST_VALUE
-    : L A S T UNDERLINE V A L U E
-    ;
-
 LAST
     : L A S T
     ;

[iotdb] 04/05: rename

Posted by ro...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rong pushed a commit to branch iotdb-1022-v2
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 98790a0d6a1dd5152ca2e368e687c16b73cbe845
Author: SteveYurongSu <st...@outlook.com>
AuthorDate: Tue May 11 22:05:17 2021 +0800

    rename
---
 .../qp/strategy/optimizer/ConcatPathOptimizer.java | 24 ++++++++--------------
 1 file changed, 9 insertions(+), 15 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
index ff75491..c77f67b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
 import org.apache.iotdb.db.qp.logical.crud.FromOperator;
 import org.apache.iotdb.db.qp.logical.crud.FunctionOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
-import org.apache.iotdb.db.qp.logical.crud.SFWOperator;
 import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
 import org.apache.iotdb.db.query.udf.core.context.UDFContext;
 import org.apache.iotdb.db.service.IoTDB;
@@ -49,20 +48,15 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
 
   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)
       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();
+    QueryOperator queryOperator = (QueryOperator) operator;
+    FromOperator from = queryOperator.getFromOperator();
     List<PartialPath> prefixPaths;
     if (from == null) {
       logger.warn(WARNING_NO_PREFIX_PATHS);
@@ -74,7 +68,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
         return operator;
       }
     }
-    SelectOperator select = sfwOperator.getSelectOperator();
+    SelectOperator select = queryOperator.getSelectOperator();
     List<PartialPath> initialSuffixPaths;
     if (select == null) {
       logger.warn(WARNING_NO_SUFFIX_PATHS);
@@ -124,19 +118,19 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
     }
 
     // concat filter
-    FilterOperator filter = sfwOperator.getFilterOperator();
+    FilterOperator filter = queryOperator.getFilterOperator();
     Set<PartialPath> filterPaths = new HashSet<>();
     if (filter == null) {
       return operator;
     }
     if (!isAlignByDevice) {
-      sfwOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
+      queryOperator.setFilterOperator(concatFilter(prefixPaths, filter, filterPaths));
     }
-    sfwOperator.getFilterOperator().setPathSet(filterPaths);
+    queryOperator.getFilterOperator().setPathSet(filterPaths);
     // GROUP_BY_DEVICE leaves the concatFilter to PhysicalGenerator to optimize filter without
     // prefix first
 
-    return sfwOperator;
+    return queryOperator;
   }
 
   private List<PartialPath> judgeSelectOperator(SelectOperator selectOperator)

[iotdb] 03/05: new SelectOperator

Posted by ro...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rong pushed a commit to branch iotdb-1022-v2
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 287c4991b09ed2530f4753d2a00150f6b4010484
Author: SteveYurongSu <st...@outlook.com>
AuthorDate: Tue May 11 21:48:21 2021 +0800

    new SelectOperator
---
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |  30 ++-
 .../main/java/org/apache/iotdb/db/qp/Planner.java  |  26 +-
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   1 -
 .../db/qp/logical/crud/DeleteDataOperator.java     |  17 +-
 .../iotdb/db/qp/logical/crud/InsertOperator.java   |  15 +-
 .../iotdb/db/qp/logical/crud/QueryOperator.java    |  43 +++-
 .../iotdb/db/qp/logical/crud/SFWOperator.java      |  98 --------
 .../iotdb/db/qp/logical/crud/SelectOperator.java   | 100 ++++----
 .../db/qp/logical/sys/CreateIndexOperator.java     |  17 +-
 .../iotdb/db/qp/logical/sys/DropIndexOperator.java |  18 +-
 .../apache/iotdb/db/qp/physical/crud/UDFPlan.java  |   4 +-
 .../apache/iotdb/db/qp/physical/crud/UDTFPlan.java |  20 +-
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    | 276 +++++++++------------
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  51 ++--
 .../qp/strategy/optimizer/ConcatPathOptimizer.java |   2 +-
 .../apache/iotdb/db/query/dataset/UDTFDataSet.java |   2 +-
 .../iotdb/db/query/expression/Expression.java      |  15 +-
 .../iotdb/db/query/expression/ResultColumn.java    |   5 +
 .../query/expression/binary/BinaryExpression.java  |   4 +-
 .../query/expression/unary/FunctionExpression.java |  43 +++-
 .../db/query/expression/unary/MinusExpression.java |   2 +-
 .../expression/unary/NumberLiteralOperand.java     |   2 +-
 .../query/expression/unary/TimeSeriesOperand.java  |   2 +-
 .../db/query/udf/core/context/UDFContext.java      | 142 -----------
 .../db/query/udf/core/executor/UDTFExecutor.java   |  17 +-
 .../query/udf/service/UDFRegistrationService.java  |   6 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   5 +-
 .../iotdb/db/qp/logical/IndexLogicalPlanTest.java  |   4 +-
 .../iotdb/db/qp/logical/LogicalPlanSmallTest.java  |   2 +-
 .../query/dataset/UDTFAlignByTimeDataSetTest.java  |  45 +++-
 30 files changed, 440 insertions(+), 574 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index 5d9d1d3..d00b933 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -107,26 +107,30 @@ statement
     ;
 
 selectClause
-   : SELECT (LAST | topClause)? complexOperationAsClause (COMMA complexOperationAsClause)*
+   : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
    ;
 
-complexOperationAsClause
-   : complexOperationClause (AS ID)?
+resultColumn
+   : expression (AS ID)?
    ;
 
-complexOperationClause
-   : realLiteral
+expression
+   : numberLiteral
    | suffixPath
    | functionClause
-   | '(' complexOperationClause ')'
-   | ('+' | '-') complexOperationClause
-   | complexOperationClause ('*' | '/' | '%') complexOperationClause
-   | complexOperationClause ('+' | '-') complexOperationClause
+   | LR_BRACKET unary=expression RR_BRACKET
+   | (PLUS | MINUS) unary=expression
+   | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
+   | leftExpression=expression (PLUS | MINUS) rightExpression=expression
+   ;
+
+numberLiteral
+   : MINUS? realLiteral
+   | MINUS? INT
    ;
 
 functionClause
-   : functionName=ID LR_BRACKET complexOperationClause (COMMA complexOperationClause)*
-     functionAttribute* RR_BRACKET
+   : functionName=ID LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
    ;
 
 functionAttribute
@@ -1273,6 +1277,10 @@ MINUS : '-';
 
 PLUS : '+';
 
+DIV : '/';
+
+MOD : M O D | '%';
+
 DOT : '.';
 
 LR_BRACKET : '(';
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
index 355b7ef..840b04f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator;
 import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
 import org.apache.iotdb.db.qp.logical.crud.FromOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
-import org.apache.iotdb.db.qp.logical.crud.SFWOperator;
 import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.strategy.LogicalGenerator;
@@ -40,6 +39,8 @@ import org.apache.iotdb.db.qp.strategy.optimizer.DnfFilterOptimizer;
 import org.apache.iotdb.db.qp.strategy.optimizer.MergeSingleFilterOptimizer;
 import org.apache.iotdb.db.qp.strategy.optimizer.RemoveNotOptimizer;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.db.query.expression.unary.TimeSeriesOperand;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
 
@@ -70,7 +71,7 @@ public class Planner {
     Operator operator = logicalGenerator.generate(sqlStr, zoneId);
     int maxDeduplicatedPathNum =
         QueryResourceManager.getInstance().getMaxDeduplicatedPathNum(fetchSize);
-    if (operator instanceof SFWOperator && ((SFWOperator) operator).isLastQuery()) {
+    if (operator instanceof QueryOperator && ((QueryOperator) operator).isLastQuery()) {
       // 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
@@ -100,7 +101,7 @@ public class Planner {
       PartialPath path = new PartialPath(p);
       fromOp.addPrefixTablePath(path);
     }
-    selectOp.addSelectPath(new PartialPath(""));
+    selectOp.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath(""))));
 
     queryOp.setSelectOperator(selectOp);
     queryOp.setFromOperator(fromOp);
@@ -133,7 +134,7 @@ public class Planner {
       // set it to Integer.MAX_VALUE - 1
       maxDeduplicatedPathNum = Integer.MAX_VALUE - 1;
     }
-    SFWOperator op = (SFWOperator) logicalOptimize(queryOp, maxDeduplicatedPathNum);
+    QueryOperator op = (QueryOperator) logicalOptimize(queryOp, maxDeduplicatedPathNum);
 
     PhysicalGenerator physicalGenerator = new PhysicalGenerator();
     return physicalGenerator.transformToPhysicalPlan(op, rawDataQueryReq.fetchSize);
@@ -181,30 +182,29 @@ public class Planner {
       case DROP_TRIGGER:
       case START_TRIGGER:
       case STOP_TRIGGER:
-        return operator;
       case QUERY:
       case DELETE:
       case CREATE_INDEX:
       case DROP_INDEX:
+        return operator;
       case QUERY_INDEX:
-        SFWOperator root = (SFWOperator) operator;
-        return optimizeSFWOperator(root, maxDeduplicatedPathNum);
+        return optimizeQueryOperator((QueryOperator) operator, maxDeduplicatedPathNum);
       default:
         throw new LogicalOperatorException(operator.getType().toString(), "");
     }
   }
 
   /**
-   * 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 maxDeduplicatedPathNum)
       throws LogicalOperatorException, PathNumOverLimitException {
     ConcatPathOptimizer concatPathOptimizer = getConcatPathOptimizer();
-    root = (SFWOperator) concatPathOptimizer.transform(root, maxDeduplicatedPathNum);
+    root = (QueryOperator) concatPathOptimizer.transform(root, maxDeduplicatedPathNum);
     FilterOperator filter = root.getFilterOperator();
     if (filter == null) {
       return root;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index e1fb10f..f3ac665 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -73,7 +73,6 @@ public abstract class Operator {
 
   /** If you want to add new OperatorType, you must add it in the last. */
   public enum OperatorType {
-    SFW,
     FILTER,
     GROUPBYTIME,
     SELECT,
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
index 3cddb30..e7a0619 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
@@ -18,10 +18,16 @@
  */
 package org.apache.iotdb.db.qp.logical.crud;
 
+import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /** this class extends {@code RootOperator} and process delete statement. */
-public class DeleteDataOperator extends SFWOperator {
+public class DeleteDataOperator extends Operator {
+
+  private final List<PartialPath> paths;
 
   private long startTime;
   private long endTime;
@@ -29,6 +35,15 @@ public class DeleteDataOperator extends SFWOperator {
   public DeleteDataOperator(int tokenIntType) {
     super(tokenIntType);
     operatorType = Operator.OperatorType.DELETE;
+    paths = new ArrayList<>();
+  }
+
+  public List<PartialPath> getPaths() {
+    return paths;
+  }
+
+  public void addPath(PartialPath path) {
+    paths.add(path);
   }
 
   public long getStartTime() {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
index bd410c6..e8c3f42 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
@@ -18,8 +18,13 @@
  */
 package org.apache.iotdb.db.qp.logical.crud;
 
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+
 /** this class extends {@code RootOperator} and process insert statement. */
-public class InsertOperator extends SFWOperator {
+public class InsertOperator extends Operator {
+
+  private PartialPath device;
 
   private long[] times;
   private String[] measurementList;
@@ -30,6 +35,14 @@ public class InsertOperator extends SFWOperator {
     operatorType = OperatorType.INSERT;
   }
 
+  public PartialPath getDevice() {
+    return device;
+  }
+
+  public void setDevice(PartialPath device) {
+    this.device = device;
+  }
+
   public String[] getMeasurementList() {
     return measurementList;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java
index 87aa1d9..f80dcbb 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java
@@ -25,8 +25,11 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 import java.util.Map;
 
-/** this class extends {@code RootOperator} and process getIndex statement */
-public class QueryOperator extends SFWOperator {
+public class QueryOperator extends Operator {
+
+  private SelectOperator selectOperator;
+  private FromOperator fromOperator;
+  private FilterOperator filterOperator;
 
   private long startTime;
   private long endTime;
@@ -67,6 +70,30 @@ public class QueryOperator extends SFWOperator {
     operatorType = Operator.OperatorType.QUERY;
   }
 
+  public SelectOperator getSelectOperator() {
+    return selectOperator;
+  }
+
+  public void setSelectOperator(SelectOperator selectOperator) {
+    this.selectOperator = selectOperator;
+  }
+
+  public FromOperator getFromOperator() {
+    return fromOperator;
+  }
+
+  public void setFromOperator(FromOperator fromOperator) {
+    this.fromOperator = fromOperator;
+  }
+
+  public FilterOperator getFilterOperator() {
+    return filterOperator;
+  }
+
+  public void setFilterOperator(FilterOperator filterOperator) {
+    this.filterOperator = filterOperator;
+  }
+
   public Map<String, Object> getProps() {
     return props;
   }
@@ -250,4 +277,16 @@ public class QueryOperator extends SFWOperator {
   public void setAscending(boolean ascending) {
     this.ascending = ascending;
   }
+
+  public boolean isLastQuery() {
+    return selectOperator.isLastQuery();
+  }
+
+  public boolean hasAggregationFunction() {
+    return selectOperator.hasAggregationFunction();
+  }
+
+  public boolean hasTimeSeriesGeneratingFunction() {
+    return selectOperator.hasTimeSeriesGeneratingFunction();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SFWOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SFWOperator.java
deleted file mode 100644
index 14d510b..0000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SFWOperator.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.qp.logical.crud;
-
-import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.qp.logical.RootOperator;
-
-import java.util.List;
-
-/**
- * SFWOperator(select-from-where) includes four subclass: INSERT,DELETE,UPDATE,QUERY. All of these
- * four statements has three partition: select clause, from clause and filter clause(where clause).
- */
-public abstract class SFWOperator extends RootOperator {
-
-  private SelectOperator selectOperator;
-  private FromOperator fromOperator;
-  private FilterOperator filterOperator;
-  private boolean hasAggregation = false;
-  private boolean hasUdf = false;
-  private boolean lastQuery = false;
-
-  public SFWOperator(int tokenIntType) {
-    super(tokenIntType);
-    operatorType = OperatorType.SFW;
-  }
-
-  public FromOperator getFromOperator() {
-    return fromOperator;
-  }
-
-  public void setFromOperator(FromOperator from) {
-    this.fromOperator = from;
-  }
-
-  public SelectOperator getSelectOperator() {
-    return selectOperator;
-  }
-
-  /** set selectOperator, then init hasAggregation according to selectOperator. */
-  public void setSelectOperator(SelectOperator sel) {
-    this.selectOperator = sel;
-    if (sel.hasAggregation()) {
-      hasAggregation = true;
-    }
-    if (sel.isUdfQuery()) {
-      hasUdf = true;
-    }
-    if (sel.isLastQuery()) {
-      lastQuery = true;
-    }
-  }
-
-  public FilterOperator getFilterOperator() {
-    return filterOperator;
-  }
-
-  public void setFilterOperator(FilterOperator filter) {
-    this.filterOperator = filter;
-  }
-
-  /**
-   * get information from SelectOperator and FromOperator and generate all table paths.
-   *
-   * @return - a list of seriesPath
-   */
-  public List<PartialPath> getSelectedPaths() {
-    return selectOperator != null ? selectOperator.getSuffixPaths() : null;
-  }
-
-  public boolean hasAggregation() {
-    return hasAggregation;
-  }
-
-  public boolean hasUdf() {
-    return hasUdf;
-  }
-
-  public boolean isLastQuery() {
-    return lastQuery;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java
index 5b34ea5..913ef74 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java
@@ -20,7 +20,9 @@ package org.apache.iotdb.db.qp.logical.crud;
 
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.query.udf.core.context.UDFContext;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
+import org.apache.iotdb.db.query.expression.unary.TimeSeriesOperand;
 
 import java.time.ZoneId;
 import java.util.ArrayList;
@@ -30,87 +32,79 @@ import java.util.List;
 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;
-  }
-
-  public void setLastQuery() {
-    lastQuery = true;
+    return isLastQuery;
   }
 
-  public List<String> getAggregations() {
-    return this.aggregations;
+  public boolean hasAggregationFunction() {
+    return hasAggregationFunction;
   }
 
-  public void setAggregations(List<String> aggregations) {
-    this.aggregations = aggregations;
+  public boolean hasTimeSeriesGeneratingFunction() {
+    return hasTimeSeriesGeneratingFunction;
   }
 
-  public boolean hasAggregation() {
-    return hasBuiltinAggregation; // todo: hasBuiltinAggregation || hasUDAF
+  public void addResultColumn(ResultColumn resultColumn) {
+    resultColumns.add(resultColumn);
+    if (resultColumn.getExpression().isAggregationFunctionExpression()) {
+      hasAggregationFunction = true;
+    }
+    if (resultColumn.getExpression().isTimeSeriesGeneratingFunctionExpression()) {
+      hasTimeSeriesGeneratingFunction = true;
+    }
   }
 
-  public void setSuffixPathList(List<PartialPath> suffixPaths) {
-    suffixList = suffixPaths;
+  public void setResultColumns(List<ResultColumn> resultColumns) {
+    this.resultColumns = resultColumns;
   }
 
-  public List<PartialPath> getSuffixPaths() {
-    return suffixList;
+  public List<ResultColumn> getResultColumns() {
+    return resultColumns;
   }
 
-  public void addUdf(UDFContext udf) {
-    if (udf != null) {
-      udfQuery = true;
+  public List<PartialPath> getPaths() {
+    if (pathsCache == null) {
+      pathsCache = new ArrayList<>();
+      for (ResultColumn resultColumn : resultColumns) {
+        pathsCache.add(((TimeSeriesOperand) resultColumn.getExpression()).getPath());
+      }
     }
-    udfList.add(udf);
-  }
-
-  public List<UDFContext> getUdfList() {
-    return udfList;
-  }
-
-  public boolean isUdfQuery() {
-    return udfQuery;
+    return pathsCache;
   }
 
-  public void setUdfList(List<UDFContext> udfList) {
-    this.udfList = udfList;
+  public List<String> getAggregationFunctions() {
+    if (aggregationFunctionsCache == null) {
+      aggregationFunctionsCache = new ArrayList<>();
+      for (ResultColumn resultColumn : resultColumns) {
+        aggregationFunctionsCache.add(
+            ((FunctionExpression) resultColumn.getExpression()).getFunctionName());
+      }
+    }
+    return aggregationFunctionsCache;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreateIndexOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreateIndexOperator.java
index 46f5cad..8934bee 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreateIndexOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreateIndexOperator.java
@@ -19,13 +19,17 @@
 package org.apache.iotdb.db.qp.logical.sys;
 
 import org.apache.iotdb.db.index.common.IndexType;
-import org.apache.iotdb.db.qp.logical.crud.SFWOperator;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
 /** this operator is to create a certain index on some time series. */
-public class CreateIndexOperator extends SFWOperator {
+public class CreateIndexOperator extends Operator {
 
+  private final List<PartialPath> paths;
   private Map<String, String> props;
   private long time;
   private IndexType indexType;
@@ -33,6 +37,15 @@ public class CreateIndexOperator extends SFWOperator {
   public CreateIndexOperator(int tokenIntType) {
     super(tokenIntType);
     operatorType = OperatorType.CREATE_INDEX;
+    paths = new ArrayList<>();
+  }
+
+  public List<PartialPath> getPaths() {
+    return paths;
+  }
+
+  public void addPath(PartialPath path) {
+    paths.add(path);
   }
 
   public long getTime() {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropIndexOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropIndexOperator.java
index 1ea6538..e538311 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropIndexOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropIndexOperator.java
@@ -19,16 +19,30 @@
 package org.apache.iotdb.db.qp.logical.sys;
 
 import org.apache.iotdb.db.index.common.IndexType;
-import org.apache.iotdb.db.qp.logical.crud.SFWOperator;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /** this operator is to drop a certain index on some time series. */
-public class DropIndexOperator extends SFWOperator {
+public class DropIndexOperator extends Operator {
 
+  private final List<PartialPath> paths;
   private IndexType indexType;
 
   public DropIndexOperator(int tokenIntType) {
     super(tokenIntType);
     operatorType = OperatorType.DROP_INDEX;
+    paths = new ArrayList<>();
+  }
+
+  public List<PartialPath> getPaths() {
+    return paths;
+  }
+
+  public void addPath(PartialPath path) {
+    paths.add(path);
   }
 
   public IndexType getIndexType() {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDFPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDFPlan.java
index b152a08..f1ed19f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDFPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDFPlan.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.qp.physical.crud;
 
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.query.udf.core.context.UDFContext;
+import org.apache.iotdb.db.query.expression.ResultColumn;
 
 import java.util.List;
 
@@ -37,7 +37,7 @@ public interface UDFPlan {
    * Build the execution plan of the executors. This method will not create any UDF instances, nor
    * will it execute user-defined logic.
    */
-  void constructUdfExecutors(List<UDFContext> udfContexts);
+  void constructUdfExecutors(List<ResultColumn> resultColumns);
 
   /** Allocate computing resources, create UDF instances, and call UDF initialization methods. */
   void initializeUdfExecutors(long queryId, float collectorMemoryBudgetInMb)
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java
index 78eb25d..6eecb73 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java
@@ -21,7 +21,8 @@ package org.apache.iotdb.db.qp.physical.crud;
 
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.query.udf.core.context.UDFContext;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
 import org.apache.iotdb.db.query.udf.core.executor.UDTFExecutor;
 import org.apache.iotdb.db.query.udf.service.UDFClassLoaderManager;
 import org.apache.iotdb.db.query.udf.service.UDFRegistrationService;
@@ -52,18 +53,15 @@ public class UDTFPlan extends RawDataQueryPlan implements UDFPlan {
   }
 
   @Override
-  public void constructUdfExecutors(List<UDFContext> udfContexts) {
-    for (int i = 0; i < udfContexts.size(); ++i) {
-      UDFContext context = udfContexts.get(i);
-      if (context == null) {
+  public void constructUdfExecutors(List<ResultColumn> resultColumns) {
+    for (int i = 0; i < resultColumns.size(); ++i) {
+      FunctionExpression expression = (FunctionExpression) resultColumns.get(i).getExpression();
+      if (expression == null) {
         continue;
       }
 
-      String columnName = context.getColumnName();
-      if (!columnName2Executor.containsKey(columnName)) {
-        UDTFExecutor executor = new UDTFExecutor(context, zoneId);
-        columnName2Executor.put(columnName, executor);
-      }
+      String columnName = expression.toString();
+      columnName2Executor.computeIfAbsent(columnName, k -> new UDTFExecutor(expression, zoneId));
       originalOutputColumnIndex2Executor.put(i, columnName2Executor.get(columnName));
     }
   }
@@ -134,7 +132,7 @@ public class UDTFPlan extends RawDataQueryPlan implements UDFPlan {
   @Override
   public String getColumnForDisplay(String columnForReader, int pathIndex) {
     if (paths.get(pathIndex) == null) {
-      return this.getExecutorByOriginalOutputColumnIndex(pathIndex).getContext().getColumnName();
+      return this.getExecutorByOriginalOutputColumnIndex(pathIndex).getExpression().toString();
     }
     return columnForReader;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
index 3888c74..a5bff6b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
@@ -78,8 +78,6 @@ import org.apache.iotdb.db.qp.logical.sys.StartTriggerOperator;
 import org.apache.iotdb.db.qp.logical.sys.StopTriggerOperator;
 import org.apache.iotdb.db.qp.logical.sys.TracingOperator;
 import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AggregationCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AggregationElementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AliasClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlignByDeviceClauseOrDisableAlignInSpecialLimitContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlignByDeviceStatementOrDisableAlignInSpecialClauseContext;
@@ -87,11 +85,8 @@ import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterTimeseriesContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterUserContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AndExpressionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AsClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AsElementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AttributeClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.AttributeClausesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.BuiltInFunctionCallContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ClearcacheContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ConstantContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountDevicesContext;
@@ -115,14 +110,15 @@ import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropIndexContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropRoleContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropTriggerContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropUserContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.ExpressionContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.FillClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.FillStatementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.FlushContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.FromClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.FullMergeContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.FullPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionAsClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionAsElementContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionAttributeContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantRoleContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantRoleToUserContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantUserContext;
@@ -141,8 +137,6 @@ import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertMultiValueContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertStatementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertValuesSpecContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.KillQueryContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LastClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LastElementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitStatementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListAllRoleOfUserContext;
@@ -172,11 +166,13 @@ import org.apache.iotdb.db.qp.sql.SqlBaseParser.PrivilegesContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.PropertyContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.PropertyValueContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.RemoveFileContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.ResultColumnContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeRoleContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeRoleFromUserContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeUserContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeWatermarkEmbeddingContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.RootOrIdContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.SelectClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.SelectStatementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.SequenceClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.SetStorageGroupContext;
@@ -204,23 +200,30 @@ import org.apache.iotdb.db.qp.sql.SqlBaseParser.StartTriggerContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.StopTriggerContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.StringLiteralContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.SuffixPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TableCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TableElementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.TagClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.TimeIntervalContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.TopClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.TracingOffContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.TracingOnContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.TriggerAttributeContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.TypeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UdfAttributeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UdfCallContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.UnsetTTLStatementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.WhereClauseContext;
 import org.apache.iotdb.db.qp.utils.DatetimeUtils;
 import org.apache.iotdb.db.query.executor.fill.IFill;
 import org.apache.iotdb.db.query.executor.fill.LinearFill;
 import org.apache.iotdb.db.query.executor.fill.PreviousFill;
-import org.apache.iotdb.db.query.udf.core.context.UDFContext;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ResultColumn;
+import org.apache.iotdb.db.query.expression.binary.AdditionExpression;
+import org.apache.iotdb.db.query.expression.binary.DivisionExpression;
+import org.apache.iotdb.db.query.expression.binary.ModuloExpression;
+import org.apache.iotdb.db.query.expression.binary.MultiplicationExpression;
+import org.apache.iotdb.db.query.expression.binary.SubtractionExpression;
+import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
+import org.apache.iotdb.db.query.expression.unary.MinusExpression;
+import org.apache.iotdb.db.query.expression.unary.NumberLiteralOperand;
+import org.apache.iotdb.db.query.expression.unary.TimeSeriesOperand;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -255,7 +258,7 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
       "For delete statement, where clause can only contain atomic expressions like : "
           + "time > XXX, time <= XXX, or two atomic expressions connected by 'AND'";
   private ZoneId zoneId;
-  QueryOperator queryOp;
+  private QueryOperator queryOp;
   private boolean isParsingSlidingStep;
 
   public void setZoneId(ZoneId zoneId) {
@@ -310,9 +313,7 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   @Override
   public Operator visitInsertStatement(InsertStatementContext ctx) {
     InsertOperator insertOp = new InsertOperator(SQLConstant.TOK_INSERT);
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
-    selectOp.addSelectPath(parsePrefixPath(ctx.prefixPath()));
-    insertOp.setSelectOperator(selectOp);
+    insertOp.setDevice(parsePrefixPath(ctx.prefixPath()));
     parseInsertColumnSpec(ctx.insertColumnsSpec(), insertOp);
     parseInsertValuesSpec(ctx.insertValuesSpec(), insertOp);
     return insertOp;
@@ -321,17 +322,13 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   @Override
   public Operator visitDeleteStatement(DeleteStatementContext ctx) {
     DeleteDataOperator deleteDataOp = new DeleteDataOperator(SQLConstant.TOK_DELETE);
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
     List<PrefixPathContext> prefixPaths = ctx.prefixPath();
     for (PrefixPathContext prefixPath : prefixPaths) {
-      PartialPath path = parsePrefixPath(prefixPath);
-      selectOp.addSelectPath(path);
+      deleteDataOp.addPath(parsePrefixPath(prefixPath));
     }
-    deleteDataOp.setSelectOperator(selectOp);
     if (ctx.whereClause() != null) {
       FilterOperator whereOp = (FilterOperator) visit(ctx.whereClause());
-      deleteDataOp.setFilterOperator(whereOp.getChildren().get(0));
-      Pair<Long, Long> timeInterval = parseDeleteTimeInterval(deleteDataOp);
+      Pair<Long, Long> timeInterval = parseDeleteTimeInterval(whereOp.getChildren().get(0));
       deleteDataOp.setStartTime(timeInterval.left);
       deleteDataOp.setEndTime(timeInterval.right);
     } else {
@@ -366,31 +363,21 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   @Override
   public Operator visitCreateIndex(CreateIndexContext ctx) {
     CreateIndexOperator createIndexOp = new CreateIndexOperator(SQLConstant.TOK_CREATE_INDEX);
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
     List<PrefixPathContext> prefixPaths = Collections.singletonList(ctx.prefixPath());
     for (PrefixPathContext prefixPath : prefixPaths) {
-      PartialPath path = parsePrefixPath(prefixPath);
-      selectOp.addSelectPath(path);
+      createIndexOp.addPath(parsePrefixPath(prefixPath));
     }
-    createIndexOp.setSelectOperator(selectOp);
     parseIndexWithClause(ctx.indexWithClause(), createIndexOp);
-    FilterOperator whereOp;
     if (ctx.whereClause() != null) {
-      whereOp = (FilterOperator) visit(ctx.whereClause());
-      createIndexOp.setFilterOperator(whereOp.getChildren().get(0));
-      long indexTime = parseCreateIndexFilter(createIndexOp);
+      FilterOperator whereOp = (FilterOperator) visit(ctx.whereClause());
+      long indexTime = parseCreateIndexFilter(whereOp.getChildren().get(0));
       createIndexOp.setTime(indexTime);
     }
     return createIndexOp;
   }
 
-  /**
-   * for create index command, time should only have an end time.
-   *
-   * @param operator create index plan
-   */
-  private long parseCreateIndexFilter(CreateIndexOperator operator) {
-    FilterOperator filterOperator = operator.getFilterOperator();
+  /** for create index command, time should only have an end time. */
+  private long parseCreateIndexFilter(FilterOperator filterOperator) {
     if (filterOperator.getTokenIntType() != SQLConstant.GREATERTHAN
         && filterOperator.getTokenIntType() != SQLConstant.GREATERTHANOREQUALTO) {
       throw new SQLParserException(
@@ -428,13 +415,10 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   @Override
   public Operator visitDropIndex(DropIndexContext ctx) {
     DropIndexOperator dropIndexOperator = new DropIndexOperator(SQLConstant.TOK_DROP_INDEX);
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
     List<PrefixPathContext> prefixPaths = Collections.singletonList(ctx.prefixPath());
     for (PrefixPathContext prefixPath : prefixPaths) {
-      PartialPath path = parsePrefixPath(prefixPath);
-      selectOp.addSelectPath(path);
+      dropIndexOperator.addPath(parsePrefixPath(prefixPath));
     }
-    dropIndexOperator.setSelectOperator(selectOp);
     try {
       dropIndexOperator.setIndexType(IndexType.getIndexType(ctx.indexName.getText()));
     } catch (UnsupportedIndexTypeException e) {
@@ -1007,24 +991,12 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   @Override
   public Operator visitSelectStatement(SelectStatementContext ctx) {
     queryOp = new QueryOperator(SQLConstant.TOK_QUERY);
-    SelectOperator selectOp = (SelectOperator) visit(ctx.selectElements());
-    queryOp.setSelectOperator(selectOp);
-    FromOperator fromOp = (FromOperator) visit(ctx.fromClause());
-    queryOp.setFromOperator(fromOp);
-    if (ctx.topClause() != null) {
-      Map<String, Object> props = new HashMap<>();
-      int top = Integer.parseInt(ctx.topClause().INT().getText());
-      if (top < 0) {
-        throw new SQLParserException("TOP <N>: N should be greater than 0.");
-      }
-      props.put(TOP_K, top);
-      queryOp.setProps(props);
-    }
+    queryOp.setSelectOperator((SelectOperator) visit(ctx.selectClause()));
+    queryOp.setFromOperator((FromOperator) visit(ctx.fromClause()));
     if (ctx.whereClause() != null) {
       Operator operator = visit(ctx.whereClause());
       if (operator instanceof FilterOperator) {
-        FilterOperator whereOp = (FilterOperator) operator;
-        queryOp.setFilterOperator(whereOp.getChildren().get(0));
+        queryOp.setFilterOperator(((FilterOperator) operator).getChildren().get(0));
       }
     }
     if (ctx.specialClause() != null) {
@@ -1034,91 +1006,98 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   }
 
   @Override
-  public Operator visitAggregationElement(AggregationElementContext ctx) {
+  public Operator visitSelectClause(SelectClauseContext ctx) {
     SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
 
-    for (AggregationCallContext aggregationCallContext : ctx.aggregationCall()) {
-      BuiltInFunctionCallContext builtInFunctionCallContext =
-          aggregationCallContext.builtInFunctionCall();
-      UdfCallContext udfCallContext = aggregationCallContext.udfCall();
-      if (builtInFunctionCallContext != null) {
-        selectOp.addClusterPath(
-            parseSuffixPath(builtInFunctionCallContext.suffixPath()),
-            builtInFunctionCallContext.functionName().getText());
-        selectOp.addUdf(null);
-      } else if (udfCallContext != null) {
-        selectOp.addClusterPath(null, null);
-        parseUdfCall(udfCallContext, selectOp);
-      }
+    if (ctx.topClause() != null) {
+      // TODO: parse info of top clause into selectOp
+      visitTopClause(ctx.topClause());
+    } else if (ctx.LAST() != null) {
+      selectOp.markAsLastQuery();
     }
 
-    return selectOp;
-  }
-
-  public void parseUdfCall(UdfCallContext udfCall, SelectOperator selectOp) {
-    String udfName = udfCall.udfName.getText();
-    UDFContext udf = new UDFContext(udfName);
-
-    for (SuffixPathContext suffixPathContext : udfCall.udfSuffixPaths().suffixPath()) {
-      udf.addPath(parseSuffixPath(suffixPathContext));
-    }
-    for (UdfAttributeContext udfAttributeContext : udfCall.udfAttribute()) {
-      udf.addAttribute(
-          removeStringQuote(udfAttributeContext.udfAttributeKey.getText()),
-          removeStringQuote(udfAttributeContext.udfAttributeValue.getText()));
+    for (ResultColumnContext resultColumnContext : ctx.resultColumn()) {
+      selectOp.addResultColumn(parseResultColumn(resultColumnContext));
     }
 
-    selectOp.addUdf(udf);
+    return selectOp;
   }
 
   @Override
-  public Operator visitLastElement(LastElementContext ctx) {
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
-    selectOp.setLastQuery();
-    LastClauseContext lastClauseContext = ctx.lastClause();
-    if (lastClauseContext.asClause().size() != 0) {
-      parseAsClause(lastClauseContext.asClause(), selectOp);
-    } else {
-      List<SuffixPathContext> suffixPaths = lastClauseContext.suffixPath();
-      for (SuffixPathContext suffixPath : suffixPaths) {
-        PartialPath path = parseSuffixPath(suffixPath);
-        selectOp.addSelectPath(path);
-      }
+  public Operator visitTopClause(TopClauseContext ctx) {
+    Map<String, Object> props = new HashMap<>();
+    int top = Integer.parseInt(ctx.INT().getText());
+    if (top < 0) {
+      throw new SQLParserException("TOP <N>: N should be greater than 0.");
     }
-    return selectOp;
+    props.put(TOP_K, top);
+    queryOp.setProps(props);
+    return queryOp;
   }
 
-  @Override
-  public Operator visitAsElement(AsElementContext ctx) {
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
-    parseAsClause(ctx.asClause(), selectOp);
-    return selectOp;
+  private ResultColumn parseResultColumn(ResultColumnContext resultColumnContext) {
+    return new ResultColumn(
+        parseExpression(resultColumnContext.expression()),
+        resultColumnContext.AS() == null ? null : resultColumnContext.ID().getText());
   }
 
-  @Override
-  public Operator visitFunctionAsElement(FunctionAsElementContext ctx) {
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
-    List<FunctionAsClauseContext> functionAsClauseContexts = ctx.functionAsClause();
-    for (FunctionAsClauseContext functionAsClauseContext : functionAsClauseContexts) {
-      BuiltInFunctionCallContext functionCallContext =
-          functionAsClauseContext.builtInFunctionCall();
-      PartialPath path = parseSuffixPath(functionCallContext.suffixPath());
-      if (functionAsClauseContext.ID() != null) {
-        path.setTsAlias(functionAsClauseContext.ID().toString());
-      }
-      selectOp.addClusterPath(path, functionCallContext.functionName().getText());
+  @SuppressWarnings("squid:S3776")
+  private Expression parseExpression(ExpressionContext context) {
+    // unary
+    if (context.numberLiteral() != null) {
+      return new NumberLiteralOperand(Double.parseDouble(context.numberLiteral().getText()));
     }
-    return selectOp;
+    if (context.suffixPath() != null) {
+      return new TimeSeriesOperand(parseSuffixPath(context.suffixPath()));
+    }
+    if (context.functionClause() != null) {
+      return parseFunctionExpression(context.functionClause());
+    }
+    if (context.unary != null) {
+      return context.MINUS() != null
+          ? new MinusExpression(parseExpression(context.expression(0)))
+          : parseExpression(context.expression(0));
+    }
+
+    // binary
+    Expression leftExpression = parseExpression(context.leftExpression);
+    Expression rightExpression = parseExpression(context.rightExpression);
+    if (context.STAR() != null) {
+      return new MultiplicationExpression(leftExpression, rightExpression);
+    }
+    if (context.DIV() != null) {
+      return new DivisionExpression(leftExpression, rightExpression);
+    }
+    if (context.MOD() != null) {
+      return new ModuloExpression(leftExpression, rightExpression);
+    }
+    if (context.PLUS() != null) {
+      return new AdditionExpression(leftExpression, rightExpression);
+    }
+    if (context.MINUS() != null) {
+      return new SubtractionExpression(leftExpression, rightExpression);
+    }
+
+    throw new UnsupportedOperationException();
   }
 
-  public void parseAsClause(List<AsClauseContext> asClauseContexts, SelectOperator selectOp) {
-    for (AsClauseContext asClauseContext : asClauseContexts) {
-      PartialPath path = parseSuffixPath(asClauseContext.suffixPath());
-      if (asClauseContext.ID() != null) {
-        path.setTsAlias(asClauseContext.ID().toString());
-      }
-      selectOp.addSelectPath(path);
+  private Expression parseFunctionExpression(FunctionClauseContext functionClause) {
+    FunctionExpression functionExpression =
+        new FunctionExpression(functionClause.functionName.getText());
+
+    // expressions
+    for (ExpressionContext expression : functionClause.expression()) {
+      functionExpression.addExpression(parseExpression(expression));
+    }
+
+    // attributes
+    for (FunctionAttributeContext functionAttribute : functionClause.functionAttribute()) {
+      functionExpression.addAttribute(
+          removeStringQuote(functionAttribute.functionAttributeKey.getText()),
+          removeStringQuote(functionAttribute.functionAttributeValue.getText()));
     }
+
+    return functionExpression;
   }
 
   @Override
@@ -1242,30 +1221,6 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   }
 
   @Override
-  public Operator visitTableElement(TableElementContext ctx) {
-    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT, zoneId);
-
-    for (TableCallContext tableCallContext : ctx.tableCall()) {
-      SuffixPathContext suffixPathContext = tableCallContext.suffixPath();
-      UdfCallContext udfCallContext = tableCallContext.udfCall();
-      if (suffixPathContext != null) {
-        selectOp.addSelectPath(parseSuffixPath(suffixPathContext));
-        selectOp.addUdf(null);
-      } else if (udfCallContext != null) {
-        selectOp.addSelectPath(null);
-        parseUdfCall(udfCallContext, selectOp);
-      } else {
-        selectOp.addSelectPath(
-            new PartialPath(
-                new String[] {tableCallContext.SINGLE_QUOTE_STRING_LITERAL().getText()}));
-        selectOp.addUdf(null);
-      }
-    }
-
-    return selectOp;
-  }
-
-  @Override
   public Operator visitFromClause(FromClauseContext ctx) {
     FromOperator fromOp = new FromOperator(SQLConstant.TOK_FROM);
     List<PrefixPathContext> prefixFromPaths = ctx.prefixPath();
@@ -1286,10 +1241,11 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
     }
     if (ctx.LIKE() != null) {
       // whole matching case
-      if (queryOp.getSelectedPaths().size() != 1) {
+      if (queryOp.getSelectOperator().getResultColumns().size() != 1) {
         throw new SQLParserException("Index query statement allows only one select path");
       }
-      if (!path.equals(queryOp.getSelectedPaths().get(0))) {
+      if (!path.equals(
+          queryOp.getSelectOperator().getResultColumns().get(0).getExpression().toString())) {
         throw new SQLParserException(
             "In the index query statement, "
                 + "the path in select element and the index predicate should be same");
@@ -1314,9 +1270,7 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
       } else {
         props = new HashMap<>();
       }
-      List<PartialPath> suffixPaths = new ArrayList<>();
-      suffixPaths.add(path);
-      queryOp.getSelectOperator().setSuffixPathList(suffixPaths);
+      queryOp.getSelectOperator().addResultColumn(new ResultColumn(new TimeSeriesOperand(path)));
       props.put(PATTERN, compositePattern);
       props.put(THRESHOLD, thresholds);
       queryOp.setIndexType(IndexType.ELB_INDEX);
@@ -1336,7 +1290,7 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   }
 
   public void parseGroupByLevelClause(GroupByLevelClauseContext ctx, QueryOperator queryOp) {
-    if (!queryOp.hasAggregation()) {
+    if (!queryOp.hasAggregationFunction()) {
       throw new SQLParserException(GroupByTimePlan.LACK_FUNC_ERROR_MESSAGE);
     }
     queryOp.setGroupByLevel(true);
@@ -1430,7 +1384,7 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   }
 
   private void parseGroupByTimeClause(GroupByTimeClauseContext ctx, QueryOperator queryOp) {
-    if (!queryOp.hasAggregation()) {
+    if (!queryOp.hasAggregationFunction()) {
       throw new SQLParserException(GroupByTimePlan.LACK_FUNC_ERROR_MESSAGE);
     }
     queryOp.setGroupByTime(true);
@@ -1458,7 +1412,7 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   }
 
   private void parseGroupByFillClause(GroupByFillClauseContext ctx, QueryOperator queryOp) {
-    if (!queryOp.hasAggregation()) {
+    if (!queryOp.hasAggregationFunction()) {
       throw new SQLParserException(GroupByTimePlan.LACK_FUNC_ERROR_MESSAGE);
     }
     queryOp.setGroupByTime(true);
@@ -1636,13 +1590,7 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
     return privileges.toArray(new String[0]);
   }
 
-  /**
-   * for delete command, time should only have an end time.
-   *
-   * @param operator delete logical plan
-   */
-  private Pair<Long, Long> parseDeleteTimeInterval(DeleteDataOperator operator) {
-    FilterOperator filterOperator = operator.getFilterOperator();
+  private Pair<Long, Long> parseDeleteTimeInterval(FilterOperator filterOperator) {
     if (!filterOperator.isLeaf() && filterOperator.getTokenIntType() != SQLConstant.KW_AND) {
       throw new SQLParserException(DELETE_RANGE_ERROR_MSG);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index fdb331b..71dc3da 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -122,7 +122,7 @@ import org.apache.iotdb.db.qp.physical.sys.ShowTriggersPlan;
 import org.apache.iotdb.db.qp.physical.sys.StartTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.StopTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.TracingPlan;
-import org.apache.iotdb.db.query.udf.core.context.UDFContext;
+import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -152,7 +152,6 @@ public class PhysicalGenerator {
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   private PhysicalPlan doTransformation(Operator operator, int fetchSize)
       throws QueryProcessException {
-    List<PartialPath> paths;
     switch (operator.getType()) {
       case AUTHOR:
         AuthorOperator author = (AuthorOperator) operator;
@@ -211,13 +210,13 @@ public class PhysicalGenerator {
       case CREATE_INDEX:
         CreateIndexOperator createIndexOp = (CreateIndexOperator) operator;
         return new CreateIndexPlan(
-            createIndexOp.getSelectedPaths(),
+            createIndexOp.getPaths(),
             createIndexOp.getProps(),
             createIndexOp.getTime(),
             createIndexOp.getIndexType());
       case DROP_INDEX:
         DropIndexOperator dropIndexOp = (DropIndexOperator) operator;
-        return new DropIndexPlan(dropIndexOp.getSelectedPaths(), dropIndexOp.getIndexType());
+        return new DropIndexPlan(dropIndexOp.getPaths(), dropIndexOp.getIndexType());
       case ALTER_TIMESERIES:
         AlterTimeSeriesOperator alterTimeSeriesOperator = (AlterTimeSeriesOperator) operator;
         return new AlterTimeSeriesPlan(
@@ -229,11 +228,9 @@ public class PhysicalGenerator {
             alterTimeSeriesOperator.getAttributesMap());
       case DELETE:
         DeleteDataOperator delete = (DeleteDataOperator) operator;
-        paths = delete.getSelectedPaths();
-        return new DeletePlan(delete.getStartTime(), delete.getEndTime(), paths);
+        return new DeletePlan(delete.getStartTime(), delete.getEndTime(), delete.getPaths());
       case INSERT:
         InsertOperator insert = (InsertOperator) operator;
-        paths = insert.getSelectedPaths();
         int measurementsNum = 0;
         for (String measurement : insert.getMeasurementList()) {
           if (measurement.startsWith("(") && measurement.endsWith(")")) {
@@ -250,7 +247,7 @@ public class PhysicalGenerator {
         }
         if (measurementsNum == insert.getValueList().length) {
           return new InsertRowPlan(
-              paths.get(0),
+              insert.getDevice(),
               insert.getTimes()[0],
               insert.getMeasurementList(),
               insert.getValueList());
@@ -259,7 +256,7 @@ public class PhysicalGenerator {
         for (int i = 0; i < insert.getTimes().length; i++) {
           insertRowsPlan.addOneInsertRowPlan(
               new InsertRowPlan(
-                  paths.get(0),
+                  insert.getDevice(),
                   insert.getTimes()[i],
                   insert.getMeasurementList(),
                   Arrays.copyOfRange(
@@ -458,17 +455,17 @@ public class PhysicalGenerator {
     return SchemaUtils.getSeriesTypesByPaths(paths);
   }
 
-  interface Transfrom {
+  interface Transform {
     QueryPlan transform(QueryOperator queryOperator) throws QueryProcessException;
   }
 
   /** agg physical plan transform */
-  public static class AggPhysicalPlanRule implements Transfrom {
+  public static class AggPhysicalPlanRule implements Transform {
 
     @Override
     public QueryPlan transform(QueryOperator queryOperator) throws QueryProcessException {
       QueryPlan queryPlan;
-      if (queryOperator.hasUdf()) {
+      if (queryOperator.hasTimeSeriesGeneratingFunction()) {
         throw new QueryProcessException(
             "User-defined and built-in hybrid aggregation is not supported.");
       }
@@ -480,7 +477,7 @@ public class PhysicalGenerator {
         queryPlan = new AggregationPlan();
       }
       ((AggregationPlan) queryPlan)
-          .setAggregations(queryOperator.getSelectOperator().getAggregations());
+          .setAggregations(queryOperator.getSelectOperator().getAggregationFunctions());
 
       if (queryOperator.isGroupByTime()) {
         GroupByTimePlan groupByTimePlan = (GroupByTimePlan) queryPlan;
@@ -519,11 +516,11 @@ public class PhysicalGenerator {
   }
 
   /** fill physical plan transfrom */
-  public static class FillPhysicalPlanRule implements Transfrom {
+  public static class FillPhysicalPlanRule implements Transform {
 
     @Override
     public QueryPlan transform(QueryOperator queryOperator) throws QueryProcessException {
-      if (queryOperator.hasUdf()) {
+      if (queryOperator.hasTimeSeriesGeneratingFunction()) {
         throw new QueryProcessException("Fill functions are not supported in UDF queries.");
       }
       FillQueryPlan queryPlan = new FillQueryPlan();
@@ -542,7 +539,7 @@ public class PhysicalGenerator {
   private PhysicalPlan transformQuery(QueryOperator queryOperator) throws QueryProcessException {
     QueryPlan queryPlan = null;
 
-    if (queryOperator.hasAggregation()) {
+    if (queryOperator.hasAggregationFunction()) {
       queryPlan = new AggPhysicalPlanRule().transform(queryOperator);
     } else if (queryOperator.isFill()) {
       queryPlan = new FillPhysicalPlanRule().transform(queryOperator);
@@ -550,9 +547,10 @@ public class PhysicalGenerator {
       queryPlan = new LastQueryPlan();
     } else if (queryOperator.getIndexType() != null) {
       queryPlan = new QueryIndexPlan();
-    } else if (queryOperator.hasUdf()) {
+    } else if (queryOperator.hasTimeSeriesGeneratingFunction()) {
       queryPlan = new UDTFPlan(queryOperator.getSelectOperator().getZoneId());
-      ((UDTFPlan) queryPlan).constructUdfExecutors(queryOperator.getSelectOperator().getUdfList());
+      ((UDTFPlan) queryPlan)
+          .constructUdfExecutors(queryOperator.getSelectOperator().getResultColumns());
     } else {
       queryPlan = new RawDataQueryPlan();
     }
@@ -560,7 +558,7 @@ public class PhysicalGenerator {
     if (queryOperator.isAlignByDevice()) {
       queryPlan = getAlignQueryPlan(queryOperator, queryPlan);
     } else {
-      queryPlan.setPaths(queryOperator.getSelectedPaths());
+      queryPlan.setPaths(queryOperator.getSelectOperator().getPaths());
       // Last query result set will not be affected by alignment
       if (queryPlan instanceof LastQueryPlan && !queryOperator.isAlignByTime()) {
         throw new QueryProcessException("Disable align cannot be applied to LAST query.");
@@ -626,8 +624,8 @@ public class PhysicalGenerator {
     List<PartialPath> prefixPaths = queryOperator.getFromOperator().getPrefixPaths();
     // remove stars in fromPaths and get deviceId with deduplication
     List<PartialPath> devices = this.removeStarsInDeviceWithUnique(prefixPaths);
-    List<PartialPath> suffixPaths = queryOperator.getSelectOperator().getSuffixPaths();
-    List<String> originAggregations = queryOperator.getSelectOperator().getAggregations();
+    List<PartialPath> suffixPaths = queryOperator.getSelectOperator().getPaths();
+    List<String> originAggregations = queryOperator.getSelectOperator().getAggregationFunctions();
 
     // to record result measurement columns
     List<String> measurements = new ArrayList<>();
@@ -886,9 +884,10 @@ public class PhysicalGenerator {
       if (path != null) { // non-udf
         indexedPaths.add(new Pair<>(paths.get(i), i));
       } else { // udf
-        UDFContext context =
-            ((UDTFPlan) queryPlan).getExecutorByOriginalOutputColumnIndex(i).getContext();
-        for (PartialPath udfPath : context.getPaths()) {
+        FunctionExpression functionExpression =
+            (FunctionExpression)
+                ((UDTFPlan) queryPlan).getExecutorByOriginalOutputColumnIndex(i).getExpression();
+        for (PartialPath udfPath : functionExpression.getPaths()) {
           indexedPaths.add(new Pair<>(udfPath, i));
         }
       }
@@ -984,12 +983,12 @@ public class PhysicalGenerator {
 
   private static boolean verifyAllAggregationDataTypesEqual(QueryOperator queryOperator)
       throws MetadataException {
-    List<String> aggregations = queryOperator.getSelectOperator().getAggregations();
+    List<String> aggregations = queryOperator.getSelectOperator().getAggregationFunctions();
     if (aggregations.isEmpty()) {
       return true;
     }
 
-    List<PartialPath> paths = queryOperator.getSelectedPaths();
+    List<PartialPath> paths = queryOperator.getSelectOperator().getPaths();
     List<TSDataType> dataTypes = SchemaUtils.getSeriesTypesByPaths(paths);
     String aggType = aggregations.get(0);
     switch (aggType) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
index 0991328..ff75491 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
@@ -105,7 +105,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
             ((QueryOperator) operator).getIndexType() == null);
       } else {
         isAlignByDevice = true;
-        if (((QueryOperator) operator).hasUdf()) {
+        if (((QueryOperator) operator).hasTimeSeriesGeneratingFunction()) {
           throw new LogicalOptimizeException(
               "ALIGN BY DEVICE clause is not supported in UDF queries.");
         }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/UDTFDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/UDTFDataSet.java
index 5761ef2..19d81c6 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/UDTFDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/UDTFDataSet.java
@@ -163,7 +163,7 @@ public abstract class UDTFDataSet extends QueryDataSet {
   }
 
   private int[] calculateReaderIndexes(UDTFExecutor executor) {
-    List<PartialPath> paths = executor.getContext().getPaths();
+    List<PartialPath> paths = executor.getExpression().getPaths();
     int[] readerIndexes = new int[paths.size()];
     for (int i = 0; i < readerIndexes.length; ++i) {
       readerIndexes[i] = udtfPlan.getReaderIndex(paths.get(i).getFullPath());
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
index e4d06d2..dc2700e 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
@@ -22,7 +22,18 @@ package org.apache.iotdb.db.query.expression;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public interface Expression {
+public abstract class Expression {
 
-  TSDataType dataType() throws MetadataException;
+  protected boolean isAggregationFunctionExpression = false;
+  protected boolean isTimeSeriesGeneratingFunctionExpression = false;
+
+  public boolean isAggregationFunctionExpression() {
+    return isAggregationFunctionExpression;
+  }
+
+  public boolean isTimeSeriesGeneratingFunctionExpression() {
+    return isTimeSeriesGeneratingFunctionExpression;
+  }
+
+  public abstract TSDataType dataType() throws MetadataException;
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/ResultColumn.java b/server/src/main/java/org/apache/iotdb/db/query/expression/ResultColumn.java
index e62350e..482d3b5 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/ResultColumn.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/ResultColumn.java
@@ -29,6 +29,11 @@ public class ResultColumn {
     this.alias = alias;
   }
 
+  public ResultColumn(Expression expression) {
+    this.expression = expression;
+    alias = null;
+  }
+
   public Expression getExpression() {
     return expression;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/BinaryExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/BinaryExpression.java
index 1106eac..741e3f5 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/BinaryExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/BinaryExpression.java
@@ -22,9 +22,9 @@ package org.apache.iotdb.db.query.expression.binary;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public abstract class BinaryExpression implements Expression {
+public abstract class BinaryExpression extends Expression {
 
-  public BinaryExpression(Expression leftExpression, Expression rightExpression) {
+  protected BinaryExpression(Expression leftExpression, Expression rightExpression) {
     this.leftExpression = leftExpression;
     this.rightExpression = rightExpression;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/FunctionExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/FunctionExpression.java
index af6186c..ab3bf3e 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/FunctionExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/FunctionExpression.java
@@ -20,6 +20,8 @@
 package org.apache.iotdb.db.query.expression.unary;
 
 import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.tsfile.exception.NotImplementedException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -31,13 +33,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-public class FunctionExpression implements Expression {
+public class FunctionExpression extends Expression {
 
   private final String functionName;
   private final Map<String, String> functionAttributes;
 
   private List<Expression> expressions;
   private List<TSDataType> dataTypes;
+  private List<PartialPath> paths;
 
   private String expressionString;
   private String parametersString;
@@ -46,13 +49,23 @@ public class FunctionExpression implements Expression {
     this.functionName = functionName;
     functionAttributes = new LinkedHashMap<>();
     expressions = new ArrayList<>();
+    setFunctionExpressionType();
   }
 
-  public FunctionExpression(String functionName, Map<String, String> functionAttributes,
-      List<Expression> expressions) {
+  public FunctionExpression(
+      String functionName, Map<String, String> functionAttributes, List<Expression> expressions) {
     this.functionName = functionName;
     this.functionAttributes = functionAttributes;
     this.expressions = expressions;
+    setFunctionExpressionType();
+  }
+
+  private void setFunctionExpressionType() {
+    if (SQLConstant.getNativeFunctionNames().contains(functionName.toLowerCase())) {
+      isAggregationFunctionExpression = true;
+    } else {
+      isTimeSeriesGeneratingFunctionExpression = true;
+    }
   }
 
   public void addAttribute(String key, String value) {
@@ -95,23 +108,35 @@ public class FunctionExpression implements Expression {
     return dataTypes;
   }
 
+  // TODO: remove this method
+  public List<PartialPath> getPaths() {
+    if (paths == null) {
+      paths = new ArrayList<>();
+      for (Expression expression : expressions) {
+        paths.add(((TimeSeriesOperand) expression).getPath());
+      }
+    }
+    return paths;
+  }
+
   @Override
   public String toString() {
     if (expressionString == null) {
-      expressionString = functionName + "(" + parametersString() + ")";
+      expressionString = functionName + "(" + getParametersString() + ")";
     }
     return expressionString;
   }
 
   /**
-   * Generates the parameter part of the udf column name.
+   * Generates the parameter part of the function column name.
    *
    * <p>Example:
-   * Full column name -> udf(root.sg.d.s1, sin(root.sg.d.s1), 'key1'='value1', 'key2'='value2')
-   * <p>
-   * The parameter part -> root.sg.d.s1, sin(root.sg.d.s1), 'key1'='value1', 'key2'='value2'
+   *
+   * <p>Full column name -> udf(root.sg.d.s1, sin(root.sg.d.s1), 'key1'='value1', 'key2'='value2')
+   *
+   * <p>The parameter part -> root.sg.d.s1, sin(root.sg.d.s1), 'key1'='value1', 'key2'='value2'
    */
-  private String parametersString() {
+  public String getParametersString() {
     if (parametersString == null) {
       StringBuilder builder = new StringBuilder();
       if (!expressions.isEmpty()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/MinusExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/MinusExpression.java
index 965e3c0..6a3145b 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/MinusExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/MinusExpression.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class MinusExpression implements Expression {
+public class MinusExpression extends Expression {
 
   protected Expression expression;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/NumberLiteralOperand.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/NumberLiteralOperand.java
index 8405947..7eaf306 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/NumberLiteralOperand.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/NumberLiteralOperand.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.query.expression.unary;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class NumberLiteralOperand implements Expression {
+public class NumberLiteralOperand extends Expression {
 
   protected double literal;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/TimeSeriesOperand.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/TimeSeriesOperand.java
index c72a4b2..bc4e635 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/TimeSeriesOperand.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/TimeSeriesOperand.java
@@ -25,7 +25,7 @@ import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class TimeSeriesOperand implements Expression {
+public class TimeSeriesOperand extends Expression {
 
   protected PartialPath path;
   protected TSDataType dataType;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/context/UDFContext.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/context/UDFContext.java
deleted file mode 100644
index 00e294e..0000000
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/core/context/UDFContext.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.query.udf.core.context;
-
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-public class UDFContext {
-
-  private final String name;
-  private final Map<String, String> attributes;
-
-  private List<PartialPath> paths;
-  private List<TSDataType> dataTypes;
-
-  private String columnParameterPart;
-  private String column;
-
-  public UDFContext(String name) {
-    this.name = name;
-    attributes = new LinkedHashMap<>();
-    paths = new ArrayList<>();
-  }
-
-  public UDFContext(String name, Map<String, String> attributes, List<PartialPath> paths) {
-    this.name = name;
-    this.attributes = attributes;
-    this.paths = paths;
-  }
-
-  public void addAttribute(String key, String value) {
-    attributes.put(key, value);
-  }
-
-  public void addPath(PartialPath path) {
-    paths.add(path);
-  }
-
-  public void setPaths(List<PartialPath> paths) {
-    this.paths = paths;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public Map<String, String> getAttributes() {
-    return attributes;
-  }
-
-  public List<PartialPath> getPaths() {
-    return paths;
-  }
-
-  public List<TSDataType> getDataTypes() throws MetadataException {
-    if (dataTypes == null) {
-      dataTypes = new ArrayList<>();
-      for (PartialPath path : paths) {
-        dataTypes.add(IoTDB.metaManager.getSeriesType(path));
-      }
-    }
-    return dataTypes;
-  }
-
-  /** Generates the column name of the udf query. */
-  public String getColumnName() {
-    if (column == null) {
-      column = name + "(" + getColumnNameParameterPart() + ")";
-    }
-    return column;
-  }
-
-  /**
-   * Generates the parameter part of the udf column name.
-   *
-   * <p>Example: <br>
-   * Full column name -> udf(root.sg.d.s1, root.sg.d.s1, 'key1'='value1', 'key2'='value2') <br>
-   * The parameter part -> root.sg.d.s1, root.sg.d.s1, 'key1'='value1', 'key2'='value2'
-   */
-  private String getColumnNameParameterPart() {
-    if (columnParameterPart == null) {
-      StringBuilder builder = new StringBuilder();
-      if (!paths.isEmpty()) {
-        builder.append(paths.get(0).getFullPath());
-        for (int i = 1; i < paths.size(); ++i) {
-          builder.append(", ").append(paths.get(i).getFullPath());
-        }
-      }
-      if (!attributes.isEmpty()) {
-        if (!paths.isEmpty()) {
-          builder.append(", ");
-        }
-        Iterator<Entry<String, String>> iterator = attributes.entrySet().iterator();
-        Entry<String, String> entry = iterator.next();
-        builder
-            .append("\"")
-            .append(entry.getKey())
-            .append("\"=\"")
-            .append(entry.getValue())
-            .append("\"");
-        while (iterator.hasNext()) {
-          entry = iterator.next();
-          builder
-              .append(", ")
-              .append("\"")
-              .append(entry.getKey())
-              .append("\"=\"")
-              .append(entry.getValue())
-              .append("\"");
-        }
-      }
-      columnParameterPart = builder.toString();
-    }
-    return columnParameterPart;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/executor/UDTFExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/executor/UDTFExecutor.java
index 9d69582..59b0a2d 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/core/executor/UDTFExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/executor/UDTFExecutor.java
@@ -20,13 +20,13 @@
 package org.apache.iotdb.db.query.udf.core.executor;
 
 import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
 import org.apache.iotdb.db.query.udf.api.UDTF;
 import org.apache.iotdb.db.query.udf.api.access.Row;
 import org.apache.iotdb.db.query.udf.api.access.RowWindow;
 import org.apache.iotdb.db.query.udf.api.customizer.config.UDTFConfigurations;
 import org.apache.iotdb.db.query.udf.api.customizer.parameter.UDFParameterValidator;
 import org.apache.iotdb.db.query.udf.api.customizer.parameter.UDFParameters;
-import org.apache.iotdb.db.query.udf.core.context.UDFContext;
 import org.apache.iotdb.db.query.udf.datastructure.tv.ElasticSerializableTVList;
 import org.apache.iotdb.db.query.udf.service.UDFRegistrationService;
 
@@ -34,21 +34,22 @@ import java.time.ZoneId;
 
 public class UDTFExecutor {
 
-  protected final UDFContext context;
+  protected final FunctionExpression expression;
   protected final UDTFConfigurations configurations;
   protected UDTF udtf;
   protected ElasticSerializableTVList collector;
 
-  public UDTFExecutor(UDFContext context, ZoneId zoneId) {
-    this.context = context;
+  public UDTFExecutor(FunctionExpression expression, ZoneId zoneId) {
+    this.expression = expression;
     configurations = new UDTFConfigurations(zoneId);
   }
 
   public void beforeStart(long queryId, float collectorMemoryBudgetInMB)
       throws QueryProcessException {
-    udtf = (UDTF) UDFRegistrationService.getInstance().reflect(context);
+    udtf = (UDTF) UDFRegistrationService.getInstance().reflect(expression);
 
-    UDFParameters parameters = new UDFParameters(context.getPaths(), context.getAttributes());
+    UDFParameters parameters =
+        new UDFParameters(expression.getPaths(), expression.getFunctionAttributes());
 
     try {
       udtf.validate(new UDFParameterValidator(parameters));
@@ -103,8 +104,8 @@ public class UDTFExecutor {
             + e);
   }
 
-  public UDFContext getContext() {
-    return context;
+  public FunctionExpression getExpression() {
+    return expression;
   }
 
   public UDTFConfigurations getConfigurations() {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/service/UDFRegistrationService.java b/server/src/main/java/org/apache/iotdb/db/query/udf/service/UDFRegistrationService.java
index 88ceaba..bc680c5 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/service/UDFRegistrationService.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/service/UDFRegistrationService.java
@@ -25,9 +25,9 @@ import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.UDFRegistrationException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
 import org.apache.iotdb.db.query.udf.api.UDF;
 import org.apache.iotdb.db.query.udf.builtin.BuiltinFunction;
-import org.apache.iotdb.db.query.udf.core.context.UDFContext;
 import org.apache.iotdb.db.service.IService;
 import org.apache.iotdb.db.service.ServiceType;
 import org.apache.iotdb.db.utils.TestOnly;
@@ -249,8 +249,8 @@ public class UDFRegistrationService implements IService {
     }
   }
 
-  public UDF reflect(UDFContext context) throws QueryProcessException {
-    String functionName = context.getName().toUpperCase();
+  public UDF reflect(FunctionExpression expression) throws QueryProcessException {
+    String functionName = expression.getFunctionName().toUpperCase();
     UDFRegistrationInformation information = registrationInformation.get(functionName);
     if (information == null) {
       String errorMessage =
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index f7b619a..0637aa2 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -1042,10 +1042,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           respColumns.add(
               paths.get(i) != null
                   ? paths.get(i).getFullPath()
-                  : udtfPlan
-                      .getExecutorByOriginalOutputColumnIndex(i)
-                      .getContext()
-                      .getColumnName());
+                  : udtfPlan.getExecutorByOriginalOutputColumnIndex(i).getExpression().toString());
           seriesTypes.add(
               paths.get(i) != null
                   ? udtfPlan.getDataTypes().get(i)
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/logical/IndexLogicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/logical/IndexLogicalPlanTest.java
index 1e09934..05e7d0a 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/logical/IndexLogicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/logical/IndexLogicalPlanTest.java
@@ -116,7 +116,7 @@ public class IndexLogicalPlanTest {
     Assert.assertEquals(QueryOperator.class, op.getClass());
     QueryOperator queryOperator = (QueryOperator) op;
     Assert.assertEquals(OperatorType.QUERY, queryOperator.getType());
-    Assert.assertEquals("Glu", queryOperator.getSelectedPaths().get(0).getFullPath());
+    Assert.assertEquals("Glu", queryOperator.getExpressions().get(0).getFullPath());
     Assert.assertEquals(
         "root.Ery.*", queryOperator.getFromOperator().getPrefixPaths().get(0).getFullPath());
     Assert.assertEquals(IndexType.RTREE_PAA, queryOperator.getIndexType());
@@ -138,7 +138,7 @@ public class IndexLogicalPlanTest {
     Assert.assertEquals(QueryOperator.class, op.getClass());
     QueryOperator queryOperator = (QueryOperator) op;
     Assert.assertEquals(OperatorType.QUERY, queryOperator.getType());
-    Assert.assertEquals("Speed", queryOperator.getSelectedPaths().get(0).getFullPath());
+    Assert.assertEquals("Speed", queryOperator.getExpressions().get(0).getFullPath());
     Assert.assertEquals(
         "root.Wind.AZQ02", queryOperator.getFromOperator().getPrefixPaths().get(0).getFullPath());
     Assert.assertEquals(IndexType.ELB_INDEX, queryOperator.getIndexType());
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/logical/LogicalPlanSmallTest.java b/server/src/test/java/org/apache/iotdb/db/qp/logical/LogicalPlanSmallTest.java
index e384edd..218bf7d 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/logical/LogicalPlanSmallTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/logical/LogicalPlanSmallTest.java
@@ -247,7 +247,7 @@ public class LogicalPlanSmallTest {
     Assert.assertEquals(QueryOperator.class, operator.getClass());
     ArrayList<PartialPath> paths = new ArrayList<>();
     paths.add(new PartialPath("*"));
-    Assert.assertEquals(paths, ((QueryOperator) operator).getSelectedPaths());
+    Assert.assertEquals(paths, ((QueryOperator) operator).getExpressions());
   }
 
   @Test
diff --git a/server/src/test/java/org/apache/iotdb/db/query/dataset/UDTFAlignByTimeDataSetTest.java b/server/src/test/java/org/apache/iotdb/db/query/dataset/UDTFAlignByTimeDataSetTest.java
index f879746..f8b8c50 100644
--- a/server/src/test/java/org/apache/iotdb/db/query/dataset/UDTFAlignByTimeDataSetTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/dataset/UDTFAlignByTimeDataSetTest.java
@@ -171,7 +171,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -233,7 +236,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -280,7 +286,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -339,7 +348,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -412,7 +424,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -468,7 +483,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -533,7 +551,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -604,7 +625,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }
@@ -657,7 +681,10 @@ public class UDTFAlignByTimeDataSetTest {
         Path path = queryPlan.getPaths().get(i);
         String columnName =
             path == null
-                ? queryPlan.getExecutorByOriginalOutputColumnIndex(i).getContext().getColumnName()
+                ? queryPlan
+                    .getExecutorByOriginalOutputColumnIndex(i)
+                    .getExpression()
+                    .getColumnName()
                 : path.getFullPath();
         originalIndex2FieldIndex.add(path2Index.get(columnName));
       }

[iotdb] 02/05: expressions for complex operations in select clauses

Posted by ro...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rong pushed a commit to branch iotdb-1022-v2
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 63e5ece09d98d09a678b9bca33ef58e35e9e1800
Author: SteveYurongSu <st...@outlook.com>
AuthorDate: Tue May 11 11:42:44 2021 +0800

    expressions for complex operations in select clauses
---
 .../iotdb/db/query/expression/Expression.java      |  28 ++++
 .../iotdb/db/query/expression/ResultColumn.java    |  39 ++++++
 .../expression/binary/AdditionExpression.java      |  34 +++++
 .../query/expression/binary/BinaryExpression.java  |  52 +++++++
 .../expression/binary/DivisionExpression.java      |  34 +++++
 .../query/expression/binary/ModuloExpression.java  |  34 +++++
 .../binary/MultiplicationExpression.java           |  34 +++++
 .../expression/binary/SubtractionExpression.java   |  34 +++++
 .../query/expression/unary/FunctionExpression.java | 150 +++++++++++++++++++++
 .../db/query/expression/unary/MinusExpression.java |  47 +++++++
 .../expression/unary/NumberLiteralOperand.java     |  42 ++++++
 .../query/expression/unary/TimeSeriesOperand.java  |  57 ++++++++
 12 files changed, 585 insertions(+)

diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
new file mode 100644
index 0000000..e4d06d2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/Expression.java
@@ -0,0 +1,28 @@
+/*
+ * 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.tsfile.file.metadata.enums.TSDataType;
+
+public interface Expression {
+
+  TSDataType dataType() throws MetadataException;
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/ResultColumn.java b/server/src/main/java/org/apache/iotdb/db/query/expression/ResultColumn.java
new file mode 100644
index 0000000..e62350e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/ResultColumn.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+public class ResultColumn {
+
+  private final Expression expression;
+  private final String alias;
+
+  public ResultColumn(Expression expression, String alias) {
+    this.expression = expression;
+    this.alias = alias;
+  }
+
+  public Expression getExpression() {
+    return expression;
+  }
+
+  public String getResultColumnName() {
+    return alias != null ? alias : expression.toString();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/AdditionExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/AdditionExpression.java
new file mode 100644
index 0000000..5500ba4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/AdditionExpression.java
@@ -0,0 +1,34 @@
+/*
+ * 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.binary;
+
+import org.apache.iotdb.db.query.expression.Expression;
+
+public class AdditionExpression extends BinaryExpression {
+
+  public AdditionExpression(Expression leftExpression, Expression rightExpression) {
+    super(leftExpression, rightExpression);
+  }
+
+  @Override
+  protected String operator() {
+    return "+";
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/BinaryExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/BinaryExpression.java
new file mode 100644
index 0000000..1106eac
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/BinaryExpression.java
@@ -0,0 +1,52 @@
+/*
+ * 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.binary;
+
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class BinaryExpression implements Expression {
+
+  public BinaryExpression(Expression leftExpression, Expression rightExpression) {
+    this.leftExpression = leftExpression;
+    this.rightExpression = rightExpression;
+  }
+
+  protected final Expression leftExpression;
+  protected final Expression rightExpression;
+
+  /**
+   * The result data type of all arithmetic operations will be DOUBLE.
+   *
+   * <p>TODO: This is just a simple implementation and should be optimized later.
+   */
+  @Override
+  public TSDataType dataType() {
+    return TSDataType.DOUBLE;
+  }
+
+  @Override
+  public final String toString() {
+    return String.format(
+        "%s %s %s", leftExpression.toString(), operator(), rightExpression.toString());
+  }
+
+  protected abstract String operator();
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/DivisionExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/DivisionExpression.java
new file mode 100644
index 0000000..e611887
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/DivisionExpression.java
@@ -0,0 +1,34 @@
+/*
+ * 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.binary;
+
+import org.apache.iotdb.db.query.expression.Expression;
+
+public class DivisionExpression extends BinaryExpression {
+
+  public DivisionExpression(Expression leftExpression, Expression rightExpression) {
+    super(leftExpression, rightExpression);
+  }
+
+  @Override
+  protected String operator() {
+    return "/";
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/ModuloExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/ModuloExpression.java
new file mode 100644
index 0000000..94dd905
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/ModuloExpression.java
@@ -0,0 +1,34 @@
+/*
+ * 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.binary;
+
+import org.apache.iotdb.db.query.expression.Expression;
+
+public class ModuloExpression extends BinaryExpression {
+
+  public ModuloExpression(Expression leftExpression, Expression rightExpression) {
+    super(leftExpression, rightExpression);
+  }
+
+  @Override
+  protected String operator() {
+    return "%";
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/MultiplicationExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/MultiplicationExpression.java
new file mode 100644
index 0000000..ae3d17b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/MultiplicationExpression.java
@@ -0,0 +1,34 @@
+/*
+ * 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.binary;
+
+import org.apache.iotdb.db.query.expression.Expression;
+
+public class MultiplicationExpression extends BinaryExpression {
+
+  public MultiplicationExpression(Expression leftExpression, Expression rightExpression) {
+    super(leftExpression, rightExpression);
+  }
+
+  @Override
+  protected String operator() {
+    return "*";
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/binary/SubtractionExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/SubtractionExpression.java
new file mode 100644
index 0000000..65bcf50
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/binary/SubtractionExpression.java
@@ -0,0 +1,34 @@
+/*
+ * 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.binary;
+
+import org.apache.iotdb.db.query.expression.Expression;
+
+public class SubtractionExpression extends BinaryExpression {
+
+  public SubtractionExpression(Expression leftExpression, Expression rightExpression) {
+    super(leftExpression, rightExpression);
+  }
+
+  @Override
+  protected String operator() {
+    return "-";
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/FunctionExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/FunctionExpression.java
new file mode 100644
index 0000000..af6186c
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/FunctionExpression.java
@@ -0,0 +1,150 @@
+/*
+ * 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.unary;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.exception.NotImplementedException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class FunctionExpression implements Expression {
+
+  private final String functionName;
+  private final Map<String, String> functionAttributes;
+
+  private List<Expression> expressions;
+  private List<TSDataType> dataTypes;
+
+  private String expressionString;
+  private String parametersString;
+
+  public FunctionExpression(String functionName) {
+    this.functionName = functionName;
+    functionAttributes = new LinkedHashMap<>();
+    expressions = new ArrayList<>();
+  }
+
+  public FunctionExpression(String functionName, Map<String, String> functionAttributes,
+      List<Expression> expressions) {
+    this.functionName = functionName;
+    this.functionAttributes = functionAttributes;
+    this.expressions = expressions;
+  }
+
+  public void addAttribute(String key, String value) {
+    functionAttributes.put(key, value);
+  }
+
+  public void addExpression(Expression expression) {
+    expressions.add(expression);
+  }
+
+  public void setExpressions(List<Expression> expressions) {
+    this.expressions = expressions;
+  }
+
+  public String getFunctionName() {
+    return functionName;
+  }
+
+  public Map<String, String> getFunctionAttributes() {
+    return functionAttributes;
+  }
+
+  public List<Expression> getExpressions() {
+    return expressions;
+  }
+
+  @Override
+  public TSDataType dataType() {
+    // TODO: the expression type is determined in runtime
+    throw new NotImplementedException();
+  }
+
+  public List<TSDataType> getDataTypes() throws MetadataException {
+    if (dataTypes == null) {
+      dataTypes = new ArrayList<>();
+      for (Expression expression : expressions) {
+        dataTypes.add(expression.dataType());
+      }
+    }
+    return dataTypes;
+  }
+
+  @Override
+  public String toString() {
+    if (expressionString == null) {
+      expressionString = functionName + "(" + parametersString() + ")";
+    }
+    return expressionString;
+  }
+
+  /**
+   * Generates the parameter part of the udf column name.
+   *
+   * <p>Example:
+   * Full column name -> udf(root.sg.d.s1, sin(root.sg.d.s1), 'key1'='value1', 'key2'='value2')
+   * <p>
+   * The parameter part -> root.sg.d.s1, sin(root.sg.d.s1), 'key1'='value1', 'key2'='value2'
+   */
+  private String parametersString() {
+    if (parametersString == null) {
+      StringBuilder builder = new StringBuilder();
+      if (!expressions.isEmpty()) {
+        builder.append(expressions.get(0).toString());
+        for (int i = 1; i < expressions.size(); ++i) {
+          builder.append(", ").append(expressions.get(i).toString());
+        }
+      }
+      if (!functionAttributes.isEmpty()) {
+        if (!expressions.isEmpty()) {
+          builder.append(", ");
+        }
+        Iterator<Entry<String, String>> iterator = functionAttributes.entrySet().iterator();
+        Entry<String, String> entry = iterator.next();
+        builder
+            .append("\"")
+            .append(entry.getKey())
+            .append("\"=\"")
+            .append(entry.getValue())
+            .append("\"");
+        while (iterator.hasNext()) {
+          entry = iterator.next();
+          builder
+              .append(", ")
+              .append("\"")
+              .append(entry.getKey())
+              .append("\"=\"")
+              .append(entry.getValue())
+              .append("\"");
+        }
+      }
+      parametersString = builder.toString();
+    }
+    return parametersString;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/MinusExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/MinusExpression.java
new file mode 100644
index 0000000..965e3c0
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/MinusExpression.java
@@ -0,0 +1,47 @@
+/*
+ * 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.unary;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class MinusExpression implements Expression {
+
+  protected Expression expression;
+
+  public MinusExpression(Expression expression) {
+    this.expression = expression;
+  }
+
+  public Expression getExpression() {
+    return expression;
+  }
+
+  @Override
+  public TSDataType dataType() throws MetadataException {
+    return expression.dataType();
+  }
+
+  @Override
+  public String toString() {
+    return "-" + expression.toString();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/NumberLiteralOperand.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/NumberLiteralOperand.java
new file mode 100644
index 0000000..8405947
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/NumberLiteralOperand.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.expression.unary;
+
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class NumberLiteralOperand implements Expression {
+
+  protected double literal;
+
+  public NumberLiteralOperand(double literal) {
+    this.literal = literal;
+  }
+
+  @Override
+  public TSDataType dataType() {
+    return TSDataType.DOUBLE;
+  }
+
+  @Override
+  public String toString() {
+    return String.valueOf(literal);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/TimeSeriesOperand.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/TimeSeriesOperand.java
new file mode 100644
index 0000000..c72a4b2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/TimeSeriesOperand.java
@@ -0,0 +1,57 @@
+/*
+ * 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.unary;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class TimeSeriesOperand implements Expression {
+
+  protected PartialPath path;
+  protected TSDataType dataType;
+
+  public TimeSeriesOperand(PartialPath path) {
+    this.path = path;
+  }
+
+  public PartialPath getPath() {
+    return path;
+  }
+
+  public void setPath(PartialPath path) {
+    this.path = path;
+  }
+
+  @Override
+  public TSDataType dataType() throws MetadataException {
+    if (dataType == null) {
+      dataType = IoTDB.metaManager.getSeriesType(path);
+    }
+    return dataType;
+  }
+
+  @Override
+  public String toString() {
+    return path.toString();
+  }
+}