You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2022/05/08 02:46:21 UTC

[iotdb] branch master updated: [IOTDB-2962] [IOTDB-3079] Refactor SQL parser and implememtation of complete Analyzer (#5819)

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

jackietien pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 104da23532 [IOTDB-2962] [IOTDB-3079] Refactor SQL parser and implememtation of complete Analyzer (#5819)
104da23532 is described below

commit 104da23532b2bbcbc9e2d0db5dddfe96706d4090
Author: liuminghui233 <36...@users.noreply.github.com>
AuthorDate: Sun May 8 10:46:17 2022 +0800

    [IOTDB-2962] [IOTDB-3079] Refactor SQL parser and implememtation of complete Analyzer (#5819)
---
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |  67 +-
 .../IoTDBSyntaxConventionStringLiteralIT.java      |   8 +-
 .../IoTDBDeletionVersionAdaptionIT.java            |   2 +
 .../IoTDBQueryVersionAdaptionIT.java               |   2 +
 .../iotdb/db/metadata/path/MeasurementPath.java    |   4 +
 .../iotdb/db/mpp/common/header/ColumnHeader.java   |  81 +-
 .../iotdb/db/mpp/common/header/DatasetHeader.java  |   2 +-
 .../db/mpp/common/schematree/DeviceSchemaInfo.java |  77 +-
 .../iotdb/db/mpp/common/schematree/SchemaTree.java |  30 +-
 .../apache/iotdb/db/mpp/plan/analyze/Analysis.java | 138 +++-
 .../apache/iotdb/db/mpp/plan/analyze/Analyzer.java | 647 ++++++++++++++--
 .../plan/analyze/ColumnPaginationController.java   |  73 ++
 .../db/mpp/plan/analyze/ConcatPathRewriter.java    | 135 ++++
 .../db/mpp/plan/analyze/ExpressionAnalyzer.java    | 854 +++++++++++++++++++++
 .../iotdb/db/mpp/plan/analyze/ExpressionUtils.java | 242 ++++++
 .../mpp/plan/analyze/GroupByLevelController.java   | 144 ++++
 .../iotdb/db/mpp/plan/analyze/TypeProvider.java    |  23 +-
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       | 686 +++--------------
 .../db/mpp/plan/parser/StatementGenerator.java     |  21 +-
 .../db/mpp/plan/planner/LogicalPlanBuilder.java    |  14 +-
 .../iotdb/db/mpp/plan/planner/LogicalPlanner.java  |  47 +-
 .../planner/SimpleFragmentParallelPlanner.java     |   6 +-
 .../plan/planner/WriteFragmentParallelPlanner.java |   6 +-
 .../plan/planner/plan/node/process/FillNode.java   |  33 +-
 .../planner/plan/parameter/FillDescriptor.java     |  30 +-
 .../plan/parameter/FilterNullParameter.java        |  14 +-
 .../plan/parameter/GroupByTimeParameter.java       |  11 +
 .../plan/rewriter/ColumnPaginationController.java  | 145 ----
 .../db/mpp/plan/rewriter/ConcatPathRewriter.java   | 190 -----
 .../db/mpp/plan/rewriter/WildcardsRemover.java     | 375 ---------
 .../db/mpp/plan/statement/StatementVisitor.java    |  35 -
 .../plan/statement/component/FillComponent.java    |  25 +-
 .../statement/component/GroupByLevelComponent.java |  20 -
 .../component/GroupByLevelController.java          | 247 ------
 .../mpp/plan/statement/component/ResultColumn.java | 113 +--
 .../plan/statement/component/SelectComponent.java  | 105 +--
 .../plan/statement/component/WhereCondition.java   |  16 +-
 .../statement/crud/AggregationQueryStatement.java  | 129 ----
 .../plan/statement/crud/FillQueryStatement.java    |  79 --
 .../statement/crud/GroupByFillQueryStatement.java  |  49 --
 .../plan/statement/crud/GroupByQueryStatement.java |  61 --
 .../plan/statement/crud/LastQueryStatement.java    |  74 --
 .../db/mpp/plan/statement/crud/QueryStatement.java | 213 ++---
 .../plan/statement/crud/UDAFQueryStatement.java    |  97 ---
 .../literal/BooleanLiteral.java}                   |  44 +-
 .../literal/DoubleLiteral.java}                    |  41 +-
 .../db/mpp/plan/statement/literal/Literal.java     |  59 ++
 .../literal/LongLiteral.java}                      |  41 +-
 .../NullLiteral.java}                              |  34 +-
 .../literal/StringLiteral.java}                    |  39 +-
 .../iotdb/db/qp/constant/FilterConstant.java       |  11 +
 .../iotdb/db/qp/logical/crud/InOperator.java       |   4 +
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    | 451 +++++------
 .../iotdb/db/query/expression/Expression.java      |  16 +-
 .../iotdb/db/query/expression/ResultColumn.java    |  12 +-
 .../query/expression/binary/BinaryExpression.java  |  42 +-
 .../db/query/expression/leaf/ConstantOperand.java  |  20 +-
 .../query/expression/leaf/TimeSeriesOperand.java   |  23 -
 .../db/query/expression/leaf/TimestampOperand.java |  17 -
 .../query/expression/multi/FunctionExpression.java |  35 -
 .../db/query/expression/unary/InExpression.java    |   8 +
 .../db/query/expression/unary/LikeExpression.java  |   8 +
 .../query/expression/unary/RegularExpression.java  |   8 +
 .../db/query/expression/unary/UnaryExpression.java |  26 -
 .../db/mpp/common/schematree/SchemaTreeTest.java   |   3 +-
 .../{AnalyzerTest.java => AnalyzeFailTest.java}    |   7 +-
 .../{AnalyzerTest.java => AnalyzeTest.java}        |  37 +-
 .../plan/plan/node/process/FillNodeSerdeTest.java  |   8 +-
 .../java/org/apache/iotdb/tsfile/utils/Pair.java   |   8 +
 69 files changed, 3043 insertions(+), 3329 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
index 92125573ae..9103af1929 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
@@ -355,7 +355,7 @@ specialClause
     | groupByTimeClause orderByTimeClause? specialLimit? #groupByTimeStatement
     | groupByFillClause orderByTimeClause? specialLimit? #groupByFillStatement
     | groupByLevelClause orderByTimeClause? specialLimit? #groupByLevelStatement
-    | fillClause slimitClause? alignByDeviceClauseOrDisableAlign? #fillStatement
+    | fillClause orderByTimeClause? specialLimit? #fillStatement
     ;
 
 specialLimit
@@ -384,9 +384,9 @@ orderByTimeClause
     ;
 
 groupByTimeClause
-    : GROUP BY LR_BRACKET timeRange COMMA DURATION_LITERAL (COMMA DURATION_LITERAL)? RR_BRACKET
+    : GROUP BY LR_BRACKET timeRange COMMA DURATION_LITERAL (COMMA DURATION_LITERAL)? fillClause? RR_BRACKET
     | GROUP BY LR_BRACKET timeRange COMMA DURATION_LITERAL (COMMA DURATION_LITERAL)? RR_BRACKET
-    COMMA LEVEL operator_eq INTEGER_LITERAL (COMMA INTEGER_LITERAL)*
+    COMMA LEVEL operator_eq INTEGER_LITERAL (COMMA INTEGER_LITERAL)* fillClause?
     ;
 
 groupByFillClause
@@ -395,7 +395,7 @@ groupByFillClause
     ;
 
 groupByLevelClause
-    : GROUP BY LEVEL operator_eq INTEGER_LITERAL (COMMA INTEGER_LITERAL)*
+    : GROUP BY LEVEL operator_eq INTEGER_LITERAL (COMMA INTEGER_LITERAL)* fillClause?
     ;
 
 fillClause
@@ -463,7 +463,7 @@ deleteStatement
     ;
 
 whereClause
-    : WHERE (orExpression | indexPredicateClause)
+    : WHERE expression
     ;
 
 /**
@@ -835,13 +835,13 @@ expression
     | constant
     | time=(TIME | TIMESTAMP)
     | fullPathInExpression
-    | functionName LR_BRACKET expression (COMMA expression)* functionAttribute* RR_BRACKET
+    | functionName LR_BRACKET expression (COMMA expression)* RR_BRACKET
     | (PLUS | MINUS | OPERATOR_NOT) expressionAfterUnaryOperator=expression
     | leftExpression=expression (STAR | DIV | MOD) rightExpression=expression
     | leftExpression=expression (PLUS | MINUS) rightExpression=expression
-    | leftExpression=expression (OPERATOR_GT | OPERATOR_GTE | OPERATOR_LT | OPERATOR_LTE | OPERATOR_DEQ | OPERATOR_NEQ) rightExpression=expression
+    | leftExpression=expression (OPERATOR_GT | OPERATOR_GTE | OPERATOR_LT | OPERATOR_LTE | OPERATOR_SEQ | OPERATOR_DEQ | OPERATOR_NEQ) rightExpression=expression
     | unaryBeforeRegularOrLikeExpression=expression (REGEXP | LIKE) STRING_LITERAL
-    | unaryBeforeInExpression=expression OPERATOR_NOT? OPERATOR_IN LR_BRACKET constant (COMMA constant)* RR_BRACKET
+    | unaryBeforeInExpression=expression OPERATOR_NOT? (OPERATOR_IN | OPERATOR_CONTAINS) LR_BRACKET constant (COMMA constant)* RR_BRACKET
     | leftExpression=expression OPERATOR_AND rightExpression=expression
     | leftExpression=expression OPERATOR_OR rightExpression=expression
     ;
@@ -851,67 +851,20 @@ functionName
     | COUNT
     ;
 
-functionAttribute
-    : COMMA functionAttributeKey=attributeKey OPERATOR_SEQ functionAttributeValue=attributeValue
-    ;
-
 containsExpression
     : name=attributeKey OPERATOR_CONTAINS value=attributeValue
     ;
 
-orExpression
-    : andExpression (OPERATOR_OR andExpression)*
-    ;
-
-andExpression
-    : predicate (OPERATOR_AND predicate)*
-    ;
-
-predicate
-    : (TIME | TIMESTAMP | suffixPath | fullPath) comparisonOperator constant
-    | (TIME | TIMESTAMP | suffixPath | fullPath) inClause
-    | OPERATOR_NOT? LR_BRACKET orExpression RR_BRACKET
-    | (suffixPath | fullPath) (REGEXP | LIKE) STRING_LITERAL
-    ;
-
 operator_eq
     : OPERATOR_SEQ
     | OPERATOR_DEQ
     ;
 
-comparisonOperator
-    : type = OPERATOR_GT
-    | type = OPERATOR_GTE
-    | type = OPERATOR_LT
-    | type = OPERATOR_LTE
-    | type = OPERATOR_DEQ
-    | type = OPERATOR_SEQ
-    | type = OPERATOR_NEQ
-    ;
-
-inClause
-    : OPERATOR_NOT? OPERATOR_IN LR_BRACKET constant (COMMA constant)* RR_BRACKET
-    ;
-
-indexPredicateClause
-    : (suffixPath | fullPath) LIKE sequenceClause
-    | (suffixPath | fullPath) CONTAIN sequenceClause
-    WITH TOLERANCE constant (CONCAT sequenceClause WITH TOLERANCE constant)*
-    ;
-
-sequenceClause
-    : LR_BRACKET constant (COMMA constant)* RR_BRACKET
-    ;
-
 
 // Select Clause
 
 selectClause
-    : SELECT (LAST | topClause)? resultColumn (COMMA resultColumn)*
-    ;
-
-topClause
-    : TOP INTEGER_LITERAL
+    : SELECT LAST? resultColumn (COMMA resultColumn)*
     ;
 
 resultColumn
@@ -988,4 +941,4 @@ slimitClause
 
 soffsetClause
     : SOFFSET INTEGER_LITERAL
-    ;
+    ;
\ No newline at end of file
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionStringLiteralIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionStringLiteralIT.java
index 0e39251fbd..de3eba8a91 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionStringLiteralIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionStringLiteralIT.java
@@ -532,9 +532,9 @@ public class IoTDBSyntaxConventionStringLiteralIT {
       assertTrue(resultSet.next());
       Assert.assertEquals("2.0", resultSet.getString(2));
 
-      resultSet = statement.executeQuery("select bottom_k(s1,k = 1) from root.vehicle.d1");
-      assertTrue(resultSet.next());
-      Assert.assertEquals("2.0", resultSet.getString(2));
+      //      resultSet = statement.executeQuery("select bottom_k(s1,k = 1) from root.vehicle.d1");
+      //      assertTrue(resultSet.next());
+      //      Assert.assertEquals("2.0", resultSet.getString(2));
     } catch (SQLException e) {
       e.printStackTrace();
       fail();
@@ -545,7 +545,7 @@ public class IoTDBSyntaxConventionStringLiteralIT {
   public void testUDFAttribute1() {
     String errorMsg =
         "401: Error occurred while parsing SQL to physical plan: "
-            + "line 1:22 token recognition error at: '` = 1) from root.vehicle.d1'";
+            + "line 1:21 extraneous input 'k' expecting {',', ')'}";
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
       // UDF attribute should be STRING_LITERAL
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBDeletionVersionAdaptionIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBDeletionVersionAdaptionIT.java
index c2d0913e3e..54f3ee7dec 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBDeletionVersionAdaptionIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBDeletionVersionAdaptionIT.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.jdbc.Constant;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -42,6 +43,7 @@ import java.util.Locale;
 import static org.junit.Assert.assertEquals;
 
 @Category({LocalStandaloneTest.class, ClusterTest.class, RemoteTest.class})
+@Ignore // No longer forward compatible since v0.14
 public class IoTDBDeletionVersionAdaptionIT {
 
   private static String[] creationSqls =
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBQueryVersionAdaptionIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBQueryVersionAdaptionIT.java
index d6f116df91..d7cfb958cc 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBQueryVersionAdaptionIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/versionadaption/IoTDBQueryVersionAdaptionIT.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.jdbc.Constant;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -52,6 +53,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 @Category({LocalStandaloneTest.class, ClusterTest.class, RemoteTest.class})
+@Ignore // No longer forward compatible since v0.14
 public class IoTDBQueryVersionAdaptionIT {
 
   private static String[] sqls =
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java
index a6fe33a8b3..5b105fd618 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java
@@ -101,6 +101,10 @@ public class MeasurementPath extends PartialPath {
     }
   }
 
+  public void removeMeasurementAlias() {
+    this.measurementAlias = null;
+  }
+
   @Override
   public boolean isMeasurementAliasExists() {
     return measurementAlias != null && !measurementAlias.isEmpty();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeader.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeader.java
index 50bb38c265..9cadcd7bcb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeader.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeader.java
@@ -19,8 +19,6 @@
 
 package org.apache.iotdb.db.mpp.common.header;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
@@ -29,12 +27,9 @@ import java.util.Objects;
 
 public class ColumnHeader {
 
-  private String pathName;
-  private String functionName;
+  private final String columnName;
   private final TSDataType dataType;
-
-  private String columnName;
-  private String alias;
+  private final String alias;
 
   public ColumnHeader(String columnName, TSDataType dataType, String alias) {
     this.columnName = columnName;
@@ -42,65 +37,54 @@ public class ColumnHeader {
     this.alias = alias;
   }
 
-  public ColumnHeader(String pathName, TSDataType dataType) {
-    this.pathName = pathName;
-    this.dataType = dataType;
-  }
-
-  public ColumnHeader(String pathName, String functionName, TSDataType dataType) {
-    this.pathName = pathName;
-    this.functionName = functionName.toLowerCase();
+  public ColumnHeader(String columnName, TSDataType dataType) {
+    this.columnName = columnName;
     this.dataType = dataType;
+    this.alias = null;
   }
 
   public String getColumnName() {
     if (alias != null) {
       return alias;
     }
-    if (columnName != null) {
-      return columnName;
-    }
-    if (functionName != null) {
-      return String.format("%s(%s)", functionName, pathName);
-    }
-    return pathName;
+    return columnName;
   }
 
   public TSDataType getColumnType() {
     return dataType;
   }
 
-  public ColumnHeader replacePathWithMeasurement() {
-    String measurement = null;
-    try {
-      measurement = new PartialPath(pathName).getMeasurement();
-    } catch (IllegalPathException e) {
-      e.printStackTrace();
-    }
-    if (functionName != null) {
-      return new ColumnHeader(measurement, functionName, dataType);
-    }
-    return new ColumnHeader(measurement, dataType);
-  }
-
   public boolean hasAlias() {
-    return alias != null;
+    return alias != null && !alias.isEmpty();
   }
 
   public void serialize(ByteBuffer byteBuffer) {
-    ReadWriteIOUtils.write(pathName, byteBuffer);
-    ReadWriteIOUtils.write(functionName, byteBuffer);
+    ReadWriteIOUtils.write(columnName, byteBuffer);
+    ReadWriteIOUtils.write(dataType.ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(hasAlias(), byteBuffer);
+    if (hasAlias()) {
+      ReadWriteIOUtils.write(alias, byteBuffer);
+    }
     dataType.serializeTo(byteBuffer);
   }
 
   public static ColumnHeader deserialize(ByteBuffer byteBuffer) {
-    String pathName = ReadWriteIOUtils.readString(byteBuffer);
-    String functionName = ReadWriteIOUtils.readString(byteBuffer);
-    TSDataType tsDataType = TSDataType.deserializeFrom(byteBuffer);
-    if (functionName == null) {
-      return new ColumnHeader(pathName, tsDataType);
+    String columnName = ReadWriteIOUtils.readString(byteBuffer);
+    TSDataType dataType = TSDataType.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    String alias = null;
+    boolean hasAlias = ReadWriteIOUtils.readBool(byteBuffer);
+    if (hasAlias) {
+      alias = ReadWriteIOUtils.readString(byteBuffer);
+    }
+    return new ColumnHeader(columnName, dataType, alias);
+  }
+
+  @Override
+  public String toString() {
+    if (hasAlias()) {
+      return String.format("%s(%s) [%s]", columnName, alias, dataType);
     }
-    return new ColumnHeader(pathName, functionName, tsDataType);
+    return String.format("%s [%s]", columnName, dataType);
   }
 
   @Override
@@ -111,15 +95,14 @@ public class ColumnHeader {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-
     ColumnHeader that = (ColumnHeader) o;
-    return Objects.equals(pathName, that.pathName)
-        && Objects.equals(functionName, that.functionName)
-        && dataType == that.dataType;
+    return Objects.equals(columnName, that.columnName)
+        && dataType == that.dataType
+        && Objects.equals(alias, that.alias);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(pathName, functionName, dataType);
+    return Objects.hash(columnName, dataType, alias);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeader.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeader.java
index 9d3e519221..f5dd5c669a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeader.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeader.java
@@ -40,7 +40,7 @@ public class DatasetHeader {
   // indicate whether the result dataset contain timestamp column
   private final boolean isIgnoreTimestamp;
 
-  // map from
+  // map from output column to output tsBlock index
   private Map<String, Integer> columnToTsBlockIndexMap;
 
   public DatasetHeader(List<ColumnHeader> columnHeaders, boolean isIgnoreTimestamp) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/DeviceSchemaInfo.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/DeviceSchemaInfo.java
index aaa1dc6e59..15d76a0cb2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/DeviceSchemaInfo.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/DeviceSchemaInfo.java
@@ -19,12 +19,16 @@
 
 package org.apache.iotdb.db.mpp.common.schematree;
 
+import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaMeasurementNode;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 public class DeviceSchemaInfo {
@@ -54,23 +58,60 @@ public class DeviceSchemaInfo {
         .collect(Collectors.toList());
   }
 
-  public List<MeasurementPath> getMeasurements() {
-    return measurementNodeList.stream()
-        .map(
-            measurementNode -> {
-              if (measurementNode == null) {
-                return null;
-              }
-              MeasurementPath measurementPath =
-                  new MeasurementPath(
-                      devicePath.concatNode(measurementNode.getName()),
-                      measurementNode.getSchema());
-              measurementPath.setUnderAlignedEntity(isAligned);
-              if (measurementNode.getAlias() != null) {
-                measurementPath.setMeasurementAlias(measurementNode.getAlias());
-              }
-              return measurementPath;
-            })
-        .collect(Collectors.toList());
+  public List<MeasurementPath> getMeasurements(Set<String> measurements) {
+    if (measurements.contains(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)) {
+      return measurementNodeList.stream()
+          .map(
+              measurementNode -> {
+                if (measurementNode == null) {
+                  return null;
+                }
+                MeasurementPath measurementPath =
+                    new MeasurementPath(
+                        devicePath.concatNode(measurementNode.getName()),
+                        measurementNode.getSchema());
+                if (measurementNode.getAlias() != null) {
+                  measurementPath.setMeasurementAlias(measurementNode.getAlias());
+                }
+                measurementPath.setUnderAlignedEntity(isAligned);
+                return measurementPath;
+              })
+          .collect(Collectors.toList());
+    }
+    List<MeasurementPath> measurementPaths = new ArrayList<>();
+    for (SchemaMeasurementNode measurementNode : measurementNodeList) {
+      MeasurementPath measurementPath =
+          new MeasurementPath(
+              devicePath.concatNode(measurementNode.getName()), measurementNode.getSchema());
+      measurementPath.setUnderAlignedEntity(isAligned);
+      if (measurements.contains(measurementNode.getName())) {
+        measurementPaths.add(measurementPath);
+      } else if (measurementNode.getAlias() != null
+          && measurements.contains(measurementNode.getAlias())) {
+        measurementPath.setMeasurementAlias(measurementNode.getAlias());
+        measurementPaths.add(measurementPath);
+      }
+    }
+    return measurementPaths;
+  }
+
+  public MeasurementPath getPathByMeasurement(String measurementName) {
+    for (SchemaMeasurementNode measurementNode : measurementNodeList) {
+      MeasurementPath measurementPath =
+          new MeasurementPath(
+              devicePath.concatNode(measurementNode.getName()), measurementNode.getSchema());
+      measurementPath.setUnderAlignedEntity(isAligned);
+      if (measurementNode.getName().equals(measurementName)) {
+        return measurementPath;
+      } else if (measurementNode.getAlias() != null
+          && measurementNode.getAlias().equals(measurementName)) {
+        measurementPath.setMeasurementAlias(measurementNode.getAlias());
+        return measurementPath;
+      }
+    }
+    throw new SemanticException(
+        String.format(
+            "ALIGN BY DEVICE: measurement '%s' does not exist in device '%s'",
+            measurementName, getDevicePath()));
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTree.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTree.java
index e4100e75e9..a1ee32b437 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTree.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.mpp.common.schematree;
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaEntityNode;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaInternalNode;
@@ -71,6 +72,17 @@ public class SchemaTree {
     return new Pair<>(visitor.getAllResult(), visitor.getNextOffset());
   }
 
+  public Pair<List<MeasurementPath>, Integer> searchMeasurementPaths(PartialPath pathPattern) {
+    SchemaTreeMeasurementVisitor visitor =
+        new SchemaTreeMeasurementVisitor(
+            root,
+            pathPattern,
+            IoTDBDescriptor.getInstance().getConfig().getMaxQueryDeduplicatedPathNum() + 1,
+            0,
+            false);
+    return new Pair<>(visitor.getAllResult(), visitor.getNextOffset());
+  }
+
   public List<MeasurementPath> getAllMeasurement() {
     return searchMeasurementPaths(ALL_MATCH_PATTERN, 0, 0, false).left;
   }
@@ -86,6 +98,11 @@ public class SchemaTree {
     return visitor.getAllResult();
   }
 
+  public List<DeviceSchemaInfo> getMatchedDevices(PartialPath pathPattern) {
+    SchemaTreeDeviceVisitor visitor = new SchemaTreeDeviceVisitor(root, pathPattern, false);
+    return visitor.getAllResult();
+  }
+
   public DeviceSchemaInfo searchDeviceSchemaInfo(
       PartialPath devicePath, List<String> measurements) {
 
@@ -241,17 +258,20 @@ public class SchemaTree {
    *
    * <p>e.g., root.sg1 is a storage group and path = root.sg1.d1, return root.sg1
    *
-   * @param path only full path, cannot be path pattern
+   * @param pathName only full path, cannot be path pattern
    * @return storage group in the given path
    */
-  public String getBelongedStorageGroup(PartialPath path) {
+  public String getBelongedStorageGroup(String pathName) {
     for (String storageGroup : storageGroups) {
-      if (path.getFullPath().startsWith(storageGroup + ".")) {
+      if (pathName.startsWith(storageGroup + ".")) {
         return storageGroup;
       }
     }
-    throw new RuntimeException(
-        "No matched storage group. Please check the path " + path.getFullPath());
+    throw new RuntimeException("No matched storage group. Please check the path " + pathName);
+  }
+
+  public String getBelongedStorageGroup(PartialPath path) {
+    return getBelongedStorageGroup(path.getFullPath());
   }
 
   public List<String> getStorageGroups() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
index b9ee50409d..1c007470d6 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
@@ -25,24 +25,24 @@ import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
 import org.apache.iotdb.db.mpp.common.schematree.SchemaTree;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FillDescriptor;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FilterNullParameter;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
-import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /** Analysis used for planning a query. TODO: This class may need to store more info for a query. */
 public class Analysis {
-  // Description for each series. Such as dataType, existence
-
-  // Data distribution info for each series. Series -> [DataRegion, DataRegion]
-
-  // Map<PartialPath, List<FullPath>> Used to remove asterisk
 
   // Statement
   private Statement statement;
 
-  // indicate whether this statement is write or read
+  // indicate whether this statement is `WRITE` or `READ`
   private QueryType queryType;
 
   private DataPartition dataPartition;
@@ -51,13 +51,43 @@ public class Analysis {
 
   private SchemaTree schemaTree;
 
-  private IExpression queryFilter;
+  // map from output column name (for every node) to its datatype
+  private TypeProvider typeProvider;
+
+  // map from device name to series/aggregation under this device
+  private Map<String, Set<Expression>> sourceExpressions;
+
+  //
+  private Set<Expression> selectExpressions;
+
+  // all aggregations that need to be calculated
+  private Map<String, Set<Expression>> AggregationExpressions;
+
+  // map from grouped path name to list of input aggregation in `GROUP BY LEVEL` clause
+  private Map<Expression, Set<Expression>> groupByLevelExpressions;
+
+  // parameter of `WITHOUT NULL` clause
+  private FilterNullParameter filterNullParameter;
+
+  // parameter of `FILL` clause
+  private FillDescriptor fillDescriptor;
+
+  // parameter of `GROUP BY TIME` clause
+  private GroupByTimeParameter groupByTimeParameter;
+
+  private Expression queryFilter;
+
+  private Map<String, Expression> deviceToQueryFilter;
+
+  // indicate is there a value filter
+  private boolean hasValueFilter = false;
+
+  // a global time filter used in `initQueryDataSource`
+  private Filter globalTimeFilter;
 
   // header of result dataset
   private DatasetHeader respDatasetHeader;
 
-  private TypeProvider typeProvider;
-
   public Analysis() {}
 
   public List<TRegionReplicaSet> getPartitionInfo(PartialPath seriesPath, Filter timefilter) {
@@ -97,12 +127,12 @@ public class Analysis {
     this.schemaTree = schemaTree;
   }
 
-  public IExpression getQueryFilter() {
-    return queryFilter;
+  public Filter getGlobalTimeFilter() {
+    return globalTimeFilter;
   }
 
-  public void setQueryFilter(IExpression expression) {
-    this.queryFilter = expression;
+  public void setGlobalTimeFilter(Filter timeFilter) {
+    this.globalTimeFilter = timeFilter;
   }
 
   public DatasetHeader getRespDatasetHeader() {
@@ -125,4 +155,84 @@ public class Analysis {
     return (dataPartition != null && !dataPartition.isEmpty())
         || (schemaPartition != null && !schemaPartition.isEmpty());
   }
+
+  public Map<String, Set<Expression>> getSourceExpressions() {
+    return sourceExpressions;
+  }
+
+  public void setSourceExpressions(Map<String, Set<Expression>> sourceExpressions) {
+    this.sourceExpressions = sourceExpressions;
+  }
+
+  public Set<Expression> getSelectExpressions() {
+    return selectExpressions;
+  }
+
+  public void setSelectExpressions(Set<Expression> selectExpressions) {
+    this.selectExpressions = selectExpressions;
+  }
+
+  public Map<String, Set<Expression>> getAggregationExpressions() {
+    return AggregationExpressions;
+  }
+
+  public void setAggregationExpressions(Map<String, Set<Expression>> aggregationExpressions) {
+    AggregationExpressions = aggregationExpressions;
+  }
+
+  public Map<Expression, Set<Expression>> getGroupByLevelExpressions() {
+    return groupByLevelExpressions;
+  }
+
+  public void setGroupByLevelExpressions(Map<Expression, Set<Expression>> groupByLevelExpressions) {
+    this.groupByLevelExpressions = groupByLevelExpressions;
+  }
+
+  public FilterNullParameter getFilterNullParameter() {
+    return filterNullParameter;
+  }
+
+  public void setFilterNullParameter(FilterNullParameter filterNullParameter) {
+    this.filterNullParameter = filterNullParameter;
+  }
+
+  public FillDescriptor getFillDescriptor() {
+    return fillDescriptor;
+  }
+
+  public void setFillDescriptor(FillDescriptor fillDescriptor) {
+    this.fillDescriptor = fillDescriptor;
+  }
+
+  public boolean isHasValueFilter() {
+    return hasValueFilter;
+  }
+
+  public void setHasValueFilter(boolean hasValueFilter) {
+    this.hasValueFilter = hasValueFilter;
+  }
+
+  public Expression getQueryFilter() {
+    return queryFilter;
+  }
+
+  public void setQueryFilter(Expression queryFilter) {
+    this.queryFilter = queryFilter;
+  }
+
+  public Map<String, Expression> getDeviceToQueryFilter() {
+    return deviceToQueryFilter;
+  }
+
+  public void setDeviceToQueryFilter(Map<String, Expression> deviceToQueryFilter) {
+    this.deviceToQueryFilter = deviceToQueryFilter;
+  }
+
+  public GroupByTimeParameter getGroupByTimeParameter() {
+    return groupByTimeParameter;
+  }
+
+  public void setGroupByTimeParameter(GroupByTimeParameter groupByTimeParameter) {
+    this.groupByTimeParameter = groupByTimeParameter;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analyzer.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analyzer.java
index 5e20cd729d..6217f23185 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analyzer.java
@@ -24,25 +24,26 @@ import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
 import org.apache.iotdb.db.exception.sql.SemanticException;
 import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
-import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
+import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
 import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
 import org.apache.iotdb.db.mpp.common.header.HeaderConstant;
+import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
 import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
 import org.apache.iotdb.db.mpp.common.schematree.SchemaTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.ConcatPathRewriter;
-import org.apache.iotdb.db.mpp.plan.rewriter.DnfFilterOptimizer;
-import org.apache.iotdb.db.mpp.plan.rewriter.MergeSingleFilterOptimizer;
-import org.apache.iotdb.db.mpp.plan.rewriter.RemoveNotOptimizer;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FillDescriptor;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FilterNullParameter;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
 import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
+import org.apache.iotdb.db.mpp.plan.statement.component.FillComponent;
+import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
+import org.apache.iotdb.db.mpp.plan.statement.component.GroupByTimeComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
-import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertMultiTabletsStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsOfOneDeviceStatement;
@@ -50,6 +51,7 @@ import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
+import org.apache.iotdb.db.mpp.plan.statement.literal.Literal;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.AlterTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountDevicesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountLevelTimeSeriesStatement;
@@ -62,11 +64,12 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowDevicesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTimeSeriesStatement;
-import org.apache.iotdb.db.qp.constant.SQLConstant;
-import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.expression.IExpression;
-import org.apache.iotdb.tsfile.read.expression.util.ExpressionOptimizer;
+import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
+import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -75,26 +78,29 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.stream.Collectors;
 
 /** Analyze the statement and generate Analysis. */
 public class Analyzer {
-
   private static final Logger logger = LoggerFactory.getLogger(Analyzer.class);
 
   private final MPPQueryContext context;
 
   private final IPartitionFetcher partitionFetcher;
   private final ISchemaFetcher schemaFetcher;
+  private final TypeProvider typeProvider;
 
   public Analyzer(
       MPPQueryContext context, IPartitionFetcher partitionFetcher, ISchemaFetcher schemaFetcher) {
     this.context = context;
     this.partitionFetcher = partitionFetcher;
     this.schemaFetcher = schemaFetcher;
+    this.typeProvider = new TypeProvider();
   }
 
   public Analysis analyze(Statement statement) {
@@ -119,92 +125,599 @@ public class Analyzer {
       Analysis analysis = new Analysis();
       try {
         // check for semantic errors
-        queryStatement.selfCheck();
+        queryStatement.semanticCheck();
 
         // concat path and construct path pattern tree
         PathPatternTree patternTree = new PathPatternTree();
         QueryStatement rewrittenStatement =
             (QueryStatement) new ConcatPathRewriter().rewrite(queryStatement, patternTree);
+        analysis.setStatement(rewrittenStatement);
 
         // request schema fetch API
         logger.info("{} fetch query schema...", getLogHeader());
         SchemaTree schemaTree = schemaFetcher.fetchSchema(patternTree);
         logger.info("{} fetch schema done", getLogHeader());
-        // (xingtanzjr) If there is no leaf node in the schema tree, the query should be completed
-        // immediately
+        // If there is no leaf node in the schema tree, the query should be completed immediately
         if (schemaTree.isEmpty()) {
           analysis.setRespDatasetHeader(new DatasetHeader(new ArrayList<>(), false));
           return analysis;
         }
-        // bind metadata, remove wildcards, and apply SLIMIT & SOFFSET
-        TypeProvider typeProvider = new TypeProvider();
-        rewrittenStatement =
-            (QueryStatement)
-                new WildcardsRemover().rewrite(rewrittenStatement, typeProvider, schemaTree);
 
-        // fetch partition information
-        Set<PartialPath> devicePathSet = new HashSet<>();
-        for (ResultColumn resultColumn : queryStatement.getSelectComponent().getResultColumns()) {
-          devicePathSet.addAll(
-              resultColumn.collectPaths().stream()
-                  .map(PartialPath::getDevicePath)
-                  .collect(Collectors.toList()));
+        List<Pair<Expression, String>> outputExpressions;
+        Set<Expression> selectExpressions = new HashSet<>();
+        Map<String, Set<Expression>> sourceExpressions = new HashMap<>();
+        // Example 1: select s1, s1 + s2 as t, udf(udf(s1)) from root.sg.d1
+        //   outputExpressions: [<root.sg.d1.s1,null>, <root.sg.d1.s1 + root.sg.d1.s2,t>,
+        //                       <udf(udf(root.sg.d1.s1)),null>]
+        //   selectExpressions: [root.sg.d1.s1, root.sg.d1.s1 + root.sg.d1.s2,
+        //                       udf(udf(root.sg.d1.s1))]
+        //   sourceExpressions: {root.sg.d1 -> [root.sg.d1.s1, root.sg.d1.s2]}
+        //
+        // Example 2: select s1, s2, s3 as t from root.sg.* align by device
+        //   outputExpressions: [<s1,null>, <s2,null>, <s1,t>]
+        //   selectExpressions: [root.sg.d1.s1, root.sg.d1.s2, root.sg.d1.s3,
+        //                       root.sg.d2.s1, root.sg.d2.s2]
+        //   sourceExpressions: {root.sg.d1 -> [root.sg.d1.s1, root.sg.d1.s2, root.sg.d1.s2],
+        //                       root.sg.d2 -> [root.sg.d2.s1, root.sg.d2.s2]}
+        //
+        // Example 3: select sum(s1) + 1 as t, count(s2) from root.sg.d1
+        //   outputExpressions: [<sum(root.sg.d1.s1) + 1,t>, <count(root.sg.d1.s2),t>]
+        //   selectExpressions: [sum(root.sg.d1.s1) + 1, count(root.sg.d1.s2)]
+        //   aggregationExpressions: {root.sg.d1 -> [sum(root.sg.d1.s1), count(root.sg.d1.s2)]}
+        //   sourceExpressions: {root.sg.d1 -> [sum(root.sg.d1.s1), count(root.sg.d1.s2)]}
+        //
+        // Example 4: select sum(s1) + 1 as t, count(s2) from root.sg.d1 where s1 > 1
+        //   outputExpressions: [<sum(root.sg.d1.s1) + 1,t>, <count(root.sg.d1.s2),t>]
+        //   selectExpressions: [sum(root.sg.d1.s1) + 1, count(root.sg.d1.s2)]
+        //   aggregationExpressions: {root.sg.d1 -> [sum(root.sg.d1.s1), count(root.sg.d1.s2)]}
+        //   sourceExpressions: {root.sg.d1 -> [root.sg.d1.s1, root.sg.d1.s2]}
+        List<DeviceSchemaInfo> deviceSchemaInfos = new ArrayList<>();
+        // a set that contains all measurement names,
+        Set<String> measurementSet = new HashSet<>();
+        if (queryStatement.isAlignByDevice()) {
+          outputExpressions =
+              analyzeFrom(
+                  queryStatement, schemaTree, deviceSchemaInfos, selectExpressions, measurementSet);
+        } else {
+          outputExpressions = analyzeSelect(queryStatement, schemaTree);
+          selectExpressions =
+              outputExpressions.stream().map(Pair::getLeft).collect(Collectors.toSet());
+        }
+
+        if (queryStatement.isGroupByLevel()) {
+          // map from grouped expression to set of input expressions
+          Map<Expression, Set<Expression>> groupByLevelExpressions =
+              analyzeGroupByLevel(queryStatement, outputExpressions, selectExpressions);
+          analysis.setGroupByLevelExpressions(groupByLevelExpressions);
+        }
+
+        if (queryStatement.isGroupByTime()) {
+          analysis.setGroupByTimeParameter(
+              new GroupByTimeParameter(queryStatement.getGroupByTimeComponent()));
         }
+
+        // extract global time filter from query filter and determine if there is a value filter
+        Pair<Filter, Boolean> resultPair = analyzeGlobalTimeFilter(queryStatement);
+        Filter globalTimeFilter = resultPair.left;
+        boolean hasValueFilter = resultPair.right;
+        analysis.setGlobalTimeFilter(globalTimeFilter);
+        analysis.setHasValueFilter(hasValueFilter);
+
+        // generate sourceExpression according to selectExpressions
+        boolean isValueFilterAggregation = queryStatement.isAggregationQuery() && hasValueFilter;
+        boolean isRawDataSource = !queryStatement.isAggregationQuery() || isValueFilterAggregation;
+        for (Expression selectExpr : selectExpressions) {
+          updateSource(selectExpr, sourceExpressions, isRawDataSource);
+        }
+        if (isValueFilterAggregation) {
+          Map<String, Set<Expression>> aggregationExpressions = new HashMap<>();
+          for (Expression selectExpr : selectExpressions) {
+            analyzeAggregation(selectExpr, aggregationExpressions);
+          }
+          analysis.setAggregationExpressions(aggregationExpressions);
+        }
+
         if (queryStatement.getWhereCondition() != null) {
-          devicePathSet.addAll(
-              queryStatement.getWhereCondition().getQueryFilter().getPathSet().stream()
-                  .filter(SQLConstant::isNotReservedPath)
-                  .map(PartialPath::getDevicePath)
-                  .collect(Collectors.toList()));
+          if (queryStatement.isAlignByDevice()) {
+            Map<String, Expression> deviceToQueryFilter =
+                analyzeWhereSplitByDevice(queryStatement, deviceSchemaInfos, measurementSet);
+            deviceToQueryFilter
+                .values()
+                .forEach(
+                    queryFilter -> updateSource(queryFilter, sourceExpressions, isRawDataSource));
+            analysis.setDeviceToQueryFilter(deviceToQueryFilter);
+          } else {
+            Expression queryFilter = analyzeWhere(queryStatement, schemaTree);
+            // update sourceExpression according to queryFilter
+            updateSource(queryFilter, sourceExpressions, isRawDataSource);
+            analysis.setQueryFilter(queryFilter);
+          }
         }
+        analysis.setSourceExpressions(sourceExpressions);
+        analysis.setSelectExpressions(selectExpressions);
+
+        if (queryStatement.getFilterNullComponent() != null) {
+          FilterNullParameter filterNullParameter =
+              analyzeWithoutNull(queryStatement, schemaTree, selectExpressions);
+          analysis.setFilterNullParameter(filterNullParameter);
+        }
+
+        if (queryStatement.getFillComponent() != null) {
+          FillComponent fillComponent = queryStatement.getFillComponent();
+          if (fillComponent.getFillPolicy() == FillPolicy.VALUE) {
+            List<Expression> fillColumnList =
+                outputExpressions.stream()
+                    .map(Pair::getLeft)
+                    .distinct()
+                    .collect(Collectors.toList());
+            for (Expression fillColumn : fillColumnList) {
+              checkDataTypeConsistencyInFill(fillColumn, fillComponent.getFillValue());
+            }
+          }
+          analysis.setFillDescriptor(
+              new FillDescriptor(fillComponent.getFillPolicy(), fillComponent.getFillValue()));
+        }
+
+        // generate result set header according to output expressions
+        DatasetHeader datasetHeader = analyzeOutput(queryStatement, outputExpressions);
+        analysis.setRespDatasetHeader(datasetHeader);
+        analysis.setTypeProvider(typeProvider);
+
+        // fetch partition information
         Map<String, List<DataPartitionQueryParam>> sgNameToQueryParamsMap = new HashMap<>();
-        for (PartialPath devicePath : devicePathSet) {
+        for (String devicePath : sourceExpressions.keySet()) {
           DataPartitionQueryParam queryParam = new DataPartitionQueryParam();
-          queryParam.setDevicePath(devicePath.getFullPath());
+          queryParam.setDevicePath(devicePath);
           sgNameToQueryParamsMap
               .computeIfAbsent(
                   schemaTree.getBelongedStorageGroup(devicePath), key -> new ArrayList<>())
               .add(queryParam);
         }
         DataPartition dataPartition = partitionFetcher.getDataPartition(sgNameToQueryParamsMap);
-
-        // optimize expressions in whereCondition
-        WhereCondition whereCondition = rewrittenStatement.getWhereCondition();
-        if (whereCondition != null) {
-          QueryFilter filter = whereCondition.getQueryFilter();
-          filter = new RemoveNotOptimizer().optimize(filter);
-          filter = new DnfFilterOptimizer().optimize(filter);
-          filter = new MergeSingleFilterOptimizer().optimize(filter);
-
-          // transform QueryFilter to expression
-          List<PartialPath> filterPaths = new ArrayList<>(filter.getPathSet());
-          HashMap<PartialPath, TSDataType> pathTSDataTypeHashMap = new HashMap<>();
-          for (PartialPath filterPath : filterPaths) {
-            pathTSDataTypeHashMap.put(
-                filterPath,
-                SQLConstant.isReservedPath(filterPath)
-                    ? TSDataType.INT64
-                    : filterPath.getSeriesType());
-          }
-          IExpression expression = filter.transformToExpression(pathTSDataTypeHashMap);
-          expression =
-              ExpressionOptimizer.getInstance()
-                  .optimize(expression, queryStatement.getSelectComponent().getDeduplicatedPaths());
-          analysis.setQueryFilter(expression);
-        }
-        analysis.setStatement(rewrittenStatement);
-        analysis.setSchemaTree(schemaTree);
-        analysis.setTypeProvider(typeProvider);
-        analysis.setRespDatasetHeader(queryStatement.constructDatasetHeader());
         analysis.setDataPartitionInfo(dataPartition);
-      } catch (StatementAnalyzeException
-          | PathNumOverLimitException
-          | QueryFilterOptimizationException e) {
+      } catch (StatementAnalyzeException e) {
+        logger.error("Meet error when analyzing the query statement: ", e);
         throw new StatementAnalyzeException("Meet error when analyzing the query statement");
       }
       return analysis;
     }
 
+    private List<Pair<Expression, String>> analyzeSelect(
+        QueryStatement queryStatement, SchemaTree schemaTree) {
+      List<Pair<Expression, String>> outputExpressions = new ArrayList<>();
+      ColumnPaginationController paginationController =
+          new ColumnPaginationController(
+              queryStatement.getSeriesLimit(),
+              queryStatement.getSeriesOffset(),
+              queryStatement.isLastQuery() || queryStatement.isGroupByLevel());
+
+      for (ResultColumn resultColumn : queryStatement.getSelectComponent().getResultColumns()) {
+        boolean hasAlias = resultColumn.hasAlias();
+        List<Expression> resultExpressions =
+            ExpressionAnalyzer.removeWildcardInExpression(resultColumn.getExpression(), schemaTree);
+        if (hasAlias && resultExpressions.size() > 1) {
+          throw new SemanticException(
+              String.format(
+                  "alias '%s' can only be matched with one time series", resultColumn.getAlias()));
+        }
+        for (Expression expression : resultExpressions) {
+          if (paginationController.hasCurOffset()) {
+            paginationController.consumeOffset();
+            continue;
+          }
+          if (paginationController.hasCurLimit()) {
+            Expression expressionWithoutAlias =
+                ExpressionAnalyzer.removeAliasFromExpression(expression);
+            String alias =
+                !Objects.equals(expressionWithoutAlias, expression)
+                    ? expression.getExpressionString()
+                    : null;
+            alias = hasAlias ? resultColumn.getAlias() : alias;
+            outputExpressions.add(new Pair<>(expressionWithoutAlias, alias));
+            ExpressionAnalyzer.updateTypeProvider(expressionWithoutAlias, typeProvider);
+            paginationController.consumeLimit();
+          } else {
+            break;
+          }
+        }
+      }
+      return outputExpressions;
+    }
+
+    private List<Pair<Expression, String>> analyzeFrom(
+        QueryStatement queryStatement,
+        SchemaTree schemaTree,
+        List<DeviceSchemaInfo> allDeviceSchemaInfos,
+        Set<Expression> selectExpressions,
+        Set<String> measurementSet) {
+      // device path patterns in FROM clause
+      List<PartialPath> devicePatternList = queryStatement.getFromComponent().getPrefixPaths();
+
+      // a list contains all selected paths
+      List<MeasurementPath> allSelectedPaths = new ArrayList<>();
+      for (PartialPath devicePattern : devicePatternList) {
+        // get matched device path and all measurement schema infos under this device
+        List<DeviceSchemaInfo> deviceSchemaInfos = schemaTree.getMatchedDevices(devicePattern);
+        allDeviceSchemaInfos.addAll(deviceSchemaInfos);
+        for (DeviceSchemaInfo deviceSchema : deviceSchemaInfos) {
+          // add matched path into allSelectedPaths
+          allSelectedPaths.addAll(deviceSchema.getMeasurements(measurementSet));
+        }
+      }
+
+      // convert allSelectedPaths to a map from measurement name to corresponding paths
+      Map<String, List<MeasurementPath>> measurementNameToPathsMap = new HashMap<>();
+      Map<String, String> aliasToMeasurementNameMap = new HashMap<>();
+      for (MeasurementPath measurementPath : allSelectedPaths) {
+        measurementNameToPathsMap
+            .computeIfAbsent(measurementPath.getMeasurement(), key -> new ArrayList<>())
+            .add(measurementPath);
+        if (measurementPath.isMeasurementAliasExists()) {
+          if (aliasToMeasurementNameMap.containsKey(measurementPath.getMeasurementAlias())
+              && !Objects.equals(
+                  aliasToMeasurementNameMap.get(measurementPath.getMeasurementAlias()),
+                  measurementPath.getMeasurement())) {
+            throw new SemanticException(
+                String.format(
+                    "ALIGN BY DEVICE: alias '%s' can only be matched with one measurement",
+                    measurementPath.getMeasurementAlias()));
+          }
+          aliasToMeasurementNameMap.put(
+              measurementPath.getMeasurementAlias(), measurementPath.getMeasurement());
+        }
+      }
+      // check whether the datatype of paths which has the same measurement name are consistent
+      // if not, throw a SemanticException
+      measurementNameToPathsMap.values().forEach(this::checkDataTypeConsistencyInAlignByDevice);
+
+      // a list of measurement name with alias (null if alias not exist)
+      List<Pair<Expression, String>> measurementWithAliasList =
+          getAllMeasurements(queryStatement, measurementSet);
+
+      // apply SLIMIT & SOFFSET and set outputExpressions & selectExpressions
+      List<Pair<Expression, String>> outputExpressions = new ArrayList<>();
+      ColumnPaginationController paginationController =
+          new ColumnPaginationController(
+              queryStatement.getSeriesLimit(), queryStatement.getSeriesOffset(), false);
+      for (Pair<Expression, String> measurementAliasPair : measurementWithAliasList) {
+        String measurement =
+            ExpressionAnalyzer.getPathInSourceExpression(measurementAliasPair.left).toString();
+        if (measurementNameToPathsMap.containsKey(measurement)) {
+          List<MeasurementPath> measurementPaths = measurementNameToPathsMap.get(measurement);
+          measurementPaths.forEach(MeasurementPath::removeMeasurementAlias);
+          TSDataType dataType = measurementPaths.get(0).getSeriesType();
+          if (paginationController.hasCurOffset()) {
+            paginationController.consumeOffset();
+            continue;
+          }
+          if (paginationController.hasCurLimit()) {
+            outputExpressions.add(measurementAliasPair);
+            typeProvider.setType(measurementAliasPair.left.getExpressionString(), dataType);
+            for (MeasurementPath measurementPath : measurementPaths) {
+              Expression tmpExpression =
+                  ExpressionAnalyzer.replacePathInSourceExpression(
+                      measurementAliasPair.left, measurementPath);
+              typeProvider.setType(tmpExpression.getExpressionString(), dataType);
+              selectExpressions.add(tmpExpression);
+            }
+            paginationController.consumeLimit();
+          } else {
+            break;
+          }
+        } else if (aliasToMeasurementNameMap.containsKey(measurement)
+            && measurementNameToPathsMap.containsKey(aliasToMeasurementNameMap.get(measurement))) {
+          List<MeasurementPath> measurementPaths =
+              measurementNameToPathsMap.get(aliasToMeasurementNameMap.get(measurement));
+          measurementPaths.forEach(MeasurementPath::removeMeasurementAlias);
+          TSDataType dataType = measurementPaths.get(0).getSeriesType();
+          Expression expressionWithAlias = measurementAliasPair.left;
+          Expression expressionWithoutAlias =
+              ExpressionAnalyzer.removeAliasInMeasurementExpression(
+                  measurementAliasPair.left, aliasToMeasurementNameMap);
+          String alias =
+              measurementAliasPair.right != null
+                  ? measurementAliasPair.right
+                  : expressionWithAlias.getExpressionString();
+          if (paginationController.hasCurOffset()) {
+            paginationController.consumeOffset();
+            continue;
+          }
+          if (paginationController.hasCurLimit()) {
+            outputExpressions.add(new Pair<>(expressionWithoutAlias, alias));
+            typeProvider.setType(expressionWithoutAlias.getExpressionString(), dataType);
+            for (MeasurementPath measurementPath : measurementPaths) {
+              Expression tmpExpression =
+                  ExpressionAnalyzer.replacePathInSourceExpression(
+                      expressionWithoutAlias, measurementPath);
+              typeProvider.setType(tmpExpression.getExpressionString(), dataType);
+              selectExpressions.add(tmpExpression);
+            }
+            paginationController.consumeLimit();
+          } else {
+            break;
+          }
+        } else if (measurement.equals(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)) {
+          for (String measurementName : measurementNameToPathsMap.keySet()) {
+            List<MeasurementPath> measurementPaths = measurementNameToPathsMap.get(measurementName);
+            measurementPaths.forEach(MeasurementPath::removeMeasurementAlias);
+            TSDataType dataType = measurementPaths.get(0).getSeriesType();
+            if (paginationController.hasCurOffset()) {
+              paginationController.consumeOffset();
+              continue;
+            }
+            if (paginationController.hasCurLimit()) {
+              // replace `*` with exact measurement name
+              Expression replacedMeasurement =
+                  ExpressionAnalyzer.replacePathInSourceExpression(
+                      measurementAliasPair.left, measurementName);
+              typeProvider.setType(replacedMeasurement.getExpressionString(), dataType);
+              outputExpressions.add(new Pair<>(replacedMeasurement, measurementAliasPair.right));
+              for (MeasurementPath measurementPath : measurementPaths) {
+                // replace `*` with exact measurement path
+                Expression tmpExpression =
+                    ExpressionAnalyzer.replacePathInSourceExpression(
+                        measurementAliasPair.left, measurementPath);
+                typeProvider.setType(tmpExpression.getExpressionString(), dataType);
+                selectExpressions.add(tmpExpression);
+              }
+              paginationController.consumeLimit();
+            } else {
+              break;
+            }
+          }
+          if (!paginationController.hasCurLimit()) {
+            break;
+          }
+        } else {
+          // do nothing or warning
+        }
+      }
+      return outputExpressions;
+    }
+
+    private List<Pair<Expression, String>> getAllMeasurements(
+        QueryStatement queryStatement, Set<String> measurementSet) {
+      List<Pair<Expression, String>> measurementWithAliasList =
+          queryStatement.getSelectComponent().getResultColumns().stream()
+              .map(
+                  resultColumn ->
+                      ExpressionAnalyzer.getMeasurementWithAliasInSourceExpression(
+                          resultColumn.getExpression(), resultColumn.getAlias()))
+              .collect(Collectors.toList());
+      measurementSet.addAll(
+          measurementWithAliasList.stream()
+              .map(Pair::getLeft)
+              .map(ExpressionAnalyzer::collectPaths)
+              .flatMap(Set::stream)
+              .map(PartialPath::getFullPath)
+              .collect(Collectors.toSet()));
+      return measurementWithAliasList;
+    }
+
+    private Pair<Filter, Boolean> analyzeGlobalTimeFilter(QueryStatement queryStatement) {
+      Filter globalTimeFilter = null;
+      boolean hasValueFilter = false;
+      if (queryStatement.getWhereCondition() != null) {
+        Pair<Filter, Boolean> resultPair =
+            ExpressionAnalyzer.transformToGlobalTimeFilter(
+                queryStatement.getWhereCondition().getPredicate());
+        globalTimeFilter = resultPair.left;
+        hasValueFilter = resultPair.right;
+      }
+      if (queryStatement.isGroupByTime()) {
+        GroupByTimeComponent groupByTimeComponent = queryStatement.getGroupByTimeComponent();
+        Filter groupByFilter =
+            new GroupByFilter(
+                groupByTimeComponent.getInterval(),
+                groupByTimeComponent.getSlidingStep(),
+                groupByTimeComponent.getStartTime(),
+                groupByTimeComponent.getEndTime());
+        if (globalTimeFilter == null) {
+          globalTimeFilter = groupByFilter;
+        } else {
+          // TODO: optimize the filter
+          globalTimeFilter = FilterFactory.and(globalTimeFilter, groupByFilter);
+        }
+      }
+      return new Pair<>(globalTimeFilter, hasValueFilter);
+    }
+
+    private void updateSource(
+        Expression selectExpr,
+        Map<String, Set<Expression>> sourceExpressions,
+        boolean isRawDataSource) {
+      for (Expression sourceExpression :
+          ExpressionAnalyzer.searchSourceExpressions(selectExpr, isRawDataSource)) {
+        sourceExpressions
+            .computeIfAbsent(
+                ExpressionAnalyzer.getDeviceNameInSourceExpression(sourceExpression),
+                key -> new HashSet<>())
+            .add(sourceExpression);
+      }
+    }
+
+    private void analyzeAggregation(
+        Expression selectExpr, Map<String, Set<Expression>> aggregationExpressions) {
+      for (Expression aggregationExpression :
+          ExpressionAnalyzer.searchAggregationExpressions(selectExpr)) {
+        aggregationExpressions
+            .computeIfAbsent(
+                ExpressionAnalyzer.getDeviceNameInSourceExpression(aggregationExpression),
+                key -> new HashSet<>())
+            .add(aggregationExpression);
+      }
+    }
+
+    private Expression analyzeWhere(QueryStatement queryStatement, SchemaTree schemaTree) {
+      List<Expression> rewrittenPredicates =
+          ExpressionAnalyzer.removeWildcardInQueryFilter(
+              queryStatement.getWhereCondition().getPredicate(),
+              queryStatement.getFromComponent().getPrefixPaths(),
+              schemaTree,
+              typeProvider);
+      return ExpressionUtils.constructQueryFilter(
+          rewrittenPredicates.stream().distinct().collect(Collectors.toList()));
+    }
+
+    private Map<String, Expression> analyzeWhereSplitByDevice(
+        QueryStatement queryStatement,
+        List<DeviceSchemaInfo> deviceSchemaInfos,
+        Set<String> measurementSet) {
+      Map<String, Expression> deviceToQueryFilter = new HashMap<>();
+      for (DeviceSchemaInfo deviceSchemaInfo : deviceSchemaInfos) {
+        List<Expression> rewrittenPredicates =
+            ExpressionAnalyzer.removeWildcardInQueryFilterByDevice(
+                queryStatement.getWhereCondition().getPredicate(),
+                deviceSchemaInfo,
+                measurementSet,
+                typeProvider);
+        deviceToQueryFilter.put(
+            deviceSchemaInfo.getDevicePath().getFullPath(),
+            ExpressionUtils.constructQueryFilter(
+                rewrittenPredicates.stream().distinct().collect(Collectors.toList())));
+      }
+      return deviceToQueryFilter;
+    }
+
+    private Map<Expression, Set<Expression>> analyzeGroupByLevel(
+        QueryStatement queryStatement,
+        List<Pair<Expression, String>> outputExpressions,
+        Set<Expression> selectExpressions) {
+      GroupByLevelController groupByLevelController =
+          new GroupByLevelController(queryStatement.getGroupByLevelComponent().getLevels());
+      for (Pair<Expression, String> measurementWithAlias : outputExpressions) {
+        groupByLevelController.control(measurementWithAlias.left, measurementWithAlias.right);
+      }
+      Map<Expression, Set<Expression>> rawGroupByLevelExpressions =
+          groupByLevelController.getGroupedPathMap();
+      // check whether the datatype of paths which has the same output column name are consistent
+      // if not, throw a SemanticException
+      rawGroupByLevelExpressions.values().forEach(this::checkDataTypeConsistencyInGroupByLevel);
+
+      Map<Expression, Set<Expression>> groupByLevelExpressions = new LinkedHashMap<>();
+      ColumnPaginationController paginationController =
+          new ColumnPaginationController(
+              queryStatement.getSeriesLimit(), queryStatement.getSeriesOffset(), false);
+      for (Expression groupedExpression : rawGroupByLevelExpressions.keySet()) {
+        if (paginationController.hasCurOffset()) {
+          paginationController.consumeOffset();
+          continue;
+        }
+        if (paginationController.hasCurLimit()) {
+          groupByLevelExpressions.put(
+              groupedExpression, rawGroupByLevelExpressions.get(groupedExpression));
+          paginationController.consumeLimit();
+        } else {
+          break;
+        }
+      }
+
+      // reset outputExpressions & selectExpressions after applying SLIMIT/SOFFSET
+      outputExpressions.clear();
+      for (Expression groupedExpression : groupByLevelExpressions.keySet()) {
+        TSDataType dataType =
+            typeProvider.getType(
+                new ArrayList<>(groupByLevelExpressions.get(groupedExpression))
+                    .get(0)
+                    .getExpressionString());
+        typeProvider.setType(groupedExpression.getExpressionString(), dataType);
+        outputExpressions.add(
+            new Pair<>(
+                groupedExpression,
+                groupByLevelController.getAlias(groupedExpression.getExpressionString())));
+      }
+      selectExpressions.clear();
+      selectExpressions.addAll(
+          groupByLevelExpressions.values().stream()
+              .flatMap(Set::stream)
+              .collect(Collectors.toSet()));
+      return groupByLevelExpressions;
+    }
+
+    private FilterNullParameter analyzeWithoutNull(
+        QueryStatement queryStatement, SchemaTree schemaTree, Set<Expression> selectExpressions) {
+      FilterNullParameter filterNullParameter = new FilterNullParameter();
+      filterNullParameter.setFilterNullPolicy(
+          queryStatement.getFilterNullComponent().getWithoutPolicyType());
+      List<Expression> resultFilterNullColumns = new ArrayList<>();
+      List<Expression> rawFilterNullColumns =
+          queryStatement.getFilterNullComponent().getWithoutNullColumns();
+      for (Expression filterNullColumn : rawFilterNullColumns) {
+        List<Expression> resultExpressions =
+            ExpressionAnalyzer.removeWildcardInExpression(filterNullColumn, schemaTree);
+        for (Expression expression : resultExpressions) {
+          Expression expressionWithoutAlias =
+              ExpressionAnalyzer.removeAliasFromExpression(expression);
+          if (!selectExpressions.contains(expressionWithoutAlias)) {
+            throw new SemanticException(
+                String.format(
+                    "The without null column '%s' don't match the columns queried.", expression));
+          }
+          resultFilterNullColumns.add(expressionWithoutAlias);
+        }
+      }
+      // don't specify columns, by default, it is effective for all columns
+      if (rawFilterNullColumns.isEmpty()) {
+        resultFilterNullColumns.addAll(selectExpressions);
+      }
+      filterNullParameter.setFilterNullColumns(resultFilterNullColumns);
+      return filterNullParameter;
+    }
+
+    private DatasetHeader analyzeOutput(
+        QueryStatement queryStatement, List<Pair<Expression, String>> outputExpressions) {
+      boolean isIgnoreTimestamp =
+          queryStatement.isAggregationQuery() && !queryStatement.isGroupByTime();
+      List<ColumnHeader> columnHeaders =
+          outputExpressions.stream()
+              .map(
+                  expressionWithAlias -> {
+                    String columnName = expressionWithAlias.left.getExpressionString();
+                    String alias = expressionWithAlias.right;
+                    return new ColumnHeader(columnName, typeProvider.getType(columnName), alias);
+                  })
+              .collect(Collectors.toList());
+      return new DatasetHeader(columnHeaders, isIgnoreTimestamp);
+    }
+
+    /**
+     * Check datatype consistency in ALIGN BY DEVICE.
+     *
+     * <p>an inconsistent example: select s0 from root.sg1.d1, root.sg1.d2 align by device, return
+     * false while root.sg1.d1.s0 is INT32 and root.sg1.d2.s0 is FLOAT.
+     */
+    private void checkDataTypeConsistencyInAlignByDevice(List<MeasurementPath> measurementPaths) {
+      TSDataType checkedDataType = measurementPaths.get(0).getSeriesType();
+      for (MeasurementPath path : measurementPaths) {
+        if (path.getSeriesType() != checkedDataType) {
+          throw new SemanticException(
+              "ALIGN BY DEVICE: the data types of the same measurement column should be the same across devices.");
+        }
+      }
+    }
+
+    /** Check datatype consistency in GROUP BY LEVEL. */
+    private void checkDataTypeConsistencyInGroupByLevel(Set<Expression> expressions) {
+      List<Expression> expressionList = new ArrayList<>(expressions);
+      TSDataType checkedDataType =
+          typeProvider.getType(expressionList.get(0).getExpressionString());
+      for (Expression expression : expressionList) {
+        if (typeProvider.getType(expression.getExpressionString()) != checkedDataType) {
+          throw new SemanticException(
+              "GROUP BY LEVEL: the data types of the same output column should be the same.");
+        }
+      }
+    }
+
+    private void checkDataTypeConsistencyInFill(Expression fillColumn, Literal fillValue) {
+      TSDataType checkedDataType = typeProvider.getType(fillColumn.getExpressionString());
+      if (!fillValue.isDataTypeConsistency(checkedDataType)) {
+        // TODO: consider type casting
+        throw new SemanticException(
+            "FILL: the data type of the fill value should be the same as the output column");
+      }
+    }
+
     @Override
     public Analysis visitInsert(InsertStatement insertStatement, MPPQueryContext context) {
       context.setQueryType(QueryType.WRITE);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ColumnPaginationController.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ColumnPaginationController.java
new file mode 100644
index 0000000000..12430396fe
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ColumnPaginationController.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.plan.analyze;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+
+/** apply MaxQueryDeduplicatedPathNum and SLIMIT & SOFFSET */
+public class ColumnPaginationController {
+
+  private int curLimit =
+      IoTDBDescriptor.getInstance().getConfig().getMaxQueryDeduplicatedPathNum() + 1;
+  private int curOffset;
+
+  // for ALIGN BY DEVICE / DISABLE ALIGN / GROUP BY LEVEL / LAST, controller does is disabled
+  private final boolean isDisabled;
+
+  public ColumnPaginationController(int seriesLimit, int seriesOffset, boolean isDisabled) {
+    // for series limit, the default value is 0, which means no limit
+    this.curLimit = seriesLimit == 0 ? this.curLimit : Math.min(seriesLimit, this.curLimit);
+    // series offset for result set. The default value is 0
+    int seriesOffset1 = this.curOffset = seriesOffset;
+    this.isDisabled = isDisabled;
+  }
+
+  public boolean hasCurOffset() {
+    if (isDisabled) {
+      return false;
+    }
+
+    return curOffset != 0;
+  }
+
+  public boolean hasCurLimit() {
+    if (isDisabled) {
+      return true;
+    }
+
+    return curLimit != 0;
+  }
+
+  public void consumeOffset() {
+    if (isDisabled) {
+      return;
+    }
+
+    curOffset--;
+  }
+
+  public void consumeLimit() {
+    if (isDisabled) {
+      return;
+    }
+
+    curLimit--;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ConcatPathRewriter.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ConcatPathRewriter.java
new file mode 100644
index 0000000000..0c7a8c7e98
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ConcatPathRewriter.java
@@ -0,0 +1,135 @@
+/*
+ * 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.mpp.plan.analyze;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
+import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
+import org.apache.iotdb.db.mpp.plan.statement.Statement;
+import org.apache.iotdb.db.mpp.plan.statement.component.FilterNullComponent;
+import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
+import org.apache.iotdb.db.mpp.plan.statement.component.SelectComponent;
+import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
+import org.apache.iotdb.db.query.expression.Expression;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * This rewriter:
+ *
+ * <p>1. Concat prefix path in SELECT, WHERE, and WITHOUT NULL clause with the suffix path in the
+ * FROM clause.
+ *
+ * <p>2. Construct a {@link PathPatternTree}.
+ */
+public class ConcatPathRewriter {
+
+  private PathPatternTree patternTree;
+
+  public Statement rewrite(Statement statement, PathPatternTree patternTree)
+      throws StatementAnalyzeException {
+    QueryStatement queryStatement = (QueryStatement) statement;
+    this.patternTree = patternTree;
+
+    // prefix paths in the FROM clause
+    List<PartialPath> prefixPaths = queryStatement.getFromComponent().getPrefixPaths();
+
+    // concat SELECT with FROM
+    List<ResultColumn> resultColumns =
+        concatSelectWithFrom(
+            queryStatement.getSelectComponent(), prefixPaths, queryStatement.isGroupByLevel());
+    queryStatement.getSelectComponent().setResultColumns(resultColumns);
+
+    // concat WITHOUT NULL with FROM
+    if (queryStatement.getFilterNullComponent() != null
+        && !queryStatement.getFilterNullComponent().getWithoutNullColumns().isEmpty()) {
+      List<Expression> withoutNullColumns =
+          concatWithoutNullColumnsWithFrom(
+              queryStatement.getFilterNullComponent(),
+              prefixPaths,
+              queryStatement.getSelectComponent().getAliasToColumnMap());
+      queryStatement.getFilterNullComponent().setWithoutNullColumns(withoutNullColumns);
+    }
+
+    // concat WHERE with FROM
+    if (queryStatement.getWhereCondition() != null) {
+      ExpressionAnalyzer.constructPatternTreeFromQueryFilter(
+          queryStatement.getWhereCondition().getPredicate(), prefixPaths, patternTree);
+    }
+    return queryStatement;
+  }
+
+  /**
+   * Concat the prefix path in the SELECT clause and the suffix path in the FROM clause into a full
+   * path pattern. And construct pattern tree.
+   */
+  private List<ResultColumn> concatSelectWithFrom(
+      SelectComponent selectComponent, List<PartialPath> prefixPaths, boolean isGroupByLevel)
+      throws StatementAnalyzeException {
+    // resultColumns after concat
+    List<ResultColumn> resultColumns = new ArrayList<>();
+    for (ResultColumn resultColumn : selectComponent.getResultColumns()) {
+      boolean needAliasCheck = resultColumn.hasAlias() && !isGroupByLevel;
+      List<Expression> resultExpressions =
+          ExpressionAnalyzer.concatExpressionWithSuffixPaths(
+              resultColumn.getExpression(), prefixPaths, patternTree);
+      if (needAliasCheck && resultExpressions.size() > 1) {
+        throw new SemanticException(
+            String.format(
+                "alias '%s' can only be matched with one time series", resultColumn.getAlias()));
+      }
+      resultColumns.addAll(
+          resultExpressions.stream()
+              .map(expression -> new ResultColumn(expression, resultColumn.getAlias()))
+              .collect(Collectors.toList()));
+    }
+    return resultColumns;
+  }
+
+  /**
+   * Concat the prefix path in the WITHOUT NULL clause and the suffix path in the FROM clause into a
+   * full path pattern. And construct pattern tree.
+   */
+  private List<Expression> concatWithoutNullColumnsWithFrom(
+      FilterNullComponent filterNullComponent,
+      List<PartialPath> prefixPaths,
+      Map<String, Expression> aliasToColumnMap)
+      throws StatementAnalyzeException {
+    // raw expression after replace alias
+    List<Expression> rawWithoutNullColumns =
+        filterNullComponent.getWithoutNullColumns().stream()
+            .map(
+                expression ->
+                    aliasToColumnMap.getOrDefault(expression.getExpressionString(), expression))
+            .collect(Collectors.toList());
+
+    // result after concat
+    return rawWithoutNullColumns.stream()
+        .map(
+            expression ->
+                ExpressionAnalyzer.concatExpressionWithSuffixPaths(
+                    expression, prefixPaths, patternTree))
+        .flatMap(List::stream)
+        .collect(Collectors.toList());
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
new file mode 100644
index 0000000000..97e9e11794
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
@@ -0,0 +1,854 @@
+/*
+ * 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.mpp.plan.analyze;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
+import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
+import org.apache.iotdb.db.mpp.common.schematree.SchemaTree;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ExpressionType;
+import org.apache.iotdb.db.query.expression.binary.BinaryExpression;
+import org.apache.iotdb.db.query.expression.binary.EqualToExpression;
+import org.apache.iotdb.db.query.expression.binary.GreaterEqualExpression;
+import org.apache.iotdb.db.query.expression.binary.GreaterThanExpression;
+import org.apache.iotdb.db.query.expression.binary.LessEqualExpression;
+import org.apache.iotdb.db.query.expression.binary.LessThanExpression;
+import org.apache.iotdb.db.query.expression.binary.LogicAndExpression;
+import org.apache.iotdb.db.query.expression.binary.LogicOrExpression;
+import org.apache.iotdb.db.query.expression.binary.NonEqualExpression;
+import org.apache.iotdb.db.query.expression.leaf.ConstantOperand;
+import org.apache.iotdb.db.query.expression.leaf.LeafOperand;
+import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.db.query.expression.leaf.TimestampOperand;
+import org.apache.iotdb.db.query.expression.multi.FunctionExpression;
+import org.apache.iotdb.db.query.expression.unary.InExpression;
+import org.apache.iotdb.db.query.expression.unary.LogicNotExpression;
+import org.apache.iotdb.db.query.expression.unary.UnaryExpression;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.lang.Validate;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils.cartesianProduct;
+import static org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils.constructTimeFilter;
+import static org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils.reconstructBinaryExpressions;
+import static org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils.reconstructFunctionExpressions;
+import static org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils.reconstructTimeSeriesOperands;
+import static org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils.reconstructUnaryExpressions;
+
+public class ExpressionAnalyzer {
+
+  /**
+   * Check if all suffix paths in expression are measurements or one-level wildcards in ALIGN BY
+   * DEVICE query. If not, throw a {@link SemanticException}.
+   *
+   * @param expression expression to be checked
+   */
+  public static void checkIsAllMeasurement(Expression expression) {
+    if (expression instanceof BinaryExpression) {
+      checkIsAllMeasurement(((BinaryExpression) expression).getLeftExpression());
+      checkIsAllMeasurement(((BinaryExpression) expression).getRightExpression());
+    } else if (expression instanceof UnaryExpression) {
+      checkIsAllMeasurement(((UnaryExpression) expression).getExpression());
+    } else if (expression instanceof FunctionExpression) {
+      for (Expression childExpression : expression.getExpressions()) {
+        checkIsAllMeasurement(childExpression);
+      }
+    } else if (expression instanceof TimeSeriesOperand) {
+      PartialPath path = ((TimeSeriesOperand) expression).getPath();
+      if (path.getNodes().length > 1
+          || path.getFullPath().equals(IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD)) {
+        throw new SemanticException(
+            "ALIGN BY DEVICE: the suffix paths can only be measurement or one-level wildcard");
+      }
+    } else if (expression instanceof TimestampOperand || expression instanceof ConstantOperand) {
+      // do nothing
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /**
+   * Check if expression is a built-in aggregation function. If not, throw a {@link
+   * SemanticException}.
+   *
+   * @param expression expression to be checked
+   */
+  public static void checkIsAllAggregation(Expression expression) {
+    if (expression instanceof BinaryExpression) {
+      checkIsAllAggregation(((BinaryExpression) expression).getLeftExpression());
+      checkIsAllAggregation(((BinaryExpression) expression).getRightExpression());
+    } else if (expression instanceof UnaryExpression) {
+      checkIsAllAggregation(((UnaryExpression) expression).getExpression());
+    } else if (expression instanceof FunctionExpression) {
+      if (expression.getExpressions().size() != 1
+          || !(expression.getExpressions().get(0) instanceof TimeSeriesOperand)) {
+        throw new SemanticException(
+            "The argument of the aggregation function must be a time series.");
+      }
+    } else if (expression instanceof TimeSeriesOperand) {
+      throw new SemanticException(
+          "Raw data queries and aggregated queries are not allowed to appear at the same time.");
+    } else if (expression instanceof TimestampOperand || expression instanceof ConstantOperand) {
+      // do nothing
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /**
+   * Concat suffix path in SELECT or WITHOUT NULL clause with the prefix path in the FROM clause.
+   * and Construct a {@link PathPatternTree}.
+   *
+   * @param expression expression in SELECT or WITHOUT NULL clause which may include suffix paths
+   * @param prefixPaths prefix paths in the FROM clause
+   * @param patternTree a PathPatternTree contains all paths to query
+   * @return the concatenated expression list
+   */
+  public static List<Expression> concatExpressionWithSuffixPaths(
+      Expression expression, List<PartialPath> prefixPaths, PathPatternTree patternTree) {
+    if (expression instanceof BinaryExpression) {
+      List<Expression> leftExpressions =
+          concatExpressionWithSuffixPaths(
+              ((BinaryExpression) expression).getLeftExpression(), prefixPaths, patternTree);
+      List<Expression> rightExpressions =
+          concatExpressionWithSuffixPaths(
+              ((BinaryExpression) expression).getRightExpression(), prefixPaths, patternTree);
+      return reconstructBinaryExpressions(
+          expression.getExpressionType(), leftExpressions, rightExpressions);
+    } else if (expression instanceof UnaryExpression) {
+      List<Expression> childExpressions =
+          concatExpressionWithSuffixPaths(
+              ((UnaryExpression) expression).getExpression(), prefixPaths, patternTree);
+      return reconstructUnaryExpressions((UnaryExpression) expression, childExpressions);
+    } else if (expression instanceof FunctionExpression) {
+      List<List<Expression>> extendedExpressions = new ArrayList<>();
+      for (Expression suffixExpression : expression.getExpressions()) {
+        extendedExpressions.add(
+            concatExpressionWithSuffixPaths(suffixExpression, prefixPaths, patternTree));
+      }
+      List<List<Expression>> childExpressionsList = new ArrayList<>();
+      cartesianProduct(extendedExpressions, childExpressionsList, 0, new ArrayList<>());
+      return reconstructFunctionExpressions((FunctionExpression) expression, childExpressionsList);
+    } else if (expression instanceof TimeSeriesOperand) {
+      PartialPath rawPath = ((TimeSeriesOperand) expression).getPath();
+      List<PartialPath> actualPaths = new ArrayList<>();
+      if (rawPath.getFullPath().startsWith(SQLConstant.ROOT + TsFileConstant.PATH_SEPARATOR)) {
+        actualPaths.add(rawPath);
+      } else {
+        for (PartialPath prefixPath : prefixPaths) {
+          PartialPath concatPath = prefixPath.concatPath(rawPath);
+          patternTree.appendPath(concatPath);
+          actualPaths.add(concatPath);
+        }
+      }
+      return reconstructTimeSeriesOperands(actualPaths);
+    } else if (expression instanceof TimestampOperand) {
+      return Collections.singletonList(expression);
+    } else if (expression instanceof ConstantOperand) {
+      return Collections.singletonList(expression);
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /**
+   * Concat suffix path in WHERE clause with the prefix path in the FROM clause and Construct a
+   * {@link PathPatternTree}. This method return void, i.e. will not rewrite the statement.
+   *
+   * @param predicate expression in WHERE clause
+   * @param prefixPaths prefix paths in the FROM clause
+   * @param patternTree a PathPatternTree contains all paths to query
+   */
+  public static void constructPatternTreeFromQueryFilter(
+      Expression predicate, List<PartialPath> prefixPaths, PathPatternTree patternTree) {
+    if (predicate instanceof BinaryExpression) {
+      constructPatternTreeFromQueryFilter(
+          ((BinaryExpression) predicate).getLeftExpression(), prefixPaths, patternTree);
+      constructPatternTreeFromQueryFilter(
+          ((BinaryExpression) predicate).getRightExpression(), prefixPaths, patternTree);
+    } else if (predicate instanceof UnaryExpression) {
+      constructPatternTreeFromQueryFilter(
+          ((UnaryExpression) predicate).getExpression(), prefixPaths, patternTree);
+    } else if (predicate instanceof FunctionExpression) {
+      for (Expression suffixExpression : predicate.getExpressions()) {
+        constructPatternTreeFromQueryFilter(suffixExpression, prefixPaths, patternTree);
+      }
+    } else if (predicate instanceof TimeSeriesOperand) {
+      PartialPath rawPath = ((TimeSeriesOperand) predicate).getPath();
+      if (rawPath.getFullPath().startsWith(SQLConstant.ROOT + TsFileConstant.PATH_SEPARATOR)) {
+        patternTree.appendPath(rawPath);
+        return;
+      }
+      for (PartialPath prefixPath : prefixPaths) {
+        PartialPath concatPath = prefixPath.concatPath(rawPath);
+        patternTree.appendPath(concatPath);
+      }
+    } else if (predicate instanceof TimestampOperand || predicate instanceof ConstantOperand) {
+      // do nothing
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + predicate.getExpressionType());
+    }
+  }
+
+  /**
+   * Bind schema ({@link PartialPath} -> {@link MeasurementPath}) and removes wildcards in
+   * Expression.
+   *
+   * @param schemaTree interface for querying schema information
+   * @return the expression list after binding schema
+   */
+  public static List<Expression> removeWildcardInExpression(
+      Expression expression, SchemaTree schemaTree) {
+    if (expression instanceof BinaryExpression) {
+      List<Expression> leftExpressions =
+          removeWildcardInExpression(
+              ((BinaryExpression) expression).getLeftExpression(), schemaTree);
+      List<Expression> rightExpressions =
+          removeWildcardInExpression(
+              ((BinaryExpression) expression).getRightExpression(), schemaTree);
+      return reconstructBinaryExpressions(
+          expression.getExpressionType(), leftExpressions, rightExpressions);
+    } else if (expression instanceof UnaryExpression) {
+      List<Expression> childExpressions =
+          removeWildcardInExpression(((UnaryExpression) expression).getExpression(), schemaTree);
+      return reconstructUnaryExpressions((UnaryExpression) expression, childExpressions);
+    } else if (expression instanceof FunctionExpression) {
+      // One by one, remove the wildcards from the input expressions. In most cases, an expression
+      // will produce multiple expressions after removing the wildcards. We use extendedExpressions
+      // to collect the produced expressions.
+      List<List<Expression>> extendedExpressions = new ArrayList<>();
+      for (Expression originExpression : expression.getExpressions()) {
+        List<Expression> actualExpressions =
+            removeWildcardInExpression(originExpression, schemaTree);
+        if (actualExpressions.isEmpty()) {
+          // Let's ignore the eval of the function which has at least one non-existence series as
+          // input. See IOTDB-1212: https://github.com/apache/iotdb/pull/3101
+          return Collections.emptyList();
+        }
+        extendedExpressions.add(actualExpressions);
+      }
+
+      // Calculate the Cartesian product of extendedExpressions to get the actual expressions after
+      // removing all wildcards. We use actualExpressions to collect them.
+      List<List<Expression>> childExpressionsList = new ArrayList<>();
+      cartesianProduct(extendedExpressions, childExpressionsList, 0, new ArrayList<>());
+      return reconstructFunctionExpressions((FunctionExpression) expression, childExpressionsList);
+    } else if (expression instanceof TimeSeriesOperand) {
+      PartialPath path = ((TimeSeriesOperand) expression).getPath();
+      List<MeasurementPath> actualPaths = schemaTree.searchMeasurementPaths(path).left;
+      return reconstructTimeSeriesOperands(actualPaths);
+    } else if (expression instanceof TimestampOperand) {
+      return Collections.singletonList(expression);
+    } else if (expression instanceof ConstantOperand) {
+      return Collections.singletonList(expression);
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /**
+   * Concat suffix path in WHERE clause with the prefix path in the FROM clause. And then, bind
+   * schema ({@link PartialPath} -> {@link MeasurementPath}) and removes wildcards in Expression.
+   *
+   * @param prefixPaths prefix paths in the FROM clause
+   * @param schemaTree interface for querying schema information
+   * @param typeProvider a map to record output symbols and their data types
+   * @return the expression list with full path and after binding schema
+   */
+  public static List<Expression> removeWildcardInQueryFilter(
+      Expression predicate,
+      List<PartialPath> prefixPaths,
+      SchemaTree schemaTree,
+      TypeProvider typeProvider) {
+    if (predicate instanceof BinaryExpression) {
+      List<Expression> leftExpressions =
+          removeWildcardInQueryFilter(
+              ((BinaryExpression) predicate).getLeftExpression(),
+              prefixPaths,
+              schemaTree,
+              typeProvider);
+      List<Expression> rightExpressions =
+          removeWildcardInQueryFilter(
+              ((BinaryExpression) predicate).getRightExpression(),
+              prefixPaths,
+              schemaTree,
+              typeProvider);
+      if (predicate.getExpressionType() == ExpressionType.LOGIC_AND) {
+        List<Expression> resultExpressions = new ArrayList<>(leftExpressions);
+        resultExpressions.addAll(rightExpressions);
+        return resultExpressions;
+      }
+      return reconstructBinaryExpressions(
+          predicate.getExpressionType(), leftExpressions, rightExpressions);
+    } else if (predicate instanceof UnaryExpression) {
+      List<Expression> childExpressions =
+          removeWildcardInQueryFilter(
+              ((UnaryExpression) predicate).getExpression(), prefixPaths, schemaTree, typeProvider);
+      return reconstructUnaryExpressions((UnaryExpression) predicate, childExpressions);
+    } else if (predicate instanceof FunctionExpression) {
+      if (predicate.isBuiltInAggregationFunctionExpression()) {
+        throw new SemanticException("aggregate functions are not supported in WHERE clause");
+      }
+      List<List<Expression>> extendedExpressions = new ArrayList<>();
+      for (Expression suffixExpression : predicate.getExpressions()) {
+        extendedExpressions.add(
+            removeWildcardInQueryFilter(suffixExpression, prefixPaths, schemaTree, typeProvider));
+      }
+      List<List<Expression>> childExpressionsList = new ArrayList<>();
+      cartesianProduct(extendedExpressions, childExpressionsList, 0, new ArrayList<>());
+      return reconstructFunctionExpressions((FunctionExpression) predicate, childExpressionsList);
+    } else if (predicate instanceof TimeSeriesOperand) {
+      PartialPath filterPath = ((TimeSeriesOperand) predicate).getPath();
+      List<PartialPath> concatPaths = new ArrayList<>();
+      if (!filterPath.getFirstNode().equals(SQLConstant.ROOT)) {
+        prefixPaths.forEach(prefix -> concatPaths.add(prefix.concatPath(filterPath)));
+      } else {
+        // do nothing in the case of "where root.d1.s1 > 5"
+        concatPaths.add(filterPath);
+      }
+
+      List<PartialPath> noStarPaths = new ArrayList<>();
+      for (PartialPath concatPath : concatPaths) {
+        List<MeasurementPath> actualPaths = schemaTree.searchMeasurementPaths(concatPath).left;
+        if (actualPaths.size() == 0) {
+          throw new SemanticException(
+              String.format("the path '%s' in WHERE clause does not exist", concatPath));
+        }
+        noStarPaths.addAll(actualPaths);
+      }
+      noStarPaths.forEach(path -> typeProvider.setType(path.getFullPath(), path.getSeriesType()));
+      return reconstructTimeSeriesOperands(noStarPaths);
+    } else if (predicate instanceof TimestampOperand) {
+      // do nothing in the case of "where time > 5"
+      return Collections.singletonList(predicate);
+    } else if (predicate instanceof ConstantOperand) {
+      return Collections.singletonList(predicate);
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + predicate.getExpressionType());
+    }
+  }
+
+  /**
+   * Concat measurement in WHERE clause with device path. And then, bind schema ({@link PartialPath}
+   * -> {@link MeasurementPath}) and removes wildcards.
+   *
+   * @param deviceSchemaInfo device path and schema infos of measurements under this device
+   * @param measurementSet
+   * @param typeProvider a map to record output symbols and their data types
+   * @return the expression list with full path and after binding schema
+   */
+  public static List<Expression> removeWildcardInQueryFilterByDevice(
+      Expression predicate,
+      DeviceSchemaInfo deviceSchemaInfo,
+      Set<String> measurementSet,
+      TypeProvider typeProvider) {
+    if (predicate instanceof BinaryExpression) {
+      List<Expression> leftExpressions =
+          removeWildcardInQueryFilterByDevice(
+              ((BinaryExpression) predicate).getLeftExpression(),
+              deviceSchemaInfo,
+              measurementSet,
+              typeProvider);
+      List<Expression> rightExpressions =
+          removeWildcardInQueryFilterByDevice(
+              ((BinaryExpression) predicate).getRightExpression(),
+              deviceSchemaInfo,
+              measurementSet,
+              typeProvider);
+      if (predicate.getExpressionType() == ExpressionType.LOGIC_AND) {
+        List<Expression> resultExpressions = new ArrayList<>(leftExpressions);
+        resultExpressions.addAll(rightExpressions);
+        return resultExpressions;
+      }
+      return reconstructBinaryExpressions(
+          predicate.getExpressionType(), leftExpressions, rightExpressions);
+    } else if (predicate instanceof UnaryExpression) {
+      List<Expression> childExpressions =
+          removeWildcardInQueryFilterByDevice(
+              ((UnaryExpression) predicate).getExpression(),
+              deviceSchemaInfo,
+              measurementSet,
+              typeProvider);
+      return reconstructUnaryExpressions((UnaryExpression) predicate, childExpressions);
+    } else if (predicate instanceof FunctionExpression) {
+      if (predicate.isBuiltInAggregationFunctionExpression()) {
+        throw new SemanticException("aggregate functions are not supported in WHERE clause");
+      }
+      List<List<Expression>> extendedExpressions = new ArrayList<>();
+      for (Expression suffixExpression : predicate.getExpressions()) {
+        extendedExpressions.add(
+            removeWildcardInQueryFilterByDevice(
+                suffixExpression, deviceSchemaInfo, measurementSet, typeProvider));
+      }
+      List<List<Expression>> childExpressionsList = new ArrayList<>();
+      cartesianProduct(extendedExpressions, childExpressionsList, 0, new ArrayList<>());
+      return reconstructFunctionExpressions((FunctionExpression) predicate, childExpressionsList);
+    } else if (predicate instanceof TimeSeriesOperand) {
+      PartialPath filterPath = ((TimeSeriesOperand) predicate).getPath();
+      String measurement = filterPath.getFullPath();
+      List<PartialPath> concatPaths = new ArrayList<>();
+      if (measurement.equals(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)) {
+        concatPaths.addAll(deviceSchemaInfo.getMeasurements(measurementSet));
+      } else {
+        MeasurementPath concatPath = deviceSchemaInfo.getPathByMeasurement(measurement);
+        if (concatPath == null) {
+          throw new SemanticException(
+              String.format(
+                  "ALIGN BY DEVICE: measurement '%s' does not exist in device '%s'",
+                  measurement, deviceSchemaInfo.getDevicePath()));
+        }
+        concatPaths.add(concatPath);
+      }
+      concatPaths.forEach(path -> typeProvider.setType(path.getFullPath(), path.getSeriesType()));
+      return reconstructTimeSeriesOperands(concatPaths);
+    } else if (predicate instanceof TimestampOperand) {
+      // do nothing in the case of "where time > 5"
+      return Collections.singletonList(predicate);
+    } else if (predicate instanceof ConstantOperand) {
+      return Collections.singletonList(predicate);
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + predicate.getExpressionType());
+    }
+  }
+
+  /**
+   * Extract global time filter from query filter.
+   *
+   * @param predicate raw query filter
+   * @return global time filter
+   */
+  public static Pair<Filter, Boolean> transformToGlobalTimeFilter(Expression predicate) {
+    if (predicate instanceof LogicAndExpression) {
+      Pair<Filter, Boolean> leftResultPair =
+          transformToGlobalTimeFilter(((BinaryExpression) predicate).getLeftExpression());
+      Pair<Filter, Boolean> rightResultPair =
+          transformToGlobalTimeFilter(((BinaryExpression) predicate).getRightExpression());
+      if (leftResultPair.left != null && rightResultPair.left != null) {
+        return new Pair<>(
+            FilterFactory.and(leftResultPair.left, rightResultPair.left),
+            leftResultPair.right || rightResultPair.right);
+      } else if (leftResultPair.left != null) {
+        return new Pair<>(leftResultPair.left, true);
+      } else if (rightResultPair.left != null) {
+        return new Pair<>(rightResultPair.left, true);
+      }
+      return new Pair<>(null, true);
+    } else if (predicate instanceof LogicOrExpression) {
+      Pair<Filter, Boolean> leftResultPair =
+          transformToGlobalTimeFilter(((BinaryExpression) predicate).getLeftExpression());
+      Pair<Filter, Boolean> rightResultPair =
+          transformToGlobalTimeFilter(((BinaryExpression) predicate).getRightExpression());
+      if (leftResultPair.left != null && rightResultPair.left != null) {
+        return new Pair<>(
+            FilterFactory.or(leftResultPair.left, rightResultPair.left),
+            leftResultPair.right || rightResultPair.right);
+      }
+      return new Pair<>(null, true);
+    } else if (predicate instanceof LogicNotExpression) {
+      Pair<Filter, Boolean> childResultPair =
+          transformToGlobalTimeFilter(((UnaryExpression) predicate).getExpression());
+      return new Pair<>(FilterFactory.not(childResultPair.left), childResultPair.right);
+    } else if (predicate instanceof GreaterEqualExpression
+        || predicate instanceof GreaterThanExpression
+        || predicate instanceof LessEqualExpression
+        || predicate instanceof LessThanExpression
+        || predicate instanceof EqualToExpression
+        || predicate instanceof NonEqualExpression) {
+      Filter timeInLeftFilter =
+          constructTimeFilter(
+              predicate.getExpressionType(),
+              ((BinaryExpression) predicate).getLeftExpression(),
+              ((BinaryExpression) predicate).getRightExpression());
+      if (timeInLeftFilter != null) {
+        return new Pair<>(timeInLeftFilter, false);
+      }
+      Filter timeInRightFilter =
+          constructTimeFilter(
+              predicate.getExpressionType(),
+              ((BinaryExpression) predicate).getRightExpression(),
+              ((BinaryExpression) predicate).getLeftExpression());
+      if (timeInRightFilter != null) {
+        return new Pair<>(timeInRightFilter, false);
+      }
+      return new Pair<>(null, true);
+    } else if (predicate instanceof InExpression) {
+      Expression timeExpression = ((InExpression) predicate).getExpression();
+      if (timeExpression instanceof TimestampOperand) {
+        return new Pair<>(
+            TimeFilter.in(
+                ((InExpression) predicate)
+                    .getValues().stream().map(Long::parseLong).collect(Collectors.toSet()),
+                ((InExpression) predicate).isNotIn()),
+            false);
+      }
+      return new Pair<>(null, true);
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + predicate.getExpressionType());
+    }
+  }
+
+  /**
+   * Search for subexpressions that can be queried natively, including time series raw data and
+   * built-in aggregate functions.
+   *
+   * @param expression expression to be searched
+   * @param isRawDataSource if true, built-in aggregate functions are not be returned
+   * @return searched subexpression list
+   */
+  public static List<Expression> searchSourceExpressions(
+      Expression expression, boolean isRawDataSource) {
+    if (expression instanceof BinaryExpression) {
+      List<Expression> resultExpressions = new ArrayList<>();
+      resultExpressions.addAll(
+          searchSourceExpressions(
+              ((BinaryExpression) expression).getLeftExpression(), isRawDataSource));
+      resultExpressions.addAll(
+          searchSourceExpressions(
+              ((BinaryExpression) expression).getRightExpression(), isRawDataSource));
+      return resultExpressions;
+    } else if (expression instanceof UnaryExpression) {
+      return searchSourceExpressions(
+          ((UnaryExpression) expression).getExpression(), isRawDataSource);
+    } else if (expression instanceof FunctionExpression) {
+      if (!isRawDataSource && expression.isBuiltInAggregationFunctionExpression()) {
+        return Collections.singletonList(expression);
+      }
+      List<Expression> resultExpressions = new ArrayList<>();
+      for (Expression childExpression : expression.getExpressions()) {
+        resultExpressions.addAll(searchSourceExpressions(childExpression, isRawDataSource));
+      }
+      return resultExpressions;
+    } else if (expression instanceof TimeSeriesOperand) {
+      return Collections.singletonList(expression);
+    } else if (expression instanceof TimestampOperand || expression instanceof ConstantOperand) {
+      return Collections.emptyList();
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /**
+   * Search for built-in aggregate functions subexpressions.
+   *
+   * @param expression expression to be searched
+   * @return searched aggregate functions list
+   */
+  public static List<Expression> searchAggregationExpressions(Expression expression) {
+    if (expression instanceof BinaryExpression) {
+      List<Expression> resultExpressions = new ArrayList<>();
+      resultExpressions.addAll(
+          searchAggregationExpressions(((BinaryExpression) expression).getLeftExpression()));
+      resultExpressions.addAll(
+          searchAggregationExpressions(((BinaryExpression) expression).getRightExpression()));
+      return resultExpressions;
+    } else if (expression instanceof UnaryExpression) {
+      return searchAggregationExpressions(((UnaryExpression) expression).getExpression());
+    } else if (expression instanceof FunctionExpression) {
+      return Collections.singletonList(expression);
+    } else if (expression instanceof LeafOperand) {
+      return Collections.emptyList();
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /** Returns all the timeseries path in the expression */
+  public static Set<PartialPath> collectPaths(Expression expression) {
+    if (expression instanceof BinaryExpression) {
+      Set<PartialPath> resultSet =
+          collectPaths(((BinaryExpression) expression).getLeftExpression());
+      resultSet.addAll(collectPaths(((BinaryExpression) expression).getRightExpression()));
+      return resultSet;
+    } else if (expression instanceof UnaryExpression) {
+      return collectPaths(((UnaryExpression) expression).getExpression());
+    } else if (expression instanceof FunctionExpression) {
+      Set<PartialPath> resultSet = new HashSet<>();
+      for (Expression childExpression : expression.getExpressions()) {
+        resultSet.addAll(collectPaths(childExpression));
+      }
+      return resultSet;
+    } else if (expression instanceof TimeSeriesOperand) {
+      return Sets.newHashSet(((TimeSeriesOperand) expression).getPath());
+    } else if (expression instanceof TimestampOperand || expression instanceof ConstantOperand) {
+      return Collections.emptySet();
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /** Update typeProvider by expression. */
+  public static void updateTypeProvider(Expression expression, TypeProvider typeProvider) {
+    if (expression instanceof BinaryExpression) {
+      updateTypeProvider(((BinaryExpression) expression).getLeftExpression(), typeProvider);
+      updateTypeProvider(((BinaryExpression) expression).getRightExpression(), typeProvider);
+    } else if (expression instanceof UnaryExpression) {
+      updateTypeProvider(((UnaryExpression) expression).getExpression(), typeProvider);
+    } else if (expression instanceof FunctionExpression) {
+      if (expression.isBuiltInAggregationFunctionExpression()) {
+        Validate.isTrue(expression.getExpressions().size() == 1);
+        Expression childExpression = expression.getExpressions().get(0);
+        PartialPath path = ((TimeSeriesOperand) childExpression).getPath();
+        typeProvider.setType(
+            expression.getExpressionString(),
+            SchemaUtils.getSeriesTypeByPath(
+                path, ((FunctionExpression) expression).getFunctionName()));
+        updateTypeProvider(childExpression, typeProvider);
+      } else {
+        for (Expression childExpression : expression.getExpressions()) {
+          updateTypeProvider(childExpression, typeProvider);
+        }
+      }
+    } else if (expression instanceof TimeSeriesOperand) {
+      PartialPath rawPath = ((TimeSeriesOperand) expression).getPath();
+      typeProvider.setType(
+          rawPath.isMeasurementAliasExists()
+              ? rawPath.getFullPathWithAlias()
+              : rawPath.getFullPath(),
+          rawPath.getSeriesType());
+    } else if (expression instanceof ConstantOperand || expression instanceof TimestampOperand) {
+      // do nothing
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /**
+   * Remove alias from expression. eg: root.sg.d1.status + 1 -> root.sg.d1.s2 + 1
+   *
+   * @return expression after removing alias
+   */
+  public static Expression removeAliasFromExpression(Expression expression) {
+    if (expression instanceof BinaryExpression) {
+      Expression leftExpression =
+          removeAliasFromExpression(((BinaryExpression) expression).getLeftExpression());
+      Expression rightExpression =
+          removeAliasFromExpression(((BinaryExpression) expression).getRightExpression());
+      return reconstructBinaryExpressions(
+              expression.getExpressionType(),
+              Collections.singletonList(leftExpression),
+              Collections.singletonList(rightExpression))
+          .get(0);
+    } else if (expression instanceof UnaryExpression) {
+      Expression childExpression =
+          removeAliasFromExpression(((UnaryExpression) expression).getExpression());
+      return reconstructUnaryExpressions(
+              (UnaryExpression) expression, Collections.singletonList(childExpression))
+          .get(0);
+    } else if (expression instanceof FunctionExpression) {
+      List<Expression> childExpressions = new ArrayList<>();
+      for (Expression suffixExpression : expression.getExpressions()) {
+        childExpressions.add(removeAliasFromExpression(suffixExpression));
+      }
+      return reconstructFunctionExpressions(
+              (FunctionExpression) expression, Collections.singletonList(childExpressions))
+          .get(0);
+    } else if (expression instanceof TimeSeriesOperand) {
+      MeasurementPath rawPath = (MeasurementPath) ((TimeSeriesOperand) expression).getPath();
+      if (rawPath.isMeasurementAliasExists()) {
+        MeasurementPath newPath = new MeasurementPath(rawPath, rawPath.getMeasurementSchema());
+        newPath.setUnderAlignedEntity(rawPath.isUnderAlignedEntity());
+        return new TimeSeriesOperand(newPath);
+      }
+      return expression;
+    } else if (expression instanceof ConstantOperand || expression instanceof TimestampOperand) {
+      // do nothing
+      return expression;
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /**
+   * Remove alias from measurement expression according to map. This method is used in ALIGN BY
+   * DEVICE query. eg: status -> s2, sum(status) -> sum(s2)
+   *
+   * @param aliasToMeasurementNameMap a map from alias to measurement name
+   * @return expression after removing alias
+   */
+  public static Expression removeAliasInMeasurementExpression(
+      Expression expression, Map<String, String> aliasToMeasurementNameMap) {
+    if (expression instanceof BinaryExpression) {
+      Expression leftExpression =
+          removeAliasInMeasurementExpression(
+              ((BinaryExpression) expression).getLeftExpression(), aliasToMeasurementNameMap);
+      Expression rightExpression =
+          removeAliasInMeasurementExpression(
+              ((BinaryExpression) expression).getRightExpression(), aliasToMeasurementNameMap);
+      return reconstructBinaryExpressions(
+              expression.getExpressionType(),
+              Collections.singletonList(leftExpression),
+              Collections.singletonList(rightExpression))
+          .get(0);
+    } else if (expression instanceof UnaryExpression) {
+      Expression childExpression =
+          removeAliasInMeasurementExpression(
+              ((UnaryExpression) expression).getExpression(), aliasToMeasurementNameMap);
+      return reconstructUnaryExpressions(
+              (UnaryExpression) expression, Collections.singletonList(childExpression))
+          .get(0);
+    } else if (expression instanceof FunctionExpression) {
+      List<Expression> childExpressions = new ArrayList<>();
+      for (Expression suffixExpression : expression.getExpressions()) {
+        childExpressions.add(
+            removeAliasInMeasurementExpression(suffixExpression, aliasToMeasurementNameMap));
+      }
+      return reconstructFunctionExpressions(
+              (FunctionExpression) expression, Collections.singletonList(childExpressions))
+          .get(0);
+    } else if (expression instanceof TimeSeriesOperand) {
+      String rawMeasurement = ((TimeSeriesOperand) expression).getPath().getFullPath();
+      if (aliasToMeasurementNameMap.containsKey(rawMeasurement)) {
+        PartialPath newPath =
+            new PartialPath(new String[] {aliasToMeasurementNameMap.get(rawMeasurement)});
+        return new TimeSeriesOperand(newPath);
+      }
+      return expression;
+    } else if (expression instanceof ConstantOperand || expression instanceof TimestampOperand) {
+      // do nothing
+      return expression;
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // Method can only be used in source expression
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public static Expression replacePathInSourceExpression(
+      Expression expression, PartialPath replacedPath) {
+    if (expression instanceof TimeSeriesOperand) {
+      return new TimeSeriesOperand(replacedPath);
+    } else if (expression instanceof FunctionExpression) {
+      return new FunctionExpression(
+          ((FunctionExpression) expression).getFunctionName(),
+          ((FunctionExpression) expression).getFunctionAttributes(),
+          Collections.singletonList(new TimeSeriesOperand(replacedPath)));
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  public static Expression replacePathInSourceExpression(
+      Expression expression, String replacedPathString) {
+    PartialPath replacedPath;
+    try {
+      replacedPath = new PartialPath(replacedPathString);
+    } catch (IllegalPathException e) {
+      throw new SemanticException("illegal path: " + replacedPathString);
+    }
+    return replacePathInSourceExpression(expression, replacedPath);
+  }
+
+  public static PartialPath getPathInSourceExpression(Expression expression) {
+    if (expression instanceof TimeSeriesOperand) {
+      return ((TimeSeriesOperand) expression).getPath();
+    } else if (expression instanceof FunctionExpression) {
+      Validate.isTrue(expression.getExpressions().size() == 1);
+      Validate.isTrue(expression.getExpressions().get(0) instanceof TimeSeriesOperand);
+      return ((TimeSeriesOperand) expression.getExpressions().get(0)).getPath();
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  public static String getDeviceNameInSourceExpression(Expression expression) {
+    if (expression instanceof TimeSeriesOperand) {
+      return ((TimeSeriesOperand) expression).getPath().getDeviceIdString();
+    } else if (expression instanceof FunctionExpression) {
+      return getDeviceNameInSourceExpression(expression.getExpressions().get(0));
+    } else {
+      throw new IllegalArgumentException(
+          "unsupported expression type: " + expression.getExpressionType());
+    }
+  }
+
+  public static Pair<Expression, String> getMeasurementWithAliasInSourceExpression(
+      Expression expression, String alias) {
+    if (expression instanceof TimeSeriesOperand) {
+      String measurement = ((TimeSeriesOperand) expression).getPath().getMeasurement();
+      if (alias != null && measurement.equals(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)) {
+        throw new SemanticException(
+            String.format(
+                "ALIGN BY DEVICE: alias '%s' can only be matched with one measurement", alias));
+      }
+      Expression measurementExpression;
+      try {
+        measurementExpression = new TimeSeriesOperand(new PartialPath(measurement));
+        return new Pair<>(measurementExpression, alias);
+      } catch (IllegalPathException e) {
+        throw new SemanticException("ALIGN BY DEVICE: illegal measurement name: " + measurement);
+      }
+    } else if (expression instanceof FunctionExpression) {
+      if (expression.getExpressions().size() > 1) {
+        throw new SemanticException(
+            "ALIGN BY DEVICE: prefix path in SELECT clause can only be one measurement or one-layer wildcard.");
+      }
+      Expression measurementFunctionExpression =
+          new FunctionExpression(
+              ((FunctionExpression) expression).getFunctionName(),
+              ((FunctionExpression) expression).getFunctionAttributes(),
+              Collections.singletonList(
+                  getMeasurementWithAliasInSourceExpression(
+                          expression.getExpressions().get(0), alias)
+                      .left));
+      return new Pair<>(measurementFunctionExpression, alias);
+    } else {
+      throw new SemanticException(
+          "ALIGN BY DEVICE: prefix path in SELECT clause can only be one measurement or one-layer wildcard.");
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionUtils.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionUtils.java
new file mode 100644
index 0000000000..2618d6ab6e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionUtils.java
@@ -0,0 +1,242 @@
+/*
+ * 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.mpp.plan.analyze;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.ExpressionType;
+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.EqualToExpression;
+import org.apache.iotdb.db.query.expression.binary.GreaterEqualExpression;
+import org.apache.iotdb.db.query.expression.binary.GreaterThanExpression;
+import org.apache.iotdb.db.query.expression.binary.LessEqualExpression;
+import org.apache.iotdb.db.query.expression.binary.LessThanExpression;
+import org.apache.iotdb.db.query.expression.binary.LogicAndExpression;
+import org.apache.iotdb.db.query.expression.binary.LogicOrExpression;
+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.NonEqualExpression;
+import org.apache.iotdb.db.query.expression.binary.SubtractionExpression;
+import org.apache.iotdb.db.query.expression.leaf.ConstantOperand;
+import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.db.query.expression.leaf.TimestampOperand;
+import org.apache.iotdb.db.query.expression.multi.FunctionExpression;
+import org.apache.iotdb.db.query.expression.unary.InExpression;
+import org.apache.iotdb.db.query.expression.unary.LikeExpression;
+import org.apache.iotdb.db.query.expression.unary.LogicNotExpression;
+import org.apache.iotdb.db.query.expression.unary.NegationExpression;
+import org.apache.iotdb.db.query.expression.unary.RegularExpression;
+import org.apache.iotdb.db.query.expression.unary.UnaryExpression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ExpressionUtils {
+
+  public static List<Expression> reconstructTimeSeriesOperands(
+      List<? extends PartialPath> actualPaths) {
+    List<Expression> resultExpressions = new ArrayList<>();
+    for (PartialPath actualPath : actualPaths) {
+      resultExpressions.add(new TimeSeriesOperand(actualPath));
+    }
+    return resultExpressions;
+  }
+
+  public static List<Expression> reconstructFunctionExpressions(
+      FunctionExpression expression, List<List<Expression>> childExpressionsList) {
+    List<Expression> resultExpressions = new ArrayList<>();
+    for (List<Expression> functionExpressions : childExpressionsList) {
+      resultExpressions.add(
+          new FunctionExpression(
+              expression.getFunctionName(),
+              expression.getFunctionAttributes(),
+              functionExpressions));
+    }
+    return resultExpressions;
+  }
+
+  public static List<Expression> reconstructUnaryExpressions(
+      UnaryExpression expression, List<Expression> childExpressions) {
+    List<Expression> resultExpressions = new ArrayList<>();
+    for (Expression childExpression : childExpressions) {
+      switch (expression.getExpressionType()) {
+        case IN:
+          resultExpressions.add(
+              new InExpression(
+                  childExpression,
+                  ((InExpression) expression).isNotIn(),
+                  ((InExpression) expression).getValues()));
+          break;
+        case LIKE:
+          resultExpressions.add(
+              new LikeExpression(
+                  childExpression,
+                  ((LikeExpression) expression).getPatternString(),
+                  ((LikeExpression) expression).getPattern()));
+          break;
+        case LOGIC_NOT:
+          resultExpressions.add(new LogicNotExpression(childExpression));
+          break;
+        case NEGATION:
+          resultExpressions.add(new NegationExpression(childExpression));
+          break;
+        case REGEXP:
+          resultExpressions.add(
+              new RegularExpression(
+                  childExpression,
+                  ((RegularExpression) expression).getPatternString(),
+                  ((RegularExpression) expression).getPattern()));
+          break;
+        default:
+          throw new IllegalArgumentException(
+              "unsupported expression type: " + expression.getExpressionType());
+      }
+    }
+    return resultExpressions;
+  }
+
+  public static List<Expression> reconstructBinaryExpressions(
+      ExpressionType expressionType,
+      List<Expression> leftExpressions,
+      List<Expression> rightExpressions) {
+    List<Expression> resultExpressions = new ArrayList<>();
+    for (Expression le : leftExpressions) {
+      for (Expression re : rightExpressions) {
+        switch (expressionType) {
+          case ADDITION:
+            resultExpressions.add(new AdditionExpression(le, re));
+            break;
+          case SUBTRACTION:
+            resultExpressions.add(new SubtractionExpression(le, re));
+            break;
+          case MULTIPLICATION:
+            resultExpressions.add(new MultiplicationExpression(le, re));
+            break;
+          case DIVISION:
+            resultExpressions.add(new DivisionExpression(le, re));
+            break;
+          case MODULO:
+            resultExpressions.add(new ModuloExpression(le, re));
+            break;
+          case LESS_THAN:
+            resultExpressions.add(new LessThanExpression(le, re));
+            break;
+          case LESS_EQUAL:
+            resultExpressions.add(new LessEqualExpression(le, re));
+            break;
+          case GREATER_THAN:
+            resultExpressions.add(new GreaterThanExpression(le, re));
+            break;
+          case GREATER_EQUAL:
+            resultExpressions.add(new GreaterEqualExpression(le, re));
+            break;
+          case EQUAL_TO:
+            resultExpressions.add(new EqualToExpression(le, re));
+            break;
+          case NON_EQUAL:
+            resultExpressions.add(new NonEqualExpression(le, re));
+            break;
+          case LOGIC_AND:
+            resultExpressions.add(new LogicAndExpression(le, re));
+            break;
+          case LOGIC_OR:
+            resultExpressions.add(new LogicOrExpression(le, re));
+            break;
+          default:
+            throw new IllegalArgumentException("unsupported expression type: " + expressionType);
+        }
+      }
+    }
+    return resultExpressions;
+  }
+
+  public static <T> void cartesianProduct(
+      List<List<T>> dimensionValue, List<List<T>> resultList, int layer, List<T> currentList) {
+    if (layer < dimensionValue.size() - 1) {
+      if (dimensionValue.get(layer).isEmpty()) {
+        cartesianProduct(dimensionValue, resultList, layer + 1, currentList);
+      } else {
+        for (int i = 0; i < dimensionValue.get(layer).size(); i++) {
+          List<T> list = new ArrayList<>(currentList);
+          list.add(dimensionValue.get(layer).get(i));
+          cartesianProduct(dimensionValue, resultList, layer + 1, list);
+        }
+      }
+    } else if (layer == dimensionValue.size() - 1) {
+      if (dimensionValue.get(layer).isEmpty()) {
+        resultList.add(currentList);
+      } else {
+        for (int i = 0; i < dimensionValue.get(layer).size(); i++) {
+          List<T> list = new ArrayList<>(currentList);
+          list.add(dimensionValue.get(layer).get(i));
+          resultList.add(list);
+        }
+      }
+    }
+  }
+
+  public static Filter constructTimeFilter(
+      ExpressionType expressionType, Expression timeExpression, Expression valueExpression) {
+    if (timeExpression instanceof TimestampOperand
+        && valueExpression instanceof ConstantOperand
+        && ((ConstantOperand) valueExpression).getDataType() == TSDataType.INT64) {
+      long value = Long.parseLong(((ConstantOperand) valueExpression).getValueString());
+      switch (expressionType) {
+        case LESS_THAN:
+          return TimeFilter.lt(value);
+        case LESS_EQUAL:
+          return TimeFilter.ltEq(value);
+        case GREATER_THAN:
+          return TimeFilter.gt(value);
+        case GREATER_EQUAL:
+          return TimeFilter.gtEq(value);
+        case EQUAL_TO:
+          return TimeFilter.eq(value);
+        case NON_EQUAL:
+          return TimeFilter.notEq(value);
+        default:
+          throw new IllegalArgumentException("unsupported expression type: " + expressionType);
+      }
+    }
+    return null;
+  }
+
+  public static Expression constructQueryFilter(List<Expression> expressions) {
+    if (expressions.size() == 1) {
+      return expressions.get(0);
+    }
+    return ExpressionUtils.constructBinaryFilterTreeWithAnd(expressions);
+  }
+
+  private static Expression constructBinaryFilterTreeWithAnd(List<Expression> expressions) {
+    // TODO: consider AVL tree
+    if (expressions.size() == 2) {
+      return new LogicAndExpression(expressions.get(0), expressions.get(1));
+    } else {
+      return new LogicAndExpression(
+          expressions.get(0),
+          constructBinaryFilterTreeWithAnd(expressions.subList(1, expressions.size())));
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/GroupByLevelController.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/GroupByLevelController.java
new file mode 100644
index 0000000000..7be09fd68e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/GroupByLevelController.java
@@ -0,0 +1,144 @@
+/*
+ * 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.mpp.plan.analyze;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.db.query.expression.multi.FunctionExpression;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class is used to control the row number of group by level query. For example, selected
+ * series[root.sg.d1.s1, root.sg.d2.s1, root.sg2.d1.s1], level = 1; the result rows will be
+ * [root.sg.*.s1, root.sg2.*.s1], sLimit and sOffset will be used to control the result numbers
+ * rather than the selected series.
+ */
+public class GroupByLevelController {
+
+  private final int[] levels;
+
+  /** count(root.sg.d1.s1) with level = 1 -> count(root.*.d1.s1) */
+  private final Map<Expression, Set<Expression>> groupedPathMap;
+
+  /** count(root.*.d1.s1) -> alias */
+  private final Map<String, String> columnToAliasMap;
+
+  /**
+   * Only used to check whether one alisa is corresponding to only one column. i.e. Different
+   * columns can't have the same alias.
+   */
+  private final Map<String, String> aliasToColumnMap;
+
+  public GroupByLevelController(int[] levels) {
+    this.levels = levels;
+    this.groupedPathMap = new LinkedHashMap<>();
+    this.columnToAliasMap = new HashMap<>();
+    this.aliasToColumnMap = new HashMap<>();
+  }
+
+  public void control(Expression expression, String alias) {
+    if (!(expression instanceof FunctionExpression
+        && expression.isBuiltInAggregationFunctionExpression())) {
+      throw new SemanticException(expression + " can't be used in group by level.");
+    }
+
+    PartialPath rawPath = ((TimeSeriesOperand) expression.getExpressions().get(0)).getPath();
+    PartialPath groupedPath = generatePartialPathByLevel(rawPath.getNodes(), levels);
+    Expression groupedExpression =
+        new FunctionExpression(
+            ((FunctionExpression) expression).getFunctionName(),
+            ((FunctionExpression) expression).getFunctionAttributes(),
+            Collections.singletonList(new TimeSeriesOperand(groupedPath)));
+    groupedPathMap.computeIfAbsent(groupedExpression, key -> new HashSet<>()).add(expression);
+
+    if (alias != null) {
+      checkAliasAndUpdateAliasMap(alias, groupedExpression.getExpressionString());
+    }
+  }
+
+  private void checkAliasAndUpdateAliasMap(String alias, String groupedExpressionString)
+      throws StatementAnalyzeException {
+    // If an alias is corresponding to more than one result column, throw an exception
+    if (columnToAliasMap.get(groupedExpressionString) == null) {
+      if (aliasToColumnMap.get(alias) != null) {
+        throw new StatementAnalyzeException(
+            String.format("alias '%s' can only be matched with one result column", alias));
+      } else {
+        columnToAliasMap.put(groupedExpressionString, alias);
+        aliasToColumnMap.put(alias, groupedExpressionString);
+      }
+      // If a result column is corresponding to more than one alias, throw an exception
+    } else if (!columnToAliasMap.get(groupedExpressionString).equals(alias)) {
+      throw new StatementAnalyzeException(
+          String.format(
+              "Result column %s with more than one alias[%s, %s]",
+              groupedExpressionString, columnToAliasMap.get(groupedExpressionString), alias));
+    }
+  }
+
+  /**
+   * Transform an originalPath to a partial path that satisfies given level. Path nodes don't
+   * satisfy the given level will be replaced by "*" except the sensor level, e.g.
+   * generatePartialPathByLevel("root.sg.dh.d1.s1", 2) will return "root.*.dh.*.s1".
+   *
+   * <p>Especially, if count(*), then the sensor level will be replaced by "*" too.
+   *
+   * @return result partial path
+   */
+  public PartialPath generatePartialPathByLevel(String[] nodes, int[] pathLevels) {
+    Set<Integer> levelSet = new HashSet<>();
+    for (int level : pathLevels) {
+      levelSet.add(level);
+    }
+
+    List<String> transformedNodes = new ArrayList<>(nodes.length);
+
+    transformedNodes.add(nodes[0]);
+    for (int k = 1; k < nodes.length - 1; k++) {
+      if (levelSet.contains(k)) {
+        transformedNodes.add(nodes[k]);
+      } else {
+        transformedNodes.add(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD);
+      }
+    }
+    transformedNodes.add(nodes[nodes.length - 1]);
+    return new PartialPath(transformedNodes.toArray(new String[0]));
+  }
+
+  public Map<Expression, Set<Expression>> getGroupedPathMap() {
+    return groupedPathMap;
+  }
+
+  public String getAlias(String columnName) {
+    return columnToAliasMap.get(columnName) != null ? columnToAliasMap.get(columnName) : null;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/TypeProvider.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/TypeProvider.java
index 32d5cee70c..34d8acfc9a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/TypeProvider.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/TypeProvider.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 
 public class TypeProvider {
 
@@ -40,11 +41,10 @@ public class TypeProvider {
   }
 
   public TSDataType getType(String path) {
-    TSDataType type = typeMap.get(path);
-    if (type == null) {
+    if (!typeMap.containsKey(path)) {
       throw new StatementAnalyzeException(String.format("no data type found for path: %s", path));
     }
-    return type;
+    return typeMap.get(path);
   }
 
   public void setType(String path, TSDataType dataType) {
@@ -74,4 +74,21 @@ public class TypeProvider {
     }
     return new TypeProvider(typeMap);
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TypeProvider that = (TypeProvider) o;
+    return Objects.equals(typeMap, that.typeMap);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(typeMap);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index b26cfe5d1f..75c5580d4d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -25,15 +25,11 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.sql.SQLParserException;
 import org.apache.iotdb.db.exception.sql.SemanticException;
-import org.apache.iotdb.db.index.common.IndexType;
 import org.apache.iotdb.db.mpp.common.filter.BasicFunctionFilter;
-import org.apache.iotdb.db.mpp.common.filter.InFilter;
-import org.apache.iotdb.db.mpp.common.filter.LikeFilter;
 import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
-import org.apache.iotdb.db.mpp.common.filter.RegexpFilter;
-import org.apache.iotdb.db.mpp.plan.constant.FilterConstant;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.db.mpp.plan.statement.component.FillComponent;
+import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
 import org.apache.iotdb.db.mpp.plan.statement.component.FilterNullComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.FilterNullPolicy;
 import org.apache.iotdb.db.mpp.plan.statement.component.FromComponent;
@@ -44,15 +40,13 @@ import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
 import org.apache.iotdb.db.mpp.plan.statement.component.ResultSetFormat;
 import org.apache.iotdb.db.mpp.plan.statement.component.SelectComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition;
-import org.apache.iotdb.db.mpp.plan.statement.crud.AggregationQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.FillQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.GroupByFillQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.GroupByQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.LastQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.UDAFQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.UDTFQueryStatement;
+import org.apache.iotdb.db.mpp.plan.statement.literal.BooleanLiteral;
+import org.apache.iotdb.db.mpp.plan.statement.literal.DoubleLiteral;
+import org.apache.iotdb.db.mpp.plan.statement.literal.Literal;
+import org.apache.iotdb.db.mpp.plan.statement.literal.LongLiteral;
+import org.apache.iotdb.db.mpp.plan.statement.literal.StringLiteral;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.AlterTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountDevicesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountLevelTimeSeriesStatement;
@@ -64,7 +58,6 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.CreateTimeSeriesStatement
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowDevicesStatement;
-import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTimeSeriesStatement;
@@ -81,10 +74,6 @@ import org.apache.iotdb.db.qp.sql.IoTDBSqlParser.CountTimeseriesContext;
 import org.apache.iotdb.db.qp.sql.IoTDBSqlParser.ExpressionContext;
 import org.apache.iotdb.db.qp.sql.IoTDBSqlParserBaseVisitor;
 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.executor.fill.ValueFill;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.query.expression.binary.AdditionExpression;
 import org.apache.iotdb.db.query.expression.binary.DivisionExpression;
@@ -118,21 +107,13 @@ import org.apache.commons.lang.StringEscapeUtils;
 
 import java.time.ZoneId;
 import java.util.ArrayList;
-import java.util.EnumMap;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Set;
 import java.util.regex.Pattern;
 
-import static org.apache.iotdb.db.index.common.IndexConstant.PATTERN;
-import static org.apache.iotdb.db.index.common.IndexConstant.THRESHOLD;
-import static org.apache.iotdb.db.index.common.IndexConstant.TOP_K;
-import static org.apache.iotdb.db.qp.constant.SQLConstant.TIME_PATH;
-
 /** Parse AST to Statement. */
 public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
@@ -149,17 +130,10 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   // TODO: add comment
   private ZoneId zoneId;
 
-  // TODO: add comment
-  private IoTDBConstant.ClientVersion clientVersion = IoTDBConstant.ClientVersion.V_0_13;
-
   public void setZoneId(ZoneId zoneId) {
     this.zoneId = zoneId;
   }
 
-  public void setClientVersion(IoTDBConstant.ClientVersion clientVersion) {
-    this.clientVersion = clientVersion;
-  }
-
   /** Top Level Description */
   @Override
   public Statement visitSingleStatement(IoTDBSqlParser.SingleStatementContext ctx) {
@@ -167,10 +141,6 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     if (ctx.DEBUG() != null) {
       statement.setDebug(true);
     }
-    if (statement instanceof ShowStatement) {
-      ((ShowStatement) statement)
-          .setPrefixPath(IoTDBConstant.ClientVersion.V_0_12.equals(clientVersion));
-    }
     return statement;
   }
 
@@ -512,24 +482,22 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
   @Override
   public Statement visitSelectStatement(IoTDBSqlParser.SelectStatementContext ctx) {
-    // visit special clause first to initialize different query statement
-    if (ctx.specialClause() != null) {
-      queryStatement = (QueryStatement) visit(ctx.specialClause());
-    }
-
-    // there is no special clause in query statement
-    if (queryStatement == null) {
-      queryStatement = new QueryStatement();
-    }
+    // initialize query statement
+    queryStatement = new QueryStatement();
 
-    // parser select, from, where clauses
+    // parser select, from
     parseSelectClause(ctx.selectClause());
     parseFromClause(ctx.fromClause());
+
+    // parse where clause
     if (ctx.whereClause() != null) {
       WhereCondition whereCondition = parseWhereClause(ctx.whereClause());
-      if (whereCondition != null) {
-        queryStatement.setWhereCondition(whereCondition);
-      }
+      queryStatement.setWhereCondition(whereCondition);
+    }
+
+    // parser special clause
+    if (ctx.specialClause() != null) {
+      queryStatement = (QueryStatement) visit(ctx.specialClause());
     }
     return queryStatement;
   }
@@ -539,62 +507,30 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   public void parseSelectClause(IoTDBSqlParser.SelectClauseContext ctx) {
     SelectComponent selectComponent = new SelectComponent(zoneId);
 
-    // parse TOP_N
-    if (ctx.topClause() != null) {
-      // TODO: parse info of top clause into selectOp
-      visitTopClause(ctx.topClause());
-    }
-
     // parse LAST
     if (ctx.LAST() != null) {
-      queryStatement = new LastQueryStatement(queryStatement);
+      selectComponent.setHasLast(true);
     }
 
     // parse resultColumn
-    Set<String> aliasSet = new HashSet<>();
+    Map<String, Expression> aliasToColumnMap = new HashMap<>();
     for (IoTDBSqlParser.ResultColumnContext resultColumnContext : ctx.resultColumn()) {
       ResultColumn resultColumn = parseResultColumn(resultColumnContext);
       if (resultColumn.hasAlias()) {
-        aliasSet.add(resultColumn.getAlias());
+        String alias = resultColumn.getAlias();
+        if (aliasToColumnMap.containsKey(alias)) {
+          throw new SemanticException("duplicate alias in select clause");
+        }
+        aliasToColumnMap.put(alias, resultColumn.getExpression());
       }
       selectComponent.addResultColumn(resultColumn);
     }
-    selectComponent.setAliasSet(aliasSet);
-
-    // judge query type
-    if (!(queryStatement instanceof GroupByQueryStatement)
-        && !(queryStatement instanceof FillQueryStatement)
-        && !(queryStatement instanceof LastQueryStatement)
-        && !(queryStatement instanceof AggregationQueryStatement)
-        && !(queryStatement instanceof UDTFQueryStatement)
-        && !(queryStatement instanceof UDAFQueryStatement)) {
-      if (selectComponent.isHasUserDefinedAggregationFunction()) {
-        queryStatement = new UDAFQueryStatement(new AggregationQueryStatement(queryStatement));
-      } else if (selectComponent.isHasBuiltInAggregationFunction()) {
-        queryStatement = new AggregationQueryStatement(queryStatement);
-      } else if (selectComponent.isHasTimeSeriesGeneratingFunction()) {
-        queryStatement = new UDTFQueryStatement(queryStatement);
-      }
-    } else if (selectComponent.isHasUserDefinedAggregationFunction()) {
-      queryStatement = new UDAFQueryStatement((AggregationQueryStatement) (queryStatement));
-    }
+    selectComponent.setAliasToColumnMap(aliasToColumnMap);
 
     // set selectComponent
     queryStatement.setSelectComponent(selectComponent);
   }
 
-  @Override
-  public Statement visitTopClause(IoTDBSqlParser.TopClauseContext ctx) {
-    int top = Integer.parseInt(ctx.INTEGER_LITERAL().getText());
-    if (top <= 0 || top > 1000) {
-      throw new SemanticException(
-          String.format(
-              "TOP <N>: N should be greater than 0 and less than 1000, current N is %d", top));
-    }
-    queryStatement.addProp(TOP_K, top);
-    return queryStatement;
-  }
-
   private ResultColumn parseResultColumn(IoTDBSqlParser.ResultColumnContext resultColumnContext) {
     Expression expression = parseExpression(resultColumnContext.expression(), false);
     if (expression.isConstantOperand()) {
@@ -622,21 +558,15 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   // Where Clause
 
   public WhereCondition parseWhereClause(IoTDBSqlParser.WhereClauseContext ctx) {
-    if (ctx.indexPredicateClause() != null) {
-      parseIndexPredicate(ctx.indexPredicateClause());
-      return null;
-    }
-    QueryFilter queryFilter = new QueryFilter();
-    queryFilter.addChildOperator(parseOrExpression(ctx.orExpression()));
-    return new WhereCondition(queryFilter.getChildren().get(0));
+    Expression predicate =
+        parseExpression(ctx.expression(), ctx.expression().OPERATOR_NOT() == null);
+    return new WhereCondition(predicate);
   }
 
   // Group By Time Clause
 
   @Override
   public Statement visitGroupByTimeStatement(IoTDBSqlParser.GroupByTimeStatementContext ctx) {
-    queryStatement = new GroupByQueryStatement();
-
     // parse group by time clause
     parseGroupByTimeClause(ctx.groupByTimeClause());
 
@@ -687,11 +617,16 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
         levels[i] = Integer.parseInt(ctx.INTEGER_LITERAL().get(i).getText());
       }
       groupByLevelComponent.setLevels(levels);
-      ((AggregationQueryStatement) queryStatement).setGroupByLevelComponent(groupByLevelComponent);
+      queryStatement.setGroupByLevelComponent(groupByLevelComponent);
+    }
+
+    // parse fill clause
+    if (ctx.fillClause() != null) {
+      parseFillClause(ctx.fillClause());
     }
 
     // set groupByTimeComponent
-    ((GroupByQueryStatement) queryStatement).setGroupByTimeComponent(groupByTimeComponent);
+    queryStatement.setGroupByTimeComponent(groupByTimeComponent);
   }
 
   /** parse time range (startTime and endTime) in group by query. */
@@ -728,8 +663,6 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   // Group By Level Clause
   @Override
   public Statement visitGroupByLevelStatement(IoTDBSqlParser.GroupByLevelStatementContext ctx) {
-    queryStatement = new AggregationQueryStatement();
-
     // parse GroupByLevel clause
     parseGroupByLevelClause(ctx.groupByLevelClause());
 
@@ -752,66 +685,22 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       levels[i] = Integer.parseInt(ctx.INTEGER_LITERAL().get(i).getText());
     }
     groupByLevelComponent.setLevels(levels);
-    ((AggregationQueryStatement) queryStatement).setGroupByLevelComponent(groupByLevelComponent);
-  }
-
-  // Fill Clause
-
-  @Override
-  public Statement visitFillStatement(IoTDBSqlParser.FillStatementContext ctx) {
-    queryStatement = new FillQueryStatement();
 
     // parse fill clause
-    parseFillClause(ctx.fillClause());
-
-    // parse slimit & soffset
-    if (ctx.slimitClause() != null) {
-      parseSlimitClause(ctx.slimitClause());
+    if (ctx.fillClause() != null) {
+      parseFillClause(ctx.fillClause());
     }
 
-    // parse AlignByDevice or DisableAlign
-    if (ctx.alignByDeviceClauseOrDisableAlign() != null) {
-      parseAlignByDeviceClauseOrDisableAlign(ctx.alignByDeviceClauseOrDisableAlign());
-    }
-
-    return queryStatement;
-  }
-
-  public void parseFillClause(IoTDBSqlParser.FillClauseContext ctx) {
-    FillComponent fillComponent = new FillComponent();
-    if (ctx.oldTypeClause().size() > 0) {
-      // old type fill logic
-      List<IoTDBSqlParser.OldTypeClauseContext> list = ctx.oldTypeClause();
-      Map<TSDataType, IFill> fillTypes = new EnumMap<>(TSDataType.class);
-      for (IoTDBSqlParser.OldTypeClauseContext typeClause : list) {
-        if (typeClause.ALL() != null) {
-          if (typeClause.linearClause() != null) {
-            throw new SemanticException("fill all doesn't support linear fill");
-          }
-          parseAllTypeClause(typeClause, fillTypes);
-          break;
-        } else {
-          parsePrimitiveTypeClause(typeClause, fillTypes);
-        }
-      }
-      fillComponent.setFillTypes(fillTypes);
-    } else {
-      // new single fill logic
-      fillComponent.setSingleFill(getSingleIFill(ctx));
-    }
-    ((FillQueryStatement) queryStatement).setFillComponent(fillComponent);
+    queryStatement.setGroupByLevelComponent(groupByLevelComponent);
   }
 
-  // Group By Time with Fill Clause
-
+  // Fill Clause
   @Override
-  public Statement visitGroupByFillStatement(IoTDBSqlParser.GroupByFillStatementContext ctx) {
-    queryStatement = new GroupByFillQueryStatement();
-
-    // parse GroupByTime with FIll
-    parseGroupByFillClause(ctx.groupByFillClause());
+  public Statement visitFillStatement(IoTDBSqlParser.FillStatementContext ctx) {
+    // parse fill
+    parseFillClause(ctx.fillClause());
 
-    // parse OrderByTime
+    // parse order by time
     if (ctx.orderByTimeClause() != null) {
       parseOrderByTimeClause(ctx.orderByTimeClause());
     }
@@ -820,215 +709,41 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     if (ctx.specialLimit() != null) {
       return visit(ctx.specialLimit());
     }
-
     return queryStatement;
   }
 
-  private void parseGroupByFillClause(IoTDBSqlParser.GroupByFillClauseContext ctx) {
-    GroupByTimeComponent groupByTimeComponent = new GroupByTimeComponent();
+  public void parseFillClause(IoTDBSqlParser.FillClauseContext ctx) {
     FillComponent fillComponent = new FillComponent();
-
-    // parse time range (start time & end time)
-    parseTimeRange(ctx.timeRange(), groupByTimeComponent);
-    groupByTimeComponent.setLeftCRightO(ctx.timeRange().LS_BRACKET() != null);
-
-    // parse time interval
-    groupByTimeComponent.setInterval(
-        parseTimeIntervalOrSlidingStep(
-            ctx.DURATION_LITERAL(0).getText(), true, groupByTimeComponent));
-
-    // parse sliding step
-    if (ctx.DURATION_LITERAL().size() == 2) {
-      groupByTimeComponent.setSlidingStep(
-          parseTimeIntervalOrSlidingStep(
-              ctx.DURATION_LITERAL(1).getText(), false, groupByTimeComponent));
-    } else {
-      groupByTimeComponent.setSlidingStep(groupByTimeComponent.getInterval());
-      groupByTimeComponent.setSlidingStepByMonth(groupByTimeComponent.isIntervalByMonth());
-    }
-
-    if (ctx.fillClause().oldTypeClause().size() > 0) {
-      // old type fill logic
-      List<IoTDBSqlParser.OldTypeClauseContext> list = ctx.fillClause().oldTypeClause();
-      Map<TSDataType, IFill> fillTypes = new EnumMap<>(TSDataType.class);
-      for (IoTDBSqlParser.OldTypeClauseContext typeClause : list) {
-        if (typeClause.ALL() != null) {
-          parseAllTypeClause(typeClause, fillTypes);
-        } else {
-          parsePrimitiveTypeClause(typeClause, fillTypes);
-        }
-      }
-
-      int usePrevious = 0;
-      int useLinear = 0;
-      int useValue = 0;
-      for (IFill iFill : fillTypes.values()) {
-        if (iFill instanceof PreviousFill) {
-          usePrevious = 1;
-        }
-        if (iFill instanceof LinearFill) {
-          useLinear = 1;
-        }
-        if (iFill instanceof ValueFill) {
-          useValue = 1;
-        }
-      }
-      if (usePrevious + useLinear + useValue > 1) {
-        throw new SemanticException("The old type logic could only use one type of fill");
-      }
-
-      fillComponent.setFillTypes(fillTypes);
-    } else {
-      fillComponent.setSingleFill(getSingleIFill(ctx.fillClause()));
-    }
-
-    ((GroupByFillQueryStatement) queryStatement).setGroupByTimeComponent(groupByTimeComponent);
-    ((GroupByFillQueryStatement) queryStatement).setFillComponent(fillComponent);
-  }
-
-  private IFill getSingleIFill(IoTDBSqlParser.FillClauseContext ctx) {
-    int defaultFillInterval = IoTDBDescriptor.getInstance().getConfig().getDefaultFillInterval();
-    if (ctx.linearClause() != null) { // linear
-      if (ctx.linearClause().DURATION_LITERAL(0) != null) {
-        String beforeStr = ctx.linearClause().DURATION_LITERAL(0).getText();
-        String afterStr = ctx.linearClause().DURATION_LITERAL(1).getText();
-        return new LinearFill(beforeStr, afterStr);
-      } else {
-        return new LinearFill(defaultFillInterval, defaultFillInterval);
-      }
-    } else if (ctx.previousClause() != null) { // previous
-      if (ctx.previousClause().DURATION_LITERAL() != null) {
-        String preRangeStr = ctx.previousClause().DURATION_LITERAL().getText();
-        return new PreviousFill(preRangeStr);
-      } else {
-        return new PreviousFill(defaultFillInterval);
-      }
-    } else if (ctx.specificValueClause() != null) { // value
-      if (ctx.specificValueClause().constant() != null) {
-        return new ValueFill(ctx.specificValueClause().constant().getText());
-      } else {
-        throw new SemanticException("fill value cannot be null");
-      }
-    } else if (ctx.previousUntilLastClause() != null) { // previous until last
-      if (ctx.previousUntilLastClause().DURATION_LITERAL() != null) {
-        String preRangeStr = ctx.previousUntilLastClause().DURATION_LITERAL().getText();
-        return new PreviousFill(preRangeStr, true);
-      } else {
-        return new PreviousFill(defaultFillInterval, true);
-      }
-    } else {
-      throw new SemanticException("unknown single fill type");
-    }
-  }
-
-  private void parseAllTypeClause(
-      IoTDBSqlParser.OldTypeClauseContext ctx, Map<TSDataType, IFill> fillTypes) {
-    IFill fill;
-    int defaultFillInterval = IoTDBDescriptor.getInstance().getConfig().getDefaultFillInterval();
-
-    if (ctx.linearClause() != null) { // linear
-      if (ctx.linearClause().DURATION_LITERAL(0) != null) {
-        String beforeStr = ctx.linearClause().DURATION_LITERAL(0).getText();
-        String afterStr = ctx.linearClause().DURATION_LITERAL(1).getText();
-        fill = new LinearFill(beforeStr, afterStr);
-      } else {
-        fill = new LinearFill(defaultFillInterval, defaultFillInterval);
-      }
-    } else if (ctx.previousClause() != null) { // previous
-      if (ctx.previousClause().DURATION_LITERAL() != null) {
-        String preRangeStr = ctx.previousClause().DURATION_LITERAL().getText();
-        fill = new PreviousFill(preRangeStr);
-      } else {
-        fill = new PreviousFill(defaultFillInterval);
-      }
+    if (ctx.linearClause() != null) {
+      fillComponent.setFillPolicy(FillPolicy.LINEAR);
+    } else if (ctx.previousClause() != null) {
+      fillComponent.setFillPolicy(FillPolicy.PREVIOUS);
     } else if (ctx.specificValueClause() != null) {
-      throw new SemanticException("fill all doesn't support value fill");
-    } else { // previous until last
-      if (ctx.previousUntilLastClause().DURATION_LITERAL() != null) {
-        String preRangeStr = ctx.previousUntilLastClause().DURATION_LITERAL().getText();
-        fill = new PreviousFill(preRangeStr, true);
-      } else {
-        fill = new PreviousFill(defaultFillInterval, true);
-      }
-    }
-
-    for (TSDataType tsDataType : TSDataType.values()) {
-      if (fill instanceof LinearFill
-          && (tsDataType == TSDataType.BOOLEAN || tsDataType == TSDataType.TEXT)) {
-        continue;
-      }
-      fillTypes.put(tsDataType, fill.copy());
-    }
-  }
-
-  private void parsePrimitiveTypeClause(
-      IoTDBSqlParser.OldTypeClauseContext ctx, Map<TSDataType, IFill> fillTypes) {
-    TSDataType dataType = parseType(ctx.dataType.getText());
-    if (dataType == TSDataType.VECTOR) {
-      throw new SemanticException(String.format("type %s cannot use fill function", dataType));
-    }
-
-    if (ctx.linearClause() != null
-        && (dataType == TSDataType.TEXT || dataType == TSDataType.BOOLEAN)) {
-      throw new SemanticException(
-          String.format(
-              "type %s cannot use %s fill function",
-              dataType, ctx.linearClause().LINEAR().getText()));
-    }
-
-    int defaultFillInterval = IoTDBDescriptor.getInstance().getConfig().getDefaultFillInterval();
-
-    if (ctx.linearClause() != null) { // linear
-      if (ctx.linearClause().DURATION_LITERAL(0) != null) {
-        String beforeRangeStr = ctx.linearClause().DURATION_LITERAL(0).getText();
-        String afterRangeStr = ctx.linearClause().DURATION_LITERAL(1).getText();
-        LinearFill fill = new LinearFill(beforeRangeStr, afterRangeStr);
-        fillTypes.put(dataType, fill);
-      } else {
-        fillTypes.put(dataType, new LinearFill(defaultFillInterval, defaultFillInterval));
-      }
-    } else if (ctx.previousClause() != null) { // previous
-      if (ctx.previousClause().DURATION_LITERAL() != null) {
-        String beforeStr = ctx.previousClause().DURATION_LITERAL().getText();
-        fillTypes.put(dataType, new PreviousFill(beforeStr));
-      } else {
-        fillTypes.put(dataType, new PreviousFill(defaultFillInterval));
-      }
-    } else if (ctx.specificValueClause() != null) { // value
+      fillComponent.setFillPolicy(FillPolicy.VALUE);
       if (ctx.specificValueClause().constant() != null) {
-        fillTypes.put(
-            dataType, new ValueFill(ctx.specificValueClause().constant().getText(), dataType));
+        Literal fillValue = parseLiteral(ctx.specificValueClause().constant());
+        fillComponent.setFillValue(fillValue);
       } else {
         throw new SemanticException("fill value cannot be null");
       }
-    } else { // previous until last
-      if (ctx.previousUntilLastClause().DURATION_LITERAL() != null) {
-        String preRangeStr = ctx.previousUntilLastClause().DURATION_LITERAL().getText();
-        fillTypes.put(dataType, new PreviousFill(preRangeStr, true));
-      } else {
-        fillTypes.put(dataType, new PreviousFill(defaultFillInterval, true));
-      }
     }
+    queryStatement.setFillComponent(fillComponent);
   }
 
-  // parse DataType
-  private TSDataType parseType(String datatype) {
-    String type = datatype.toLowerCase();
-    switch (type) {
-      case "int32":
-        return TSDataType.INT32;
-      case "int64":
-        return TSDataType.INT64;
-      case "float":
-        return TSDataType.FLOAT;
-      case "double":
-        return TSDataType.DOUBLE;
-      case "boolean":
-        return TSDataType.BOOLEAN;
-      case "text":
-        return TSDataType.TEXT;
-      default:
-        throw new SemanticException("not a valid fill type : " + type);
+  private Literal parseLiteral(ConstantContext constantContext) {
+    String text = constantContext.getText();
+    if (constantContext.BOOLEAN_LITERAL() != null) {
+      return new BooleanLiteral(text);
+    } else if (constantContext.STRING_LITERAL() != null) {
+      return new StringLiteral(parseStringLiteral(text));
+    } else if (constantContext.INTEGER_LITERAL() != null) {
+      return new LongLiteral(text);
+    } else if (constantContext.realLiteral() != null) {
+      return new DoubleLiteral(text);
+    } else if (constantContext.dateExpression() != null) {
+      return new LongLiteral(parseDateExpression(constantContext.dateExpression()));
+    } else {
+      throw new SQLParserException("Unsupported constant operand: " + text);
     }
   }
 
@@ -1040,10 +755,6 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
   @Override
   public Statement visitLimitStatement(IoTDBSqlParser.LimitStatementContext ctx) {
-    if (queryStatement == null) {
-      queryStatement = new QueryStatement();
-    }
-
     // parse LIMIT
     parseLimitClause(ctx.limitClause(), queryStatement);
 
@@ -1107,10 +818,6 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
   @Override
   public Statement visitSlimitStatement(IoTDBSqlParser.SlimitStatementContext ctx) {
-    if (queryStatement == null) {
-      queryStatement = new QueryStatement();
-    }
-
     // parse SLIMIT
     parseSlimitClause(ctx.slimitClause());
 
@@ -1164,10 +871,6 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
   @Override
   public Statement visitWithoutNullStatement(IoTDBSqlParser.WithoutNullStatementContext ctx) {
-    if (queryStatement == null) {
-      queryStatement = new QueryStatement();
-    }
-
     // parse WITHOUT NULL
     parseWithoutNullClause(ctx.withoutNullClause());
 
@@ -1214,8 +917,6 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
   @Override
   public Statement visitOrderByTimeStatement(IoTDBSqlParser.OrderByTimeStatementContext ctx) {
-    queryStatement = new QueryStatement();
-
     // parse ORDER BY TIME
     parseOrderByTimeClause(ctx.orderByTimeClause());
 
@@ -1238,9 +939,6 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   @Override
   public Statement visitAlignByDeviceClauseOrDisableAlignStatement(
       IoTDBSqlParser.AlignByDeviceClauseOrDisableAlignStatementContext ctx) {
-    if (queryStatement == null) {
-      queryStatement = new QueryStatement();
-    }
     parseAlignByDeviceClauseOrDisableAlign(ctx.alignByDeviceClauseOrDisableAlign());
     return queryStatement;
   }
@@ -1795,7 +1493,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       if (context.OPERATOR_LTE() != null) {
         return new LessEqualExpression(leftExpression, rightExpression);
       }
-      if (context.OPERATOR_DEQ() != null) {
+      if (context.OPERATOR_DEQ() != null || context.OPERATOR_SEQ() != null) {
         return new EqualToExpression(leftExpression, rightExpression);
       }
       if (context.OPERATOR_NEQ() != null) {
@@ -1856,25 +1554,27 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       if (!subexpression.isConstantOperand()) {
         hasNonPureConstantSubExpression = true;
       }
-      functionExpression.addExpression(subexpression);
+      if (subexpression instanceof EqualToExpression
+          && ((EqualToExpression) subexpression).getLeftExpression().isConstantOperand()
+          && ((EqualToExpression) subexpression).getRightExpression().isConstantOperand()) {
+        // parse attribute
+        functionExpression.addAttribute(
+            ((ConstantOperand) ((EqualToExpression) subexpression).getLeftExpression())
+                .getValueString(),
+            ((ConstantOperand) ((EqualToExpression) subexpression).getRightExpression())
+                .getValueString());
+      } else {
+        functionExpression.addExpression(subexpression);
+      }
     }
 
     // It is not allowed to have function expressions like F(1, 1.0). There should be at least one
     // non-pure-constant sub-expression, otherwise the timestamp of the row cannot be inferred.
     if (!hasNonPureConstantSubExpression) {
-      throw new SQLParserException(
+      throw new SemanticException(
           "Invalid function expression, all the arguments are constant operands: "
               + functionClause.getText());
     }
-
-    // attributes
-    for (IoTDBSqlParser.FunctionAttributeContext functionAttribute :
-        functionClause.functionAttribute()) {
-      functionExpression.addAttribute(
-          parseAttributeKey(functionAttribute.attributeKey()),
-          parseAttributeValue(functionAttribute.attributeValue()));
-    }
-
     return functionExpression;
   }
 
@@ -1894,216 +1594,42 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     Expression childExpression = parseExpression(context.unaryBeforeInExpression, inWithoutNull);
     LinkedHashSet<String> values = new LinkedHashSet<>();
     for (ConstantContext constantContext : context.constant()) {
-      String text = constantContext.getText();
-      values.add(constantContext.STRING_LITERAL() != null ? parseStringLiteral(text) : text);
+      values.add(parseConstant(constantContext));
     }
     return new InExpression(childExpression, context.OPERATOR_NOT() != null, values);
   }
 
-  private Expression parseConstantOperand(ConstantContext constantContext) {
-    try {
-      String text = constantContext.getText();
-      if (clientVersion.equals(IoTDBConstant.ClientVersion.V_0_13)) {
-        if (constantContext.BOOLEAN_LITERAL() != null) {
-          return new ConstantOperand(TSDataType.BOOLEAN, text);
-        } else if (constantContext.STRING_LITERAL() != null) {
-          return new ConstantOperand(TSDataType.TEXT, parseStringLiteral(text));
-        } else if (constantContext.INTEGER_LITERAL() != null) {
-          return new ConstantOperand(TSDataType.INT64, text);
-        } else if (constantContext.realLiteral() != null) {
-          return new ConstantOperand(TSDataType.DOUBLE, text);
-        } else {
-          throw new SQLParserException("Unsupported constant operand: " + text);
-        }
-      } else if (clientVersion.equals(IoTDBConstant.ClientVersion.V_0_12)) {
-        // if client version is before 0.13, node name in expression may be a constant
-        return new TimeSeriesOperand(convertConstantToPath(text));
-      } else {
-        throw new UnsupportedOperationException();
-      }
-    } catch (IllegalPathException e) {
-      throw new SQLParserException(e.getMessage());
-    }
-  }
-
-  private QueryFilter parseOrExpression(IoTDBSqlParser.OrExpressionContext ctx) {
-    if (ctx.andExpression().size() == 1) {
-      return parseAndExpression(ctx.andExpression(0));
-    }
-    QueryFilter binaryOp = new QueryFilter(FilterConstant.FilterType.KW_OR);
-    if (ctx.andExpression().size() > 2) {
-      binaryOp.addChildOperator(parseAndExpression(ctx.andExpression(0)));
-      binaryOp.addChildOperator(parseAndExpression(ctx.andExpression(1)));
-      for (int i = 2; i < ctx.andExpression().size(); i++) {
-        QueryFilter op = new QueryFilter(FilterConstant.FilterType.KW_OR);
-        op.addChildOperator(binaryOp);
-        op.addChildOperator(parseAndExpression(ctx.andExpression(i)));
-        binaryOp = op;
-      }
+  private String parseConstant(ConstantContext constantContext) {
+    String text = constantContext.getText();
+    if (constantContext.BOOLEAN_LITERAL() != null
+        || constantContext.INTEGER_LITERAL() != null
+        || constantContext.realLiteral() != null) {
+      return text;
+    } else if (constantContext.STRING_LITERAL() != null) {
+      return parseStringLiteral(text);
+    } else if (constantContext.dateExpression() != null) {
+      return String.valueOf(parseDateExpression(constantContext.dateExpression()));
     } else {
-      for (IoTDBSqlParser.AndExpressionContext andExpressionContext : ctx.andExpression()) {
-        binaryOp.addChildOperator(parseAndExpression(andExpressionContext));
-      }
+      throw new IllegalArgumentException("Unsupported constant operand: " + text);
     }
-    return binaryOp;
   }
 
-  private QueryFilter parseAndExpression(IoTDBSqlParser.AndExpressionContext ctx) {
-    if (ctx.predicate().size() == 1) {
-      return parsePredicate(ctx.predicate(0));
-    }
-    QueryFilter binaryOp = new QueryFilter(FilterConstant.FilterType.KW_AND);
-    int size = ctx.predicate().size();
-    if (size > 2) {
-      binaryOp.addChildOperator(parsePredicate(ctx.predicate(0)));
-      binaryOp.addChildOperator(parsePredicate(ctx.predicate(1)));
-      for (int i = 2; i < size; i++) {
-        QueryFilter op = new QueryFilter(FilterConstant.FilterType.KW_AND);
-        op.addChildOperator(binaryOp);
-        op.addChildOperator(parsePredicate(ctx.predicate(i)));
-        binaryOp = op;
-      }
-    } else {
-      for (IoTDBSqlParser.PredicateContext predicateContext : ctx.predicate()) {
-        binaryOp.addChildOperator(parsePredicate(predicateContext));
-      }
-    }
-    return binaryOp;
-  }
-
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private QueryFilter parsePredicate(IoTDBSqlParser.PredicateContext ctx) {
-    PartialPath path = null;
-    if (ctx.OPERATOR_NOT() != null) {
-      QueryFilter notOp = new QueryFilter(FilterConstant.FilterType.KW_NOT);
-      notOp.addChildOperator(parseOrExpression(ctx.orExpression()));
-      return notOp;
-    } else if (ctx.LR_BRACKET() != null && ctx.OPERATOR_NOT() == null) {
-      return parseOrExpression(ctx.orExpression());
-    } else if (ctx.REGEXP() != null || ctx.LIKE() != null) {
-      if (ctx.suffixPath() != null) {
-        path = parseSuffixPath(ctx.suffixPath());
-      } else if (ctx.fullPath() != null) {
-        path = parseFullPath(ctx.fullPath());
-      }
-      if (path == null) {
-        throw new SQLParserException("Path is null, please check the sql.");
-      }
-      return ctx.REGEXP() != null
-          ? new RegexpFilter(FilterConstant.FilterType.REGEXP, path, ctx.STRING_LITERAL().getText())
-          : new LikeFilter(FilterConstant.FilterType.LIKE, path, ctx.STRING_LITERAL().getText());
-    } else {
-      if (ctx.TIME() != null || ctx.TIMESTAMP() != null) {
-        path = new PartialPath(SQLConstant.getSingleTimeArray());
-      }
-      if (ctx.fullPath() != null) {
-        path = parseFullPath(ctx.fullPath());
-      }
-      if (ctx.suffixPath() != null) {
-        path = parseSuffixPath(ctx.suffixPath());
-      }
-      if (path == null) {
-        throw new SQLParserException("Path is null, please check the sql.");
-      }
-      if (ctx.inClause() != null) {
-        return parseInOperator(ctx.inClause(), path);
-      } else {
-        return parseBasicFunctionOperator(ctx, path);
-      }
-    }
-  }
-
-  private QueryFilter parseBasicFunctionOperator(
-      IoTDBSqlParser.PredicateContext ctx, PartialPath path) {
-    BasicFunctionFilter basic;
-    if (ctx.constant().dateExpression() != null) {
-      if (!path.equals(TIME_PATH)) {
-        throw new SQLParserException(path.getFullPath(), "Date can only be used to time");
-      }
-      basic =
-          new BasicFunctionFilter(
-              FilterConstant.lexerToFilterType.get(ctx.comparisonOperator().type.getType()),
-              path,
-              Long.toString(parseDateExpression(ctx.constant().dateExpression())));
-    } else {
-      basic =
-          new BasicFunctionFilter(
-              FilterConstant.lexerToFilterType.get(ctx.comparisonOperator().type.getType()),
-              path,
-              parseStringLiteral(ctx.constant().getText()));
-    }
-    return basic;
-  }
-
-  private QueryFilter parseInOperator(IoTDBSqlParser.InClauseContext ctx, PartialPath path) {
-    Set<String> values = new HashSet<>();
-    boolean not = ctx.OPERATOR_NOT() != null;
-    for (IoTDBSqlParser.ConstantContext constant : ctx.constant()) {
-      if (constant.dateExpression() != null) {
-        if (!path.equals(TIME_PATH)) {
-          throw new SQLParserException(path.getFullPath(), "Date can only be used to time");
-        }
-        values.add(Long.toString(parseDateExpression(constant.dateExpression())));
-      } else {
-        values.add(constant.getText());
-      }
-    }
-    return new InFilter(FilterConstant.FilterType.IN, path, not, values);
-  }
-
-  private void parseIndexPredicate(IoTDBSqlParser.IndexPredicateClauseContext ctx) {
-    Map<String, Object> props;
-    PartialPath path;
-    if (ctx.suffixPath() != null) {
-      path = parseSuffixPath(ctx.suffixPath());
-    } else {
-      path = parseFullPath(ctx.fullPath());
-    }
-    if (ctx.LIKE() != null) {
-      // whole matching case
-      if (queryStatement.getSelectComponent().getResultColumns().size() != 1) {
-        throw new SemanticException("Index query statement allows only one select path");
-      }
-      if (!path.equals(
-          queryStatement
-              .getSelectComponent()
-              .getResultColumns()
-              .get(0)
-              .getExpression()
-              .toString())) {
-        throw new SemanticException(
-            "In the index query statement, "
-                + "the path in select element and the index predicate should be same");
-      }
-      queryStatement.addProp(PATTERN, parseSequence(ctx.sequenceClause(0)));
-      queryStatement.setIndexType(IndexType.RTREE_PAA);
-    } else if (ctx.CONTAIN() != null) {
-      // subsequence matching case
-      List<double[]> compositePattern = new ArrayList<>();
-      List<Double> thresholds = new ArrayList<>();
-      for (int i = 0; i < ctx.sequenceClause().size(); i++) {
-        compositePattern.add(parseSequence(ctx.sequenceClause(i)));
-        thresholds.add(Double.parseDouble(ctx.constant(i).getText()));
-      }
-
-      List<ResultColumn> resultColumns = new ArrayList<>();
-      resultColumns.add(new ResultColumn(new TimeSeriesOperand(path)));
-      queryStatement.getSelectComponent().setResultColumns(resultColumns);
-      queryStatement.addProp(PATTERN, compositePattern);
-      queryStatement.addProp(THRESHOLD, thresholds);
-      queryStatement.setIndexType(IndexType.ELB_INDEX);
+  private Expression parseConstantOperand(ConstantContext constantContext) {
+    String text = constantContext.getText();
+    if (constantContext.BOOLEAN_LITERAL() != null) {
+      return new ConstantOperand(TSDataType.BOOLEAN, text);
+    } else if (constantContext.STRING_LITERAL() != null) {
+      return new ConstantOperand(TSDataType.TEXT, parseStringLiteral(text));
+    } else if (constantContext.INTEGER_LITERAL() != null) {
+      return new ConstantOperand(TSDataType.INT64, text);
+    } else if (constantContext.realLiteral() != null) {
+      return new ConstantOperand(TSDataType.DOUBLE, text);
+    } else if (constantContext.dateExpression() != null) {
+      return new ConstantOperand(
+          TSDataType.INT64, String.valueOf(parseDateExpression(constantContext.dateExpression())));
     } else {
-      throw new SQLParserException("Unknown index predicate: " + ctx);
-    }
-  }
-
-  private double[] parseSequence(IoTDBSqlParser.SequenceClauseContext ctx) {
-    int seqLen = ctx.constant().size();
-    double[] sequence = new double[seqLen];
-    for (int i = 0; i < seqLen; i++) {
-      sequence[i] = Double.parseDouble(ctx.constant(i).getText());
+      throw new SQLParserException("Unsupported constant operand: " + text);
     }
-    return sequence;
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
index ee86e7f259..5a4eeb0153 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.db.mpp.plan.parser;
 
-import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
@@ -36,7 +35,6 @@ import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsOfOneDeviceStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.LastQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CreateTimeSeriesStatement;
 import org.apache.iotdb.db.qp.sql.IoTDBSqlParser;
@@ -76,13 +74,8 @@ import static org.apache.iotdb.commons.conf.IoTDBConstant.TIME;
 /** Convert SQL and RPC requests to {@link Statement}. */
 public class StatementGenerator {
 
-  public static Statement createStatement(
-      String sql, ZoneId zoneId, IoTDBConstant.ClientVersion clientVersion) {
-    return invokeParser(sql, zoneId, clientVersion);
-  }
-
   public static Statement createStatement(String sql, ZoneId zoneId) {
-    return invokeParser(sql, zoneId, IoTDBConstant.ClientVersion.V_0_13);
+    return invokeParser(sql, zoneId);
   }
 
   public static Statement createStatement(TSRawDataQueryReq rawDataQueryReq, ZoneId zoneId)
@@ -121,7 +114,7 @@ public class StatementGenerator {
             Long.toString(rawDataQueryReq.getEndTime()));
     queryFilter.addChildOperator(left);
     queryFilter.addChildOperator(right);
-    whereCondition.setQueryFilter(queryFilter);
+    //    whereCondition.setQueryFilter(queryFilter);
 
     queryStatement.setSelectComponent(selectComponent);
     queryStatement.setFromComponent(fromComponent);
@@ -132,11 +125,13 @@ public class StatementGenerator {
   public static Statement createStatement(TSLastDataQueryReq lastDataQueryReq, ZoneId zoneId)
       throws IllegalPathException {
     // construct query statement
-    LastQueryStatement lastQueryStatement = new LastQueryStatement();
+    QueryStatement lastQueryStatement = new QueryStatement();
     SelectComponent selectComponent = new SelectComponent(zoneId);
     FromComponent fromComponent = new FromComponent();
     WhereCondition whereCondition = new WhereCondition();
 
+    selectComponent.setHasLast(true);
+
     // iterate the path list and add it to from operator
     for (String pathStr : lastDataQueryReq.getPaths()) {
       PartialPath path = new PartialPath(pathStr);
@@ -151,7 +146,7 @@ public class StatementGenerator {
             FilterConstant.FilterType.GREATERTHANOREQUALTO,
             timePath,
             Long.toString(lastDataQueryReq.getTime()));
-    whereCondition.setQueryFilter(basicFunctionFilter);
+    //    whereCondition.setQueryFilter(basicFunctionFilter);
 
     lastQueryStatement.setSelectComponent(selectComponent);
     lastQueryStatement.setFromComponent(fromComponent);
@@ -342,11 +337,9 @@ public class StatementGenerator {
     return statement;
   }
 
-  private static Statement invokeParser(
-      String sql, ZoneId zoneId, IoTDBConstant.ClientVersion clientVersion) {
+  private static Statement invokeParser(String sql, ZoneId zoneId) {
     ASTVisitor astVisitor = new ASTVisitor();
     astVisitor.setZoneId(zoneId);
-    astVisitor.setClientVersion(clientVersion);
 
     CharStream charStream1 = CharStreams.fromString(sql);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
index dba6c1fbe9..62a8b7f83e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
@@ -40,6 +40,8 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.OffsetNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.TimeJoinNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.SeriesScanNode;
 import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -64,16 +66,20 @@ public class LogicalPlanBuilder {
   }
 
   public LogicalPlanBuilder planRawDataQuerySource(
-      Map<String, Set<PartialPath>> deviceNameToPathsMap,
+      Map<String, Set<Expression>> deviceNameToPathsMap,
       OrderBy scanOrder,
       boolean isAlignByDevice) {
     Map<String, List<PlanNode>> deviceNameToSourceNodesMap = new HashMap<>();
 
-    for (Map.Entry<String, Set<PartialPath>> entry : deviceNameToPathsMap.entrySet()) {
+    for (Map.Entry<String, Set<Expression>> entry : deviceNameToPathsMap.entrySet()) {
       String deviceName = entry.getKey();
       Set<String> allSensors =
-          entry.getValue().stream().map(PartialPath::getMeasurement).collect(Collectors.toSet());
-      for (PartialPath path : entry.getValue()) {
+          entry.getValue().stream()
+              .map(expression -> ((TimeSeriesOperand) expression).getPath())
+              .map(PartialPath::getMeasurement)
+              .collect(Collectors.toSet());
+      for (Expression expression : entry.getValue()) {
+        PartialPath path = ((TimeSeriesOperand) expression).getPath();
         deviceNameToSourceNodesMap
             .computeIfAbsent(deviceName, k -> new ArrayList<>())
             .add(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanner.java
index 6a7041325e..064c59971f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanner.java
@@ -33,19 +33,12 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowsOfOneDevic
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
 import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.crud.AggregationQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.FillQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.GroupByFillQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.GroupByQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertMultiTabletsStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsOfOneDeviceStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.LastQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.UDAFQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.UDTFQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.AlterTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountDevicesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountLevelTimeSeriesStatement;
@@ -105,7 +98,7 @@ public class LogicalPlanner {
     public PlanNode visitQuery(QueryStatement queryStatement, MPPQueryContext context) {
       return new LogicalPlanBuilder(context)
           .planRawDataQuerySource(
-              queryStatement.getDeviceNameToDeduplicatedPathsMap(),
+              analysis.getSourceExpressions(),
               queryStatement.getResultOrder(),
               queryStatement.isAlignByDevice())
           .planOffset(queryStatement.getRowOffset())
@@ -113,44 +106,6 @@ public class LogicalPlanner {
           .getRoot();
     }
 
-    @Override
-    public PlanNode visitAggregationQuery(
-        AggregationQueryStatement queryStatement, MPPQueryContext context) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public PlanNode visitGroupByQuery(
-        GroupByQueryStatement queryStatement, MPPQueryContext context) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public PlanNode visitGroupByFillQuery(
-        GroupByFillQueryStatement queryStatement, MPPQueryContext context) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public PlanNode visitFillQuery(FillQueryStatement queryStatement, MPPQueryContext context) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public PlanNode visitLastQuery(LastQueryStatement queryStatement, MPPQueryContext context) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public PlanNode visitUDTFQuery(UDTFQueryStatement queryStatement, MPPQueryContext context) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public PlanNode visitUDAFQuery(UDAFQueryStatement queryStatement, MPPQueryContext context) {
-      throw new UnsupportedOperationException();
-    }
-
     @Override
     public PlanNode visitCreateTimeseries(
         CreateTimeSeriesStatement createTimeSeriesStatement, MPPQueryContext context) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/SimpleFragmentParallelPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/SimpleFragmentParallelPlanner.java
index 27f8ba0968..5b06be36fe 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/SimpleFragmentParallelPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/SimpleFragmentParallelPlanner.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeUtil;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.ExchangeNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.sink.FragmentSinkNode;
-import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 import java.util.ArrayList;
@@ -83,10 +82,7 @@ public class SimpleFragmentParallelPlanner implements IFragmentParallelPlaner {
     // If one PlanFragment will produce several FragmentInstance, the instanceIdx will be increased
     // one by one
     PlanNode rootCopy = PlanNodeUtil.deepCopy(fragment.getRoot());
-    Filter timeFilter =
-        analysis.getQueryFilter() == null
-            ? null
-            : ((GlobalTimeExpression) analysis.getQueryFilter()).getFilter();
+    Filter timeFilter = analysis.getGlobalTimeFilter();
     FragmentInstance fragmentInstance =
         new FragmentInstance(
             new PlanFragment(fragment.getId(), rootCopy),
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/WriteFragmentParallelPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/WriteFragmentParallelPlanner.java
index ffa1b7c4a5..6925194e9c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/WriteFragmentParallelPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/WriteFragmentParallelPlanner.java
@@ -26,7 +26,6 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.PlanFragment;
 import org.apache.iotdb.db.mpp.plan.planner.plan.SubPlan;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.WritePlanNode;
-import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 import java.util.ArrayList;
@@ -48,10 +47,7 @@ public class WriteFragmentParallelPlanner implements IFragmentParallelPlaner {
   @Override
   public List<FragmentInstance> parallelPlan() {
     PlanFragment fragment = subPlan.getPlanFragment();
-    Filter timeFilter =
-        analysis.getQueryFilter() != null
-            ? ((GlobalTimeExpression) analysis.getQueryFilter()).getFilter()
-            : null;
+    Filter timeFilter = analysis.getGlobalTimeFilter();
     PlanNode node = fragment.getRoot();
     if (!(node instanceof WritePlanNode)) {
       throw new IllegalArgumentException("PlanNode should be IWritePlanNode in WRITE operation");
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FillNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FillNode.java
index 2a166e9e63..72b6c1cdf9 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FillNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FillNode.java
@@ -23,12 +23,10 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FillDescriptor;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import com.google.common.collect.ImmutableList;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 
@@ -36,7 +34,7 @@ import java.util.Objects;
 public class FillNode extends ProcessNode {
 
   // descriptions of how null values are filled
-  private List<FillDescriptor> fillDescriptorList;
+  private FillDescriptor fillDescriptor;
 
   private PlanNode child;
 
@@ -44,13 +42,13 @@ public class FillNode extends ProcessNode {
     super(id);
   }
 
-  public FillNode(PlanNodeId id, List<FillDescriptor> fillDescriptorList) {
+  public FillNode(PlanNodeId id, FillDescriptor fillDescriptor) {
     this(id);
-    this.fillDescriptorList = fillDescriptorList;
+    this.fillDescriptor = fillDescriptor;
   }
 
-  public FillNode(PlanNodeId id, PlanNode child, List<FillDescriptor> fillDescriptorList) {
-    this(id, fillDescriptorList);
+  public FillNode(PlanNodeId id, PlanNode child, FillDescriptor fillDescriptor) {
+    this(id, fillDescriptor);
     this.child = child;
   }
 
@@ -71,7 +69,7 @@ public class FillNode extends ProcessNode {
 
   @Override
   public PlanNode clone() {
-    return new FillNode(getPlanNodeId(), fillDescriptorList);
+    return new FillNode(getPlanNodeId(), fillDescriptor);
   }
 
   @Override
@@ -87,21 +85,13 @@ public class FillNode extends ProcessNode {
   @Override
   protected void serializeAttributes(ByteBuffer byteBuffer) {
     PlanNodeType.FILL.serialize(byteBuffer);
-    ReadWriteIOUtils.write(fillDescriptorList.size(), byteBuffer);
-    for (FillDescriptor fillDescriptor : fillDescriptorList) {
-      fillDescriptor.serialize(byteBuffer);
-    }
+    fillDescriptor.serialize(byteBuffer);
   }
 
   public static FillNode deserialize(ByteBuffer byteBuffer) {
-    int fillDescriptorsSize = ReadWriteIOUtils.readInt(byteBuffer);
-    List<FillDescriptor> fillDescriptorList = new ArrayList<>();
-    while (fillDescriptorsSize > 0) {
-      fillDescriptorList.add(FillDescriptor.deserialize(byteBuffer));
-      fillDescriptorsSize--;
-    }
+    FillDescriptor fillDescriptor = FillDescriptor.deserialize(byteBuffer);
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
-    return new FillNode(planNodeId, fillDescriptorList);
+    return new FillNode(planNodeId, fillDescriptor);
   }
 
   @Override
@@ -116,11 +106,12 @@ public class FillNode extends ProcessNode {
       return false;
     }
     FillNode fillNode = (FillNode) o;
-    return fillDescriptorList.equals(fillNode.fillDescriptorList) && child.equals(fillNode.child);
+    return Objects.equals(fillDescriptor, fillNode.fillDescriptor)
+        && Objects.equals(child, fillNode.child);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(super.hashCode(), fillDescriptorList, child);
+    return Objects.hash(super.hashCode(), fillDescriptor, child);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
index 26d91956f1..e0ae3faf43 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
 
 import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
-import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.mpp.plan.statement.literal.Literal;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.nio.ByteBuffer;
@@ -31,23 +31,33 @@ public class FillDescriptor {
   // policy of fill null values
   private final FillPolicy fillPolicy;
 
-  // target column for fill
-  private final Expression expression;
+  // filled value when fillPolicy is VALUE
+  private Literal fillValue;
 
-  public FillDescriptor(FillPolicy fillPolicy, Expression expression) {
+  public FillDescriptor(FillPolicy fillPolicy) {
     this.fillPolicy = fillPolicy;
-    this.expression = expression;
+  }
+
+  public FillDescriptor(FillPolicy fillPolicy, Literal fillValue) {
+    this.fillPolicy = fillPolicy;
+    this.fillValue = fillValue;
   }
 
   public void serialize(ByteBuffer byteBuffer) {
     ReadWriteIOUtils.write(fillPolicy.ordinal(), byteBuffer);
-    Expression.serialize(expression, byteBuffer);
+    if (fillPolicy == FillPolicy.VALUE) {
+      fillValue.serialize(byteBuffer);
+    }
   }
 
   public static FillDescriptor deserialize(ByteBuffer byteBuffer) {
     FillPolicy fillPolicy = FillPolicy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
-    Expression expression = Expression.deserialize(byteBuffer);
-    return new FillDescriptor(fillPolicy, expression);
+    if (fillPolicy == FillPolicy.VALUE) {
+      Literal fillValue = Literal.deserialize(byteBuffer);
+      return new FillDescriptor(fillPolicy, fillValue);
+    } else {
+      return new FillDescriptor(fillPolicy);
+    }
   }
 
   @Override
@@ -59,11 +69,11 @@ public class FillDescriptor {
       return false;
     }
     FillDescriptor that = (FillDescriptor) o;
-    return fillPolicy == that.fillPolicy && Objects.equals(expression, that.expression);
+    return fillPolicy == that.fillPolicy && Objects.equals(fillValue, that.fillValue);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(fillPolicy, expression);
+    return Objects.hash(fillPolicy, fillValue);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FilterNullParameter.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FilterNullParameter.java
index b936202297..cf8b61f10f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FilterNullParameter.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FilterNullParameter.java
@@ -31,10 +31,12 @@ import java.util.Objects;
 public class FilterNullParameter {
 
   // The policy to discard the result from upstream npde
-  private final FilterNullPolicy filterNullPolicy;
+  private FilterNullPolicy filterNullPolicy;
 
   // indicate columns used to filter null
-  private final List<Expression> filterNullColumns;
+  private List<Expression> filterNullColumns;
+
+  public FilterNullParameter() {}
 
   public FilterNullParameter(
       FilterNullPolicy filterNullPolicy, List<Expression> filterNullColumns) {
@@ -50,6 +52,14 @@ public class FilterNullParameter {
     return filterNullColumns;
   }
 
+  public void setFilterNullPolicy(FilterNullPolicy filterNullPolicy) {
+    this.filterNullPolicy = filterNullPolicy;
+  }
+
+  public void setFilterNullColumns(List<Expression> filterNullColumns) {
+    this.filterNullColumns = filterNullColumns;
+  }
+
   public void serialize(ByteBuffer byteBuffer) {
     ReadWriteIOUtils.write(filterNullPolicy.ordinal(), byteBuffer);
     ReadWriteIOUtils.write(filterNullColumns.size(), byteBuffer);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/GroupByTimeParameter.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/GroupByTimeParameter.java
index e53c4dc404..468b36e6a2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/GroupByTimeParameter.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/GroupByTimeParameter.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
 
+import org.apache.iotdb.db.mpp.plan.statement.component.GroupByTimeComponent;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.nio.ByteBuffer;
@@ -68,6 +69,16 @@ public class GroupByTimeParameter {
     this.leftCRightO = leftCRightO;
   }
 
+  public GroupByTimeParameter(GroupByTimeComponent groupByTimeComponent) {
+    this.startTime = groupByTimeComponent.getStartTime();
+    this.endTime = groupByTimeComponent.getEndTime();
+    this.interval = groupByTimeComponent.getInterval();
+    this.slidingStep = groupByTimeComponent.getSlidingStep();
+    this.isIntervalByMonth = groupByTimeComponent.isIntervalByMonth();
+    this.isSlidingStepByMonth = groupByTimeComponent.isSlidingStepByMonth();
+    this.leftCRightO = groupByTimeComponent.isLeftCRightO();
+  }
+
   public long getStartTime() {
     return startTime;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/ColumnPaginationController.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/ColumnPaginationController.java
deleted file mode 100644
index 37e65d3183..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/ColumnPaginationController.java
+++ /dev/null
@@ -1,145 +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.mpp.plan.rewriter;
-
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
-
-import java.util.List;
-
-/** apply MaxQueryDeduplicatedPathNum and SLIMIT & SOFFSET */
-public class ColumnPaginationController {
-
-  // series offset for result set. The default value is 0
-  private final int seriesOffset;
-
-  // for ALIGN BY DEVICE / DISABLE ALIGN / GROUP BY LEVEL / LAST, controller does is disabled
-  private final boolean isDisabled;
-
-  private int curLimit =
-      IoTDBDescriptor.getInstance().getConfig().getMaxQueryDeduplicatedPathNum() + 1;
-  private int curOffset;
-
-  // records the path number that the SchemaTree totally returned
-  private int consumed = 0;
-
-  public ColumnPaginationController(int seriesLimit, int seriesOffset, boolean isDisabled) {
-    // for series limit, the default value is 0, which means no limit
-    this.curLimit = seriesLimit == 0 ? this.curLimit : Math.min(seriesLimit, this.curLimit);
-    this.seriesOffset = this.curOffset = seriesOffset;
-    this.isDisabled = isDisabled;
-  }
-
-  public int getCurLimit() {
-    if (isDisabled) {
-      return 0;
-    }
-
-    return curLimit;
-  }
-
-  public int getCurOffset() {
-    if (isDisabled) {
-      return 0;
-    }
-
-    return curOffset;
-  }
-
-  /** @return should break the loop or not */
-  public boolean checkIfPathNumberIsOverLimit(List<ResultColumn> resultColumns)
-      throws PathNumOverLimitException {
-    if (resultColumns.size()
-        > IoTDBDescriptor.getInstance().getConfig().getMaxQueryDeduplicatedPathNum()) {
-      throw new PathNumOverLimitException();
-    }
-    if (isDisabled) {
-      return false;
-    }
-
-    return curLimit == 0;
-  }
-
-  public void checkIfSoffsetIsExceeded(List<ResultColumn> resultColumns)
-      throws StatementAnalyzeException {
-    if (isDisabled) {
-      return;
-    }
-
-    if (consumed == 0 ? seriesOffset != 0 : resultColumns.isEmpty()) {
-      throw new StatementAnalyzeException(
-          String.format(
-              "The value of SOFFSET (%d) is equal to or exceeds the number of sequences (%d) that can actually be returned.",
-              seriesOffset, consumed));
-    }
-  }
-
-  public void consume(int limit, int offset) {
-    if (isDisabled) {
-      return;
-    }
-
-    consumed += offset;
-    curOffset -= Math.min(curOffset, offset);
-    curLimit -= limit;
-  }
-
-  public boolean hasCurOffset() {
-    if (isDisabled) {
-      return false;
-    }
-
-    return curOffset != 0;
-  }
-
-  public boolean hasCurLimit() {
-    if (isDisabled) {
-      return true;
-    }
-
-    return curLimit != 0;
-  }
-
-  public void decCurOffset() {
-    if (isDisabled) {
-      return;
-    }
-
-    curOffset--;
-  }
-
-  public void decCurLimit() {
-    if (isDisabled) {
-      return;
-    }
-
-    curLimit--;
-  }
-
-  public void incConsumed(int num) {
-    if (isDisabled) {
-      return;
-    }
-
-    consumed += num;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/ConcatPathRewriter.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/ConcatPathRewriter.java
deleted file mode 100644
index a79d114ab4..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/ConcatPathRewriter.java
+++ /dev/null
@@ -1,190 +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.mpp.plan.rewriter;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.utils.PathUtils;
-import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.filter.FunctionFilter;
-import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.statement.Statement;
-import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
-import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
-import org.apache.iotdb.db.qp.constant.SQLConstant;
-import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-/**
- * This rewriter:
- *
- * <p>1. Concat prefix path in SELECT, WHERE, and WITHOUT NULL clause with the suffix path in the
- * FROM clause.
- *
- * <p>2. Construct a {@link PathPatternTree}.
- */
-public class ConcatPathRewriter {
-
-  private PathPatternTree patternTree;
-
-  public Statement rewrite(Statement statement, PathPatternTree patternTree)
-      throws StatementAnalyzeException, PathNumOverLimitException {
-    QueryStatement queryStatement = (QueryStatement) statement;
-    this.patternTree = patternTree;
-
-    // concat SELECT with FROM
-    concatSelectWithFrom(queryStatement);
-
-    // concat WITHOUT NULL with FROM
-    if (queryStatement.getFilterNullComponent() != null
-        && !queryStatement.getFilterNullComponent().getWithoutNullColumns().isEmpty()) {
-      concatWithoutNullColumnsWithFrom(queryStatement);
-    }
-
-    // concat WHERE with FROM
-    if (queryStatement.getWhereCondition() != null) {
-      constructPatternTreeFromWhereWithFrom(queryStatement);
-    }
-    return queryStatement;
-  }
-
-  /**
-   * Concat the prefix path in the SELECT clause and the suffix path in the FROM clause into a full
-   * path pattern. And construct pattern tree.
-   */
-  private void concatSelectWithFrom(QueryStatement queryStatement)
-      throws StatementAnalyzeException {
-    // prefix paths in the FROM clause
-    List<PartialPath> prefixPaths = queryStatement.getFromComponent().getPrefixPaths();
-
-    // resultColumns after concat
-    List<ResultColumn> resultColumns = new ArrayList<>();
-    for (ResultColumn suffixPath : queryStatement.getSelectComponent().getResultColumns()) {
-      boolean needAliasCheck = suffixPath.hasAlias() && !queryStatement.isGroupByLevel();
-      suffixPath.concat(prefixPaths, resultColumns, needAliasCheck, patternTree);
-    }
-    queryStatement.getSelectComponent().setResultColumns(resultColumns);
-  }
-
-  /**
-   * Concat the prefix path in the WITHOUT NULL clause and the suffix path in the FROM clause into a
-   * full path pattern. And construct pattern tree.
-   */
-  private void concatWithoutNullColumnsWithFrom(QueryStatement queryStatement)
-      throws StatementAnalyzeException {
-    // prefix paths in the FROM clause
-    List<PartialPath> prefixPaths = queryStatement.getFromComponent().getPrefixPaths();
-
-    // result after concat
-    List<Expression> withoutNullColumns = new ArrayList<>();
-    for (Expression expression : queryStatement.getFilterNullComponent().getWithoutNullColumns()) {
-      concatWithoutNullColumnsWithFrom(
-          prefixPaths,
-          expression,
-          withoutNullColumns,
-          queryStatement.getSelectComponent().getAliasSet());
-    }
-    queryStatement.getFilterNullComponent().setWithoutNullColumns(withoutNullColumns);
-  }
-
-  private void concatWithoutNullColumnsWithFrom(
-      List<PartialPath> prefixPaths,
-      Expression expression,
-      List<Expression> withoutNullColumns,
-      Set<String> aliasSet)
-      throws StatementAnalyzeException {
-    if (expression instanceof TimeSeriesOperand) {
-      TimeSeriesOperand timeSeriesOperand = (TimeSeriesOperand) expression;
-      if (timeSeriesOperand
-          .getPath()
-          .getFullPath()
-          .startsWith(SQLConstant.ROOT + ".")) { // start with "root." don't concat
-        // because the full path that starts with 'root.' won't be split
-        // so we need to split it.
-        if (((TimeSeriesOperand) expression).getPath().getNodeLength() == 1) { // no split
-          try {
-            ((TimeSeriesOperand) expression)
-                .setPath(
-                    new PartialPath(
-                        PathUtils.splitPathToDetachedPath(
-                            ((TimeSeriesOperand) expression)
-                                .getPath()
-                                .getFirstNode()))); // split path To nodes
-          } catch (IllegalPathException e) {
-            throw new StatementAnalyzeException(e.getMessage());
-          }
-        }
-        patternTree.appendPath(((TimeSeriesOperand) expression).getPath());
-        withoutNullColumns.add(expression);
-      } else {
-        if (!aliasSet.contains(expression.getExpressionString())) { // not alias, concat
-          List<Expression> resultExpressions = new ArrayList<>();
-          expression.concat(prefixPaths, resultExpressions, patternTree);
-          withoutNullColumns.addAll(resultExpressions);
-        } else { // alias, don't concat
-          withoutNullColumns.add(expression);
-        }
-      }
-    } else {
-      List<Expression> resultExpressions = new ArrayList<>();
-      expression.concat(prefixPaths, resultExpressions, patternTree);
-      withoutNullColumns.addAll(resultExpressions);
-    }
-  }
-
-  /**
-   * Concat the prefix path in the WHERE clause and the suffix path in the FROM clause into a full
-   * path pattern. And construct pattern tree.
-   */
-  private void constructPatternTreeFromWhereWithFrom(QueryStatement queryStatement) {
-    constructPatternTreeFromWhereWithFrom(
-        queryStatement.getFromComponent().getPrefixPaths(),
-        queryStatement.getWhereCondition().getQueryFilter());
-  }
-
-  private void constructPatternTreeFromWhereWithFrom(
-      List<PartialPath> fromPaths, QueryFilter filter) {
-    if (!filter.isLeaf()) {
-      for (QueryFilter child : filter.getChildren()) {
-        constructPatternTreeFromWhereWithFrom(fromPaths, child);
-      }
-      return;
-    }
-
-    FunctionFilter functionOperator = (FunctionFilter) filter;
-    PartialPath filterPath = functionOperator.getSinglePath();
-    List<PartialPath> concatPaths = new ArrayList<>();
-    if (SQLConstant.isReservedPath(filterPath)) {
-      // do nothing in the case of "where time > 5"
-      return;
-    } else if (filterPath.getFirstNode().startsWith(SQLConstant.ROOT)) {
-      // do nothing in the case of "where root.d1.s1 > 5"
-      concatPaths.add(filterPath);
-    } else {
-      fromPaths.forEach(fromPath -> concatPaths.add(fromPath.concatPath(filterPath)));
-    }
-    concatPaths.forEach(concatPath -> patternTree.appendPath(concatPath));
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/WildcardsRemover.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/WildcardsRemover.java
deleted file mode 100644
index 2b8214a3ff..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/rewriter/WildcardsRemover.java
+++ /dev/null
@@ -1,375 +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.mpp.plan.rewriter;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
-import org.apache.iotdb.db.exception.sql.SQLParserException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.metadata.path.MeasurementPath;
-import org.apache.iotdb.db.mpp.common.filter.BasicFunctionFilter;
-import org.apache.iotdb.db.mpp.common.filter.FunctionFilter;
-import org.apache.iotdb.db.mpp.common.filter.InFilter;
-import org.apache.iotdb.db.mpp.common.filter.LikeFilter;
-import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
-import org.apache.iotdb.db.mpp.common.filter.RegexpFilter;
-import org.apache.iotdb.db.mpp.common.schematree.SchemaTree;
-import org.apache.iotdb.db.mpp.plan.analyze.TypeProvider;
-import org.apache.iotdb.db.mpp.plan.constant.FilterConstant;
-import org.apache.iotdb.db.mpp.plan.statement.Statement;
-import org.apache.iotdb.db.mpp.plan.statement.component.GroupByLevelController;
-import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
-import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition;
-import org.apache.iotdb.db.mpp.plan.statement.crud.AggregationQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.LastQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
-import org.apache.iotdb.db.qp.constant.SQLConstant;
-import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-import org.apache.iotdb.tsfile.utils.Pair;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-
-/**
- * This rewriter:
- *
- * <p>1. Bind metadata to paths in SELECT, WHERE, and WITHOUT NULL clauses.
- *
- * <p>2. Remove wildcards and apply SLIMIT & SOFFSET.
- */
-public class WildcardsRemover {
-
-  private SchemaTree schemaTree;
-  private TypeProvider typeProvider;
-
-  private ColumnPaginationController paginationController;
-
-  public Statement rewrite(Statement statement, TypeProvider typeProvider, SchemaTree schemaTree)
-      throws StatementAnalyzeException, PathNumOverLimitException {
-    QueryStatement queryStatement = (QueryStatement) statement;
-    this.paginationController =
-        new ColumnPaginationController(
-            queryStatement.getSeriesLimit(),
-            queryStatement.getSeriesOffset(),
-            queryStatement.isAlignByDevice()
-                || queryStatement.disableAlign()
-                || queryStatement instanceof LastQueryStatement
-                || queryStatement.isGroupByLevel());
-    this.schemaTree = schemaTree;
-    this.typeProvider = typeProvider;
-
-    if (queryStatement.getIndexType() == null) {
-      // remove wildcards in SELECT clause
-      removeWildcardsInSelectPaths(queryStatement);
-
-      // remove wildcards in WITHOUT NULL clause
-      if (queryStatement.getFilterNullComponent() != null
-          && !queryStatement.getFilterNullComponent().getWithoutNullColumns().isEmpty()) {
-        removeWildcardsWithoutNullColumns(queryStatement);
-      }
-    }
-
-    // remove wildcards in WHERE clause
-    if (queryStatement.getWhereCondition() != null) {
-      removeWildcardsInQueryFilter(queryStatement);
-    }
-
-    return queryStatement;
-  }
-
-  private void removeWildcardsInSelectPaths(QueryStatement queryStatement)
-      throws StatementAnalyzeException, PathNumOverLimitException {
-    List<ResultColumn> resultColumns = new ArrayList<>();
-
-    // Only used for group by level
-    GroupByLevelController groupByLevelController = null;
-    if (queryStatement.isGroupByLevel()) {
-      groupByLevelController = new GroupByLevelController(queryStatement);
-      queryStatement.resetSLimitOffset();
-      resultColumns = new LinkedList<>();
-    }
-
-    for (ResultColumn resultColumn : queryStatement.getSelectComponent().getResultColumns()) {
-      boolean needAliasCheck = resultColumn.hasAlias() && !queryStatement.isGroupByLevel();
-      resultColumn.removeWildcards(this, resultColumns, needAliasCheck);
-      if (groupByLevelController != null) {
-        groupByLevelController.control(resultColumn, resultColumns);
-      }
-      if (paginationController.checkIfPathNumberIsOverLimit(resultColumns)) {
-        break;
-      }
-    }
-    paginationController.checkIfSoffsetIsExceeded(resultColumns);
-    queryStatement.getSelectComponent().setResultColumns(resultColumns);
-    if (groupByLevelController != null) {
-      ((AggregationQueryStatement) queryStatement)
-          .getGroupByLevelComponent()
-          .setGroupByLevelController(groupByLevelController);
-    }
-  }
-
-  private void removeWildcardsWithoutNullColumns(QueryStatement queryStatement)
-      throws StatementAnalyzeException {
-
-    List<Expression> expressions = queryStatement.getFilterNullComponent().getWithoutNullColumns();
-
-    // because timeSeries path may be with "*", so need to remove it for getting some actual
-    // timeSeries paths
-    // actualExpressions store the actual timeSeries paths
-    List<Expression> actualExpressions = new ArrayList<>();
-    List<Expression> resultExpressions = new ArrayList<>();
-
-    // because expression.removeWildcards will ignore the TimeSeries path that exists in the meta
-    // so we need to recognise the alias, just simply add to the resultExpressions
-    for (Expression expression : expressions) {
-      if (queryStatement
-          .getSelectComponent()
-          .getAliasSet()
-          .contains(expression.getExpressionString())) {
-        resultExpressions.add(expression);
-        continue;
-      }
-      expression.removeWildcards(this, actualExpressions);
-    }
-
-    // group by level, use groupedPathMap
-    if (queryStatement.isGroupByLevel()) {
-      GroupByLevelController groupByLevelController =
-          ((AggregationQueryStatement) queryStatement)
-              .getGroupByLevelComponent()
-              .getGroupByLevelController();
-      for (Expression expression : actualExpressions) {
-        String groupedPath =
-            groupByLevelController.getGroupedPath(expression.getExpressionString());
-        if (groupedPath != null) {
-          try {
-            resultExpressions.add(new TimeSeriesOperand(new PartialPath(groupedPath)));
-          } catch (IllegalPathException e) {
-            throw new StatementAnalyzeException(e.getMessage());
-          }
-        } else {
-          resultExpressions.add(expression);
-        }
-      }
-    } else {
-      resultExpressions.addAll(actualExpressions);
-    }
-    queryStatement.getFilterNullComponent().setWithoutNullColumns(resultExpressions);
-  }
-
-  private void removeWildcardsInQueryFilter(QueryStatement queryStatement)
-      throws StatementAnalyzeException {
-    WhereCondition whereCondition = queryStatement.getWhereCondition();
-    List<PartialPath> fromPaths = queryStatement.getFromComponent().getPrefixPaths();
-
-    Set<PartialPath> resultPaths = new HashSet<>();
-    whereCondition.setQueryFilter(
-        removeWildcardsInQueryFilter(whereCondition.getQueryFilter(), fromPaths, resultPaths));
-    whereCondition.getQueryFilter().setPathSet(resultPaths);
-  }
-
-  private QueryFilter removeWildcardsInQueryFilter(
-      QueryFilter filter, List<PartialPath> fromPaths, Set<PartialPath> resultPaths)
-      throws StatementAnalyzeException {
-    if (!filter.isLeaf()) {
-      List<QueryFilter> newFilterList = new ArrayList<>();
-      for (QueryFilter child : filter.getChildren()) {
-        newFilterList.add(removeWildcardsInQueryFilter(child, fromPaths, resultPaths));
-      }
-      filter.setChildren(newFilterList);
-      return filter;
-    }
-    FunctionFilter functionFilter = (FunctionFilter) filter;
-    PartialPath filterPath = functionFilter.getSinglePath();
-
-    List<PartialPath> concatPaths = new ArrayList<>();
-    if (SQLConstant.isReservedPath(filterPath)) {
-      // do nothing in the case of "where time > 5"
-      resultPaths.add(filterPath);
-      return filter;
-    } else if (filterPath.getFirstNode().startsWith(SQLConstant.ROOT)) {
-      // do nothing in the case of "where root.d1.s1 > 5"
-      concatPaths.add(filterPath);
-    } else {
-      fromPaths.forEach(fromPath -> concatPaths.add(fromPath.concatPath(filterPath)));
-    }
-
-    List<PartialPath> noStarPaths = removeWildcardsInConcatPaths(concatPaths);
-    resultPaths.addAll(noStarPaths);
-    if (noStarPaths.size() == 1) {
-      // Transform "select s1 from root.car.* where s1 > 10" to
-      // "select s1 from root.car.* where root.car.*.s1 > 10"
-      functionFilter.setSinglePath(noStarPaths.get(0));
-      return filter;
-    } else {
-      // Transform "select s1 from root.car.d1, root.car.d2 where s1 > 10" to
-      // "select s1 from root.car.d1, root.car.d2 where root.car.d1.s1 > 10 and root.car.d2.s1 > 10"
-      // Note that, two fork tree has to be maintained for DnfFilterOptimizer.
-      return constructBinaryFilterTreeWithAnd(noStarPaths, filter);
-    }
-  }
-
-  private QueryFilter constructBinaryFilterTreeWithAnd(
-      List<PartialPath> noStarPaths, QueryFilter filter) throws StatementAnalyzeException {
-    QueryFilter filterBinaryTree = new QueryFilter(FilterConstant.FilterType.KW_AND);
-    QueryFilter currentNode = filterBinaryTree;
-    for (int i = 0; i < noStarPaths.size(); i++) {
-      if (i > 0 && i < noStarPaths.size() - 1) {
-        QueryFilter newInnerNode = new QueryFilter(FilterConstant.FilterType.KW_AND);
-        currentNode.addChildOperator(newInnerNode);
-        currentNode = newInnerNode;
-      }
-      try {
-        if (filter instanceof InFilter) {
-          currentNode.addChildOperator(
-              new InFilter(
-                  filter.getFilterType(),
-                  noStarPaths.get(i),
-                  ((InFilter) filter).getNot(),
-                  ((InFilter) filter).getValues()));
-        } else if (filter instanceof LikeFilter) {
-          currentNode.addChildOperator(
-              new LikeFilter(
-                  filter.getFilterType(), noStarPaths.get(i), ((LikeFilter) filter).getValue()));
-        } else if (filter instanceof RegexpFilter) {
-          currentNode.addChildOperator(
-              new RegexpFilter(
-                  filter.getFilterType(), noStarPaths.get(i), ((RegexpFilter) filter).getValue()));
-        } else {
-          currentNode.addChildOperator(
-              new BasicFunctionFilter(
-                  filter.getFilterType(),
-                  noStarPaths.get(i),
-                  ((BasicFunctionFilter) filter).getValue()));
-        }
-      } catch (SQLParserException e) {
-        throw new StatementAnalyzeException(e.getMessage());
-      }
-    }
-    return filterBinaryTree;
-  }
-
-  public List<MeasurementPath> removeWildcardInPath(PartialPath path)
-      throws StatementAnalyzeException {
-    try {
-      Pair<List<MeasurementPath>, Integer> pair =
-          schemaTree.searchMeasurementPaths(
-              path, paginationController.getCurLimit(), paginationController.getCurOffset(), false);
-      paginationController.consume(pair.left.size(), pair.right);
-      pair.left.forEach(
-          measurementPath ->
-              typeProvider.setType(measurementPath.getFullPath(), measurementPath.getSeriesType()));
-      return pair.left;
-    } catch (Exception e) {
-      e.printStackTrace();
-      throw new StatementAnalyzeException(
-          "error occurred when removing wildcard: " + e.getMessage());
-    }
-  }
-
-  public List<List<Expression>> removeWildcardsInExpressions(List<Expression> expressions)
-      throws StatementAnalyzeException {
-    // One by one, remove the wildcards from the input expressions. In most cases, an expression
-    // will produce multiple expressions after removing the wildcards. We use extendedExpressions to
-    // collect the produced expressions.
-    List<List<Expression>> extendedExpressions = new ArrayList<>();
-    for (Expression originExpression : expressions) {
-      List<Expression> actualExpressions = new ArrayList<>();
-      originExpression.removeWildcards(this, actualExpressions);
-      if (actualExpressions.isEmpty()) {
-        // Let's ignore the eval of the function which has at least one non-existence series as
-        // input. See IOTDB-1212: https://github.com/apache/iotdb/pull/3101
-        return Collections.emptyList();
-      }
-      extendedExpressions.add(actualExpressions);
-    }
-
-    // Calculate the Cartesian product of extendedExpressions to get the actual expressions after
-    // removing all wildcards. We use actualExpressions to collect them.
-    List<List<Expression>> actualExpressions = new ArrayList<>();
-    cartesianProduct(extendedExpressions, actualExpressions, 0, new ArrayList<>());
-
-    // Apply the soffset & slimit control to the actualExpressions and return the remaining
-    // expressions.
-    List<List<Expression>> remainingExpressions = new ArrayList<>();
-    for (List<Expression> actualExpression : actualExpressions) {
-      if (paginationController.hasCurOffset()) {
-        paginationController.decCurOffset();
-      } else if (paginationController.hasCurLimit()) {
-        remainingExpressions.add(actualExpression);
-        paginationController.decCurLimit();
-      } else {
-        break;
-      }
-    }
-    paginationController.incConsumed(actualExpressions.size());
-    return remainingExpressions;
-  }
-
-  private List<PartialPath> removeWildcardsInConcatPaths(List<PartialPath> originalPaths)
-      throws StatementAnalyzeException {
-    HashSet<PartialPath> actualPaths = new HashSet<>();
-    try {
-      for (PartialPath originalPath : originalPaths) {
-        List<MeasurementPath> all =
-            schemaTree.searchMeasurementPaths(originalPath, 0, 0, false).left;
-        if (all.isEmpty()) {
-          throw new StatementAnalyzeException(
-              String.format("Unknown time series %s in `where clause`", originalPath));
-        }
-        actualPaths.addAll(all);
-      }
-    } catch (StatementAnalyzeException e) {
-      throw new StatementAnalyzeException("error when remove star: " + e.getMessage());
-    }
-    return new ArrayList<>(actualPaths);
-  }
-
-  public static <T> void cartesianProduct(
-      List<List<T>> dimensionValue, List<List<T>> resultList, int layer, List<T> currentList) {
-    if (layer < dimensionValue.size() - 1) {
-      if (dimensionValue.get(layer).isEmpty()) {
-        cartesianProduct(dimensionValue, resultList, layer + 1, currentList);
-      } else {
-        for (int i = 0; i < dimensionValue.get(layer).size(); i++) {
-          List<T> list = new ArrayList<>(currentList);
-          list.add(dimensionValue.get(layer).get(i));
-          cartesianProduct(dimensionValue, resultList, layer + 1, list);
-        }
-      }
-    } else if (layer == dimensionValue.size() - 1) {
-      if (dimensionValue.get(layer).isEmpty()) {
-        resultList.add(currentList);
-      } else {
-        for (int i = 0; i < dimensionValue.get(layer).size(); i++) {
-          List<T> list = new ArrayList<>(currentList);
-          list.add(dimensionValue.get(layer).get(i));
-          resultList.add(list);
-        }
-      }
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
index 0d3ece4a7e..59f16004b3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
@@ -19,20 +19,13 @@
 
 package org.apache.iotdb.db.mpp.plan.statement;
 
-import org.apache.iotdb.db.mpp.plan.statement.crud.AggregationQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.FillQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.GroupByFillQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.GroupByQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertMultiTabletsStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsOfOneDeviceStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.LastQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.UDAFQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.UDTFQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.AlterTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountDevicesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountLevelTimeSeriesStatement;
@@ -113,34 +106,6 @@ public abstract class StatementVisitor<R, C> {
     return visitStatement(queryStatement, context);
   }
 
-  public R visitAggregationQuery(AggregationQueryStatement queryStatement, C context) {
-    return visitQuery(queryStatement, context);
-  }
-
-  public R visitFillQuery(FillQueryStatement queryStatement, C context) {
-    return visitQuery(queryStatement, context);
-  }
-
-  public R visitGroupByQuery(GroupByQueryStatement queryStatement, C context) {
-    return visitQuery(queryStatement, context);
-  }
-
-  public R visitGroupByFillQuery(GroupByFillQueryStatement queryStatement, C context) {
-    return visitQuery(queryStatement, context);
-  }
-
-  public R visitLastQuery(LastQueryStatement queryStatement, C context) {
-    return visitQuery(queryStatement, context);
-  }
-
-  public R visitUDTFQuery(UDTFQueryStatement queryStatement, C context) {
-    return visitQuery(queryStatement, context);
-  }
-
-  public R visitUDAFQuery(UDAFQueryStatement queryStatement, C context) {
-    return visitQuery(queryStatement, context);
-  }
-
   // Insert Statement
   public R visitInsert(InsertStatement insertStatement, C context) {
     return visitStatement(insertStatement, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/FillComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/FillComponent.java
index 1be46d9933..c1b29c1aff 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/FillComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/FillComponent.java
@@ -20,32 +20,29 @@
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
-import org.apache.iotdb.db.query.executor.fill.IFill;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import java.util.Map;
+import org.apache.iotdb.db.mpp.plan.statement.literal.Literal;
 
 /** This class maintains information of {@code FILL} clause. */
 public class FillComponent extends StatementNode {
 
-  private Map<TSDataType, IFill> fillTypes;
-  private IFill singleFill;
+  private FillPolicy fillPolicy;
+  private Literal fillValue;
 
   public FillComponent() {}
 
-  public Map<TSDataType, IFill> getFillTypes() {
-    return fillTypes;
+  public FillPolicy getFillPolicy() {
+    return fillPolicy;
   }
 
-  public IFill getSingleFill() {
-    return singleFill;
+  public void setFillPolicy(FillPolicy fillPolicy) {
+    this.fillPolicy = fillPolicy;
   }
 
-  public void setFillTypes(Map<TSDataType, IFill> fillTypes) {
-    this.fillTypes = fillTypes;
+  public Literal getFillValue() {
+    return fillValue;
   }
 
-  public void setSingleFill(IFill singleFill) {
-    this.singleFill = singleFill;
+  public void setFillValue(Literal fillValue) {
+    this.fillValue = fillValue;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/GroupByLevelComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/GroupByLevelComponent.java
index 2b3d22f3cf..fa7192a0fe 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/GroupByLevelComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/GroupByLevelComponent.java
@@ -19,15 +19,11 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
-import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
 
-import java.util.Map;
-
 /** This class maintains information of {@code GROUP BY LEVEL} clause. */
 public class GroupByLevelComponent extends StatementNode {
 
-  protected GroupByLevelController groupByLevelController;
   protected int[] levels;
 
   public int[] getLevels() {
@@ -37,20 +33,4 @@ public class GroupByLevelComponent extends StatementNode {
   public void setLevels(int[] levels) {
     this.levels = levels;
   }
-
-  public void setGroupByLevelController(GroupByLevelController groupByLevelController) {
-    this.groupByLevelController = groupByLevelController;
-  }
-
-  public GroupByLevelController getGroupByLevelController() {
-    return groupByLevelController;
-  }
-
-  public Map<String, String> getGroupedPathMap() {
-    return groupByLevelController.getGroupedPathMap();
-  }
-
-  public Map<ColumnHeader, ColumnHeader> getGroupedHeaderMap() {
-    return groupByLevelController.getGroupedHeaderMap();
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/GroupByLevelController.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/GroupByLevelController.java
deleted file mode 100644
index edf865bc82..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/GroupByLevelController.java
+++ /dev/null
@@ -1,247 +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.mpp.plan.statement.component;
-
-import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
-import org.apache.iotdb.db.mpp.plan.statement.crud.AggregationQueryStatement;
-import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
-import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.multi.FunctionExpression;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * This class is used to control the row number of group by level query. For example, selected
- * series[root.sg.d1.s1, root.sg.d2.s1, root.sg2.d1.s1], level = 1; the result rows will be
- * [root.sg.*.s1, root.sg2.*.s1], sLimit and sOffset will be used to control the result numbers
- * rather than the selected series.
- */
-public class GroupByLevelController {
-
-  public static String ALIAS_ERROR_MESSAGE1 =
-      "alias '%s' can only be matched with one result column";
-  public static String ALIAS_ERROR_MESSAGE2 = "Result column %s with more than one alias[%s, %s]";
-  private final int seriesLimit;
-  private int seriesOffset;
-  Set<String> limitPaths;
-  Set<String> offsetPaths;
-  private final int[] levels;
-  int prevSize = 0;
-  /** count(root.sg.d1.s1) with level = 1 -> count(root.*.d1.s1) */
-  private final Map<String, String> groupedPathMap;
-
-  private final Map<ColumnHeader, ColumnHeader> groupedHeaderMap;
-  /** count(root.*.d1.s1) -> alias */
-  private Map<String, String> columnToAliasMap;
-  /**
-   * Only used to check whether one alisa is corresponding to only one column. i.e. Different
-   * columns can't have the same alias.
-   */
-  private Map<String, String> aliasToColumnMap;
-
-  public GroupByLevelController(QueryStatement queryStatement) {
-    this.seriesLimit = queryStatement.getSeriesLimit();
-    this.seriesOffset = queryStatement.getSeriesOffset();
-    this.limitPaths = seriesLimit > 0 ? new HashSet<>() : null;
-    this.offsetPaths = seriesOffset > 0 ? new HashSet<>() : null;
-    this.groupedPathMap = new LinkedHashMap<>();
-    this.groupedHeaderMap = new LinkedHashMap<>();
-    this.levels =
-        ((AggregationQueryStatement) queryStatement).getGroupByLevelComponent().getLevels();
-  }
-
-  public String getGroupedPath(String rawPath) {
-    return groupedPathMap.get(rawPath);
-  }
-
-  public String getAlias(String originName) {
-    return columnToAliasMap != null && columnToAliasMap.get(originName) != null
-        ? columnToAliasMap.get(originName)
-        : null;
-  }
-
-  public void control(ResultColumn rawColumn, List<ResultColumn> resultColumns)
-      throws StatementAnalyzeException {
-    Set<Integer> countWildcardIterIndices = getCountStarIndices(rawColumn);
-
-    // `resultColumns` includes all result columns after removing wildcards, so we need to skip
-    // those we have processed
-    Iterator<ResultColumn> iterator = resultColumns.iterator();
-    for (int i = 0; i < prevSize; i++) {
-      iterator.next();
-    }
-
-    while (iterator.hasNext()) {
-      ResultColumn resultColumn = iterator.next();
-      Expression rootExpression = resultColumn.getExpression();
-      boolean hasAggregation = false;
-      int idx = 0;
-      for (Iterator<Expression> it = rootExpression.iterator(); it.hasNext(); ) {
-        Expression expression = it.next();
-        if (expression instanceof FunctionExpression
-            && expression.isBuiltInAggregationFunctionExpression()) {
-          hasAggregation = true;
-          List<PartialPath> paths = ((FunctionExpression) expression).getPaths();
-          String functionName = ((FunctionExpression) expression).getFunctionName();
-          boolean isCountStar = countWildcardIterIndices.contains(idx++);
-          String groupedPath =
-              generatePartialPathByLevel(isCountStar, paths.get(0).getNodes(), levels);
-          TSDataType dataType = paths.get(0).getSeriesType();
-          String rawPath = String.format("%s(%s)", functionName, paths.get(0).getFullPath());
-          String pathWithFunction = String.format("%s(%s)", functionName, groupedPath);
-
-          ColumnHeader rawPathHeader =
-              new ColumnHeader(paths.get(0).getFullPath(), functionName, dataType);
-          ColumnHeader groupedPathHeader = new ColumnHeader(groupedPath, functionName, dataType);
-
-          if (seriesLimit == 0 && seriesOffset == 0) {
-            groupedPathMap.put(rawPath, pathWithFunction);
-            groupedHeaderMap.put(rawPathHeader, groupedPathHeader);
-            checkAliasAndUpdateAliasMap(rawColumn, pathWithFunction);
-          } else {
-            // We cannot judge whether the path after grouping exists until we add it to set
-            if (seriesOffset > 0 && offsetPaths != null) {
-              offsetPaths.add(pathWithFunction);
-              if (offsetPaths.size() <= seriesOffset) {
-                iterator.remove();
-                if (offsetPaths.size() == seriesOffset) {
-                  seriesOffset = 0;
-                }
-              }
-            } else if (offsetPaths == null || !offsetPaths.contains(pathWithFunction)) {
-              limitPaths.add(pathWithFunction);
-              if (seriesLimit > 0 && limitPaths.size() > seriesLimit) {
-                iterator.remove();
-                limitPaths.remove(pathWithFunction);
-              } else {
-                groupedPathMap.put(rawPath, pathWithFunction);
-                groupedHeaderMap.put(rawPathHeader, groupedPathHeader);
-                checkAliasAndUpdateAliasMap(rawColumn, pathWithFunction);
-              }
-            } else {
-              iterator.remove();
-            }
-          }
-        }
-      }
-      if (!hasAggregation) {
-        throw new StatementAnalyzeException(rootExpression + " can't be used in group by level.");
-      }
-    }
-    prevSize = resultColumns.size();
-  }
-
-  // As one expression may have many aggregation results in the tree leaf, here we should traverse
-  // all the successor expressions and record the count(*) indices
-  private Set<Integer> getCountStarIndices(ResultColumn rawColumn) {
-    Set<Integer> countWildcardIterIndices = new HashSet<>();
-    int idx = 0;
-    for (Iterator<Expression> it = rawColumn.getExpression().iterator(); it.hasNext(); ) {
-      Expression expression = it.next();
-      if (expression instanceof FunctionExpression
-          && expression.isBuiltInAggregationFunctionExpression()
-          && ((FunctionExpression) expression).isCountStar()) {
-        countWildcardIterIndices.add(idx);
-      }
-      idx++;
-    }
-    return countWildcardIterIndices;
-  }
-
-  private void checkAliasAndUpdateAliasMap(ResultColumn rawColumn, String originName)
-      throws StatementAnalyzeException {
-    if (!rawColumn.hasAlias()) {
-      return;
-    } else if (columnToAliasMap == null) {
-      columnToAliasMap = new HashMap<>();
-      aliasToColumnMap = new HashMap<>();
-    }
-    // If an alias is corresponding to more than one result column, throw an exception
-    if (columnToAliasMap.get(originName) == null) {
-      if (aliasToColumnMap.get(rawColumn.getAlias()) != null) {
-        throw new StatementAnalyzeException(
-            String.format(ALIAS_ERROR_MESSAGE1, rawColumn.getAlias()));
-      } else {
-        columnToAliasMap.put(originName, rawColumn.getAlias());
-        aliasToColumnMap.put(rawColumn.getAlias(), originName);
-      }
-      // If a result column is corresponding to more than one alias, throw an exception
-    } else if (!columnToAliasMap.get(originName).equals(rawColumn.getAlias())) {
-      throw new StatementAnalyzeException(
-          String.format(
-              ALIAS_ERROR_MESSAGE2,
-              originName,
-              columnToAliasMap.get(originName),
-              rawColumn.getAlias()));
-    }
-  }
-
-  /**
-   * Transform an originalPath to a partial path that satisfies given level. Path nodes don't
-   * satisfy the given level will be replaced by "*" except the sensor level, e.g.
-   * generatePartialPathByLevel("root.sg.dh.d1.s1", 2) will return "root.*.dh.*.s1".
-   *
-   * <p>Especially, if count(*), then the sensor level will be replaced by "*" too.
-   *
-   * @return result partial path
-   */
-  public String generatePartialPathByLevel(boolean isCountStar, String[] nodes, int[] pathLevels) {
-    Set<Integer> levelSet = new HashSet<>();
-    for (int level : pathLevels) {
-      levelSet.add(level);
-    }
-
-    StringBuilder transformedPath = new StringBuilder();
-    transformedPath.append(nodes[0]).append(TsFileConstant.PATH_SEPARATOR);
-    for (int k = 1; k < nodes.length - 1; k++) {
-      if (levelSet.contains(k)) {
-        transformedPath.append(nodes[k]);
-      } else {
-        transformedPath.append(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD);
-      }
-      transformedPath.append(TsFileConstant.PATH_SEPARATOR);
-    }
-    if (isCountStar) {
-      transformedPath.append(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD);
-    } else {
-      transformedPath.append(nodes[nodes.length - 1]);
-    }
-    return transformedPath.toString();
-  }
-
-  public Map<String, String> getGroupedPathMap() {
-    return groupedPathMap;
-  }
-
-  public Map<ColumnHeader, ColumnHeader> getGroupedHeaderMap() {
-    return groupedHeaderMap;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java
index 05066baa24..82a8ab4af3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java
@@ -19,20 +19,10 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
 import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
+import java.util.Objects;
 
 /**
  * This class is used to represent a result column of a query.
@@ -76,10 +66,6 @@ public class ResultColumn extends StatementNode {
   private final Expression expression;
   private final String alias;
 
-  private TSDataType dataType;
-
-  private List<PartialPath> allPathsInExpression;
-
   public ResultColumn(Expression expression, String alias) {
     this.expression = expression;
     this.alias = alias;
@@ -90,15 +76,6 @@ public class ResultColumn extends StatementNode {
     alias = null;
   }
 
-  public List<PartialPath> collectPaths() {
-    if (allPathsInExpression == null) {
-      Set<PartialPath> pathSet = new HashSet<>();
-      expression.collectPaths(pathSet);
-      allPathsInExpression = new ArrayList<>(pathSet);
-    }
-    return allPathsInExpression;
-  }
-
   public Expression getExpression() {
     return expression;
   }
@@ -111,95 +88,25 @@ public class ResultColumn extends StatementNode {
     return alias;
   }
 
-  public String getResultColumnName() {
-    return alias != null ? alias : expression.getExpressionString();
-  }
-
-  public String getExpressionString() {
-    return expression.getExpressionString();
-  }
-
-  public void setDataType(TSDataType dataType) {
-    this.dataType = dataType;
-  }
-
-  public TSDataType getDataType() {
-    return dataType;
-  }
-
-  /**
-   * @param prefixPaths prefix paths in the from clause
-   * @param resultColumns used to collect the result columns
-   * @param needAliasCheck used to skip illegal alias judgement here. Including !isGroupByLevel
-   *     because count(*) may be * unfolded to more than one expression, but it still can be
-   *     aggregated together later.
-   */
-  public void concat(
-      List<PartialPath> prefixPaths,
-      List<ResultColumn> resultColumns,
-      boolean needAliasCheck,
-      PathPatternTree patternTree)
-      throws StatementAnalyzeException {
-    List<Expression> resultExpressions = new ArrayList<>();
-    expression.concat(prefixPaths, resultExpressions, patternTree);
-    if (needAliasCheck && 1 < resultExpressions.size()) {
-      throw new StatementAnalyzeException(
-          String.format("alias '%s' can only be matched with one time series", alias));
-    }
-    for (Expression resultExpression : resultExpressions) {
-      resultColumns.add(new ResultColumn(resultExpression, alias));
-    }
-  }
-
-  /**
-   * @param wildcardsRemover used to remove wildcards from {@code expression} and apply slimit &
-   *     soffset control
-   * @param resultColumns used to collect the result columns
-   * @param needAliasCheck used to skip illegal alias judgement here. Including !isGroupByLevel
-   *     because count(*) may be * unfolded to more than one expression, but it still can be
-   *     aggregated together later.
-   */
-  public void removeWildcards(
-      WildcardsRemover wildcardsRemover, List<ResultColumn> resultColumns, boolean needAliasCheck)
-      throws StatementAnalyzeException {
-    List<Expression> resultExpressions = new ArrayList<>();
-    expression.removeWildcards(wildcardsRemover, resultExpressions);
-    if (needAliasCheck && 1 < resultExpressions.size()) {
-      throw new StatementAnalyzeException(
-          String.format("alias '%s' can only be matched with one time series", alias));
-    }
-    for (Expression resultExpression : resultExpressions) {
-      resultColumns.add(new ResultColumn(resultExpression, alias));
-    }
-  }
-
-  public ColumnHeader constructColumnHeader() {
-    return new ColumnHeader(
-        this.getExpressionString(),
-        ((TimeSeriesOperand) this.getExpression()).getPath().getSeriesType(),
-        this.getAlias());
-  }
-
   @Override
   public String toString() {
     return "ResultColumn{" + "expression=" + expression + ", alias='" + alias + '\'' + '}';
   }
 
   @Override
-  public final int hashCode() {
-    return alias == null ? getResultColumnName().hashCode() : alias.hashCode();
-  }
-
-  @Override
-  public final boolean equals(Object o) {
+  public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
-
-    if (!(o instanceof org.apache.iotdb.db.query.expression.ResultColumn)) {
+    if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    return getResultColumnName()
-        .equals(((org.apache.iotdb.db.query.expression.ResultColumn) o).getResultColumnName());
+    ResultColumn that = (ResultColumn) o;
+    return Objects.equals(expression, that.expression) && Objects.equals(alias, that.alias);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(expression, alias);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java
index 740ec8c7be..b104041390 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java
@@ -19,54 +19,33 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
 import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-import org.apache.iotdb.db.query.expression.multi.FunctionExpression;
-import org.apache.iotdb.tsfile.read.common.Path;
 
 import java.time.ZoneId;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 /** This class maintains information of {@code SELECT} clause. */
 public class SelectComponent extends StatementNode {
 
   private final ZoneId zoneId;
 
+  private boolean hasLast = false;
+
   private boolean hasBuiltInAggregationFunction = false;
   private boolean hasTimeSeriesGeneratingFunction = false;
   private boolean hasUserDefinedAggregationFunction = false;
 
-  protected List<ResultColumn> resultColumns = new ArrayList<>();
-
-  Set<String> aliasSet;
+  private List<ResultColumn> resultColumns = new ArrayList<>();
 
-  private List<PartialPath> pathsCache;
-  private List<String> aggregationFunctionsCache;
-  private Map<String, List<PartialPath>> deviceNameToPathsCache;
-  private Map<String, Set<PartialPath>> deviceNameToDeduplicatedPathsCache;
+  private Map<String, Expression> aliasToColumnMap;
 
   public SelectComponent(ZoneId zoneId) {
     this.zoneId = zoneId;
   }
 
-  public SelectComponent(SelectComponent another) {
-    zoneId = another.getZoneId();
-
-    hasBuiltInAggregationFunction = another.isHasBuiltInAggregationFunction();
-    hasTimeSeriesGeneratingFunction = another.isHasTimeSeriesGeneratingFunction();
-    hasUserDefinedAggregationFunction = another.isHasUserDefinedAggregationFunction();
-
-    resultColumns.addAll(another.getResultColumns());
-    aliasSet.addAll(another.getAliasSet());
-  }
-
   public ZoneId getZoneId() {
     return zoneId;
   }
@@ -104,79 +83,19 @@ public class SelectComponent extends StatementNode {
     return resultColumns;
   }
 
-  public void setAliasSet(Set<String> aliasSet) {
-    this.aliasSet = aliasSet;
+  public Map<String, Expression> getAliasToColumnMap() {
+    return aliasToColumnMap;
   }
 
-  public Set<String> getAliasSet() {
-    return aliasSet;
+  public void setAliasToColumnMap(Map<String, Expression> aliasToColumnMap) {
+    this.aliasToColumnMap = aliasToColumnMap;
   }
 
-  public List<PartialPath> getPaths() {
-    if (pathsCache == null) {
-      pathsCache = new ArrayList<>();
-      for (ResultColumn resultColumn : resultColumns) {
-        Expression expression = resultColumn.getExpression();
-        if (expression instanceof TimeSeriesOperand) {
-          pathsCache.add(((TimeSeriesOperand) expression).getPath());
-        } else if (expression instanceof FunctionExpression
-            && expression.isBuiltInAggregationFunctionExpression()) {
-          pathsCache.add(((TimeSeriesOperand) expression.getExpressions().get(0)).getPath());
-        } else {
-          pathsCache.add(null);
-        }
-      }
-    }
-    return pathsCache;
+  public boolean isHasLast() {
+    return hasLast;
   }
 
-  public List<String> getAggregationFunctions() {
-    if (aggregationFunctionsCache == null) {
-      aggregationFunctionsCache = new ArrayList<>();
-      for (ResultColumn resultColumn : resultColumns) {
-        Expression expression = resultColumn.getExpression();
-        aggregationFunctionsCache.add(
-            expression instanceof FunctionExpression
-                ? ((FunctionExpression) resultColumn.getExpression()).getFunctionName()
-                : null);
-      }
-    }
-    return aggregationFunctionsCache;
-  }
-
-  public Map<String, Set<PartialPath>> getDeviceNameToDeduplicatedPathsMap() {
-    if (deviceNameToDeduplicatedPathsCache == null) {
-      deviceNameToDeduplicatedPathsCache = new HashMap<>();
-      for (ResultColumn resultColumn : resultColumns) {
-        for (PartialPath path : resultColumn.collectPaths()) {
-          deviceNameToDeduplicatedPathsCache
-              .computeIfAbsent(path.getDeviceIdString(), k -> new HashSet<>())
-              .add(path);
-        }
-      }
-    }
-    return deviceNameToDeduplicatedPathsCache;
-  }
-
-  public Map<String, List<PartialPath>> getDeviceNameToPathsMap() {
-    if (deviceNameToPathsCache == null) {
-      deviceNameToPathsCache = new HashMap<>();
-      for (ResultColumn resultColumn : resultColumns) {
-        for (PartialPath path : resultColumn.collectPaths()) {
-          deviceNameToPathsCache
-              .computeIfAbsent(path.getDeviceIdString(), k -> new ArrayList<>())
-              .add(path);
-        }
-      }
-    }
-    return deviceNameToPathsCache;
-  }
-
-  public List<Path> getDeduplicatedPaths() {
-    Set<Path> deduplicatedPaths = new HashSet<>();
-    for (ResultColumn resultColumn : resultColumns) {
-      deduplicatedPaths.addAll(resultColumn.collectPaths());
-    }
-    return new ArrayList<>(deduplicatedPaths);
+  public void setHasLast(boolean hasLast) {
+    this.hasLast = hasLast;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/WhereCondition.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/WhereCondition.java
index 01831dd90b..cd62ab8742 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/WhereCondition.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/WhereCondition.java
@@ -19,25 +19,25 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
-import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
+import org.apache.iotdb.db.query.expression.Expression;
 
 /** This class maintains information of {@code WHERE} clause. */
 public class WhereCondition extends StatementNode {
 
-  private QueryFilter queryFilter;
+  private Expression predicate;
 
   public WhereCondition() {}
 
-  public WhereCondition(QueryFilter queryFilter) {
-    this.queryFilter = queryFilter;
+  public WhereCondition(Expression predicate) {
+    this.predicate = predicate;
   }
 
-  public QueryFilter getQueryFilter() {
-    return queryFilter;
+  public Expression getPredicate() {
+    return predicate;
   }
 
-  public void setQueryFilter(QueryFilter queryFilter) {
-    this.queryFilter = queryFilter;
+  public void setPredicate(Expression predicate) {
+    this.predicate = predicate;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/AggregationQueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/AggregationQueryStatement.java
deleted file mode 100644
index ee74479fc4..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/AggregationQueryStatement.java
+++ /dev/null
@@ -1,129 +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.mpp.plan.statement.crud;
-
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.sql.SemanticException;
-import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
-import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
-import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.GroupByLevelComponent;
-import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
-import org.apache.iotdb.db.mpp.plan.statement.component.SelectComponent;
-import org.apache.iotdb.db.query.aggregation.AggregationType;
-import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-import org.apache.iotdb.db.query.expression.multi.FunctionExpression;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class AggregationQueryStatement extends QueryStatement {
-
-  // GROUP BY LEVEL clause
-  protected GroupByLevelComponent groupByLevelComponent;
-
-  public AggregationQueryStatement() {
-    super();
-  }
-
-  public AggregationQueryStatement(QueryStatement queryStatement) {
-    super(queryStatement);
-  }
-
-  public GroupByLevelComponent getGroupByLevelComponent() {
-    return groupByLevelComponent;
-  }
-
-  public void setGroupByLevelComponent(GroupByLevelComponent groupByLevelComponent) {
-    this.groupByLevelComponent = groupByLevelComponent;
-  }
-
-  @Override
-  public boolean isGroupByLevel() {
-    return groupByLevelComponent != null && groupByLevelComponent.getLevels().length > 0;
-  }
-
-  public Map<String, Map<PartialPath, Set<AggregationType>>> getDeviceNameToAggregationsMap() {
-    Map<String, Map<PartialPath, Set<AggregationType>>> deviceNameToAggregationsMap =
-        new HashMap<>();
-    for (ResultColumn resultColumn : getSelectComponent().getResultColumns()) {
-      FunctionExpression expression = (FunctionExpression) resultColumn.getExpression();
-      PartialPath path = expression.getPaths().get(0);
-      String functionName = expression.getFunctionName();
-      deviceNameToAggregationsMap
-          .computeIfAbsent(path.getDeviceIdString(), key -> new HashMap<>())
-          .computeIfAbsent(path, key -> new HashSet<>())
-          .add(AggregationType.valueOf(functionName.toUpperCase()));
-    }
-    return deviceNameToAggregationsMap;
-  }
-
-  @Override
-  public DatasetHeader constructDatasetHeader() {
-    List<ColumnHeader> columnHeaders = new ArrayList<>();
-    // TODO: consider Aggregation
-    return new DatasetHeader(columnHeaders, true);
-  }
-
-  @Override
-  public void selfCheck() {
-    super.selfCheck();
-
-    if (disableAlign()) {
-      throw new SemanticException("AGGREGATION doesn't support disable align clause.");
-    }
-    checkSelectComponent(selectComponent);
-    if (isGroupByLevel() && isAlignByDevice()) {
-      throw new SemanticException("group by level does not support align by device now.");
-    }
-  }
-
-  protected void checkSelectComponent(SelectComponent selectComponent) throws SemanticException {
-    if (hasTimeSeriesGeneratingFunction()) {
-      throw new SemanticException(
-          "User-defined and built-in hybrid aggregation is not supported together.");
-    }
-
-    for (ResultColumn resultColumn : selectComponent.getResultColumns()) {
-      Expression expression = resultColumn.getExpression();
-      if (expression instanceof TimeSeriesOperand) {
-        throw new SemanticException(
-            "Common queries and aggregated queries are not allowed to appear at the same time.");
-      }
-      // Currently, the aggregation function expression can only contain a timeseries operand.
-      if (expression instanceof FunctionExpression
-          && (expression.getExpressions().size() != 1
-              || !(expression.getExpressions().get(0) instanceof TimeSeriesOperand))) {
-        throw new SemanticException(
-            "The argument of the aggregation function must be a time series.");
-      }
-    }
-  }
-
-  @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitAggregationQuery(this, context);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/FillQueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/FillQueryStatement.java
deleted file mode 100644
index b9b44f1b3c..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/FillQueryStatement.java
+++ /dev/null
@@ -1,79 +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.mpp.plan.statement.crud;
-
-import org.apache.iotdb.db.exception.sql.SemanticException;
-import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
-import org.apache.iotdb.db.mpp.plan.constant.FilterConstant;
-import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.FillComponent;
-import org.apache.iotdb.db.qp.constant.SQLConstant;
-
-import java.util.Arrays;
-
-public class FillQueryStatement extends QueryStatement {
-
-  FillComponent fillComponent;
-
-  public FillQueryStatement() {
-    super();
-  }
-
-  public FillComponent getFillComponent() {
-    return fillComponent;
-  }
-
-  public void setFillComponent(FillComponent fillComponent) {
-    this.fillComponent = fillComponent;
-  }
-
-  @Override
-  public void selfCheck() {
-    super.selfCheck();
-
-    if (disableAlign()) {
-      throw new SemanticException("FILL doesn't support disable align clause.");
-    }
-
-    if (hasTimeSeriesGeneratingFunction() || hasUserDefinedAggregationFunction()) {
-      throw new SemanticException("Fill functions are not supported in UDF queries.");
-    }
-
-    if (whereCondition == null || whereCondition.getQueryFilter() == null) {
-      throw new SemanticException("FILL must be used with a WHERE clause");
-    }
-
-    QueryFilter queryFilter = whereCondition.getQueryFilter();
-    if (!queryFilter.isLeaf()
-        || queryFilter.getFilterType() != FilterConstant.FilterType.EQUAL
-        || !Arrays.equals(
-            SQLConstant.getSingleTimeArray(),
-            whereCondition.getQueryFilter().getSinglePath().getNodes())) {
-      throw new SemanticException("The condition of WHERE clause must be like time=constant");
-    } else if (!queryFilter.isSingle()) {
-      throw new SemanticException("Slice query must select a single time point");
-    }
-  }
-
-  @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitFillQuery(this, context);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/GroupByFillQueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/GroupByFillQueryStatement.java
deleted file mode 100644
index 4f050f39d3..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/GroupByFillQueryStatement.java
+++ /dev/null
@@ -1,49 +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.mpp.plan.statement.crud;
-
-import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.FillComponent;
-
-public class GroupByFillQueryStatement extends GroupByQueryStatement {
-
-  FillComponent fillComponent;
-
-  public GroupByFillQueryStatement() {
-    super();
-  }
-
-  public GroupByFillQueryStatement(QueryStatement queryStatement) {
-    super(queryStatement);
-  }
-
-  public FillComponent getFillComponent() {
-    return fillComponent;
-  }
-
-  public void setFillComponent(FillComponent fillComponent) {
-    this.fillComponent = fillComponent;
-  }
-
-  @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitGroupByFillQuery(this, context);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/GroupByQueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/GroupByQueryStatement.java
deleted file mode 100644
index ccf9c20680..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/GroupByQueryStatement.java
+++ /dev/null
@@ -1,61 +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.mpp.plan.statement.crud;
-
-import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
-import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
-import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.GroupByTimeComponent;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class GroupByQueryStatement extends AggregationQueryStatement {
-
-  protected GroupByTimeComponent groupByTimeComponent;
-
-  public GroupByQueryStatement() {
-    super();
-  }
-
-  public GroupByQueryStatement(QueryStatement queryStatement) {
-    super(queryStatement);
-  }
-
-  public GroupByTimeComponent getGroupByTimeComponent() {
-    return groupByTimeComponent;
-  }
-
-  public void setGroupByTimeComponent(GroupByTimeComponent groupByTimeComponent) {
-    this.groupByTimeComponent = groupByTimeComponent;
-  }
-
-  @Override
-  public DatasetHeader constructDatasetHeader() {
-    List<ColumnHeader> columnHeaders = new ArrayList<>();
-    // TODO: consider GROUP BY
-    return new DatasetHeader(columnHeaders, false);
-  }
-
-  @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitGroupByQuery(this, context);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/LastQueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/LastQueryStatement.java
deleted file mode 100644
index c2776b7037..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/LastQueryStatement.java
+++ /dev/null
@@ -1,74 +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.mpp.plan.statement.crud;
-
-import org.apache.iotdb.db.exception.sql.SemanticException;
-import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
-import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
-import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
-import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class LastQueryStatement extends QueryStatement {
-
-  public LastQueryStatement() {
-    super();
-  }
-
-  public LastQueryStatement(QueryStatement queryStatement) {
-    super(queryStatement);
-  }
-
-  @Override
-  public DatasetHeader constructDatasetHeader() {
-    List<ColumnHeader> columnHeaders = new ArrayList<>();
-    // TODO: consider LAST
-    return new DatasetHeader(columnHeaders, false);
-  }
-
-  @Override
-  public void selfCheck() {
-    super.selfCheck();
-
-    if (isAlignByDevice()) {
-      throw new SemanticException("Last query doesn't support align by device.");
-    }
-
-    if (disableAlign()) {
-      throw new SemanticException("Disable align cannot be applied to LAST query.");
-    }
-
-    for (ResultColumn resultColumn : selectComponent.getResultColumns()) {
-      Expression expression = resultColumn.getExpression();
-      if (!(expression instanceof TimeSeriesOperand)) {
-        throw new SemanticException("Last queries can only be applied on raw time series.");
-      }
-    }
-  }
-
-  @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitLastQuery(this, context);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java
index 3c635cfdf4..75dbc40439 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java
@@ -21,34 +21,24 @@ package org.apache.iotdb.db.mpp.plan.statement.crud;
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.sql.SemanticException;
-import org.apache.iotdb.db.index.common.IndexType;
-import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
-import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
-import org.apache.iotdb.db.mpp.common.header.HeaderConstant;
+import org.apache.iotdb.db.mpp.plan.analyze.ExpressionAnalyzer;
 import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
+import org.apache.iotdb.db.mpp.plan.statement.component.FillComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.FilterNullComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.FromComponent;
+import org.apache.iotdb.db.mpp.plan.statement.component.GroupByLevelComponent;
+import org.apache.iotdb.db.mpp.plan.statement.component.GroupByTimeComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
 import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
 import org.apache.iotdb.db.mpp.plan.statement.component.ResultSetFormat;
 import org.apache.iotdb.db.mpp.plan.statement.component.SelectComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition;
-import org.apache.iotdb.db.qp.constant.SQLConstant;
-import org.apache.iotdb.db.qp.physical.crud.MeasurementInfo;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-import org.apache.iotdb.db.query.expression.multi.FunctionExpression;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Base class of SELECT statement.
@@ -57,13 +47,12 @@ import java.util.stream.Collectors;
  *
  * <ul>
  *   SELECT
- *   <li>[LAST] [TOP k]
- *   <li>resultColumn [, resultColumn] ...
+ *   <li>[LAST] resultColumn [, resultColumn] ...
  *   <li>FROM prefixPath [, prefixPath] ...
  *   <li>WHERE whereCondition
  *   <li>[GROUP BY ([startTime, endTime), interval, slidingStep)]
  *   <li>[GROUP BY LEVEL = levelNum [, levelNum] ...]
- *   <li>[FILL ({PREVIOUS, beforeRange | LINEAR, beforeRange, afterRange | constant})]
+ *   <li>[FILL ({PREVIOUS | LINEAR | constant})]
  *   <li>[LIMIT rowLimit] [OFFSET rowOffset]
  *   <li>[SLIMIT seriesLimit] [SOFFSET seriesOffset]
  *   <li>[WITHOUT NULL {ANY | ALL} [resultColumn [, resultColumn] ...]]
@@ -87,17 +76,19 @@ public class QueryStatement extends Statement {
   // series offset for result set. The default value is 0
   protected int seriesOffset = 0;
 
+  protected FillComponent fillComponent;
+
   protected FilterNullComponent filterNullComponent;
 
   protected OrderBy resultOrder = OrderBy.TIMESTAMP_ASC;
 
   protected ResultSetFormat resultSetFormat = ResultSetFormat.ALIGN_BY_TIME;
 
-  // used for TOP_N, LIKE, CONTAIN
-  protected Map<String, Object> props;
+  // `GROUP BY TIME` clause
+  protected GroupByTimeComponent groupByTimeComponent;
 
-  // TODO: add comments
-  protected IndexType indexType;
+  // `GROUP BY LEVEL` clause
+  protected GroupByLevelComponent groupByLevelComponent;
 
   public QueryStatement() {
     this.statementType = StatementType.QUERY;
@@ -108,22 +99,6 @@ public class QueryStatement extends Statement {
     return fromComponent.getPrefixPaths();
   }
 
-  public QueryStatement(QueryStatement another) {
-    this.statementType = StatementType.QUERY;
-    this.selectComponent = another.getSelectComponent();
-    this.fromComponent = another.getFromComponent();
-    this.whereCondition = another.getWhereCondition();
-    this.rowLimit = another.getRowLimit();
-    this.rowOffset = another.getRowOffset();
-    this.seriesLimit = another.getSeriesLimit();
-    this.seriesOffset = another.getSeriesOffset();
-    this.filterNullComponent = another.getFilterNullComponent();
-    this.resultOrder = another.getResultOrder();
-    this.resultSetFormat = another.getResultSetFormat();
-    this.props = another.getProps();
-    this.indexType = another.getIndexType();
-  }
-
   public SelectComponent getSelectComponent() {
     return selectComponent;
   }
@@ -180,10 +155,12 @@ public class QueryStatement extends Statement {
     this.seriesOffset = seriesOffset;
   }
 
-  /** Reset sLimit and sOffset. */
-  public void resetSLimitOffset() {
-    this.seriesLimit = 0;
-    this.seriesOffset = 0;
+  public FillComponent getFillComponent() {
+    return fillComponent;
+  }
+
+  public void setFillComponent(FillComponent fillComponent) {
+    this.fillComponent = fillComponent;
   }
 
   public FilterNullComponent getFilterNullComponent() {
@@ -210,31 +187,36 @@ public class QueryStatement extends Statement {
     this.resultSetFormat = resultSetFormat;
   }
 
-  public Map<String, Object> getProps() {
-    return props;
+  public GroupByTimeComponent getGroupByTimeComponent() {
+    return groupByTimeComponent;
   }
 
-  public void addProp(String prop, Object value) {
-    if (props == null) {
-      props = new HashMap<>();
-    }
-    props.put(prop, value);
+  public void setGroupByTimeComponent(GroupByTimeComponent groupByTimeComponent) {
+    this.groupByTimeComponent = groupByTimeComponent;
   }
 
-  public void setProps(Map<String, Object> props) {
-    this.props = props;
+  public GroupByLevelComponent getGroupByLevelComponent() {
+    return groupByLevelComponent;
   }
 
-  public IndexType getIndexType() {
-    return indexType;
+  public void setGroupByLevelComponent(GroupByLevelComponent groupByLevelComponent) {
+    this.groupByLevelComponent = groupByLevelComponent;
   }
 
-  public void setIndexType(IndexType indexType) {
-    this.indexType = indexType;
+  public boolean isLastQuery() {
+    return selectComponent.isHasLast();
+  }
+
+  public boolean isAggregationQuery() {
+    return selectComponent.isHasBuiltInAggregationFunction();
   }
 
   public boolean isGroupByLevel() {
-    return false;
+    return groupByLevelComponent != null;
+  }
+
+  public boolean isGroupByTime() {
+    return groupByTimeComponent != null;
   }
 
   public boolean isAlignByDevice() {
@@ -245,6 +227,10 @@ public class QueryStatement extends Statement {
     return resultSetFormat == ResultSetFormat.DISABLE_ALIGN;
   }
 
+  public boolean HasBuiltInAggregationFunction() {
+    return selectComponent.isHasBuiltInAggregationFunction();
+  }
+
   public boolean hasTimeSeriesGeneratingFunction() {
     return selectComponent.isHasTimeSeriesGeneratingFunction();
   }
@@ -253,98 +239,47 @@ public class QueryStatement extends Statement {
     return selectComponent.isHasUserDefinedAggregationFunction();
   }
 
-  public Map<String, Set<PartialPath>> getDeviceNameToDeduplicatedPathsMap() {
-    Map<String, Set<PartialPath>> deviceNameToDeduplicatedPathsMap =
-        new HashMap<>(getSelectComponent().getDeviceNameToDeduplicatedPathsMap());
-    if (getWhereCondition() != null) {
-      for (PartialPath path :
-          getWhereCondition().getQueryFilter().getPathSet().stream()
-              .filter(SQLConstant::isNotReservedPath)
-              .collect(Collectors.toList())) {
-        deviceNameToDeduplicatedPathsMap
-            .computeIfAbsent(path.getDeviceIdString(), k -> new HashSet<>())
-            .add(path);
+  public void semanticCheck() {
+    if (isAlignByDevice()) {
+      // the paths can only be measurement or one-level wildcard in ALIGN BY DEVICE
+      for (ResultColumn resultColumn : selectComponent.getResultColumns()) {
+        ExpressionAnalyzer.checkIsAllMeasurement(resultColumn.getExpression());
+      }
+      if (getWhereCondition() != null) {
+        ExpressionAnalyzer.checkIsAllMeasurement(getWhereCondition().getPredicate());
       }
     }
-    return deviceNameToDeduplicatedPathsMap;
-  }
-
-  public List<String> getSelectedPathNames() {
-    Set<String> pathSet = new HashSet<>();
-    for (ResultColumn resultColumn : getSelectComponent().getResultColumns()) {
-      pathSet.addAll(
-          resultColumn.collectPaths().stream()
-              .map(PartialPath::getFullPath)
-              .collect(Collectors.toList()));
-    }
-    return new ArrayList<>(pathSet);
-  }
-
-  public DatasetHeader constructDatasetHeader() {
-    List<ColumnHeader> columnHeaders = new ArrayList<>();
-    if (this.isAlignByDevice()) {
-      // add DEVICE column
-      columnHeaders.add(new ColumnHeader(HeaderConstant.COLUMN_DEVICE, TSDataType.TEXT, null));
-
-      // TODO: consider ALIGN BY DEVICE
-    } else {
-      columnHeaders.addAll(
-          this.getSelectComponent().getResultColumns().stream()
-              .map(ResultColumn::constructColumnHeader)
-              .collect(Collectors.toList()));
-    }
-    return new DatasetHeader(columnHeaders, false);
-  }
-
-  /**
-   * If path is a vectorPartialPath, we return its measurementId + subMeasurement as the final
-   * measurement. e.g. path: root.sg.d1.vector1[s1], return "vector1.s1".
-   */
-  private String getMeasurementName(PartialPath path, String aggregation) {
-    String initialMeasurement = path.getMeasurement();
-    if (aggregation != null) {
-      initialMeasurement = aggregation + "(" + initialMeasurement + ")";
-    }
-    return initialMeasurement;
-  }
-
-  private PartialPath getPathFromExpression(Expression expression) {
-    return expression instanceof TimeSeriesOperand
-        ? ((TimeSeriesOperand) expression).getPath()
-        : (((FunctionExpression) expression).getPaths().get(0));
-  }
-
-  private String getAggregationFromExpression(Expression expression) {
-    return expression.isBuiltInAggregationFunctionExpression()
-        ? ((FunctionExpression) expression).getFunctionName()
-        : null;
-  }
 
-  /** semantic check */
-  public void selfCheck() {
-    if (isAlignByDevice()) {
-      if (hasTimeSeriesGeneratingFunction() || hasUserDefinedAggregationFunction()) {
-        throw new SemanticException("The ALIGN BY DEVICE clause is not supported in UDF queries.");
+    if (isLastQuery()) {
+      if (isAlignByDevice()) {
+        throw new SemanticException("Last query doesn't support align by device.");
       }
-
-      for (PartialPath path : selectComponent.getPaths()) {
-        if (path.getNodes().length > 1) {
-          throw new SemanticException(
-              "The paths of the SELECT clause can only be measurements or wildcard.");
+      if (disableAlign()) {
+        throw new SemanticException("Disable align cannot be applied to LAST query.");
+      }
+      for (ResultColumn resultColumn : selectComponent.getResultColumns()) {
+        Expression expression = resultColumn.getExpression();
+        if (!(expression instanceof TimeSeriesOperand)) {
+          throw new SemanticException("Last queries can only be applied on raw time series.");
         }
       }
     }
-  }
 
-  /**
-   * Check datatype consistency in ALIGN BY DEVICE.
-   *
-   * <p>an inconsistent example: select s0 from root.sg1.d1, root.sg1.d2 align by device, return
-   * false while root.sg1.d1.s0 is INT32 and root.sg1.d2.s0 is FLOAT.
-   */
-  private boolean checkDataTypeConsistency(
-      TSDataType checkedDataType, MeasurementInfo measurementInfo) {
-    return measurementInfo == null || checkedDataType.equals(measurementInfo.getColumnDataType());
+    if (isAggregationQuery()) {
+      if (disableAlign()) {
+        throw new SemanticException("AGGREGATION doesn't support disable align clause.");
+      }
+      if (isGroupByLevel() && isAlignByDevice()) {
+        throw new SemanticException("group by level does not support align by device now.");
+      }
+      if (hasTimeSeriesGeneratingFunction()) {
+        throw new SemanticException(
+            "User-defined and built-in hybrid aggregation is not supported together.");
+      }
+      for (ResultColumn resultColumn : selectComponent.getResultColumns()) {
+        ExpressionAnalyzer.checkIsAllAggregation(resultColumn.getExpression());
+      }
+    }
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/UDAFQueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/UDAFQueryStatement.java
deleted file mode 100644
index 766e0b4177..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/UDAFQueryStatement.java
+++ /dev/null
@@ -1,97 +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.mpp.plan.statement.crud;
-
-import org.apache.iotdb.db.exception.sql.SemanticException;
-import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
-import org.apache.iotdb.db.mpp.plan.statement.component.SelectComponent;
-import org.apache.iotdb.db.query.expression.Expression;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
-
-import java.util.List;
-
-/**
- * For a UDAF Statement, we construct an inner AggregationStatement for it. Example: select
- * count(a)/count(b),count(a)+sum(b) from root.sg To init inner AggregationStatement, we will
- * convert it to statement: select count(a),count(b),count(a),sum(b) from root.sg
- * innerResultColumnsCache will be [count(a),count(b),sum(b)]
- */
-public class UDAFQueryStatement extends QueryStatement {
-
-  private List<ResultColumn> innerResultColumnsCache;
-
-  private AggregationQueryStatement innerAggregationQueryStatement;
-
-  public UDAFQueryStatement(AggregationQueryStatement querySatement) {
-    super(querySatement);
-    this.innerAggregationQueryStatement = querySatement;
-  }
-
-  @Override
-  public void selfCheck() {
-    super.selfCheck();
-
-    if (!disableAlign()) {
-      throw new SemanticException("AGGREGATION doesn't support disable align clause.");
-    }
-    checkSelectComponent(selectComponent);
-    if (isGroupByLevel()) {
-      throw new SemanticException(
-          "UDF nesting aggregations in GROUP BY query does not support grouping by level now.");
-    }
-    if (innerAggregationQueryStatement instanceof GroupByQueryStatement) {
-      throw new SemanticException(
-          "UDF nesting aggregations in GROUP BY query does not support FILL now.");
-    }
-  }
-
-  private void checkSelectComponent(SelectComponent selectComponent) throws SemanticException {
-    for (ResultColumn resultColumn : selectComponent.getResultColumns()) {
-      Expression expression = resultColumn.getExpression();
-      checkEachExpression(expression);
-    }
-  }
-
-  private void checkEachExpression(Expression expression) throws SemanticException {
-    if (expression instanceof TimeSeriesOperand) {
-      throw new SemanticException(
-          "Common queries and aggregated queries are not allowed to appear at the same time.");
-    }
-    // Currently, the aggregation function expression can only contain a timeseries operand.
-    if (expression.isBuiltInAggregationFunctionExpression()) {
-      if (expression.getExpressions().size() == 1
-          && expression.getExpressions().get(0) instanceof TimeSeriesOperand) {
-        return;
-      }
-      throw new SemanticException(
-          "The argument of the aggregation function must be a time series.");
-    }
-
-    for (Expression childExp : expression.getExpressions()) {
-      checkEachExpression(childExp);
-    }
-  }
-
-  @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitUDAFQuery(this, context);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/BooleanLiteral.java
similarity index 53%
copy from server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/BooleanLiteral.java
index 26d91956f1..dc8657555d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/BooleanLiteral.java
@@ -17,37 +17,41 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
+package org.apache.iotdb.db.mpp.plan.statement.literal;
 
-import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
-import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
-public class FillDescriptor {
+public class BooleanLiteral extends Literal {
+  public static final BooleanLiteral TRUE_LITERAL = new BooleanLiteral("true");
+  public static final BooleanLiteral FALSE_LITERAL = new BooleanLiteral("false");
 
-  // policy of fill null values
-  private final FillPolicy fillPolicy;
+  private final boolean value;
 
-  // target column for fill
-  private final Expression expression;
+  public BooleanLiteral(String value) {
+    this.value = Boolean.parseBoolean(value);
+  }
+
+  public BooleanLiteral(boolean value) {
+    this.value = value;
+  }
 
-  public FillDescriptor(FillPolicy fillPolicy, Expression expression) {
-    this.fillPolicy = fillPolicy;
-    this.expression = expression;
+  public boolean getValue() {
+    return value;
   }
 
+  @Override
   public void serialize(ByteBuffer byteBuffer) {
-    ReadWriteIOUtils.write(fillPolicy.ordinal(), byteBuffer);
-    Expression.serialize(expression, byteBuffer);
+    ReadWriteIOUtils.write(LiteralType.BOOLEAN.ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(value, byteBuffer);
   }
 
-  public static FillDescriptor deserialize(ByteBuffer byteBuffer) {
-    FillPolicy fillPolicy = FillPolicy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
-    Expression expression = Expression.deserialize(byteBuffer);
-    return new FillDescriptor(fillPolicy, expression);
+  @Override
+  public boolean isDataTypeConsistency(TSDataType dataType) {
+    return dataType == TSDataType.BOOLEAN;
   }
 
   @Override
@@ -58,12 +62,12 @@ public class FillDescriptor {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    FillDescriptor that = (FillDescriptor) o;
-    return fillPolicy == that.fillPolicy && Objects.equals(expression, that.expression);
+    BooleanLiteral that = (BooleanLiteral) o;
+    return value == that.value;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(fillPolicy, expression);
+    return Objects.hash(value);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/DoubleLiteral.java
similarity index 53%
copy from server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/DoubleLiteral.java
index 26d91956f1..de596dca37 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/DoubleLiteral.java
@@ -17,37 +17,38 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
+package org.apache.iotdb.db.mpp.plan.statement.literal;
 
-import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
-import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
-public class FillDescriptor {
+public class DoubleLiteral extends Literal {
+  private final double value;
 
-  // policy of fill null values
-  private final FillPolicy fillPolicy;
+  public DoubleLiteral(String value) {
+    this.value = Double.parseDouble(value);
+  }
 
-  // target column for fill
-  private final Expression expression;
+  public DoubleLiteral(double value) {
+    this.value = value;
+  }
 
-  public FillDescriptor(FillPolicy fillPolicy, Expression expression) {
-    this.fillPolicy = fillPolicy;
-    this.expression = expression;
+  public double getValue() {
+    return value;
   }
 
+  @Override
   public void serialize(ByteBuffer byteBuffer) {
-    ReadWriteIOUtils.write(fillPolicy.ordinal(), byteBuffer);
-    Expression.serialize(expression, byteBuffer);
+    ReadWriteIOUtils.write(LiteralType.DOUBLE.ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(value, byteBuffer);
   }
 
-  public static FillDescriptor deserialize(ByteBuffer byteBuffer) {
-    FillPolicy fillPolicy = FillPolicy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
-    Expression expression = Expression.deserialize(byteBuffer);
-    return new FillDescriptor(fillPolicy, expression);
+  @Override
+  public boolean isDataTypeConsistency(TSDataType dataType) {
+    return dataType == TSDataType.FLOAT || dataType == TSDataType.DOUBLE;
   }
 
   @Override
@@ -58,12 +59,12 @@ public class FillDescriptor {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    FillDescriptor that = (FillDescriptor) o;
-    return fillPolicy == that.fillPolicy && Objects.equals(expression, that.expression);
+    DoubleLiteral that = (DoubleLiteral) o;
+    return Double.compare(that.value, value) == 0;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(fillPolicy, expression);
+    return Objects.hash(value);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/Literal.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/Literal.java
new file mode 100644
index 0000000000..e35a8cbf2e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/Literal.java
@@ -0,0 +1,59 @@
+/*
+ * 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.mpp.plan.statement.literal;
+
+import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.nio.ByteBuffer;
+
+public abstract class Literal extends StatementNode {
+
+  public enum LiteralType {
+    BOOLEAN,
+    DOUBLE,
+    LONG,
+    STRING,
+    NULL
+  }
+
+  public static Literal deserialize(ByteBuffer byteBuffer) {
+    LiteralType type = LiteralType.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    switch (type) {
+      case BOOLEAN:
+        return new BooleanLiteral(ReadWriteIOUtils.readBool(byteBuffer));
+      case DOUBLE:
+        return new DoubleLiteral(ReadWriteIOUtils.readDouble(byteBuffer));
+      case LONG:
+        return new LongLiteral(ReadWriteIOUtils.readLong(byteBuffer));
+      case STRING:
+        return new StringLiteral(ReadWriteIOUtils.readString(byteBuffer));
+      case NULL:
+        return new NullLiteral();
+      default:
+        throw new IllegalArgumentException(String.format("Unknown literal type: %s", type));
+    }
+  }
+
+  public abstract void serialize(ByteBuffer byteBuffer);
+
+  public abstract boolean isDataTypeConsistency(TSDataType dataType);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/LongLiteral.java
similarity index 53%
copy from server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/LongLiteral.java
index 26d91956f1..4a177d8ed6 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/LongLiteral.java
@@ -17,37 +17,38 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
+package org.apache.iotdb.db.mpp.plan.statement.literal;
 
-import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
-import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
-public class FillDescriptor {
+public class LongLiteral extends Literal {
+  private final long value;
 
-  // policy of fill null values
-  private final FillPolicy fillPolicy;
+  public LongLiteral(String value) {
+    this.value = Long.parseLong(value);
+  }
 
-  // target column for fill
-  private final Expression expression;
+  public LongLiteral(long value) {
+    this.value = value;
+  }
 
-  public FillDescriptor(FillPolicy fillPolicy, Expression expression) {
-    this.fillPolicy = fillPolicy;
-    this.expression = expression;
+  public long getValue() {
+    return value;
   }
 
+  @Override
   public void serialize(ByteBuffer byteBuffer) {
-    ReadWriteIOUtils.write(fillPolicy.ordinal(), byteBuffer);
-    Expression.serialize(expression, byteBuffer);
+    ReadWriteIOUtils.write(LiteralType.LONG.ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(value, byteBuffer);
   }
 
-  public static FillDescriptor deserialize(ByteBuffer byteBuffer) {
-    FillPolicy fillPolicy = FillPolicy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
-    Expression expression = Expression.deserialize(byteBuffer);
-    return new FillDescriptor(fillPolicy, expression);
+  @Override
+  public boolean isDataTypeConsistency(TSDataType dataType) {
+    return dataType == TSDataType.INT32 || dataType == TSDataType.INT64;
   }
 
   @Override
@@ -58,12 +59,12 @@ public class FillDescriptor {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    FillDescriptor that = (FillDescriptor) o;
-    return fillPolicy == that.fillPolicy && Objects.equals(expression, that.expression);
+    LongLiteral that = (LongLiteral) o;
+    return value == that.value;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(fillPolicy, expression);
+    return Objects.hash(value);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/UDTFQueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/NullLiteral.java
similarity index 53%
rename from server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/UDTFQueryStatement.java
rename to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/NullLiteral.java
index d99d4be93c..655c848288 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/UDTFQueryStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/NullLiteral.java
@@ -17,18 +17,38 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.mpp.plan.statement.crud;
+package org.apache.iotdb.db.mpp.plan.statement.literal;
 
-import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
-public class UDTFQueryStatement extends QueryStatement {
+import java.nio.ByteBuffer;
 
-  public UDTFQueryStatement(QueryStatement queryStatement) {
-    super(queryStatement);
+public class NullLiteral extends Literal {
+
+  @Override
+  public void serialize(ByteBuffer byteBuffer) {
+    ReadWriteIOUtils.write(LiteralType.NULL.ordinal(), byteBuffer);
+  }
+
+  @Override
+  public boolean isDataTypeConsistency(TSDataType dataType) {
+    return false;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    return true;
   }
 
   @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitUDTFQuery(this, context);
+  public int hashCode() {
+    return getClass().hashCode();
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/StringLiteral.java
similarity index 53%
copy from server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/StringLiteral.java
index 26d91956f1..f932e0d094 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/FillDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/literal/StringLiteral.java
@@ -17,37 +17,34 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
+package org.apache.iotdb.db.mpp.plan.statement.literal;
 
-import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
-import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
-public class FillDescriptor {
+public class StringLiteral extends Literal {
+  private final String value;
 
-  // policy of fill null values
-  private final FillPolicy fillPolicy;
-
-  // target column for fill
-  private final Expression expression;
+  public StringLiteral(String value) {
+    this.value = value;
+  }
 
-  public FillDescriptor(FillPolicy fillPolicy, Expression expression) {
-    this.fillPolicy = fillPolicy;
-    this.expression = expression;
+  public String getValue() {
+    return value;
   }
 
+  @Override
   public void serialize(ByteBuffer byteBuffer) {
-    ReadWriteIOUtils.write(fillPolicy.ordinal(), byteBuffer);
-    Expression.serialize(expression, byteBuffer);
+    ReadWriteIOUtils.write(LiteralType.STRING.ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(value, byteBuffer);
   }
 
-  public static FillDescriptor deserialize(ByteBuffer byteBuffer) {
-    FillPolicy fillPolicy = FillPolicy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
-    Expression expression = Expression.deserialize(byteBuffer);
-    return new FillDescriptor(fillPolicy, expression);
+  @Override
+  public boolean isDataTypeConsistency(TSDataType dataType) {
+    return dataType == TSDataType.TEXT;
   }
 
   @Override
@@ -58,12 +55,12 @@ public class FillDescriptor {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    FillDescriptor that = (FillDescriptor) o;
-    return fillPolicy == that.fillPolicy && Objects.equals(expression, that.expression);
+    StringLiteral that = (StringLiteral) o;
+    return Objects.equals(value, that.value);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(fillPolicy, expression);
+    return Objects.hash(value);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/FilterConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/FilterConstant.java
index 5fa0416b71..2e4ebbe383 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/FilterConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/FilterConstant.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.db.qp.constant;
 
 import org.apache.iotdb.db.qp.sql.SqlLexer;
+import org.apache.iotdb.db.query.expression.ExpressionType;
 
 import java.util.EnumMap;
 import java.util.HashMap;
@@ -28,6 +29,7 @@ import java.util.Map;
 public class FilterConstant {
 
   public static final Map<Integer, FilterType> lexerToFilterType = new HashMap<>();
+  public static final Map<ExpressionType, FilterType> ExpressionToFilterType = new HashMap<>();
   public static final Map<FilterType, String> filterSymbol = new EnumMap<>(FilterType.class);
   public static final Map<FilterType, String> filterNames = new EnumMap<>(FilterType.class);
   public static final Map<FilterType, FilterType> filterReverseWords =
@@ -62,6 +64,15 @@ public class FilterConstant {
     lexerToFilterType.put(SqlLexer.LIKE, FilterType.LIKE);
   }
 
+  static {
+    ExpressionToFilterType.put(ExpressionType.EQUAL_TO, FilterType.EQUAL);
+    ExpressionToFilterType.put(ExpressionType.NON_EQUAL, FilterType.NOTEQUAL);
+    ExpressionToFilterType.put(ExpressionType.LESS_EQUAL, FilterType.LESSTHANOREQUALTO);
+    ExpressionToFilterType.put(ExpressionType.LESS_THAN, FilterType.LESSTHAN);
+    ExpressionToFilterType.put(ExpressionType.GREATER_EQUAL, FilterType.GREATERTHANOREQUALTO);
+    ExpressionToFilterType.put(ExpressionType.GREATER_THAN, FilterType.GREATERTHAN);
+  }
+
   static {
     filterSymbol.put(FilterType.KW_AND, "&");
     filterSymbol.put(FilterType.KW_OR, "|");
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InOperator.java
index 4a588ac127..d3beaefa58 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InOperator.java
@@ -72,6 +72,10 @@ public class InOperator extends FunctionOperator {
     return not;
   }
 
+  public void setNot(boolean not) {
+    this.not = not;
+  }
+
   @Override
   public void reverseFunc() {
     not = !not;
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 3d776fdd43..f4481bfc0b 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
@@ -24,7 +24,6 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.trigger.executor.TriggerEvent;
 import org.apache.iotdb.db.exception.sql.SQLParserException;
-import org.apache.iotdb.db.index.common.IndexType;
 import org.apache.iotdb.db.qp.constant.FilterConstant;
 import org.apache.iotdb.db.qp.constant.FilterConstant.FilterType;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
@@ -130,6 +129,7 @@ import org.apache.iotdb.db.query.executor.fill.ValueFill;
 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.BinaryExpression;
 import org.apache.iotdb.db.query.expression.binary.DivisionExpression;
 import org.apache.iotdb.db.query.expression.binary.EqualToExpression;
 import org.apache.iotdb.db.query.expression.binary.GreaterEqualExpression;
@@ -177,9 +177,6 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import static org.apache.iotdb.db.index.common.IndexConstant.PATTERN;
-import static org.apache.iotdb.db.index.common.IndexConstant.THRESHOLD;
-import static org.apache.iotdb.db.index.common.IndexConstant.TOP_K;
 import static org.apache.iotdb.db.qp.constant.SQLConstant.TIME_PATH;
 import static org.apache.iotdb.db.qp.constant.SQLConstant.TOK_KILL_QUERY;
 
@@ -1097,9 +1094,7 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     parseFromClause(ctx.fromClause());
     if (ctx.whereClause() != null) {
       WhereComponent whereComponent = parseWhereClause(ctx.whereClause());
-      if (whereComponent != null) {
-        queryOp.setWhereComponent(whereComponent);
-      }
+      queryOp.setWhereComponent(whereComponent);
     }
     queryOp.setEnableTracing(ctx.TRACING() != null);
     // 4. Check whether it's a select-into clause
@@ -1237,11 +1232,11 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   public Operator visitFillStatement(IoTDBSqlParser.FillStatementContext ctx) {
     queryOp = new FillQueryOperator();
     parseFillClause(ctx.fillClause());
-    if (ctx.slimitClause() != null) {
-      parseSlimitClause(ctx.slimitClause());
+    if (ctx.orderByTimeClause() != null) {
+      parseOrderByTimeClause(ctx.orderByTimeClause());
     }
-    if (ctx.alignByDeviceClauseOrDisableAlign() != null) {
-      parseAlignByDeviceClauseOrDisableAlign(ctx.alignByDeviceClauseOrDisableAlign());
+    if (ctx.specialLimit() != null) {
+      return visit(ctx.specialLimit());
     }
     return queryOp;
   }
@@ -1438,7 +1433,9 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
       levels[i] = Integer.parseInt(ctx.INTEGER_LITERAL().get(i).getText());
     }
     groupByLevelClauseComponent.setLevels(levels);
-
+    if (ctx.fillClause() != null) {
+      parseFillClause(ctx.fillClause());
+    }
     queryOp.setSpecialClauseComponent(groupByLevelClauseComponent);
   }
 
@@ -2304,9 +2301,7 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
 
     // parse select
     IoTDBSqlParser.SelectClauseContext selectCtx = ctx.selectClause();
-    if (selectCtx.LAST() != null
-        || selectCtx.topClause() != null
-        || selectCtx.resultColumn().size() != 1) {
+    if (selectCtx.LAST() != null || selectCtx.resultColumn().size() != 1) {
       throw new SQLParserException("Not support for this sql in pipe.");
     }
     IoTDBSqlParser.ResultColumnContext resultColumnCtx = selectCtx.resultColumn(0);
@@ -2326,30 +2321,28 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     // parse where
     IoTDBSqlParser.WhereClauseContext whereCtx = ctx.whereClause();
     if (whereCtx != null) {
-      if (whereCtx.orExpression() == null || whereCtx.indexPredicateClause() != null) {
+      Expression predicate =
+          parseExpression(whereCtx.expression(), whereCtx.expression().OPERATOR_NOT() == null);
+      if (!((predicate instanceof GreaterThanExpression)
+          || (predicate instanceof GreaterEqualExpression))) {
         throw new SQLParserException("Not support for this sql in pipe.");
       }
-      IoTDBSqlParser.OrExpressionContext orExpressionCtx = whereCtx.orExpression();
-      if (orExpressionCtx.andExpression().size() != 1) {
+      Expression left = ((BinaryExpression) predicate).getLeftExpression();
+      Expression right = ((BinaryExpression) predicate).getRightExpression();
+      if (!(left instanceof TimeSeriesOperand)) {
         throw new SQLParserException("Not support for this sql in pipe.");
       }
-      IoTDBSqlParser.AndExpressionContext andExpressionCtx = orExpressionCtx.andExpression(0);
-      if (andExpressionCtx.predicate().size() != 1) {
+      if (!SQLConstant.isReservedPath(((TimeSeriesOperand) left).getPath())) {
         throw new SQLParserException("Not support for this sql in pipe.");
       }
-      IoTDBSqlParser.PredicateContext predicateCtx = andExpressionCtx.predicate(0);
-      if (predicateCtx.comparisonOperator() == null
-          || (predicateCtx.comparisonOperator().OPERATOR_GTE() == null
-              && predicateCtx.comparisonOperator().OPERATOR_GT() == null)
-          || predicateCtx.suffixPath() != null
-          || predicateCtx.fullPath() != null) {
+      if (!(right instanceof ConstantOperand)) {
         throw new SQLParserException("Not support for this sql in pipe.");
       }
-      IoTDBSqlParser.ConstantContext constantCtx = predicateCtx.constant();
-      if (constantCtx.dateExpression() == null) {
-        throw new SQLParserException("data type error for time limit");
+      if (((ConstantOperand) right).getDataType() != TSDataType.INT64) {
+        throw new SQLParserException("Not support for this sql in pipe.");
       }
-      operator.setStartTime(parseDateExpression(constantCtx.dateExpression()));
+      long startTime = Long.parseLong(((ConstantOperand) right).getValueString());
+      operator.setStartTime(startTime);
     }
   }
 
@@ -2546,7 +2539,7 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
 
   @SuppressWarnings("squid:S3776")
   private Expression parseExpression(
-      IoTDBSqlParser.ExpressionContext context, boolean inWithoutNull) {
+      IoTDBSqlParser.ExpressionContext context, boolean inWithoutNull, boolean isQueryFilter) {
     if (context.unaryInBracket != null) {
       return parseExpression(context.unaryInBracket, inWithoutNull);
     }
@@ -2593,7 +2586,7 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
       if (context.OPERATOR_LTE() != null) {
         return new LessEqualExpression(leftExpression, rightExpression);
       }
-      if (context.OPERATOR_DEQ() != null) {
+      if (context.OPERATOR_DEQ() != null || context.OPERATOR_SEQ() != null) {
         return new EqualToExpression(leftExpression, rightExpression);
       }
       if (context.OPERATOR_NEQ() != null) {
@@ -2610,10 +2603,10 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
 
     if (context.unaryBeforeRegularOrLikeExpression != null) {
       if (context.REGEXP() != null) {
-        return parseRegularExpression(context, inWithoutNull);
+        return parseRegularExpression(context, inWithoutNull, isQueryFilter);
       }
       if (context.LIKE() != null) {
-        return parseLikeExpression(context, inWithoutNull);
+        return parseLikeExpression(context, inWithoutNull, isQueryFilter);
       }
       throw new UnsupportedOperationException();
     }
@@ -2642,6 +2635,11 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     throw new UnsupportedOperationException();
   }
 
+  private Expression parseExpression(
+      IoTDBSqlParser.ExpressionContext context, boolean inWithoutNull) {
+    return parseExpression(context, inWithoutNull, false);
+  }
+
   private Expression parseFunctionExpression(
       IoTDBSqlParser.ExpressionContext functionClause, boolean inWithoutNull) {
     FunctionExpression functionExpression =
@@ -2654,7 +2652,18 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
       if (!subexpression.isConstantOperand()) {
         hasNonPureConstantSubExpression = true;
       }
-      functionExpression.addExpression(subexpression);
+      if (subexpression instanceof EqualToExpression
+          && ((EqualToExpression) subexpression).getLeftExpression().isConstantOperand()
+          && ((EqualToExpression) subexpression).getRightExpression().isConstantOperand()) {
+        // parse attribute
+        functionExpression.addAttribute(
+            ((ConstantOperand) ((EqualToExpression) subexpression).getLeftExpression())
+                .getValueString(),
+            ((ConstantOperand) ((EqualToExpression) subexpression).getRightExpression())
+                .getValueString());
+      } else {
+        functionExpression.addExpression(subexpression);
+      }
     }
 
     // It is not allowed to have function expressions like F(1, 1.0). There should be at least one
@@ -2664,25 +2673,28 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
           "Invalid function expression, all the arguments are constant operands: "
               + functionClause.getText());
     }
-
-    // attributes
-    for (IoTDBSqlParser.FunctionAttributeContext functionAttribute :
-        functionClause.functionAttribute()) {
-      functionExpression.addAttribute(
-          parseAttributeKey(functionAttribute.attributeKey()),
-          parseAttributeValue(functionAttribute.attributeValue()));
-    }
-
     return functionExpression;
   }
 
-  private Expression parseRegularExpression(ExpressionContext context, boolean inWithoutNull) {
+  private Expression parseRegularExpression(
+      ExpressionContext context, boolean inWithoutNull, boolean isQueryFilter) {
+    if (isQueryFilter) {
+      return new RegularExpression(
+          parseExpression(context.unaryBeforeRegularOrLikeExpression, inWithoutNull),
+          context.STRING_LITERAL().getText());
+    }
     return new RegularExpression(
         parseExpression(context.unaryBeforeRegularOrLikeExpression, inWithoutNull),
         parseStringLiteral(context.STRING_LITERAL().getText()));
   }
 
-  private Expression parseLikeExpression(ExpressionContext context, boolean inWithoutNull) {
+  private Expression parseLikeExpression(
+      ExpressionContext context, boolean inWithoutNull, boolean isQueryFilter) {
+    if (isQueryFilter) {
+      return new LikeExpression(
+          parseExpression(context.unaryBeforeRegularOrLikeExpression, inWithoutNull),
+          context.STRING_LITERAL().getText());
+    }
     return new LikeExpression(
         parseExpression(context.unaryBeforeRegularOrLikeExpression, inWithoutNull),
         parseStringLiteral(context.STRING_LITERAL().getText()));
@@ -2692,231 +2704,31 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     Expression childExpression = parseExpression(context.unaryBeforeInExpression, inWithoutNull);
     LinkedHashSet<String> values = new LinkedHashSet<>();
     for (ConstantContext constantContext : context.constant()) {
-      String text = constantContext.getText();
-      values.add(constantContext.STRING_LITERAL() != null ? parseStringLiteral(text) : text);
+      values.add(parseConstant(constantContext));
     }
     return new InExpression(childExpression, context.OPERATOR_NOT() != null, values);
   }
 
-  private Expression parseConstantOperand(ConstantContext constantContext) {
-    try {
-      String text = constantContext.getText();
-      if (clientVersion.equals(IoTDBConstant.ClientVersion.V_0_13)) {
-        if (constantContext.BOOLEAN_LITERAL() != null) {
-          return new ConstantOperand(TSDataType.BOOLEAN, text);
-        } else if (constantContext.STRING_LITERAL() != null) {
-          return new ConstantOperand(TSDataType.TEXT, parseStringLiteral(text));
-        } else if (constantContext.INTEGER_LITERAL() != null) {
-          return new ConstantOperand(TSDataType.INT64, text);
-        } else if (constantContext.realLiteral() != null) {
-          return new ConstantOperand(TSDataType.DOUBLE, text);
-        } else {
-          throw new SQLParserException("Unsupported constant operand: " + text);
-        }
-      } else if (clientVersion.equals(IoTDBConstant.ClientVersion.V_0_12)) {
-        // if client version is before 0.13, node name in expression may be a constant
-        return new TimeSeriesOperand(convertConstantToPath(text));
-      } else {
-        throw new UnsupportedOperationException();
-      }
-    } catch (IllegalPathException e) {
-      throw new SQLParserException(e.getMessage());
-    }
-  }
-
-  private FilterOperator parseOrExpression(IoTDBSqlParser.OrExpressionContext ctx) {
-    if (ctx.andExpression().size() == 1) {
-      return parseAndExpression(ctx.andExpression(0));
-    }
-    FilterOperator binaryOp = new FilterOperator(FilterType.KW_OR);
-    if (ctx.andExpression().size() > 2) {
-      binaryOp.addChildOperator(parseAndExpression(ctx.andExpression(0)));
-      binaryOp.addChildOperator(parseAndExpression(ctx.andExpression(1)));
-      for (int i = 2; i < ctx.andExpression().size(); i++) {
-        FilterOperator op = new FilterOperator(FilterType.KW_OR);
-        op.addChildOperator(binaryOp);
-        op.addChildOperator(parseAndExpression(ctx.andExpression(i)));
-        binaryOp = op;
-      }
-    } else {
-      for (IoTDBSqlParser.AndExpressionContext andExpressionContext : ctx.andExpression()) {
-        binaryOp.addChildOperator(parseAndExpression(andExpressionContext));
-      }
-    }
-    return binaryOp;
-  }
-
-  private FilterOperator parseAndExpression(IoTDBSqlParser.AndExpressionContext ctx) {
-    if (ctx.predicate().size() == 1) {
-      return parsePredicate(ctx.predicate(0));
-    }
-    FilterOperator binaryOp = new FilterOperator(FilterType.KW_AND);
-    int size = ctx.predicate().size();
-    if (size > 2) {
-      binaryOp.addChildOperator(parsePredicate(ctx.predicate(0)));
-      binaryOp.addChildOperator(parsePredicate(ctx.predicate(1)));
-      for (int i = 2; i < size; i++) {
-        FilterOperator op = new FilterOperator(FilterType.KW_AND);
-        op.addChildOperator(binaryOp);
-        op.addChildOperator(parsePredicate(ctx.predicate(i)));
-        binaryOp = op;
-      }
-    } else {
-      for (IoTDBSqlParser.PredicateContext predicateContext : ctx.predicate()) {
-        binaryOp.addChildOperator(parsePredicate(predicateContext));
-      }
-    }
-    return binaryOp;
-  }
-
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private FilterOperator parsePredicate(IoTDBSqlParser.PredicateContext ctx) {
-    PartialPath path = null;
-    if (ctx.OPERATOR_NOT() != null) {
-      FilterOperator notOp = new FilterOperator(FilterType.KW_NOT);
-      notOp.addChildOperator(parseOrExpression(ctx.orExpression()));
-      return notOp;
-    } else if (ctx.LR_BRACKET() != null && ctx.OPERATOR_NOT() == null) {
-      return parseOrExpression(ctx.orExpression());
-    } else if (ctx.REGEXP() != null || ctx.LIKE() != null) {
-      if (ctx.suffixPath() != null) {
-        path = parseSuffixPath(ctx.suffixPath());
-      } else if (ctx.fullPath() != null) {
-        path = parseFullPath(ctx.fullPath());
-      }
-      if (path == null) {
-        throw new SQLParserException("Path is null, please check the sql.");
-      }
-      return ctx.REGEXP() != null
-          ? new RegexpOperator(FilterType.REGEXP, path, ctx.STRING_LITERAL().getText())
-          : new LikeOperator(FilterType.LIKE, path, ctx.STRING_LITERAL().getText());
-    } else {
-      if (ctx.TIME() != null || ctx.TIMESTAMP() != null) {
-        path = new PartialPath(SQLConstant.getSingleTimeArray());
-      }
-      if (ctx.fullPath() != null) {
-        path = parseFullPath(ctx.fullPath());
-      }
-      if (ctx.suffixPath() != null) {
-        path = parseSuffixPath(ctx.suffixPath());
-      }
-      if (path == null) {
-        throw new SQLParserException("Path is null, please check the sql.");
-      }
-      if (ctx.inClause() != null) {
-        return parseInOperator(ctx.inClause(), path);
-      } else {
-        return parseBasicFunctionOperator(ctx, path);
-      }
-    }
-  }
-
-  private FilterOperator parseBasicFunctionOperator(
-      IoTDBSqlParser.PredicateContext ctx, PartialPath path) {
-    BasicFunctionOperator basic;
-    if (ctx.constant().dateExpression() != null) {
-      if (!path.equals(TIME_PATH)) {
-        throw new SQLParserException(path.getFullPath(), "Date can only be used to time");
-      }
-      basic =
-          new BasicFunctionOperator(
-              FilterConstant.lexerToFilterType.get(ctx.comparisonOperator().type.getType()),
-              path,
-              Long.toString(parseDateExpression(ctx.constant().dateExpression())));
-    } else {
-      basic =
-          new BasicFunctionOperator(
-              FilterConstant.lexerToFilterType.get(ctx.comparisonOperator().type.getType()),
-              path,
-              parseStringLiteral(ctx.constant().getText()));
-    }
-    return basic;
-  }
-
-  private FilterOperator parseInOperator(IoTDBSqlParser.InClauseContext ctx, PartialPath path) {
-    Set<String> values = new HashSet<>();
-    boolean not = ctx.OPERATOR_NOT() != null;
-    for (IoTDBSqlParser.ConstantContext constant : ctx.constant()) {
-      if (constant.dateExpression() != null) {
-        if (!path.equals(TIME_PATH)) {
-          throw new SQLParserException(path.getFullPath(), "Date can only be used to time");
-        }
-        values.add(Long.toString(parseDateExpression(constant.dateExpression())));
-      } else {
-        values.add(constant.getText());
-      }
-    }
-    return new InOperator(FilterType.IN, path, not, values);
-  }
-
-  private void parseIndexPredicate(IoTDBSqlParser.IndexPredicateClauseContext ctx) {
-    Map<String, Object> props;
-    PartialPath path;
-    if (ctx.suffixPath() != null) {
-      path = parseSuffixPath(ctx.suffixPath());
-    } else {
-      path = parseFullPath(ctx.fullPath());
-    }
-    if (ctx.LIKE() != null) {
-      // whole matching case
-      if (queryOp.getSelectComponent().getResultColumns().size() != 1) {
-        throw new SQLParserException("Index query statement allows only one select path");
-      }
-      if (!path.equals(
-          queryOp.getSelectComponent().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");
-      }
-      if (queryOp.getProps() != null) {
-        props = queryOp.getProps();
-      } else {
-        props = new HashMap<>();
-      }
-      props.put(PATTERN, parseSequence(ctx.sequenceClause(0)));
-      queryOp.setIndexType(IndexType.RTREE_PAA);
-    } else if (ctx.CONTAIN() != null) {
-      // subsequence matching case
-      List<double[]> compositePattern = new ArrayList<>();
-      List<Double> thresholds = new ArrayList<>();
-      for (int i = 0; i < ctx.sequenceClause().size(); i++) {
-        compositePattern.add(parseSequence(ctx.sequenceClause(i)));
-        thresholds.add(Double.parseDouble(ctx.constant(i).getText()));
-      }
-      if (queryOp.getProps() != null) {
-        props = queryOp.getProps();
-      } else {
-        props = new HashMap<>();
-      }
-      List<ResultColumn> resultColumns = new ArrayList<>();
-      resultColumns.add(new ResultColumn(new TimeSeriesOperand(path)));
-      queryOp.getSelectComponent().setResultColumns(resultColumns);
-      props.put(PATTERN, compositePattern);
-      props.put(THRESHOLD, thresholds);
-      queryOp.setIndexType(IndexType.ELB_INDEX);
+  private String parseConstant(ConstantContext constantContext) {
+    String text = constantContext.getText();
+    if (constantContext.BOOLEAN_LITERAL() != null
+        || constantContext.INTEGER_LITERAL() != null
+        || constantContext.realLiteral() != null) {
+      return text;
+    } else if (constantContext.STRING_LITERAL() != null) {
+      return parseStringLiteral(text);
+    } else if (constantContext.dateExpression() != null) {
+      return String.valueOf(parseDateExpression(constantContext.dateExpression()));
     } else {
-      throw new SQLParserException("Unknown index predicate: " + ctx);
-    }
-    queryOp.setProps(props);
-  }
-
-  private double[] parseSequence(IoTDBSqlParser.SequenceClauseContext ctx) {
-    int seqLen = ctx.constant().size();
-    double[] sequence = new double[seqLen];
-    for (int i = 0; i < seqLen; i++) {
-      sequence[i] = Double.parseDouble(ctx.constant(i).getText());
+      throw new IllegalArgumentException("Unsupported constant operand: " + text);
     }
-    return sequence;
   }
 
   // Select Clause
 
   public void parseSelectClause(IoTDBSqlParser.SelectClauseContext ctx) {
     SelectComponent selectComponent = new SelectComponent(zoneId);
-    if (ctx.topClause() != null) {
-      // TODO: parse info of top clause into selectOp
-      visitTopClause(ctx.topClause());
-    } else if (ctx.LAST() != null) {
+    if (ctx.LAST() != null) {
       queryOp = new LastQueryOperator(queryOp);
     }
 
@@ -2946,20 +2758,6 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     queryOp.setSelectComponent(selectComponent);
   }
 
-  @Override
-  public Operator visitTopClause(IoTDBSqlParser.TopClauseContext ctx) {
-    Map<String, Object> props = new HashMap<>();
-    int top = Integer.parseInt(ctx.INTEGER_LITERAL().getText());
-    if (top <= 0 || top > 1000) {
-      throw new SQLParserException(
-          String.format(
-              "TOP <N>: N should be greater than 0 and less than 1000, current N is %d", top));
-    }
-    props.put(TOP_K, top);
-    queryOp.setProps(props);
-    return queryOp;
-  }
-
   private ResultColumn parseResultColumn(IoTDBSqlParser.ResultColumnContext resultColumnContext) {
     Expression expression = parseExpression(resultColumnContext.expression(), false);
     if (expression.isConstantOperand()) {
@@ -2987,13 +2785,78 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   // Where Clause
 
   public WhereComponent parseWhereClause(IoTDBSqlParser.WhereClauseContext ctx) {
-    if (ctx.indexPredicateClause() != null) {
-      parseIndexPredicate(ctx.indexPredicateClause());
-      return null;
+    Expression predicate =
+        parseExpression(ctx.expression(), ctx.expression().OPERATOR_NOT() == null, true);
+    return new WhereComponent(convertExpressionToFilter(predicate));
+  }
+
+  private FilterOperator convertExpressionToFilter(Expression predicate) {
+    FilterOperator filter;
+    if (predicate instanceof LogicAndExpression) {
+      filter = new FilterOperator(FilterType.KW_AND);
+      filter.addChildOperator(
+          convertExpressionToFilter(((BinaryExpression) predicate).getLeftExpression()));
+      filter.addChildOperator(
+          convertExpressionToFilter(((BinaryExpression) predicate).getRightExpression()));
+    } else if (predicate instanceof LogicOrExpression) {
+      filter = new FilterOperator(FilterType.KW_OR);
+      filter.addChildOperator(
+          convertExpressionToFilter(((BinaryExpression) predicate).getLeftExpression()));
+      filter.addChildOperator(
+          convertExpressionToFilter(((BinaryExpression) predicate).getRightExpression()));
+    } else if (predicate instanceof LogicNotExpression) {
+      filter = new FilterOperator(FilterType.KW_NOT);
+      filter.addChildOperator(
+          convertExpressionToFilter(((LogicNotExpression) predicate).getExpression()));
+    } else if (predicate instanceof LikeExpression) {
+      filter =
+          new LikeOperator(
+              FilterType.LIKE,
+              parsePathFromExpression(((LikeExpression) predicate).getExpression()),
+              ((LikeExpression) predicate).getPatternString());
+    } else if (predicate instanceof RegularExpression) {
+      filter =
+          new RegexpOperator(
+              FilterType.REGEXP,
+              parsePathFromExpression(((RegularExpression) predicate).getExpression()),
+              ((RegularExpression) predicate).getPatternString());
+    } else if (predicate instanceof InExpression) {
+      filter =
+          new InOperator(
+              FilterType.IN,
+              parsePathFromExpression(((InExpression) predicate).getExpression()),
+              ((InExpression) predicate).isNotIn(),
+              ((InExpression) predicate).getValues());
+    } else if (FilterConstant.ExpressionToFilterType.containsKey(predicate.getExpressionType())) {
+      filter =
+          new BasicFunctionOperator(
+              FilterConstant.ExpressionToFilterType.get(predicate.getExpressionType()),
+              parsePathFromExpression(((BinaryExpression) predicate).getLeftExpression()),
+              parseValueFromExpression(((BinaryExpression) predicate).getRightExpression()));
+    } else {
+      throw new IllegalArgumentException(
+          "Unsupported expression type: " + predicate.getExpressionType());
+    }
+    return filter;
+  }
+
+  private PartialPath parsePathFromExpression(Expression expression) {
+    if (expression instanceof TimeSeriesOperand) {
+      return ((TimeSeriesOperand) expression).getPath();
+    } else if (expression instanceof TimestampOperand) {
+      return TIME_PATH;
+    } else {
+      throw new IllegalArgumentException(
+          "Unsupported expression type: " + expression.getExpressionType());
     }
-    FilterOperator whereOp = new FilterOperator();
-    whereOp.addChildOperator(parseOrExpression(ctx.orExpression()));
-    return new WhereComponent(whereOp.getChildren().get(0));
+  }
+
+  private String parseValueFromExpression(Expression expression) {
+    if (!(expression instanceof ConstantOperand)) {
+      throw new IllegalArgumentException(
+          "Unsupported expression type: " + expression.getExpressionType());
+    }
+    return ((ConstantOperand) expression).getValueString();
   }
 
   // Tag & Property & Attribute
@@ -3183,6 +3046,36 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     return DatetimeUtils.convertDurationStrToLong(durationStr);
   }
 
+  private Expression parseConstantOperand(ConstantContext constantContext) {
+    try {
+      String text = constantContext.getText();
+      if (clientVersion.equals(IoTDBConstant.ClientVersion.V_0_13)) {
+        if (constantContext.BOOLEAN_LITERAL() != null) {
+          return new ConstantOperand(TSDataType.BOOLEAN, text);
+        } else if (constantContext.STRING_LITERAL() != null) {
+          return new ConstantOperand(TSDataType.TEXT, parseStringLiteral(text));
+        } else if (constantContext.INTEGER_LITERAL() != null) {
+          return new ConstantOperand(TSDataType.INT64, text);
+        } else if (constantContext.realLiteral() != null) {
+          return new ConstantOperand(TSDataType.DOUBLE, text);
+        } else if (constantContext.dateExpression() != null) {
+          return new ConstantOperand(
+              TSDataType.INT64,
+              String.valueOf(parseDateExpression(constantContext.dateExpression())));
+        } else {
+          throw new SQLParserException("Unsupported constant operand: " + text);
+        }
+      } else if (clientVersion.equals(IoTDBConstant.ClientVersion.V_0_12)) {
+        // if client version is before 0.13, node name in expression may be a constant
+        return new TimeSeriesOperand(convertConstantToPath(text));
+      } else {
+        throw new UnsupportedOperationException();
+      }
+    } catch (IllegalPathException e) {
+      throw new SQLParserException(e.getMessage());
+    }
+  }
+
   private void setMap(IoTDBSqlParser.AlterClauseContext ctx, Map<String, String> alterMap) {
     List<IoTDBSqlParser.AttributePairContext> tagsList = ctx.attributePair();
     if (ctx.attributePair(0) != null) {
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 2779a735e9..14d5007739 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,9 +22,6 @@ package org.apache.iotdb.db.query.expression;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.query.expression.binary.AdditionExpression;
 import org.apache.iotdb.db.query.expression.binary.DivisionExpression;
@@ -89,24 +86,19 @@ public abstract class Expression {
   // Operations for time series paths
   /////////////////////////////////////////////////////////////////////////////////////////////////
 
-  public abstract void concat(
-      List<PartialPath> prefixPaths,
-      List<Expression> resultExpressions,
-      PathPatternTree patternTree);
-
   // TODO: remove after MPP finish
+  @Deprecated
   public abstract void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions);
 
-  public abstract void removeWildcards(
-      WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)
-      throws StatementAnalyzeException;
-
   // TODO: remove after MPP finish
+  @Deprecated
   public abstract void removeWildcards(
       org.apache.iotdb.db.qp.utils.WildcardsRemover wildcardsRemover,
       List<Expression> resultExpressions)
       throws LogicalOptimizeException;
 
+  // TODO: remove after MPP finish
+  @Deprecated
   public abstract void collectPaths(Set<PartialPath> pathSet);
 
   /////////////////////////////////////////////////////////////////////////////////////////////////
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 12187c8846..32f7f92718 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
@@ -39,15 +39,15 @@ import java.util.Set;
  *
  * <ul>
  *   Example 1: select a, a + b, udf(udf(b)) from root.sg.d, root.sg.e;
- *   <li>Step 1: constructed by sql visitor in logical operator: <br>
+ *   <li>Step 1: constructed by ASTVisitor in StatementGenerator: <br>
  *       result columns: <br>
  *       [a, a + b, udf(udf(b))]
- *   <li>Step 2: concatenated with prefix paths in logical optimizer:<br>
+ *   <li>Step 2: concatenated with prefix paths:<br>
  *       result columns: <br>
  *       [root.sg.d.a, root.sg.e.a, root.sg.d.a + root.sg.d.b, root.sg.d.a + root.sg.e.b,
  *       root.sg.e.a + root.sg.d.b, root.sg.e.a + root.sg.e.b, udf(udf(root.sg.d.b)),
  *       udf(udf(root.sg.e.b))]
- *   <li>Step 3: remove wildcards in logical optimizer:<br>
+ *   <li>Step 3: remove wildcards:<br>
  *       result columns: <br>
  *       [root.sg.d.a, root.sg.e.a, root.sg.d.a + root.sg.d.b, root.sg.d.a + root.sg.e.b,
  *       root.sg.e.a + root.sg.d.b, root.sg.e.a + root.sg.e.b, udf(udf(root.sg.d.b)),
@@ -56,13 +56,13 @@ import java.util.Set;
  *
  * <ul>
  *   Example 2: select *, a + *, udf(udf(*)) from root.sg.d;
- *   <li>Step 1: constructed by sql visitor in logical operator: <br>
+ *   <li>Step 1: constructed by ASTVisitor in StatementGenerator: <br>
  *       result columns: <br>
  *       [*, a + * , udf(udf(*))]
- *   <li>Step 2: concatenated with prefix paths in logical optimizer:<br>
+ *   <li>Step 2: concatenated with prefix paths:<br>
  *       result columns: <br>
  *       [root.sg.d.*, root.sg.d.a + root.sg.d.*, udf(udf(root.sg.d.*))]
- *   <li>Step 3: remove wildcards in logical optimizer:<br>
+ *   <li>Step 3: remove wildcards:<br>
  *       result columns: <br>
  *       [root.sg.d.a, root.sg.d.b, root.sg.d.a + root.sg.d.a, root.sg.d.a + root.sg.d.b,
  *       udf(udf(root.sg.d.a)), udf(udf(root.sg.d.b))]
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 b603a28376..d9a5836d9b 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,6 @@ package org.apache.iotdb.db.query.expression.binary;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.query.udf.core.executor.UDTFContext;
@@ -50,8 +47,8 @@ import java.util.Set;
 
 public abstract class BinaryExpression extends Expression {
 
-  protected final Expression leftExpression;
-  protected final Expression rightExpression;
+  protected Expression leftExpression;
+  protected Expression rightExpression;
 
   protected BinaryExpression(Expression leftExpression, Expression rightExpression) {
     this.leftExpression = leftExpression;
@@ -71,6 +68,14 @@ public abstract class BinaryExpression extends Expression {
     return rightExpression;
   }
 
+  public void setLeftExpression(Expression leftExpression) {
+    this.leftExpression = leftExpression;
+  }
+
+  public void setRightExpression(Expression rightExpression) {
+    this.rightExpression = rightExpression;
+  }
+
   @Override
   public boolean isConstantOperandInternal() {
     return leftExpression.isConstantOperand() && rightExpression.isConstantOperand();
@@ -94,20 +99,6 @@ public abstract class BinaryExpression extends Expression {
     return Arrays.asList(leftExpression, rightExpression);
   }
 
-  @Override
-  public final void concat(
-      List<PartialPath> prefixPaths,
-      List<Expression> resultExpressions,
-      PathPatternTree patternTree) {
-    List<Expression> leftExpressions = new ArrayList<>();
-    leftExpression.concat(prefixPaths, leftExpressions, patternTree);
-
-    List<Expression> rightExpressions = new ArrayList<>();
-    rightExpression.concat(prefixPaths, rightExpressions, patternTree);
-
-    reconstruct(leftExpressions, rightExpressions, resultExpressions);
-  }
-
   @Override
   public final void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions) {
     List<Expression> leftExpressions = new ArrayList<>();
@@ -119,19 +110,6 @@ public abstract class BinaryExpression extends Expression {
     reconstruct(leftExpressions, rightExpressions, resultExpressions);
   }
 
-  @Override
-  public final void removeWildcards(
-      WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)
-      throws StatementAnalyzeException {
-    List<Expression> leftExpressions = new ArrayList<>();
-    leftExpression.removeWildcards(wildcardsRemover, leftExpressions);
-
-    List<Expression> rightExpressions = new ArrayList<>();
-    rightExpression.removeWildcards(wildcardsRemover, rightExpressions);
-
-    reconstruct(leftExpressions, rightExpressions, resultExpressions);
-  }
-
   @Override
   public final void removeWildcards(
       org.apache.iotdb.db.qp.utils.WildcardsRemover wildcardsRemover,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/ConstantOperand.java b/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/ConstantOperand.java
index 3583dc596c..d92c8021c3 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/ConstantOperand.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/ConstantOperand.java
@@ -21,8 +21,6 @@ package org.apache.iotdb.db.query.expression.leaf;
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.query.expression.ExpressionType;
@@ -61,6 +59,10 @@ public class ConstantOperand extends LeafOperand {
     return dataType;
   }
 
+  public String getValueString() {
+    return valueString;
+  }
+
   public boolean isNegativeNumber() {
     return !dataType.equals(TSDataType.TEXT)
         && !dataType.equals(TSDataType.BOOLEAN)
@@ -72,25 +74,11 @@ public class ConstantOperand extends LeafOperand {
     return true;
   }
 
-  @Override
-  public void concat(
-      List<PartialPath> prefixPaths,
-      List<Expression> resultExpressions,
-      PathPatternTree patternTree) {
-    resultExpressions.add(this);
-  }
-
   @Override
   public void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions) {
     resultExpressions.add(this);
   }
 
-  @Override
-  public void removeWildcards(
-      WildcardsRemover wildcardsRemover, List<Expression> resultExpressions) {
-    resultExpressions.add(this);
-  }
-
   @Override
   public void removeWildcards(
       org.apache.iotdb.db.qp.utils.WildcardsRemover wildcardsRemover,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimeSeriesOperand.java b/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimeSeriesOperand.java
index e7ffa3e341..11a6b72912 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimeSeriesOperand.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimeSeriesOperand.java
@@ -22,10 +22,7 @@ package org.apache.iotdb.db.query.expression.leaf;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
 import org.apache.iotdb.db.metadata.path.PathDeserializeUtil;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.query.expression.ExpressionType;
@@ -68,18 +65,6 @@ public class TimeSeriesOperand extends LeafOperand {
     return false;
   }
 
-  @Override
-  public void concat(
-      List<PartialPath> prefixPaths,
-      List<Expression> resultExpressions,
-      PathPatternTree patternTree) {
-    for (PartialPath prefixPath : prefixPaths) {
-      TimeSeriesOperand resultExpression = new TimeSeriesOperand(prefixPath.concatPath(path));
-      patternTree.appendPath(resultExpression.getPath());
-      resultExpressions.add(resultExpression);
-    }
-  }
-
   @Override
   public void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions) {
     for (PartialPath prefixPath : prefixPaths) {
@@ -87,14 +72,6 @@ public class TimeSeriesOperand extends LeafOperand {
     }
   }
 
-  @Override
-  public void removeWildcards(WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)
-      throws StatementAnalyzeException {
-    for (PartialPath actualPath : wildcardsRemover.removeWildcardInPath(path)) {
-      resultExpressions.add(new TimeSeriesOperand(actualPath));
-    }
-  }
-
   @Override
   public void removeWildcards(
       org.apache.iotdb.db.qp.utils.WildcardsRemover wildcardsRemover,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimestampOperand.java b/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimestampOperand.java
index 200a12ae33..73c77a2b37 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimestampOperand.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/leaf/TimestampOperand.java
@@ -22,9 +22,6 @@ package org.apache.iotdb.db.query.expression.leaf;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.query.expression.ExpressionType;
@@ -60,25 +57,11 @@ public class TimestampOperand extends LeafOperand {
     return true;
   }
 
-  @Override
-  public void concat(
-      List<PartialPath> prefixPaths,
-      List<Expression> resultExpressions,
-      PathPatternTree patternTree) {
-    resultExpressions.add(this);
-  }
-
   @Override
   public void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions) {
     resultExpressions.add(this);
   }
 
-  @Override
-  public void removeWildcards(WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)
-      throws StatementAnalyzeException {
-    resultExpressions.add(this);
-  }
-
   @Override
   public void removeWildcards(
       org.apache.iotdb.db.qp.utils.WildcardsRemover wildcardsRemover,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/multi/FunctionExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/multi/FunctionExpression.java
index 46cfab6f6f..b2bd93b05c 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/multi/FunctionExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/multi/FunctionExpression.java
@@ -23,9 +23,6 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
@@ -190,28 +187,6 @@ public class FunctionExpression extends Expression {
     return expressions;
   }
 
-  @Override
-  public void concat(
-      List<PartialPath> prefixPaths,
-      List<Expression> resultExpressions,
-      PathPatternTree patternTree) {
-    List<List<Expression>> resultExpressionsForRecursionList = new ArrayList<>();
-
-    for (Expression suffixExpression : expressions) {
-      List<Expression> resultExpressionsForRecursion = new ArrayList<>();
-      suffixExpression.concat(prefixPaths, resultExpressionsForRecursion, patternTree);
-      resultExpressionsForRecursionList.add(resultExpressionsForRecursion);
-    }
-
-    List<List<Expression>> functionExpressions = new ArrayList<>();
-    ConcatPathOptimizer.cartesianProduct(
-        resultExpressionsForRecursionList, functionExpressions, 0, new ArrayList<>());
-    for (List<Expression> functionExpression : functionExpressions) {
-      resultExpressions.add(
-          new FunctionExpression(functionName, functionAttributes, functionExpression));
-    }
-  }
-
   @Override
   public void concat(List<PartialPath> prefixPaths, List<Expression> resultExpressions) {
     List<List<Expression>> resultExpressionsForRecursionList = new ArrayList<>();
@@ -231,16 +206,6 @@ public class FunctionExpression extends Expression {
     }
   }
 
-  @Override
-  public void removeWildcards(WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)
-      throws StatementAnalyzeException {
-    for (List<Expression> functionExpression :
-        wildcardsRemover.removeWildcardsInExpressions(expressions)) {
-      resultExpressions.add(
-          new FunctionExpression(functionName, functionAttributes, functionExpression));
-    }
-  }
-
   @Override
   public void removeWildcards(
       org.apache.iotdb.db.qp.utils.WildcardsRemover wildcardsRemover,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/InExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/InExpression.java
index 2d311cba1d..9f329b3728 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/InExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/InExpression.java
@@ -51,6 +51,14 @@ public class InExpression extends UnaryExpression {
     }
   }
 
+  public boolean isNotIn() {
+    return isNotIn;
+  }
+
+  public LinkedHashSet<String> getValues() {
+    return values;
+  }
+
   @Override
   protected String getExpressionStringInternal() {
     StringBuilder valuesStringBuilder = new StringBuilder();
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/LikeExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/LikeExpression.java
index ede6283ce5..2d91ac0720 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/LikeExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/LikeExpression.java
@@ -52,6 +52,14 @@ public class LikeExpression extends UnaryExpression {
     pattern = compile();
   }
 
+  public String getPatternString() {
+    return patternString;
+  }
+
+  public Pattern getPattern() {
+    return pattern;
+  }
+
   /**
    * The main idea of this part comes from
    * https://codereview.stackexchange.com/questions/36861/convert-sql-like-to-regex/36864
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/RegularExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/RegularExpression.java
index 0c990731bd..f6027a9c8e 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/RegularExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/RegularExpression.java
@@ -54,6 +54,14 @@ public class RegularExpression extends UnaryExpression {
     pattern = Pattern.compile(Validate.notNull(patternString));
   }
 
+  public String getPatternString() {
+    return patternString;
+  }
+
+  public Pattern getPattern() {
+    return pattern;
+  }
+
   @Override
   protected Transformer constructTransformer(LayerPointReader pointReader) {
     return new RegularTransformer(pointReader, pattern);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/UnaryExpression.java b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/UnaryExpression.java
index 136157d8c3..f82d980e7c 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/expression/unary/UnaryExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/expression/unary/UnaryExpression.java
@@ -22,8 +22,6 @@ package org.apache.iotdb.db.query.expression.unary;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.qp.utils.WildcardsRemover;
 import org.apache.iotdb.db.query.expression.Expression;
@@ -152,30 +150,6 @@ public abstract class UnaryExpression extends Expression {
     }
   }
 
-  @Override
-  public final void concat(
-      List<PartialPath> prefixPaths,
-      List<Expression> resultExpressions,
-      PathPatternTree patternTree) {
-    List<Expression> resultExpressionsForRecursion = new ArrayList<>();
-    expression.concat(prefixPaths, resultExpressionsForRecursion, patternTree);
-    for (Expression resultExpression : resultExpressionsForRecursion) {
-      resultExpressions.add(constructExpression(resultExpression));
-    }
-  }
-
-  @Override
-  public final void removeWildcards(
-      org.apache.iotdb.db.mpp.plan.rewriter.WildcardsRemover wildcardsRemover,
-      List<Expression> resultExpressions)
-      throws StatementAnalyzeException {
-    List<Expression> resultExpressionsForRecursion = new ArrayList<>();
-    expression.removeWildcards(wildcardsRemover, resultExpressionsForRecursion);
-    for (Expression resultExpression : resultExpressionsForRecursion) {
-      resultExpressions.add(constructExpression(resultExpression));
-    }
-  }
-
   @Override
   public final void removeWildcards(
       WildcardsRemover wildcardsRemover, List<Expression> resultExpressions)
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java
index 416a5990ff..4bad672aa4 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java
@@ -32,6 +32,7 @@ import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.internal.util.collections.Sets;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -431,7 +432,7 @@ public class SchemaTreeTest {
     DeviceSchemaInfo deviceSchemaInfo = deviceSchemaInfoList.get(0);
     Assert.assertEquals(new PartialPath("root.sg.d2.a"), deviceSchemaInfo.getDevicePath());
     Assert.assertTrue(deviceSchemaInfo.isAligned());
-    Assert.assertEquals(2, deviceSchemaInfo.getMeasurements().size());
+    Assert.assertEquals(2, deviceSchemaInfo.getMeasurements(Sets.newSet("*")).size());
 
     deviceSchemaInfoList = schemaTree.getMatchedDevices(new PartialPath("root.sg.*"), false);
     Assert.assertEquals(2, deviceSchemaInfoList.size());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeFailTest.java
similarity index 93%
copy from server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzerTest.java
copy to server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeFailTest.java
index 0f506824e9..6211ca4c78 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeFailTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.mpp.plan.analyze;
 
+import org.apache.iotdb.db.exception.sql.SemanticException;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.QueryId;
 import org.apache.iotdb.db.mpp.plan.parser.StatementGenerator;
@@ -30,7 +31,7 @@ import java.time.ZonedDateTime;
 
 import static org.junit.Assert.fail;
 
-public class AnalyzerTest {
+public class AnalyzeFailTest {
 
   @Test
   public void samePropertyKeyTest() {
@@ -42,7 +43,7 @@ public class AnalyzerTest {
   @Test
   public void sameMeasurementsInAlignedTest() {
     assertAnalyzeSemanticException(
-        "CREATE ALIGNED TIMESERIES root.ln.wf01.GPS(latitude FLOAT encoding=PLAIN  compressor=SNAPPY, latitude FLOAT encoding=PLAIN compressor=SNAPPY)",
+        "CREATE ALIGNED TIMESERIES root.ln.wf01.GPS(latitude FLOAT, latitude FLOAT)",
         "Measurement under an aligned device is not allowed to have the same measurement name");
   }
 
@@ -55,7 +56,7 @@ public class AnalyzerTest {
               new FakeSchemaFetcherImpl());
       analyzer.analyze(StatementGenerator.createStatement(sql, ZonedDateTime.now().getOffset()));
       fail();
-    } catch (RuntimeException e) {
+    } catch (SemanticException e) {
       Assert.assertTrue(e.getMessage().contains(message));
     }
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeTest.java
similarity index 52%
rename from server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzerTest.java
rename to server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeTest.java
index 0f506824e9..04b7fa534e 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeTest.java
@@ -22,41 +22,34 @@ package org.apache.iotdb.db.mpp.plan.analyze;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.QueryId;
 import org.apache.iotdb.db.mpp.plan.parser.StatementGenerator;
+import org.apache.iotdb.db.mpp.plan.statement.Statement;
 
-import org.junit.Assert;
 import org.junit.Test;
 
 import java.time.ZonedDateTime;
 
 import static org.junit.Assert.fail;
 
-public class AnalyzerTest {
+public class AnalyzeTest {
 
   @Test
-  public void samePropertyKeyTest() {
-    assertAnalyzeSemanticException(
-        "CREATE TIMESERIES root.sg1.d1.s1 INT32 TAGS('a'='1') ATTRIBUTES('a'='1')",
-        "Tag and attribute shouldn't have the same property key");
+  public void testRawDataQuery() {
+    // TODO: @lmh add UTs
   }
 
-  @Test
-  public void sameMeasurementsInAlignedTest() {
-    assertAnalyzeSemanticException(
-        "CREATE ALIGNED TIMESERIES root.ln.wf01.GPS(latitude FLOAT encoding=PLAIN  compressor=SNAPPY, latitude FLOAT encoding=PLAIN compressor=SNAPPY)",
-        "Measurement under an aligned device is not allowed to have the same measurement name");
-  }
-
-  private void assertAnalyzeSemanticException(String sql, String message) {
+  private Analysis analyzeSQL(String sql) {
     try {
+      Statement statement =
+          StatementGenerator.createStatement(sql, ZonedDateTime.now().getOffset());
+      MPPQueryContext context = new MPPQueryContext(new QueryId("test_query"));
       Analyzer analyzer =
-          new Analyzer(
-              new MPPQueryContext(new QueryId("test_query")),
-              new FakePartitionFetcherImpl(),
-              new FakeSchemaFetcherImpl());
-      analyzer.analyze(StatementGenerator.createStatement(sql, ZonedDateTime.now().getOffset()));
-      fail();
-    } catch (RuntimeException e) {
-      Assert.assertTrue(e.getMessage().contains(message));
+          new Analyzer(context, new FakePartitionFetcherImpl(), new FakeSchemaFetcherImpl());
+      return analyzer.analyze(statement);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
     }
+    fail();
+    return null;
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FillNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FillNodeSerdeTest.java
index 544e24e635..84f7605d88 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FillNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FillNodeSerdeTest.java
@@ -19,7 +19,6 @@
 package org.apache.iotdb.db.mpp.plan.plan.node.process;
 
 import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.plan.plan.node.PlanNodeDeserializeHelper;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.FillNode;
@@ -27,12 +26,11 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.TimeJoinNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FillDescriptor;
 import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
 import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
-import org.apache.iotdb.db.query.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.db.mpp.plan.statement.literal.LongLiteral;
 
 import org.junit.Test;
 
 import java.nio.ByteBuffer;
-import java.util.Collections;
 
 import static org.junit.Assert.assertEquals;
 
@@ -46,9 +44,7 @@ public class FillNodeSerdeTest {
         new FillNode(
             new PlanNodeId("TestFillNode"),
             timeJoinNode,
-            Collections.singletonList(
-                new FillDescriptor(
-                    FillPolicy.PREVIOUS, new TimeSeriesOperand(new PartialPath("root.sg.d1.s1")))));
+            new FillDescriptor(FillPolicy.VALUE, new LongLiteral("100")));
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
     fillNode.serialize(byteBuffer);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Pair.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Pair.java
index c81debf70c..140d43bc0a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Pair.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Pair.java
@@ -38,6 +38,14 @@ public class Pair<L, R> implements Serializable {
     right = r;
   }
 
+  public L getLeft() {
+    return left;
+  }
+
+  public R getRight() {
+    return right;
+  }
+
   @Override
   public int hashCode() {
     final int prime = 31;