You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hu...@apache.org on 2022/07/20 03:18:50 UTC

[iotdb] branch lmh/orderBySensor created (now 574153d818)

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

hui pushed a change to branch lmh/orderBySensor
in repository https://gitbox.apache.org/repos/asf/iotdb.git


      at 574153d818 add mergeOrders in LastQueryMergeNode

This branch includes the following new commits:

     new c53a1d49ed refactor Order by in SQL parser & QueryStatement
     new 9024ee0338 add semantic check
     new 6dace94ef0 replace OrderBy with Ordering in PlanNode
     new 574153d818 add mergeOrders in LastQueryMergeNode

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



[iotdb] 03/04: replace OrderBy with Ordering in PlanNode

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

hui pushed a commit to branch lmh/orderBySensor
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 6dace94ef049789544598f5ee5fd69903f571627
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Wed Jul 20 10:59:04 2022 +0800

    replace OrderBy with Ordering in PlanNode
---
 .../operator/process/TimeJoinOperator.java         |   7 +-
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  |   4 +-
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |  10 +-
 .../db/mpp/plan/planner/LocalExecutionPlanner.java |  39 +++---
 .../db/mpp/plan/planner/LogicalPlanBuilder.java    |  41 ++++---
 .../db/mpp/plan/planner/LogicalPlanVisitor.java    |  30 ++---
 .../planner/plan/node/process/AggregationNode.java |  12 +-
 .../planner/plan/node/process/DeviceMergeNode.java |  31 +++--
 .../planner/plan/node/process/DeviceViewNode.java  |  33 +++--
 .../plan/planner/plan/node/process/FillNode.java   |  13 +-
 .../plan/planner/plan/node/process/FilterNode.java |   8 +-
 .../plan/node/process/GroupByLevelNode.java        |  12 +-
 .../node/process/SlidingWindowAggregationNode.java |  12 +-
 .../plan/planner/plan/node/process/SortNode.java   |  12 +-
 .../planner/plan/node/process/TimeJoinNode.java    |  12 +-
 .../planner/plan/node/process/TransformNode.java   |  12 +-
 .../source/AlignedSeriesAggregationScanNode.java   |  10 +-
 .../plan/node/source/AlignedSeriesScanNode.java    |  12 +-
 .../node/source/SeriesAggregationScanNode.java     |  10 +-
 .../node/source/SeriesAggregationSourceNode.java   |   6 +-
 .../planner/plan/node/source/SeriesScanNode.java   |  14 +--
 .../plan/statement/component/OrderByComponent.java |  33 ++++-
 .../db/mpp/plan/statement/component/Ordering.java  |  33 +++++
 .../db/mpp/plan/statement/component/SortItem.java  |  42 +++----
 .../db/mpp/plan/statement/component/SortKey.java   |  26 ++++
 .../db/mpp/plan/statement/crud/QueryStatement.java |   8 ++
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    |  10 +-
 .../org/apache/iotdb/db/utils/SchemaUtils.java     |   6 +-
 .../iotdb/db/mpp/execution/DataDriverTest.java     |   4 +-
 .../operator/AlignedSeriesScanOperatorTest.java    |   6 +-
 .../mpp/execution/operator/LimitOperatorTest.java  |   4 +-
 .../mpp/execution/operator/OffsetOperatorTest.java |   8 +-
 .../operator/RawDataAggregationOperatorTest.java   |   4 +-
 .../execution/operator/TimeJoinOperatorTest.java   |   8 +-
 .../mpp/plan/plan/FragmentInstanceSerdeTest.java   |  11 +-
 .../db/mpp/plan/plan/QueryLogicalPlanUtil.java     | 136 +++++++++++----------
 .../distribution/AggregationDistributionTest.java  |  34 +++---
 .../distribution/DistributionPlannerBasicTest.java |  42 +++----
 .../distribution/NoDataRegionPlanningTest.java     |  20 +--
 .../node/process/AggregationNodeSerdeTest.java     |   6 +-
 .../plan/node/process/DeviceViewNodeSerdeTest.java |  14 ++-
 .../plan/node/process/ExchangeNodeSerdeTest.java   |   5 +-
 .../plan/plan/node/process/FillNodeSerdeTest.java  |   7 +-
 .../plan/node/process/FilterNodeSerdeTest.java     |   7 +-
 .../node/process/GroupByLevelNodeSerdeTest.java    |   8 +-
 .../plan/plan/node/process/LimitNodeSerdeTest.java |   4 +-
 .../plan/node/process/OffsetNodeSerdeTest.java     |   4 +-
 .../plan/plan/node/process/SortNodeSerdeTest.java  |   7 +-
 .../plan/node/process/TimeJoinNodeSerdeTest.java   |   9 +-
 .../source/SeriesAggregationScanNodeSerdeTest.java |   4 +-
 .../plan/node/source/SeriesScanNodeSerdeTest.java  |   4 +-
 51 files changed, 483 insertions(+), 371 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/TimeJoinOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/TimeJoinOperator.java
index 5a576ea5cc..6775c66447 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/TimeJoinOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/TimeJoinOperator.java
@@ -22,7 +22,7 @@ import org.apache.iotdb.db.mpp.execution.operator.Operator;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
 import org.apache.iotdb.db.mpp.execution.operator.process.merge.ColumnMerger;
 import org.apache.iotdb.db.mpp.execution.operator.process.merge.TimeComparator;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.utils.datastructure.TimeSelector;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.block.TsBlock;
@@ -82,7 +82,7 @@ public class TimeJoinOperator implements ProcessOperator {
   public TimeJoinOperator(
       OperatorContext operatorContext,
       List<Operator> children,
-      OrderBy mergeOrder,
+      Ordering mergeOrder,
       List<TSDataType> dataTypes,
       List<ColumnMerger> mergers,
       TimeComparator comparator) {
@@ -96,8 +96,7 @@ public class TimeJoinOperator implements ProcessOperator {
     this.inputIndex = new int[this.inputOperatorsCount];
     this.shadowInputIndex = new int[this.inputOperatorsCount];
     this.noMoreTsBlocks = new boolean[this.inputOperatorsCount];
-    this.timeSelector =
-        new TimeSelector(this.inputOperatorsCount << 1, OrderBy.TIMESTAMP_ASC == mergeOrder);
+    this.timeSelector = new TimeSelector(this.inputOperatorsCount << 1, Ordering.ASC == mergeOrder);
     this.outputColumnCount = dataTypes.size();
     this.dataTypes = dataTypes;
     this.tsBlockBuilder = new TsBlockBuilder(dataTypes);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index 59ad88efa7..d010131552 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -45,7 +45,7 @@ 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.GroupByTimeComponent;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
 import org.apache.iotdb.db.mpp.plan.statement.crud.DeleteDataStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertMultiTabletsStatement;
@@ -373,7 +373,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
         GroupByTimeComponent groupByTimeComponent = queryStatement.getGroupByTimeComponent();
         if ((groupByTimeComponent.isIntervalByMonth()
                 || groupByTimeComponent.isSlidingStepByMonth())
-            && queryStatement.getResultOrder() == OrderBy.TIMESTAMP_DESC) {
+            && queryStatement.getResultTimeOrder() == Ordering.ASC) {
           throw new SemanticException("Group by month doesn't support order by time desc now.");
         }
         analysis.setGroupByTimeParameter(new GroupByTimeParameter(groupByTimeComponent));
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 8d5f002540..5ca1454d64 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
@@ -65,10 +65,12 @@ 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.OrderByComponent;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 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.SortItem;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortKey;
 import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition;
 import org.apache.iotdb.db.mpp.plan.statement.crud.DeleteDataStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement;
@@ -1197,12 +1199,12 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   // parse ORDER BY TIME
   private void parseOrderByClause(IoTDBSqlParser.OrderByClauseContext ctx) {
     OrderByComponent orderByComponent = new OrderByComponent();
-    Set<SortItem.SortKey> sortKeySet = new HashSet<>();
+    Set<SortKey> sortKeySet = new HashSet<>();
     for (IoTDBSqlParser.OrderByAttributeClauseContext orderByAttributeClauseContext :
         ctx.orderByAttributeClause()) {
       SortItem sortItem = parseOrderByAttributeClause(orderByAttributeClauseContext);
 
-      SortItem.SortKey sortKey = sortItem.getSortKey();
+      SortKey sortKey = sortItem.getSortKey();
       if (sortKeySet.contains(sortKey)) {
         throw new SemanticException(String.format("ORDER BY: duplicate sort key '%s'", sortKey));
       } else {
@@ -1215,8 +1217,8 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
   private SortItem parseOrderByAttributeClause(IoTDBSqlParser.OrderByAttributeClauseContext ctx) {
     return new SortItem(
-        SortItem.SortKey.valueOf(ctx.sortKey().getText().toUpperCase()),
-        ctx.DESC() != null ? SortItem.Ordering.DESC : SortItem.Ordering.ASC);
+        SortKey.valueOf(ctx.sortKey().getText().toUpperCase()),
+        ctx.DESC() != null ? Ordering.DESC : Ordering.ASC);
   }
 
   // ResultSetFormat Clause
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java
index ad7734fa8f..22a3791884 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java
@@ -154,7 +154,9 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByLevelDescripto
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.InputLocation;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.OutputColumn;
 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.mpp.plan.statement.component.Ordering;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortKey;
 import org.apache.iotdb.db.mpp.plan.statement.literal.Literal;
 import org.apache.iotdb.db.utils.datastructure.TimeSelector;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -267,7 +269,7 @@ public class LocalExecutionPlanner {
     @Override
     public Operator visitSeriesScan(SeriesScanNode node, LocalExecutionPlanContext context) {
       PartialPath seriesPath = node.getSeriesPath();
-      boolean ascending = node.getScanOrder() == OrderBy.TIMESTAMP_ASC;
+      boolean ascending = node.getScanOrder() == Ordering.ASC;
       OperatorContext operatorContext =
           context.instanceContext.addOperatorContext(
               context.getNextOperatorId(),
@@ -295,7 +297,7 @@ public class LocalExecutionPlanner {
     public Operator visitAlignedSeriesScan(
         AlignedSeriesScanNode node, LocalExecutionPlanContext context) {
       AlignedPath seriesPath = node.getAlignedPath();
-      boolean ascending = node.getScanOrder() == OrderBy.TIMESTAMP_ASC;
+      boolean ascending = node.getScanOrder() == Ordering.ASC;
       OperatorContext operatorContext =
           context.instanceContext.addOperatorContext(
               context.getNextOperatorId(),
@@ -323,7 +325,7 @@ public class LocalExecutionPlanner {
     public Operator visitAlignedSeriesAggregationScan(
         AlignedSeriesAggregationScanNode node, LocalExecutionPlanContext context) {
       AlignedPath seriesPath = node.getAlignedPath();
-      boolean ascending = node.getScanOrder() == OrderBy.TIMESTAMP_ASC;
+      boolean ascending = node.getScanOrder() == Ordering.ASC;
       OperatorContext operatorContext =
           context.instanceContext.addOperatorContext(
               context.getNextOperatorId(),
@@ -576,7 +578,7 @@ public class LocalExecutionPlanner {
     public Operator visitSeriesAggregationScan(
         SeriesAggregationScanNode node, LocalExecutionPlanContext context) {
       PartialPath seriesPath = node.getSeriesPath();
-      boolean ascending = node.getScanOrder() == OrderBy.TIMESTAMP_ASC;
+      boolean ascending = node.getScanOrder() == Ordering.ASC;
       OperatorContext operatorContext =
           context.instanceContext.addOperatorContext(
               context.getNextOperatorId(),
@@ -647,16 +649,17 @@ public class LocalExecutionPlanner {
       List<TSDataType> dataTypes = getOutputColumnTypes(node, context.getTypeProvider());
       TimeSelector selector = null;
       TimeComparator timeComparator = null;
-      for (OrderBy orderBy : node.getMergeOrders()) {
-        switch (orderBy) {
-          case TIMESTAMP_ASC:
+      for (SortItem sortItem : node.getMergeOrders()) {
+        if (sortItem.getSortKey() == SortKey.TIME) {
+          Ordering ordering = sortItem.getOrdering();
+          if (ordering == Ordering.ASC) {
             selector = new TimeSelector(node.getChildren().size() << 1, true);
             timeComparator = ASC_TIME_COMPARATOR;
-            break;
-          case TIMESTAMP_DESC:
+          } else {
             selector = new TimeSelector(node.getChildren().size() << 1, false);
             timeComparator = DESC_TIME_COMPARATOR;
-            break;
+          }
+          break;
         }
       }
 
@@ -814,7 +817,7 @@ public class LocalExecutionPlanner {
             node.isKeepNull(),
             node.getZoneId(),
             context.getTypeProvider(),
-            node.getScanOrder() == OrderBy.TIMESTAMP_ASC);
+            node.getScanOrder() == Ordering.ASC);
       } catch (QueryProcessException | IOException e) {
         throw new RuntimeException(e);
       }
@@ -844,7 +847,7 @@ public class LocalExecutionPlanner {
             node.isKeepNull(),
             node.getZoneId(),
             context.getTypeProvider(),
-            node.getScanOrder() == OrderBy.TIMESTAMP_ASC);
+            node.getScanOrder() == Ordering.ASC);
       } catch (QueryProcessException | IOException e) {
         throw new RuntimeException(e);
       }
@@ -859,7 +862,7 @@ public class LocalExecutionPlanner {
           node.getChildren().stream()
               .map(child -> child.accept(this, context))
               .collect(Collectors.toList());
-      boolean ascending = node.getScanOrder() == OrderBy.TIMESTAMP_ASC;
+      boolean ascending = node.getScanOrder() == Ordering.ASC;
       List<Aggregator> aggregators = new ArrayList<>();
       Map<String, List<InputLocation>> layout = makeLayout(node);
       for (GroupByLevelDescriptor descriptor : node.getGroupByLevelDescriptors()) {
@@ -899,7 +902,7 @@ public class LocalExecutionPlanner {
               node.getPlanNodeId(),
               SlidingWindowAggregationOperator.class.getSimpleName());
       Operator child = node.getChild().accept(this, context);
-      boolean ascending = node.getScanOrder() == OrderBy.TIMESTAMP_ASC;
+      boolean ascending = node.getScanOrder() == Ordering.ASC;
       List<Aggregator> aggregators = new ArrayList<>();
       Map<String, List<InputLocation>> layout = makeLayout(node);
       for (AggregationDescriptor descriptor : node.getAggregationDescriptorList()) {
@@ -956,7 +959,7 @@ public class LocalExecutionPlanner {
           node.getChildren().stream()
               .map(child -> child.accept(this, context))
               .collect(Collectors.toList());
-      boolean ascending = node.getScanOrder() == OrderBy.TIMESTAMP_ASC;
+      boolean ascending = node.getScanOrder() == Ordering.ASC;
       List<Aggregator> aggregators = new ArrayList<>();
       Map<String, List<InputLocation>> layout = makeLayout(node);
       for (AggregationDescriptor descriptor : node.getAggregationDescriptorList()) {
@@ -1045,9 +1048,7 @@ public class LocalExecutionPlanner {
               node.getPlanNodeId(),
               TimeJoinOperator.class.getSimpleName());
       TimeComparator timeComparator =
-          node.getMergeOrder() == OrderBy.TIMESTAMP_ASC
-              ? ASC_TIME_COMPARATOR
-              : DESC_TIME_COMPARATOR;
+          node.getMergeOrder() == Ordering.ASC ? ASC_TIME_COMPARATOR : DESC_TIME_COMPARATOR;
       List<OutputColumn> outputColumns = generateOutputColumns(node);
       List<ColumnMerger> mergers = createColumnMergers(outputColumns, timeComparator);
       List<TSDataType> outputColumnTypes = getOutputColumnTypes(node, context.getTypeProvider());
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 b65d87b4ea..cb86295df9 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
@@ -70,7 +70,9 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationStep;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FillDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByLevelDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortKey;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
@@ -110,7 +112,7 @@ public class LogicalPlanBuilder {
   }
 
   public LogicalPlanBuilder planRawDataSource(
-      Set<Expression> sourceExpressions, OrderBy scanOrder, Filter timeFilter) {
+      Set<Expression> sourceExpressions, Ordering scanOrder, Filter timeFilter) {
     List<PlanNode> sourceNodeList = new ArrayList<>();
     List<PartialPath> selectedPaths =
         sourceExpressions.stream()
@@ -162,7 +164,7 @@ public class LogicalPlanBuilder {
   public LogicalPlanBuilder planAggregationSource(
       Set<Expression> sourceExpressions,
       AggregationStep curStep,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       Filter timeFilter,
       GroupByTimeParameter groupByTimeParameter,
       Set<Expression> aggregationExpressions,
@@ -203,7 +205,7 @@ public class LogicalPlanBuilder {
   public LogicalPlanBuilder planAggregationSourceWithIndexAdjust(
       Set<Expression> sourceExpressions,
       AggregationStep curStep,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       Filter timeFilter,
       GroupByTimeParameter groupByTimeParameter,
       Set<Expression> aggregationExpressions,
@@ -267,7 +269,7 @@ public class LogicalPlanBuilder {
   private AggregationDescriptor createAggregationDescriptor(
       FunctionExpression sourceExpression,
       AggregationStep curStep,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       boolean needCheckAscending,
       TypeProvider typeProvider,
       Map<PartialPath, List<AggregationDescriptor>> ascendingAggregations,
@@ -297,7 +299,7 @@ public class LogicalPlanBuilder {
   private List<PlanNode> constructSourceNodeFromAggregationDescriptors(
       Map<PartialPath, List<AggregationDescriptor>> ascendingAggregations,
       Map<PartialPath, List<AggregationDescriptor>> descendingAggregations,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       Filter timeFilter,
       GroupByTimeParameter groupByTimeParameter) {
     List<PlanNode> sourceNodeList = new ArrayList<>();
@@ -335,7 +337,7 @@ public class LogicalPlanBuilder {
   private LogicalPlanBuilder convergeAggregationSource(
       List<PlanNode> sourceNodeList,
       AggregationStep curStep,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       GroupByTimeParameter groupByTimeParameter,
       Set<Expression> aggregationExpressions,
       Map<Expression, Set<Expression>> groupByLevelExpressions) {
@@ -393,7 +395,7 @@ public class LogicalPlanBuilder {
     }
   }
 
-  private PlanNode convergeWithTimeJoin(List<PlanNode> sourceNodes, OrderBy mergeOrder) {
+  private PlanNode convergeWithTimeJoin(List<PlanNode> sourceNodes, Ordering mergeOrder) {
     PlanNode tmpNode;
     if (sourceNodes.size() == 1) {
       tmpNode = sourceNodes.get(0);
@@ -407,11 +409,12 @@ public class LogicalPlanBuilder {
       Map<String, PlanNode> deviceNameToSourceNodesMap,
       List<String> outputColumnNames,
       Map<String, List<Integer>> deviceToMeasurementIndexesMap,
-      OrderBy mergeOrder) {
+      Ordering mergeOrder) {
     DeviceViewNode deviceViewNode =
         new DeviceViewNode(
             context.getQueryId().genPlanNodeId(),
-            Arrays.asList(OrderBy.DEVICE_ASC, mergeOrder),
+            Arrays.asList(
+                new SortItem(SortKey.DEVICE, Ordering.ASC), new SortItem(SortKey.TIME, mergeOrder)),
             outputColumnNames,
             deviceToMeasurementIndexesMap);
     for (Map.Entry<String, PlanNode> entry : deviceNameToSourceNodesMap.entrySet()) {
@@ -428,7 +431,7 @@ public class LogicalPlanBuilder {
       Map<Expression, Set<Expression>> groupByLevelExpressions,
       AggregationStep curStep,
       GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     if (groupByLevelExpressions == null) {
       return this;
     }
@@ -448,7 +451,7 @@ public class LogicalPlanBuilder {
       GroupByTimeParameter groupByTimeParameter,
       AggregationStep curStep,
       TypeProvider typeProvider,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     if (aggregationExpressions == null) {
       return this;
     }
@@ -475,7 +478,7 @@ public class LogicalPlanBuilder {
       Set<Expression> aggregationExpressions,
       GroupByTimeParameter groupByTimeParameter,
       AggregationStep curStep,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     if (aggregationExpressions == null) {
       return this;
     }
@@ -491,7 +494,7 @@ public class LogicalPlanBuilder {
       Set<Expression> aggregationExpressions,
       GroupByTimeParameter groupByTimeParameter,
       AggregationStep curStep,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     List<AggregationDescriptor> aggregationDescriptorList =
         constructAggregationDescriptorList(aggregationExpressions, curStep);
     return new SlidingWindowAggregationNode(
@@ -507,7 +510,7 @@ public class LogicalPlanBuilder {
       Map<Expression, Set<Expression>> groupByLevelExpressions,
       AggregationStep curStep,
       GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     List<GroupByLevelDescriptor> groupByLevelDescriptors = new ArrayList<>();
     for (Expression groupedExpression : groupByLevelExpressions.keySet()) {
       groupByLevelDescriptors.add(
@@ -531,7 +534,7 @@ public class LogicalPlanBuilder {
   private SeriesAggregationSourceNode createAggregationScanNode(
       PartialPath selectPath,
       List<AggregationDescriptor> aggregationDescriptorList,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       GroupByTimeParameter groupByTimeParameter,
       Filter timeFilter) {
     if (selectPath instanceof MeasurementPath) { // non-aligned series
@@ -578,7 +581,7 @@ public class LogicalPlanBuilder {
       Set<Expression> selectExpressions,
       boolean isGroupByTime,
       ZoneId zoneId,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     if (queryFilter == null) {
       return this;
     }
@@ -599,7 +602,7 @@ public class LogicalPlanBuilder {
       Set<Expression> transformExpressions,
       boolean isGroupByTime,
       ZoneId zoneId,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     boolean needTransform = false;
     for (Expression expression : transformExpressions) {
       if (ExpressionAnalyzer.checkIsNeedTransform(expression)) {
@@ -622,7 +625,7 @@ public class LogicalPlanBuilder {
     return this;
   }
 
-  public LogicalPlanBuilder planFill(FillDescriptor fillDescriptor, OrderBy scanOrder) {
+  public LogicalPlanBuilder planFill(FillDescriptor fillDescriptor, Ordering scanOrder) {
     if (fillDescriptor == null) {
       return this;
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
index 38044f2fe2..f35f51bcb4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
@@ -124,7 +124,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                   .distinct()
                   .collect(Collectors.toList()),
               analysis.getDeviceToMeasurementIndexesMap(),
-              queryStatement.getResultOrder());
+              queryStatement.getResultTimeOrder());
     } else {
       planBuilder =
           planBuilder.withNewRoot(
@@ -143,7 +143,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
     // other common upstream node
     planBuilder =
         planBuilder
-            .planFill(analysis.getFillDescriptor(), queryStatement.getResultOrder())
+            .planFill(analysis.getFillDescriptor(), queryStatement.getResultTimeOrder())
             .planOffset(queryStatement.getRowOffset())
             .planLimit(queryStatement.getRowLimit());
 
@@ -166,7 +166,9 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
     if (isRawDataSource) {
       planBuilder =
           planBuilder.planRawDataSource(
-              sourceExpressions, queryStatement.getResultOrder(), analysis.getGlobalTimeFilter());
+              sourceExpressions,
+              queryStatement.getResultTimeOrder(),
+              analysis.getGlobalTimeFilter());
 
       if (queryStatement.isAggregationQuery()) {
         if (analysis.hasValueFilter()) {
@@ -176,14 +178,14 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                   aggregationTransformExpressions,
                   queryStatement.isGroupByTime(),
                   queryStatement.getSelectComponent().getZoneId(),
-                  queryStatement.getResultOrder());
+                  queryStatement.getResultTimeOrder());
         } else {
           planBuilder =
               planBuilder.planTransform(
                   aggregationTransformExpressions,
                   queryStatement.isGroupByTime(),
                   queryStatement.getSelectComponent().getZoneId(),
-                  queryStatement.getResultOrder());
+                  queryStatement.getResultTimeOrder());
         }
 
         boolean outputPartial =
@@ -197,7 +199,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                 analysis.getGroupByTimeParameter(),
                 curStep,
                 analysis.getTypeProvider(),
-                queryStatement.getResultOrder());
+                queryStatement.getResultTimeOrder());
 
         if (curStep.isOutputPartial()) {
           if (queryStatement.isGroupByTime() && analysis.getGroupByTimeParameter().hasOverlap()) {
@@ -210,7 +212,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                     aggregationExpressions,
                     analysis.getGroupByTimeParameter(),
                     curStep,
-                    queryStatement.getResultOrder());
+                    queryStatement.getResultTimeOrder());
           }
 
           if (queryStatement.isGroupByLevel()) {
@@ -220,7 +222,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                     analysis.getGroupByLevelExpressions(),
                     curStep,
                     analysis.getGroupByTimeParameter(),
-                    queryStatement.getResultOrder());
+                    queryStatement.getResultTimeOrder());
           }
         }
 
@@ -229,7 +231,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                 transformExpressions,
                 queryStatement.isGroupByTime(),
                 queryStatement.getSelectComponent().getZoneId(),
-                queryStatement.getResultOrder());
+                queryStatement.getResultTimeOrder());
       } else {
         if (analysis.hasValueFilter()) {
           planBuilder =
@@ -238,14 +240,14 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                   transformExpressions,
                   queryStatement.isGroupByTime(),
                   queryStatement.getSelectComponent().getZoneId(),
-                  queryStatement.getResultOrder());
+                  queryStatement.getResultTimeOrder());
         } else {
           planBuilder =
               planBuilder.planTransform(
                   transformExpressions,
                   queryStatement.isGroupByTime(),
                   queryStatement.getSelectComponent().getZoneId(),
-                  queryStatement.getResultOrder());
+                  queryStatement.getResultTimeOrder());
         }
       }
     } else {
@@ -269,7 +271,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
             planBuilder.planAggregationSourceWithIndexAdjust(
                 sourceExpressions,
                 curStep,
-                queryStatement.getResultOrder(),
+                queryStatement.getResultTimeOrder(),
                 analysis.getGlobalTimeFilter(),
                 analysis.getGroupByTimeParameter(),
                 aggregationExpressions,
@@ -282,7 +284,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                 .planAggregationSource(
                     sourceExpressions,
                     curStep,
-                    queryStatement.getResultOrder(),
+                    queryStatement.getResultTimeOrder(),
                     analysis.getGlobalTimeFilter(),
                     analysis.getGroupByTimeParameter(),
                     aggregationExpressions,
@@ -292,7 +294,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
                     transformExpressions,
                     queryStatement.isGroupByTime(),
                     queryStatement.getSelectComponent().getZoneId(),
-                    queryStatement.getResultOrder());
+                    queryStatement.getResultTimeOrder());
       }
     }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/AggregationNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/AggregationNode.java
index 00ad3a6341..837be95747 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/AggregationNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/AggregationNode.java
@@ -24,7 +24,7 @@ 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.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import javax.annotation.Nullable;
@@ -55,13 +55,13 @@ public class AggregationNode extends MultiChildNode {
   // Its value will be null if there is no `group by time` clause.
   @Nullable protected GroupByTimeParameter groupByTimeParameter;
 
-  protected OrderBy scanOrder;
+  protected Ordering scanOrder;
 
   public AggregationNode(
       PlanNodeId id,
       List<AggregationDescriptor> aggregationDescriptorList,
       @Nullable GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id, new ArrayList<>());
     this.aggregationDescriptorList = getDeduplicatedDescriptors(aggregationDescriptorList);
     this.groupByTimeParameter = groupByTimeParameter;
@@ -73,7 +73,7 @@ public class AggregationNode extends MultiChildNode {
       List<PlanNode> children,
       List<AggregationDescriptor> aggregationDescriptorList,
       @Nullable GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     this(id, aggregationDescriptorList, groupByTimeParameter, scanOrder);
     this.children = children;
   }
@@ -87,7 +87,7 @@ public class AggregationNode extends MultiChildNode {
     return groupByTimeParameter;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
@@ -173,7 +173,7 @@ public class AggregationNode extends MultiChildNode {
     if (isNull == 1) {
       groupByTimeParameter = GroupByTimeParameter.deserialize(byteBuffer);
     }
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new AggregationNode(
         planNodeId, aggregationDescriptorList, groupByTimeParameter, scanOrder);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java
index cdcfedab24..37a71fa58c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 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.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.io.DataOutputStream;
@@ -38,27 +38,27 @@ public class DeviceMergeNode extends MultiChildNode {
 
   // The result output order, which could sort by device and time.
   // The size of this list is 2 and the first OrderBy in this list has higher priority.
-  private final List<OrderBy> mergeOrders;
+  private final List<SortItem> mergeOrders;
 
   // the list of selected devices
   private final List<String> devices;
 
   public DeviceMergeNode(
-      PlanNodeId id, List<PlanNode> children, List<OrderBy> mergeOrders, List<String> devices) {
+      PlanNodeId id, List<PlanNode> children, List<SortItem> mergeOrders, List<String> devices) {
     super(id);
     this.children = children;
     this.mergeOrders = mergeOrders;
     this.devices = devices;
   }
 
-  public DeviceMergeNode(PlanNodeId id, List<OrderBy> mergeOrders, List<String> devices) {
+  public DeviceMergeNode(PlanNodeId id, List<SortItem> mergeOrders, List<String> devices) {
     super(id);
     this.children = new ArrayList<>();
     this.mergeOrders = mergeOrders;
     this.devices = devices;
   }
 
-  public List<OrderBy> getMergeOrders() {
+  public List<SortItem> getMergeOrders() {
     return mergeOrders;
   }
 
@@ -103,8 +103,10 @@ public class DeviceMergeNode extends MultiChildNode {
   @Override
   protected void serializeAttributes(ByteBuffer byteBuffer) {
     PlanNodeType.DEVICE_MERGE.serialize(byteBuffer);
-    ReadWriteIOUtils.write(mergeOrders.get(0).ordinal(), byteBuffer);
-    ReadWriteIOUtils.write(mergeOrders.get(1).ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(mergeOrders.size(), byteBuffer);
+    for (SortItem mergeOrder : mergeOrders) {
+      mergeOrder.serialize(byteBuffer);
+    }
     ReadWriteIOUtils.write(devices.size(), byteBuffer);
     for (String deviceName : devices) {
       ReadWriteIOUtils.write(deviceName, byteBuffer);
@@ -114,8 +116,10 @@ public class DeviceMergeNode extends MultiChildNode {
   @Override
   protected void serializeAttributes(DataOutputStream stream) throws IOException {
     PlanNodeType.DEVICE_MERGE.serialize(stream);
-    ReadWriteIOUtils.write(mergeOrders.get(0).ordinal(), stream);
-    ReadWriteIOUtils.write(mergeOrders.get(1).ordinal(), stream);
+    ReadWriteIOUtils.write(mergeOrders.size(), stream);
+    for (SortItem mergeOrder : mergeOrders) {
+      mergeOrder.serialize(stream);
+    }
     ReadWriteIOUtils.write(devices.size(), stream);
     for (String deviceName : devices) {
       ReadWriteIOUtils.write(deviceName, stream);
@@ -123,9 +127,12 @@ public class DeviceMergeNode extends MultiChildNode {
   }
 
   public static DeviceMergeNode deserialize(ByteBuffer byteBuffer) {
-    List<OrderBy> mergeOrders = new ArrayList<>();
-    mergeOrders.add(OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)]);
-    mergeOrders.add(OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)]);
+    int mergeOrdersSize = ReadWriteIOUtils.readInt(byteBuffer);
+    List<SortItem> mergeOrders = new ArrayList<>(mergeOrdersSize);
+    while (mergeOrdersSize > 0) {
+      mergeOrders.add(SortItem.deserialize(byteBuffer));
+      mergeOrdersSize--;
+    }
     int devicesSize = ReadWriteIOUtils.readInt(byteBuffer);
     List<String> devices = new ArrayList<>();
     while (devicesSize > 0) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java
index 3a54be0b34..fb074122ba 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java
@@ -22,7 +22,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 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.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.io.DataOutputStream;
@@ -46,7 +46,7 @@ public class DeviceViewNode extends MultiChildNode {
 
   // The result output order, which could sort by device and time.
   // The size of this list is 2 and the first OrderBy in this list has higher priority.
-  private final List<OrderBy> mergeOrders;
+  private final List<SortItem> mergeOrders;
 
   // The size devices and children should be the same.
   private final List<String> devices = new ArrayList<>();
@@ -56,11 +56,11 @@ public class DeviceViewNode extends MultiChildNode {
 
   // e.g. [s1,s2,s3] is query, but [s1, s3] exists in device1, then device1 -> [1, 3], s1 is 1 but
   // not 0 because device is the first column
-  private Map<String, List<Integer>> deviceToMeasurementIndexesMap;
+  private final Map<String, List<Integer>> deviceToMeasurementIndexesMap;
 
   public DeviceViewNode(
       PlanNodeId id,
-      List<OrderBy> mergeOrders,
+      List<SortItem> mergeOrders,
       List<String> outputColumnNames,
       Map<String, List<Integer>> deviceToMeasurementIndexesMap) {
     super(id);
@@ -71,7 +71,7 @@ public class DeviceViewNode extends MultiChildNode {
 
   public DeviceViewNode(
       PlanNodeId id,
-      List<OrderBy> mergeOrders,
+      List<SortItem> mergeOrders,
       List<String> outputColumnNames,
       List<String> devices,
       Map<String, List<Integer>> deviceToMeasurementIndexesMap) {
@@ -116,7 +116,7 @@ public class DeviceViewNode extends MultiChildNode {
         getPlanNodeId(), mergeOrders, outputColumnNames, devices, deviceToMeasurementIndexesMap);
   }
 
-  public List<OrderBy> getMergeOrders() {
+  public List<SortItem> getMergeOrders() {
     return mergeOrders;
   }
 
@@ -133,8 +133,10 @@ public class DeviceViewNode extends MultiChildNode {
   @Override
   protected void serializeAttributes(ByteBuffer byteBuffer) {
     PlanNodeType.DEVICE_VIEW.serialize(byteBuffer);
-    ReadWriteIOUtils.write(mergeOrders.get(0).ordinal(), byteBuffer);
-    ReadWriteIOUtils.write(mergeOrders.get(1).ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(mergeOrders.size(), byteBuffer);
+    for (SortItem mergeOrder : mergeOrders) {
+      mergeOrder.serialize(byteBuffer);
+    }
     ReadWriteIOUtils.write(outputColumnNames.size(), byteBuffer);
     for (String column : outputColumnNames) {
       ReadWriteIOUtils.write(column, byteBuffer);
@@ -156,8 +158,10 @@ public class DeviceViewNode extends MultiChildNode {
   @Override
   protected void serializeAttributes(DataOutputStream stream) throws IOException {
     PlanNodeType.DEVICE_VIEW.serialize(stream);
-    ReadWriteIOUtils.write(mergeOrders.get(0).ordinal(), stream);
-    ReadWriteIOUtils.write(mergeOrders.get(1).ordinal(), stream);
+    ReadWriteIOUtils.write(mergeOrders.size(), stream);
+    for (SortItem mergeOrder : mergeOrders) {
+      mergeOrder.serialize(stream);
+    }
     ReadWriteIOUtils.write(outputColumnNames.size(), stream);
     for (String column : outputColumnNames) {
       ReadWriteIOUtils.write(column, stream);
@@ -177,9 +181,12 @@ public class DeviceViewNode extends MultiChildNode {
   }
 
   public static DeviceViewNode deserialize(ByteBuffer byteBuffer) {
-    List<OrderBy> mergeOrders = new ArrayList<>();
-    mergeOrders.add(OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)]);
-    mergeOrders.add(OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)]);
+    int mergeOrdersSize = ReadWriteIOUtils.readInt(byteBuffer);
+    List<SortItem> mergeOrders = new ArrayList<>(mergeOrdersSize);
+    while (mergeOrdersSize > 0) {
+      mergeOrders.add(SortItem.deserialize(byteBuffer));
+      mergeOrdersSize--;
+    }
     int columnSize = ReadWriteIOUtils.readInt(byteBuffer);
     List<String> outputColumnNames = new ArrayList<>();
     while (columnSize > 0) {
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 d5a7fce186..5acb18a9de 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,7 +23,7 @@ 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.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import com.google.common.collect.ImmutableList;
@@ -40,7 +40,7 @@ public class FillNode extends ProcessNode {
   // descriptions of how null values are filled
   private FillDescriptor fillDescriptor;
 
-  private OrderBy scanOrder;
+  private Ordering scanOrder;
 
   private PlanNode child;
 
@@ -48,13 +48,14 @@ public class FillNode extends ProcessNode {
     super(id);
   }
 
-  public FillNode(PlanNodeId id, FillDescriptor fillDescriptor, OrderBy scanOrder) {
+  public FillNode(PlanNodeId id, FillDescriptor fillDescriptor, Ordering scanOrder) {
     this(id);
     this.fillDescriptor = fillDescriptor;
     this.scanOrder = scanOrder;
   }
 
-  public FillNode(PlanNodeId id, PlanNode child, FillDescriptor fillDescriptor, OrderBy scanOrder) {
+  public FillNode(
+      PlanNodeId id, PlanNode child, FillDescriptor fillDescriptor, Ordering scanOrder) {
     this(id, fillDescriptor, scanOrder);
     this.child = child;
   }
@@ -109,7 +110,7 @@ public class FillNode extends ProcessNode {
 
   public static FillNode deserialize(ByteBuffer byteBuffer) {
     FillDescriptor fillDescriptor = FillDescriptor.deserialize(byteBuffer);
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new FillNode(planNodeId, fillDescriptor, scanOrder);
   }
@@ -140,7 +141,7 @@ public class FillNode extends ProcessNode {
     return fillDescriptor;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FilterNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FilterNode.java
index 10df916541..0e513afbbc 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FilterNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/FilterNode.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 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.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.io.DataOutputStream;
@@ -43,7 +43,7 @@ public class FilterNode extends TransformNode {
       Expression predicate,
       boolean keepNull,
       ZoneId zoneId,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id, childPlanNode, outputExpressions, keepNull, zoneId, scanOrder);
     this.predicate = predicate;
   }
@@ -54,7 +54,7 @@ public class FilterNode extends TransformNode {
       Expression predicate,
       boolean keepNull,
       ZoneId zoneId,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id, outputExpressions, keepNull, zoneId, scanOrder);
     this.predicate = predicate;
   }
@@ -105,7 +105,7 @@ public class FilterNode extends TransformNode {
     Expression predicate = Expression.deserialize(byteBuffer);
     boolean keepNull = ReadWriteIOUtils.readBool(byteBuffer);
     ZoneId zoneId = ZoneId.of(Objects.requireNonNull(ReadWriteIOUtils.readString(byteBuffer)));
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new FilterNode(planNodeId, outputExpressions, predicate, keepNull, zoneId, scanOrder);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/GroupByLevelNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/GroupByLevelNode.java
index 2919877a55..3c7c653b70 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/GroupByLevelNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/GroupByLevelNode.java
@@ -25,7 +25,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByLevelDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import javax.annotation.Nullable;
@@ -62,14 +62,14 @@ public class GroupByLevelNode extends MultiChildNode {
   // Its value will be null if there is no `group by time` clause.
   @Nullable protected GroupByTimeParameter groupByTimeParameter;
 
-  protected OrderBy scanOrder;
+  protected Ordering scanOrder;
 
   public GroupByLevelNode(
       PlanNodeId id,
       List<PlanNode> children,
       List<GroupByLevelDescriptor> groupByLevelDescriptors,
       GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id, children);
     this.groupByLevelDescriptors = groupByLevelDescriptors;
     this.groupByTimeParameter = groupByTimeParameter;
@@ -80,7 +80,7 @@ public class GroupByLevelNode extends MultiChildNode {
       PlanNodeId id,
       List<GroupByLevelDescriptor> groupByLevelDescriptors,
       GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id);
     this.groupByLevelDescriptors = groupByLevelDescriptors;
     this.groupByTimeParameter = groupByTimeParameter;
@@ -173,7 +173,7 @@ public class GroupByLevelNode extends MultiChildNode {
     if (isNull == 1) {
       groupByTimeParameter = GroupByTimeParameter.deserialize(byteBuffer);
     }
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new GroupByLevelNode(
         planNodeId, groupByLevelDescriptors, groupByTimeParameter, scanOrder);
@@ -184,7 +184,7 @@ public class GroupByLevelNode extends MultiChildNode {
     return groupByTimeParameter;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SlidingWindowAggregationNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SlidingWindowAggregationNode.java
index 0e860b335b..a2f4935791 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SlidingWindowAggregationNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SlidingWindowAggregationNode.java
@@ -25,7 +25,7 @@ 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.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import com.google.common.collect.ImmutableList;
@@ -47,7 +47,7 @@ public class SlidingWindowAggregationNode extends ProcessNode {
   // The parameter of `group by time`.
   private final GroupByTimeParameter groupByTimeParameter;
 
-  protected OrderBy scanOrder = OrderBy.TIMESTAMP_ASC;
+  protected Ordering scanOrder;
 
   private PlanNode child;
 
@@ -55,7 +55,7 @@ public class SlidingWindowAggregationNode extends ProcessNode {
       PlanNodeId id,
       List<AggregationDescriptor> aggregationDescriptorList,
       GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id);
     this.aggregationDescriptorList = aggregationDescriptorList;
     this.groupByTimeParameter = groupByTimeParameter;
@@ -67,7 +67,7 @@ public class SlidingWindowAggregationNode extends ProcessNode {
       PlanNode child,
       List<AggregationDescriptor> aggregationDescriptorList,
       GroupByTimeParameter groupByTimeParameter,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     this(id, aggregationDescriptorList, groupByTimeParameter, scanOrder);
     this.child = child;
   }
@@ -84,7 +84,7 @@ public class SlidingWindowAggregationNode extends ProcessNode {
     return groupByTimeParameter;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
@@ -170,7 +170,7 @@ public class SlidingWindowAggregationNode extends ProcessNode {
     if (isNull == 1) {
       groupByTimeParameter = GroupByTimeParameter.deserialize(byteBuffer);
     }
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new SlidingWindowAggregationNode(
         planNodeId, aggregationDescriptorList, groupByTimeParameter, scanOrder);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SortNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SortNode.java
index 509d7b9398..283987eb97 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SortNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/SortNode.java
@@ -22,7 +22,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 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.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import com.google.common.collect.ImmutableList;
@@ -41,19 +41,19 @@ public class SortNode extends ProcessNode {
 
   private PlanNode child;
 
-  private final OrderBy sortOrder;
+  private final Ordering sortOrder;
 
-  public SortNode(PlanNodeId id, OrderBy sortOrder) {
+  public SortNode(PlanNodeId id, Ordering sortOrder) {
     super(id);
     this.sortOrder = sortOrder;
   }
 
-  public SortNode(PlanNodeId id, PlanNode child, OrderBy sortOrder) {
+  public SortNode(PlanNodeId id, PlanNode child, Ordering sortOrder) {
     this(id, sortOrder);
     this.child = child;
   }
 
-  public OrderBy getSortOrder() {
+  public Ordering getSortOrder() {
     return sortOrder;
   }
 
@@ -100,7 +100,7 @@ public class SortNode extends ProcessNode {
   }
 
   public static SortNode deserialize(ByteBuffer byteBuffer) {
-    OrderBy orderBy = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering orderBy = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new SortNode(planNodeId, orderBy);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TimeJoinNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TimeJoinNode.java
index 1e9fe6424b..5685e6d95b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TimeJoinNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TimeJoinNode.java
@@ -22,7 +22,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 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.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.io.DataOutputStream;
@@ -41,19 +41,19 @@ import java.util.stream.Collectors;
 public class TimeJoinNode extends MultiChildNode {
 
   // This parameter indicates the order when executing multiway merge sort.
-  private final OrderBy mergeOrder;
+  private final Ordering mergeOrder;
 
-  public TimeJoinNode(PlanNodeId id, OrderBy mergeOrder) {
+  public TimeJoinNode(PlanNodeId id, Ordering mergeOrder) {
     super(id, new ArrayList<>());
     this.mergeOrder = mergeOrder;
   }
 
-  public TimeJoinNode(PlanNodeId id, OrderBy mergeOrder, List<PlanNode> children) {
+  public TimeJoinNode(PlanNodeId id, Ordering mergeOrder, List<PlanNode> children) {
     super(id, children);
     this.mergeOrder = mergeOrder;
   }
 
-  public OrderBy getMergeOrder() {
+  public Ordering getMergeOrder() {
     return mergeOrder;
   }
 
@@ -104,7 +104,7 @@ public class TimeJoinNode extends MultiChildNode {
   }
 
   public static TimeJoinNode deserialize(ByteBuffer byteBuffer) {
-    OrderBy mergeOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering mergeOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new TimeJoinNode(planNodeId, mergeOrder);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TransformNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TransformNode.java
index 5d07be4174..f6233e2e9d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TransformNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/TransformNode.java
@@ -24,7 +24,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 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.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import com.google.common.collect.ImmutableList;
@@ -46,7 +46,7 @@ public class TransformNode extends ProcessNode {
   protected final boolean keepNull;
   protected final ZoneId zoneId;
 
-  protected final OrderBy scanOrder;
+  protected final Ordering scanOrder;
 
   private List<String> outputColumnNames;
 
@@ -56,7 +56,7 @@ public class TransformNode extends ProcessNode {
       Expression[] outputExpressions,
       boolean keepNull,
       ZoneId zoneId,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id);
     this.childPlanNode = childPlanNode;
     this.outputExpressions = outputExpressions;
@@ -70,7 +70,7 @@ public class TransformNode extends ProcessNode {
       Expression[] outputExpressions,
       boolean keepNull,
       ZoneId zoneId,
-      OrderBy scanOrder) {
+      Ordering scanOrder) {
     super(id);
     this.outputExpressions = outputExpressions;
     this.keepNull = keepNull;
@@ -146,7 +146,7 @@ public class TransformNode extends ProcessNode {
     }
     boolean keepNull = ReadWriteIOUtils.readBool(byteBuffer);
     ZoneId zoneId = ZoneId.of(Objects.requireNonNull(ReadWriteIOUtils.readString(byteBuffer)));
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
     return new TransformNode(planNodeId, outputExpressions, keepNull, zoneId, scanOrder);
   }
@@ -163,7 +163,7 @@ public class TransformNode extends ProcessNode {
     return zoneId;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesAggregationScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesAggregationScanNode.java
index 0bcc14f957..d2a4f83ab4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesAggregationScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesAggregationScanNode.java
@@ -31,7 +31,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.AggregationNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -70,7 +70,7 @@ public class AlignedSeriesAggregationScanNode extends SeriesAggregationSourceNod
       PlanNodeId id,
       AlignedPath alignedPath,
       List<AggregationDescriptor> aggregationDescriptorList,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       @Nullable GroupByTimeParameter groupByTimeParameter) {
     this(id, alignedPath, aggregationDescriptorList);
     this.scanOrder = scanOrder;
@@ -81,7 +81,7 @@ public class AlignedSeriesAggregationScanNode extends SeriesAggregationSourceNod
       PlanNodeId id,
       AlignedPath alignedPath,
       List<AggregationDescriptor> aggregationDescriptorList,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       @Nullable Filter timeFilter,
       @Nullable GroupByTimeParameter groupByTimeParameter,
       TRegionReplicaSet dataRegionReplicaSet) {
@@ -94,7 +94,7 @@ public class AlignedSeriesAggregationScanNode extends SeriesAggregationSourceNod
     return alignedPath;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
@@ -222,7 +222,7 @@ public class AlignedSeriesAggregationScanNode extends SeriesAggregationSourceNod
     for (int i = 0; i < aggregateDescriptorSize; i++) {
       aggregationDescriptorList.add(AggregationDescriptor.deserialize(byteBuffer));
     }
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     byte isNull = ReadWriteIOUtils.readByte(byteBuffer);
     Filter timeFilter = null;
     if (isNull == 1) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesScanNode.java
index 5ebab844fc..307ae2c16d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/AlignedSeriesScanNode.java
@@ -28,7 +28,7 @@ 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.PlanNodeUtil;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
@@ -53,7 +53,7 @@ public class AlignedSeriesScanNode extends SeriesSourceNode {
   // The order to traverse the data.
   // Currently, we only support TIMESTAMP_ASC and TIMESTAMP_DESC here.
   // The default order is TIMESTAMP_ASC, which means "order by timestamp asc"
-  private OrderBy scanOrder = OrderBy.TIMESTAMP_ASC;
+  private Ordering scanOrder = Ordering.ASC;
 
   // time filter for current series, could be null if doesn't exist
   @Nullable private Filter timeFilter;
@@ -75,7 +75,7 @@ public class AlignedSeriesScanNode extends SeriesSourceNode {
     this.alignedPath = alignedPath;
   }
 
-  public AlignedSeriesScanNode(PlanNodeId id, AlignedPath alignedPath, OrderBy scanOrder) {
+  public AlignedSeriesScanNode(PlanNodeId id, AlignedPath alignedPath, Ordering scanOrder) {
     this(id, alignedPath);
     this.scanOrder = scanOrder;
   }
@@ -83,7 +83,7 @@ public class AlignedSeriesScanNode extends SeriesSourceNode {
   public AlignedSeriesScanNode(
       PlanNodeId id,
       AlignedPath alignedPath,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       @Nullable Filter timeFilter,
       @Nullable Filter valueFilter,
       int limit,
@@ -101,7 +101,7 @@ public class AlignedSeriesScanNode extends SeriesSourceNode {
     return alignedPath;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
@@ -230,7 +230,7 @@ public class AlignedSeriesScanNode extends SeriesSourceNode {
 
   public static AlignedSeriesScanNode deserialize(ByteBuffer byteBuffer) {
     AlignedPath alignedPath = (AlignedPath) PathDeserializeUtil.deserialize(byteBuffer);
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     byte isNull = ReadWriteIOUtils.readByte(byteBuffer);
     Filter timeFilter = null;
     if (isNull == 1) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationScanNode.java
index 7c4c7563f9..9efbb9a3cf 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationScanNode.java
@@ -30,7 +30,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.AggregationNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -82,7 +82,7 @@ public class SeriesAggregationScanNode extends SeriesAggregationSourceNode {
       PlanNodeId id,
       MeasurementPath seriesPath,
       List<AggregationDescriptor> aggregationDescriptorList,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       @Nullable GroupByTimeParameter groupByTimeParameter) {
     this(id, seriesPath, aggregationDescriptorList);
     this.scanOrder = scanOrder;
@@ -93,7 +93,7 @@ public class SeriesAggregationScanNode extends SeriesAggregationSourceNode {
       PlanNodeId id,
       MeasurementPath seriesPath,
       List<AggregationDescriptor> aggregationDescriptorList,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       @Nullable Filter timeFilter,
       @Nullable GroupByTimeParameter groupByTimeParameter,
       TRegionReplicaSet dataRegionReplicaSet) {
@@ -102,7 +102,7 @@ public class SeriesAggregationScanNode extends SeriesAggregationSourceNode {
     this.regionReplicaSet = dataRegionReplicaSet;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
@@ -233,7 +233,7 @@ public class SeriesAggregationScanNode extends SeriesAggregationSourceNode {
     for (int i = 0; i < aggregateDescriptorSize; i++) {
       aggregationDescriptorList.add(AggregationDescriptor.deserialize(byteBuffer));
     }
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     byte isNull = ReadWriteIOUtils.readByte(byteBuffer);
     Filter timeFilter = null;
     if (isNull == 1) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationSourceNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationSourceNode.java
index d0087b2303..863f597832 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationSourceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesAggregationSourceNode.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.mpp.plan.planner.plan.node.source;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 import javax.annotation.Nullable;
@@ -38,7 +38,7 @@ public abstract class SeriesAggregationSourceNode extends SeriesSourceNode {
   // The order to traverse the data.
   // Currently, we only support TIMESTAMP_ASC and TIMESTAMP_DESC here.
   // The default order is TIMESTAMP_ASC, which means "order by timestamp asc"
-  protected OrderBy scanOrder = OrderBy.TIMESTAMP_ASC;
+  protected Ordering scanOrder = Ordering.ASC;
 
   // time filter for current series, could be null if doesn't exist
   @Nullable protected Filter timeFilter;
@@ -61,7 +61,7 @@ public abstract class SeriesAggregationSourceNode extends SeriesSourceNode {
     this.aggregationDescriptorList = aggregationDescriptorList;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesScanNode.java
index 03ff5dfb23..b38a6df4b0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/source/SeriesScanNode.java
@@ -27,7 +27,7 @@ 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.PlanNodeUtil;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -57,7 +57,7 @@ public class SeriesScanNode extends SeriesSourceNode {
   // The order to traverse the data.
   // Currently, we only support TIMESTAMP_ASC and TIMESTAMP_DESC here.
   // The default order is TIMESTAMP_ASC, which means "order by timestamp asc"
-  private OrderBy scanOrder = OrderBy.TIMESTAMP_ASC;
+  private Ordering scanOrder = Ordering.ASC;
 
   // time filter for current series, could be null if doesn't exist
   @Nullable private Filter timeFilter;
@@ -79,7 +79,7 @@ public class SeriesScanNode extends SeriesSourceNode {
     this.seriesPath = seriesPath;
   }
 
-  public SeriesScanNode(PlanNodeId id, MeasurementPath seriesPath, OrderBy scanOrder) {
+  public SeriesScanNode(PlanNodeId id, MeasurementPath seriesPath, Ordering scanOrder) {
     this(id, seriesPath);
     this.scanOrder = scanOrder;
   }
@@ -87,7 +87,7 @@ public class SeriesScanNode extends SeriesSourceNode {
   public SeriesScanNode(
       PlanNodeId id,
       MeasurementPath seriesPath,
-      OrderBy scanOrder,
+      Ordering scanOrder,
       @Nullable Filter timeFilter,
       @Nullable Filter valueFilter,
       int limit,
@@ -133,11 +133,11 @@ public class SeriesScanNode extends SeriesSourceNode {
     this.offset = offset;
   }
 
-  public OrderBy getScanOrder() {
+  public Ordering getScanOrder() {
     return scanOrder;
   }
 
-  public void setScanOrder(OrderBy scanOrder) {
+  public void setScanOrder(Ordering scanOrder) {
     this.scanOrder = scanOrder;
   }
 
@@ -241,7 +241,7 @@ public class SeriesScanNode extends SeriesSourceNode {
 
   public static SeriesScanNode deserialize(ByteBuffer byteBuffer) {
     MeasurementPath partialPath = (MeasurementPath) PathDeserializeUtil.deserialize(byteBuffer);
-    OrderBy scanOrder = OrderBy.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
     byte isNull = ReadWriteIOUtils.readByte(byteBuffer);
     Filter timeFilter = null;
     if (isNull == 1) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
index 8fe478d918..8b968408a2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
@@ -19,17 +19,26 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
+import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
+
 import java.util.ArrayList;
 import java.util.List;
 
+import static com.google.common.base.Preconditions.checkState;
+
 /** The order of query result set */
-public class OrderByComponent {
+public class OrderByComponent extends StatementNode {
 
   private final List<SortItem> sortItemList;
 
   private boolean orderByTime = false;
+  private int timeOrderPriority = -1;
+
   private boolean orderByTimeseries = false;
+  private int timeseriesOrderPriority = -1;
+
   private boolean orderByDevice = false;
+  private int deviceOrderPriority = -1;
 
   public OrderByComponent() {
     this.sortItemList = new ArrayList<>();
@@ -38,12 +47,15 @@ public class OrderByComponent {
   public void addSortItem(SortItem sortItem) {
     this.sortItemList.add(sortItem);
 
-    if (sortItem.getSortKey() == SortItem.SortKey.TIME) {
+    if (sortItem.getSortKey() == SortKey.TIME) {
       orderByTime = true;
-    } else if (sortItem.getSortKey() == SortItem.SortKey.TIMESERIES) {
+      timeOrderPriority = sortItemList.size() - 1;
+    } else if (sortItem.getSortKey() == SortKey.TIMESERIES) {
       orderByTimeseries = true;
+      timeseriesOrderPriority = sortItemList.size() - 1;
     } else {
       orderByDevice = true;
+      deviceOrderPriority = sortItemList.size() - 1;
     }
   }
 
@@ -55,11 +67,26 @@ public class OrderByComponent {
     return orderByTime;
   }
 
+  public Ordering getTimeOrder() {
+    checkState(timeOrderPriority != -1, "The time order is not specified.");
+    return sortItemList.get(timeOrderPriority).getOrdering();
+  }
+
   public boolean isOrderByTimeseries() {
     return orderByTimeseries;
   }
 
+  public Ordering getTimeseriesOrder() {
+    checkState(timeOrderPriority != -1, "The timeseries order is not specified.");
+    return sortItemList.get(timeseriesOrderPriority).getOrdering();
+  }
+
   public boolean isOrderByDevice() {
     return orderByDevice;
   }
+
+  public Ordering getDeviceOrder() {
+    checkState(timeOrderPriority != -1, "The device order is not specified.");
+    return sortItemList.get(deviceOrderPriority).getOrdering();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/Ordering.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/Ordering.java
new file mode 100644
index 0000000000..141a394567
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/Ordering.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+public enum Ordering {
+  ASC,
+  DESC;
+
+  public Ordering reverse() {
+    if (this == ASC) {
+      return DESC;
+    } else {
+      return ASC;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortItem.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortItem.java
index bbdff9f01b..41fc47dd47 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortItem.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortItem.java
@@ -19,30 +19,14 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
-import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Objects;
 
-public class SortItem extends StatementNode {
-
-  public enum SortKey {
-    TIME,
-    TIMESERIES,
-    DEVICE
-  }
-
-  public enum Ordering {
-    ASC,
-    DESC;
-
-    public Ordering reverse() {
-      if (this == ASC) {
-        return DESC;
-      } else {
-        return ASC;
-      }
-    }
-  }
+public class SortItem {
 
   private final SortKey sortKey;
   private final Ordering ordering;
@@ -64,6 +48,22 @@ public class SortItem extends StatementNode {
     return new SortItem(getSortKey(), getOrdering().reverse());
   }
 
+  public void serialize(ByteBuffer byteBuffer) {
+    ReadWriteIOUtils.write(sortKey.ordinal(), byteBuffer);
+    ReadWriteIOUtils.write(ordering.ordinal(), byteBuffer);
+  }
+
+  public void serialize(DataOutputStream stream) throws IOException {
+    ReadWriteIOUtils.write(sortKey.ordinal(), stream);
+    ReadWriteIOUtils.write(ordering.ordinal(), stream);
+  }
+
+  public static SortItem deserialize(ByteBuffer byteBuffer) {
+    SortKey sortKey = SortKey.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    Ordering ordering = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
+    return new SortItem(sortKey, ordering);
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortKey.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortKey.java
new file mode 100644
index 0000000000..e4a876c85a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortKey.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+public enum SortKey {
+  TIME,
+  TIMESERIES,
+  DEVICE
+}
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 ea99bd3fae..aa4f01cd68 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
@@ -32,6 +32,7 @@ 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.OrderByComponent;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 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;
@@ -227,6 +228,13 @@ public class QueryStatement extends Statement {
     return orderByComponent != null && orderByComponent.isOrderByDevice();
   }
 
+  public Ordering getResultTimeOrder() {
+    if (orderByComponent == null || !orderByComponent.isOrderByTime()) {
+      return Ordering.ASC;
+    }
+    return orderByComponent.getTimeOrder();
+  }
+
   public void semanticCheck() {
     if (isAggregationQuery()) {
       if (disableAlign()) {
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 d4c58d998c..34d1dc6c38 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
@@ -53,7 +53,9 @@ import org.apache.iotdb.db.mpp.plan.expression.unary.LikeExpression;
 import org.apache.iotdb.db.mpp.plan.expression.unary.LogicNotExpression;
 import org.apache.iotdb.db.mpp.plan.expression.unary.NegationExpression;
 import org.apache.iotdb.db.mpp.plan.expression.unary.RegularExpression;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortKey;
 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;
@@ -1486,10 +1488,10 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
       throw new SQLParserException("Sorting by multiple fields is not supported.");
     }
     SortItem sortItem = parseOrderByAttributeClause(ctx.orderByAttributeClause(0));
-    if (sortItem.getSortKey() != SortItem.SortKey.TIME) {
+    if (sortItem.getSortKey() != SortKey.TIME) {
       throw new SQLParserException("It only supports sorting by time.");
     }
-    if (sortItem.getOrdering() == SortItem.Ordering.DESC) {
+    if (sortItem.getOrdering() == Ordering.DESC) {
       SpecialClauseComponent specialClauseComponent = queryOp.getSpecialClauseComponent();
       if (specialClauseComponent == null) {
         specialClauseComponent = new SpecialClauseComponent();
@@ -1501,8 +1503,8 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
 
   private SortItem parseOrderByAttributeClause(IoTDBSqlParser.OrderByAttributeClauseContext ctx) {
     return new SortItem(
-        SortItem.SortKey.valueOf(ctx.sortKey().getText().toUpperCase()),
-        ctx.DESC() != null ? SortItem.Ordering.DESC : SortItem.Ordering.ASC);
+        SortKey.valueOf(ctx.sortKey().getText().toUpperCase()),
+        ctx.DESC() != null ? Ordering.DESC : Ordering.ASC);
   }
 
   private void parseGroupByTimeClause(IoTDBSqlParser.GroupByTimeClauseContext ctx) {
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
index ae6923c08a..df8bd373ea 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
@@ -27,7 +27,7 @@ import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.db.service.IoTDB;
@@ -213,8 +213,8 @@ public class SchemaUtils {
    * data
    */
   public static boolean isConsistentWithScanOrder(
-      AggregationType aggregationFunction, OrderBy scanOrder) {
-    boolean ascending = scanOrder == OrderBy.TIMESTAMP_ASC;
+      AggregationType aggregationFunction, Ordering scanOrder) {
+    boolean ascending = scanOrder == Ordering.ASC;
     switch (aggregationFunction) {
       case MIN_TIME:
       case FIRST_VALUE:
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java
index 0d19e99889..dc13943bef 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java
@@ -44,7 +44,7 @@ import org.apache.iotdb.db.mpp.execution.operator.process.merge.SingleColumnMerg
 import org.apache.iotdb.db.mpp.execution.operator.source.SeriesScanOperator;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.InputLocation;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderTestUtil;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -149,7 +149,7 @@ public class DataDriverTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(2),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java
index 97121b80e3..b45a11f668 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java
@@ -38,7 +38,7 @@ import org.apache.iotdb.db.mpp.execution.operator.source.AlignedSeriesScanOperat
 import org.apache.iotdb.db.mpp.execution.operator.source.SeriesScanOperator;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.InputLocation;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.block.TsBlock;
@@ -347,7 +347,7 @@ public class AlignedSeriesScanOperatorTest {
                   seriesScanOperator6,
                   seriesScanOperator7,
                   seriesScanOperator8),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(
                   TSDataType.BOOLEAN,
                   TSDataType.INT32,
@@ -637,7 +637,7 @@ public class AlignedSeriesScanOperatorTest {
                   seriesScanOperator6,
                   seriesScanOperator7,
                   seriesScanOperator8),
-              OrderBy.TIMESTAMP_DESC,
+              Ordering.DESC,
               Arrays.asList(
                   TSDataType.BOOLEAN,
                   TSDataType.INT32,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java
index 7af69e9dbe..6625f296e6 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java
@@ -36,7 +36,7 @@ import org.apache.iotdb.db.mpp.execution.operator.process.merge.SingleColumnMerg
 import org.apache.iotdb.db.mpp.execution.operator.source.SeriesScanOperator;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.InputLocation;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderTestUtil;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -138,7 +138,7 @@ public class LimitOperatorTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(2),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java
index a796f89cdd..9256c852c9 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java
@@ -37,7 +37,7 @@ import org.apache.iotdb.db.mpp.execution.operator.process.merge.SingleColumnMerg
 import org.apache.iotdb.db.mpp.execution.operator.source.SeriesScanOperator;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.InputLocation;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderTestUtil;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -141,7 +141,7 @@ public class OffsetOperatorTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(2),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
@@ -255,7 +255,7 @@ public class OffsetOperatorTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(2),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
@@ -360,7 +360,7 @@ public class OffsetOperatorTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(2),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java
index 9cf7c6331f..9b9b81a856 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java
@@ -42,7 +42,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationStep;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.InputLocation;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderTestUtil;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
@@ -364,7 +364,7 @@ public class RawDataAggregationOperatorTest {
         new TimeJoinOperator(
             fragmentInstanceContext.getOperatorContexts().get(2),
             Arrays.asList(seriesScanOperator1, seriesScanOperator2),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             Arrays.asList(TSDataType.INT32, TSDataType.INT32),
             Arrays.asList(
                 new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java
index 06ef37f927..6030229577 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java
@@ -36,7 +36,7 @@ import org.apache.iotdb.db.mpp.execution.operator.process.merge.SingleColumnMerg
 import org.apache.iotdb.db.mpp.execution.operator.source.SeriesScanOperator;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.InputLocation;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderTestUtil;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -136,7 +136,7 @@ public class TimeJoinOperatorTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(2),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
@@ -251,7 +251,7 @@ public class TimeJoinOperatorTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(3),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2, seriesScanOperator3),
-              OrderBy.TIMESTAMP_ASC,
+              Ordering.ASC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new AscTimeComparator()),
@@ -369,7 +369,7 @@ public class TimeJoinOperatorTest {
           new TimeJoinOperator(
               fragmentInstanceContext.getOperatorContexts().get(3),
               Arrays.asList(seriesScanOperator1, seriesScanOperator2, seriesScanOperator3),
-              OrderBy.TIMESTAMP_DESC,
+              Ordering.DESC,
               Arrays.asList(TSDataType.INT32, TSDataType.INT32, TSDataType.INT32),
               Arrays.asList(
                   new SingleColumnMerger(new InputLocation(0, 0), new DescTimeComparator()),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/FragmentInstanceSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/FragmentInstanceSerdeTest.java
index ebcd805c0b..7ab2894a06 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/FragmentInstanceSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/FragmentInstanceSerdeTest.java
@@ -35,7 +35,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.LimitNode;
 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.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
 
 import com.google.common.collect.ImmutableList;
@@ -115,17 +115,16 @@ public class FragmentInstanceSerdeTest {
     OffsetNode offsetNode = new OffsetNode(new PlanNodeId("OffsetNode"), 100);
     LimitNode limitNode = new LimitNode(new PlanNodeId("LimitNode"), 100);
 
-    TimeJoinNode timeJoinNode =
-        new TimeJoinNode(new PlanNodeId("TimeJoinNode"), OrderBy.TIMESTAMP_DESC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(new PlanNodeId("TimeJoinNode"), Ordering.DESC);
     SeriesScanNode seriesScanNode1 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode1"), new MeasurementPath("root.sg.d1.s2"));
-    seriesScanNode1.setScanOrder(OrderBy.TIMESTAMP_DESC);
+    seriesScanNode1.setScanOrder(Ordering.DESC);
     SeriesScanNode seriesScanNode2 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode2"), new MeasurementPath("root.sg.d2.s1"));
-    seriesScanNode2.setScanOrder(OrderBy.TIMESTAMP_DESC);
+    seriesScanNode2.setScanOrder(Ordering.DESC);
     SeriesScanNode seriesScanNode3 =
         new SeriesScanNode(new PlanNodeId("SeriesScanNode3"), new MeasurementPath("root.sg.d2.s2"));
-    seriesScanNode3.setScanOrder(OrderBy.TIMESTAMP_DESC);
+    seriesScanNode3.setScanOrder(Ordering.DESC);
 
     // build tree
     timeJoinNode.addChild(seriesScanNode1);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java
index 753dee8c19..e098262c81 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java
@@ -49,7 +49,9 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.SeriesScanNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationStep;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByLevelDescriptor;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortKey;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
@@ -154,24 +156,22 @@ public class QueryLogicalPlanUtil {
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s1"),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     sourceNodeList.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s2"),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     sourceNodeList.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s4"),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     sourceNodeList.add(
         new AlignedSeriesScanNode(
-            queryId.genPlanNodeId(),
-            (AlignedPath) schemaMap.get("root.sg.d2.a"),
-            OrderBy.TIMESTAMP_ASC));
+            queryId.genPlanNodeId(), (AlignedPath) schemaMap.get("root.sg.d2.a"), Ordering.ASC));
     TimeJoinNode timeJoinNode =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC, sourceNodeList);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC, sourceNodeList);
     OffsetNode offsetNode = new OffsetNode(queryId.genPlanNodeId(), timeJoinNode, 10);
     LimitNode limitNode = new LimitNode(queryId.genPlanNodeId(), offsetNode, 10);
 
@@ -191,22 +191,22 @@ public class QueryLogicalPlanUtil {
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s1"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList.forEach(
         planNode -> ((SeriesScanNode) planNode).setTimeFilter(TimeFilter.gt(100)));
 
     TimeJoinNode timeJoinNode =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList);
 
     GreaterThanExpression timeFilter =
         new GreaterThanExpression(
@@ -230,7 +230,7 @@ public class QueryLogicalPlanUtil {
             predicate,
             false,
             ZonedDateTime.now().getOffset(),
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     OffsetNode offsetNode = new OffsetNode(queryId.genPlanNodeId(), filterNode, 100);
     LimitNode limitNode = new LimitNode(queryId.genPlanNodeId(), offsetNode, 100);
@@ -251,22 +251,22 @@ public class QueryLogicalPlanUtil {
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s3"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList1.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s1"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList1.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList1.forEach(
         planNode -> ((SeriesScanNode) planNode).setTimeFilter(TimeFilter.gt(100)));
 
     TimeJoinNode timeJoinNode1 =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList1);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList1);
 
     GreaterThanExpression timeFilter =
         new GreaterThanExpression(
@@ -289,29 +289,29 @@ public class QueryLogicalPlanUtil {
             predicate1,
             false,
             ZonedDateTime.now().getOffset(),
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     List<PlanNode> sourceNodeList2 = new ArrayList<>();
     sourceNodeList2.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s1"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList2.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList2.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s4"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList2.forEach(
         planNode -> ((SeriesScanNode) planNode).setTimeFilter(TimeFilter.gt(100)));
 
     TimeJoinNode timeJoinNode2 =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList2);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList2);
 
     GreaterThanExpression valueFilter2 =
         new GreaterThanExpression(
@@ -331,7 +331,7 @@ public class QueryLogicalPlanUtil {
             predicate2,
             false,
             ZonedDateTime.now().getOffset(),
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     Map<String, List<Integer>> deviceToMeasurementIndexesMap = new HashMap<>();
     deviceToMeasurementIndexesMap.put("root.sg.d1", Arrays.asList(1, 2, 3));
@@ -339,7 +339,9 @@ public class QueryLogicalPlanUtil {
     DeviceViewNode deviceViewNode =
         new DeviceViewNode(
             queryId.genPlanNodeId(),
-            Arrays.asList(OrderBy.DEVICE_ASC, OrderBy.TIMESTAMP_DESC),
+            Arrays.asList(
+                new SortItem(SortKey.DEVICE, Ordering.ASC),
+                new SortItem(SortKey.TIME, Ordering.DESC)),
             Arrays.asList(HeaderConstant.COLUMN_DEVICE, "s3", "s1", "s2", "s4"),
             deviceToMeasurementIndexesMap);
     deviceViewNode.addChildDeviceNode("root.sg.d1", filterNode1);
@@ -375,7 +377,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.a.s2"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -387,7 +389,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s1"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -399,7 +401,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s1"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -411,7 +413,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s2"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -423,7 +425,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s2"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             null));
     sourceNodeList.add(
         new AlignedSeriesAggregationScanNode(
@@ -435,7 +437,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.a.s1"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -447,7 +449,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s1"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -459,7 +461,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s1"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.forEach(
         node -> {
@@ -471,7 +473,7 @@ public class QueryLogicalPlanUtil {
         });
 
     TimeJoinNode timeJoinNode =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC, sourceNodeList);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC, sourceNodeList);
     OffsetNode offsetNode = new OffsetNode(queryId.genPlanNodeId(), timeJoinNode, 10);
     LimitNode limitNode = new LimitNode(queryId.genPlanNodeId(), offsetNode, 10);
 
@@ -508,7 +510,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.PARTIAL,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.a.s2"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -525,7 +527,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.PARTIAL,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s1"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -542,7 +544,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.PARTIAL,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s1"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -554,7 +556,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.PARTIAL,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s2"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.add(
         new SeriesAggregationScanNode(
@@ -566,7 +568,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.PARTIAL,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s2"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList.forEach(
         node -> {
@@ -622,7 +624,7 @@ public class QueryLogicalPlanUtil {
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.a.s1"))),
                     new TimeSeriesOperand(schemaMap.get("root.sg.*.*.s1")))),
             null,
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     OffsetNode offsetNode = new OffsetNode(queryId.genPlanNodeId(), groupByLevelNode, 100);
     LimitNode limitNode = new LimitNode(queryId.genPlanNodeId(), offsetNode, 100);
@@ -655,7 +657,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s1"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList1.add(
         new SeriesAggregationScanNode(
@@ -667,12 +669,12 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s2"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList1.forEach(node -> ((SeriesAggregationScanNode) node).setTimeFilter(timeFilter));
 
     TimeJoinNode timeJoinNode1 =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList1);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList1);
 
     List<PlanNode> sourceNodeList2 = new ArrayList<>();
     sourceNodeList2.add(
@@ -690,7 +692,7 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s1"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList2.add(
         new SeriesAggregationScanNode(
@@ -702,12 +704,12 @@ public class QueryLogicalPlanUtil {
                     AggregationStep.SINGLE,
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s2"))))),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null));
     sourceNodeList2.forEach(node -> ((SeriesAggregationScanNode) node).setTimeFilter(timeFilter));
 
     TimeJoinNode timeJoinNode2 =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList2);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList2);
 
     Map<String, List<Integer>> deviceToMeasurementIndexesMap = new HashMap<>();
     deviceToMeasurementIndexesMap.put("root.sg.d1", Arrays.asList(1, 3, 2));
@@ -715,7 +717,9 @@ public class QueryLogicalPlanUtil {
     DeviceViewNode deviceViewNode =
         new DeviceViewNode(
             queryId.genPlanNodeId(),
-            Arrays.asList(OrderBy.DEVICE_ASC, OrderBy.TIMESTAMP_DESC),
+            Arrays.asList(
+                new SortItem(SortKey.DEVICE, Ordering.ASC),
+                new SortItem(SortKey.TIME, Ordering.DESC)),
             Arrays.asList(
                 HeaderConstant.COLUMN_DEVICE, "count(s1)", "max_value(s2)", "last_value(s1)"),
             deviceToMeasurementIndexesMap);
@@ -741,27 +745,27 @@ public class QueryLogicalPlanUtil {
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s1"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s1"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList.forEach(
         planNode -> ((SeriesScanNode) planNode).setTimeFilter(TimeFilter.gt(100)));
 
     TimeJoinNode timeJoinNode =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList);
 
     GreaterThanExpression timeFilter =
         new GreaterThanExpression(
@@ -789,7 +793,7 @@ public class QueryLogicalPlanUtil {
             predicate,
             false,
             ZonedDateTime.now().getOffset(),
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     AggregationNode aggregationNode =
         new AggregationNode(
@@ -827,7 +831,7 @@ public class QueryLogicalPlanUtil {
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s1"))))),
             null,
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     GroupByLevelNode groupByLevelNode =
         new GroupByLevelNode(
@@ -856,7 +860,7 @@ public class QueryLogicalPlanUtil {
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s1"))),
                     new TimeSeriesOperand(schemaMap.get("root.sg.*.s1")))),
             null,
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     OffsetNode offsetNode = new OffsetNode(queryId.genPlanNodeId(), groupByLevelNode, 100);
     LimitNode limitNode = new LimitNode(queryId.genPlanNodeId(), offsetNode, 100);
@@ -877,19 +881,19 @@ public class QueryLogicalPlanUtil {
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s1"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList1.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d1.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList1.forEach(
         planNode -> {
           ((SeriesScanNode) planNode).setTimeFilter(TimeFilter.gt(100));
         });
 
     TimeJoinNode timeJoinNode1 =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList1);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList1);
 
     GreaterThanExpression timeFilter =
         new GreaterThanExpression(
@@ -911,7 +915,7 @@ public class QueryLogicalPlanUtil {
             predicate1,
             false,
             ZonedDateTime.now().getOffset(),
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     AggregationNode aggregationNode1 =
         new AggregationNode(
@@ -934,24 +938,24 @@ public class QueryLogicalPlanUtil {
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d1.s1"))))),
             null,
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     List<PlanNode> sourceNodeList2 = new ArrayList<>();
     sourceNodeList2.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s1"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList2.add(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             (MeasurementPath) schemaMap.get("root.sg.d2.s2"),
-            OrderBy.TIMESTAMP_DESC));
+            Ordering.DESC));
     sourceNodeList2.forEach(
         planNode -> ((SeriesScanNode) planNode).setTimeFilter(TimeFilter.gt(100)));
 
     TimeJoinNode timeJoinNode2 =
-        new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_DESC, sourceNodeList2);
+        new TimeJoinNode(queryId.genPlanNodeId(), Ordering.DESC, sourceNodeList2);
 
     GreaterThanExpression valueFilter2 =
         new GreaterThanExpression(
@@ -970,7 +974,7 @@ public class QueryLogicalPlanUtil {
             predicate2,
             false,
             ZonedDateTime.now().getOffset(),
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     AggregationNode aggregationNode2 =
         new AggregationNode(
@@ -993,7 +997,7 @@ public class QueryLogicalPlanUtil {
                     Collections.singletonList(
                         new TimeSeriesOperand(schemaMap.get("root.sg.d2.s1"))))),
             null,
-            OrderBy.TIMESTAMP_DESC);
+            Ordering.DESC);
 
     Map<String, List<Integer>> deviceToMeasurementIndexesMap = new HashMap<>();
     deviceToMeasurementIndexesMap.put("root.sg.d1", Arrays.asList(1, 2, 3));
@@ -1001,7 +1005,9 @@ public class QueryLogicalPlanUtil {
     DeviceViewNode deviceViewNode =
         new DeviceViewNode(
             queryId.genPlanNodeId(),
-            Arrays.asList(OrderBy.DEVICE_ASC, OrderBy.TIMESTAMP_DESC),
+            Arrays.asList(
+                new SortItem(SortKey.DEVICE, Ordering.ASC),
+                new SortItem(SortKey.TIME, Ordering.DESC)),
             Arrays.asList(
                 HeaderConstant.COLUMN_DEVICE, "count(s1)", "max_value(s2)", "last_value(s1)"),
             deviceToMeasurementIndexesMap);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/AggregationDistributionTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/AggregationDistributionTest.java
index c250dd7e13..82d20e210b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/AggregationDistributionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/AggregationDistributionTest.java
@@ -48,7 +48,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationStep;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByLevelDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
@@ -70,7 +70,7 @@ public class AggregationDistributionTest {
   @Test
   public void testTimeJoinAggregationSinglePerRegion() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query_time_join_aggregation");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
     String d1s1Path = "root.sg.d1.s1";
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d1s1Path, AggregationType.COUNT));
 
@@ -109,7 +109,7 @@ public class AggregationDistributionTest {
   @Test
   public void testTimeJoinAggregationWithSlidingWindow() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query_time_join_agg_with_sliding");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
     String d1s1Path = "root.sg.d1.s1";
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d1s1Path, AggregationType.COUNT));
 
@@ -157,7 +157,7 @@ public class AggregationDistributionTest {
   @Test
   public void testTimeJoinAggregationMultiPerRegion() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query_time_join_aggregation");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
     String d1s1Path = "root.sg.d1.s1";
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d1s1Path, AggregationType.COUNT));
 
@@ -181,7 +181,7 @@ public class AggregationDistributionTest {
   @Test
   public void testTimeJoinAggregationMultiPerRegion2() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query_time_join_aggregation");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
     String d3s1Path = "root.sg.d333.s1";
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d3s1Path, AggregationType.COUNT));
 
@@ -223,7 +223,7 @@ public class AggregationDistributionTest {
                         new TimeSeriesOperand(new PartialPath(d2s1Path))),
                     new TimeSeriesOperand(new PartialPath(groupedPath)))),
             null,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
     Analysis analysis = Util.constructAnalysis();
     MPPQueryContext context =
         new MPPQueryContext("", queryId, null, new TEndPoint(), new TEndPoint());
@@ -260,7 +260,7 @@ public class AggregationDistributionTest {
                         new TimeSeriesOperand(new PartialPath(d4s1Path))),
                     new TimeSeriesOperand(new PartialPath(groupedPath)))),
             null,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
     Analysis analysis = Util.constructAnalysis();
     MPPQueryContext context =
         new MPPQueryContext("", queryId, null, new TEndPoint(), new TEndPoint());
@@ -301,7 +301,7 @@ public class AggregationDistributionTest {
             AggregationType.COUNT,
             AggregationStep.PARTIAL,
             null);
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d3s1Path, AggregationType.COUNT));
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d4s1Path, AggregationType.COUNT));
     slidingWindowAggregationNode.addChild(timeJoinNode);
@@ -319,7 +319,7 @@ public class AggregationDistributionTest {
                         new TimeSeriesOperand(new PartialPath(d4s1Path))),
                     new TimeSeriesOperand(new PartialPath(groupedPath)))),
             null,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
 
     Analysis analysis = Util.constructAnalysis();
     MPPQueryContext context =
@@ -396,7 +396,7 @@ public class AggregationDistributionTest {
                     Collections.singletonList(new TimeSeriesOperand(new PartialPath(d1s2Path))),
                     new TimeSeriesOperand(new PartialPath(groupedPathS2)))),
             null,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
     Analysis analysis = Util.constructAnalysis();
     MPPQueryContext context =
         new MPPQueryContext("", queryId, null, new TEndPoint(), new TEndPoint());
@@ -455,7 +455,7 @@ public class AggregationDistributionTest {
                     Collections.singletonList(new TimeSeriesOperand(new PartialPath(d1s2Path))),
                     new TimeSeriesOperand(new PartialPath(groupedPathS2)))),
             null,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
     Analysis analysis = Util.constructAnalysis();
     MPPQueryContext context =
         new MPPQueryContext("", queryId, null, new TEndPoint(), new TEndPoint());
@@ -494,7 +494,7 @@ public class AggregationDistributionTest {
     String groupedPathS1 = "root.sg.*.s1";
     String groupedPathS2 = "root.sg.*.s2";
 
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d1s1Path, AggregationType.COUNT));
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d1s2Path, AggregationType.COUNT));
     timeJoinNode.addChild(genAggregationSourceNode(queryId, d2s1Path, AggregationType.COUNT));
@@ -527,7 +527,7 @@ public class AggregationDistributionTest {
                     Collections.singletonList(new TimeSeriesOperand(new PartialPath(d1s2Path))),
                     new TimeSeriesOperand(new PartialPath(groupedPathS2)))),
             null,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
     Analysis analysis = Util.constructAnalysis();
     MPPQueryContext context =
         new MPPQueryContext("", queryId, null, new TEndPoint(), new TEndPoint());
@@ -654,7 +654,7 @@ public class AggregationDistributionTest {
                         Collections.singletonList(new TimeSeriesOperand(path))))
             .collect(Collectors.toList()),
         groupByTimeParameter,
-        OrderBy.TIMESTAMP_ASC);
+        Ordering.ASC);
   }
 
   private SeriesAggregationSourceNode genAggregationSourceNode(
@@ -673,18 +673,18 @@ public class AggregationDistributionTest {
   @Test
   public void testParallelPlanWithAlignedSeries() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query_aligned");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
 
     timeJoinNode.addChild(
         new AlignedSeriesScanNode(
             queryId.genPlanNodeId(),
             new AlignedPath("root.sg.d1", Arrays.asList("s1", "s2")),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d333.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
 
     LimitNode root = new LimitNode(queryId.genPlanNodeId(), timeJoinNode, 10);
     Analysis analysis = Util.constructAnalysis();
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/DistributionPlannerBasicTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/DistributionPlannerBasicTest.java
index ea762fb2c4..8227fc49c1 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/DistributionPlannerBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/DistributionPlannerBasicTest.java
@@ -42,7 +42,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.AlignedSeriesScanNo
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.SeriesScanNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowsNode;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
@@ -61,7 +61,7 @@ public class DistributionPlannerBasicTest {
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
 
     Analysis analysis = Util.constructAnalysis();
 
@@ -80,7 +80,7 @@ public class DistributionPlannerBasicTest {
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
 
     Analysis analysis = Util.constructAnalysis();
 
@@ -96,23 +96,23 @@ public class DistributionPlannerBasicTest {
   public void testRewriteSourceNode() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query");
 
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
 
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s2", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d22.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
 
     LimitNode root = new LimitNode(queryId.genPlanNodeId(), timeJoinNode, 10);
 
@@ -173,23 +173,23 @@ public class DistributionPlannerBasicTest {
   @Test
   public void testAddExchangeNode() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
 
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s2", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d22.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
 
     LimitNode root = new LimitNode(queryId.genPlanNodeId(), timeJoinNode, 10);
 
@@ -210,23 +210,23 @@ public class DistributionPlannerBasicTest {
   @Test
   public void testSplitFragment() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
 
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s2", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d22.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
 
     LimitNode root = new LimitNode(queryId.genPlanNodeId(), timeJoinNode, 10);
 
@@ -245,23 +245,23 @@ public class DistributionPlannerBasicTest {
   @Test
   public void testParallelPlan() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
 
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d1.s2", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
             queryId.genPlanNodeId(),
             new MeasurementPath("root.sg.d333.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
 
     LimitNode root = new LimitNode(queryId.genPlanNodeId(), timeJoinNode, 10);
 
@@ -278,13 +278,13 @@ public class DistributionPlannerBasicTest {
   @Test
   public void testSingleAlignedSeries() throws IllegalPathException {
     QueryId queryId = new QueryId("test_query_aligned");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
 
     timeJoinNode.addChild(
         new AlignedSeriesScanNode(
             queryId.genPlanNodeId(),
             new AlignedPath("root.sg.d22", Arrays.asList("s1", "s2")),
-            OrderBy.TIMESTAMP_ASC));
+            Ordering.ASC));
 
     LimitNode root = new LimitNode(queryId.genPlanNodeId(), timeJoinNode, 10);
     Analysis analysis = Util.constructAnalysis();
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/NoDataRegionPlanningTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/NoDataRegionPlanningTest.java
index 1caaedf559..4e1e692ea5 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/NoDataRegionPlanningTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/NoDataRegionPlanningTest.java
@@ -31,7 +31,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.LogicalQueryPlan;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.LimitNode;
 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.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 import org.junit.Test;
@@ -47,28 +47,20 @@ public class NoDataRegionPlanningTest {
     String d5s1 = "root.sg.d55555.s1";
 
     QueryId queryId = new QueryId("test_query");
-    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(queryId.genPlanNodeId(), Ordering.ASC);
 
     timeJoinNode.addChild(
         new SeriesScanNode(
-            queryId.genPlanNodeId(),
-            new MeasurementPath(d1s1, TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            queryId.genPlanNodeId(), new MeasurementPath(d1s1, TSDataType.INT32), Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
-            queryId.genPlanNodeId(),
-            new MeasurementPath(d1s2, TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            queryId.genPlanNodeId(), new MeasurementPath(d1s2, TSDataType.INT32), Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
-            queryId.genPlanNodeId(),
-            new MeasurementPath(d3s1, TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            queryId.genPlanNodeId(), new MeasurementPath(d3s1, TSDataType.INT32), Ordering.ASC));
     timeJoinNode.addChild(
         new SeriesScanNode(
-            queryId.genPlanNodeId(),
-            new MeasurementPath(d5s1, TSDataType.INT32),
-            OrderBy.TIMESTAMP_ASC));
+            queryId.genPlanNodeId(), new MeasurementPath(d5s1, TSDataType.INT32), Ordering.ASC));
 
     LimitNode root = new LimitNode(queryId.genPlanNodeId(), timeJoinNode, 10);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/AggregationNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/AggregationNodeSerdeTest.java
index 5208c0913f..effb1dc5bd 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/AggregationNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/AggregationNodeSerdeTest.java
@@ -29,7 +29,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.SeriesAggregationSc
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationStep;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.operator.In;
@@ -61,7 +61,7 @@ public class AggregationNodeSerdeTest {
                     AggregationStep.INTERMEDIATE,
                     Collections.singletonList(
                         new TimeSeriesOperand(new PartialPath("root.sg.d1.s1"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             new In<>(Sets.newHashSet("s1", "s2"), VALUE_FILTER, true),
             groupByTimeParameter,
             null);
@@ -76,7 +76,7 @@ public class AggregationNodeSerdeTest {
                     Collections.singletonList(
                         new TimeSeriesOperand(new PartialPath("root.sg.d1.s1"))))),
             groupByTimeParameter,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
     aggregationNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/DeviceViewNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/DeviceViewNodeSerdeTest.java
index b8ed4919fa..c405fbc17b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/DeviceViewNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/DeviceViewNodeSerdeTest.java
@@ -23,7 +23,9 @@ 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.DeviceViewNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.TimeJoinNode;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortKey;
 
 import org.junit.Test;
 
@@ -36,14 +38,14 @@ import static org.junit.Assert.assertEquals;
 public class DeviceViewNodeSerdeTest {
   @Test
   public void testSerializeAndDeserialize() throws IllegalPathException {
-    TimeJoinNode timeJoinNode1 =
-        new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), OrderBy.TIMESTAMP_ASC);
-    TimeJoinNode timeJoinNode2 =
-        new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode1 = new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), Ordering.ASC);
+    TimeJoinNode timeJoinNode2 = new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), Ordering.ASC);
     DeviceViewNode deviceViewNode =
         new DeviceViewNode(
             new PlanNodeId("TestDeviceMergeNode"),
-            Arrays.asList(OrderBy.DEVICE_ASC, OrderBy.TIMESTAMP_ASC),
+            Arrays.asList(
+                new SortItem(SortKey.DEVICE, Ordering.ASC),
+                new SortItem(SortKey.TIME, Ordering.DESC)),
             Arrays.asList("s1", "s2"),
             new HashMap<>());
     deviceViewNode.addChildDeviceNode("root.sg.d1", timeJoinNode1);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/ExchangeNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/ExchangeNodeSerdeTest.java
index 4211aebb32..5b6a411386 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/ExchangeNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/ExchangeNodeSerdeTest.java
@@ -27,7 +27,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.ExchangeNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.TimeJoinNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.sink.FragmentSinkNode;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 
 import org.junit.Test;
 
@@ -39,8 +39,7 @@ public class ExchangeNodeSerdeTest {
 
   @Test
   public void testSerializeAndDeserialize() throws IllegalPathException {
-    TimeJoinNode timeJoinNode =
-        new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), Ordering.ASC);
 
     ExchangeNode exchangeNode = new ExchangeNode(new PlanNodeId("TestExchangeNode"));
     FragmentSinkNode fragmentSinkNode =
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 4c984e0dd1..ed84dfe5ec 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
@@ -25,7 +25,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.FillNode;
 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.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.mpp.plan.statement.literal.LongLiteral;
 
 import org.junit.Test;
@@ -38,14 +38,13 @@ public class FillNodeSerdeTest {
 
   @Test
   public void testSerializeAndDeserialize() throws IllegalPathException {
-    TimeJoinNode timeJoinNode =
-        new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), Ordering.ASC);
     FillNode fillNode =
         new FillNode(
             new PlanNodeId("TestFillNode"),
             timeJoinNode,
             new FillDescriptor(FillPolicy.VALUE, new LongLiteral("100")),
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
     fillNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FilterNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FilterNodeSerdeTest.java
index bac6d75cd7..fa3d7ec140 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FilterNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/FilterNodeSerdeTest.java
@@ -28,7 +28,7 @@ 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.FilterNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.TimeJoinNode;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 import org.junit.Test;
@@ -42,8 +42,7 @@ public class FilterNodeSerdeTest {
 
   @Test
   public void testSerializeAndDeserialize() throws IllegalPathException {
-    TimeJoinNode timeJoinNode =
-        new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), Ordering.ASC);
     FilterNode filterNode =
         new FilterNode(
             new PlanNodeId("TestFilterNode"),
@@ -54,7 +53,7 @@ public class FilterNodeSerdeTest {
                 new ConstantOperand(TSDataType.INT64, "100")),
             false,
             ZoneId.systemDefault(),
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
     filterNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/GroupByLevelNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/GroupByLevelNodeSerdeTest.java
index 943402ac45..7d7dadb7c0 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/GroupByLevelNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/GroupByLevelNodeSerdeTest.java
@@ -30,7 +30,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationStep;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByLevelDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
@@ -61,7 +61,7 @@ public class GroupByLevelNodeSerdeTest {
                     AggregationStep.FINAL,
                     Collections.singletonList(
                         new TimeSeriesOperand(new PartialPath("root.sg.d1.s1"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             null,
             groupByTimeParameter,
             null);
@@ -75,7 +75,7 @@ public class GroupByLevelNodeSerdeTest {
                     AggregationStep.FINAL,
                     Collections.singletonList(
                         new TimeSeriesOperand(new PartialPath("root.sg.d2.s1"))))),
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             null,
             groupByTimeParameter,
             null);
@@ -93,7 +93,7 @@ public class GroupByLevelNodeSerdeTest {
                         new TimeSeriesOperand(new PartialPath("root.sg.d2.s1"))),
                     new TimeSeriesOperand(new PartialPath("root.sg.*.s1")))),
             groupByTimeParameter,
-            OrderBy.TIMESTAMP_ASC);
+            Ordering.ASC);
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(2048);
     groupByLevelNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/LimitNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/LimitNodeSerdeTest.java
index 9834f97fad..8b4299c036 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/LimitNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/LimitNodeSerdeTest.java
@@ -25,7 +25,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.LimitNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.OffsetNode;
 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.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
 
@@ -43,7 +43,7 @@ public class LimitNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             new GroupByFilter(1, 2, 3, 4),
             null,
             100,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/OffsetNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/OffsetNodeSerdeTest.java
index b22acc0a04..66ef69f31d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/OffsetNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/OffsetNodeSerdeTest.java
@@ -24,7 +24,7 @@ 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.OffsetNode;
 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.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
 
@@ -42,7 +42,7 @@ public class OffsetNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             new GroupByFilter(1, 2, 3, 4),
             null,
             100,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/SortNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/SortNodeSerdeTest.java
index 13262c20c8..9bc72ea4e1 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/SortNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/SortNodeSerdeTest.java
@@ -24,7 +24,7 @@ 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.SortNode;
 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.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
 
@@ -42,14 +42,13 @@ public class SortNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             new GroupByFilter(1, 2, 3, 4),
             null,
             100,
             100,
             null);
-    SortNode sortNode =
-        new SortNode(new PlanNodeId("TestSortNode"), seriesScanNode, OrderBy.TIMESTAMP_ASC);
+    SortNode sortNode = new SortNode(new PlanNodeId("TestSortNode"), seriesScanNode, Ordering.ASC);
 
     ByteBuffer byteBuffer = ByteBuffer.allocate(2048);
     sortNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/TimeJoinNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/TimeJoinNodeSerdeTest.java
index 9f809e2705..1ff7342c16 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/TimeJoinNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/process/TimeJoinNodeSerdeTest.java
@@ -24,7 +24,7 @@ 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.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.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.ValueFilter;
@@ -42,7 +42,7 @@ public class TimeJoinNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             TimeFilter.gt(100),
             null,
             100,
@@ -52,15 +52,14 @@ public class TimeJoinNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new MeasurementPath("root.sg.d1.s2", TSDataType.INT32),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             null,
             ValueFilter.gt(100),
             100,
             100,
             null);
 
-    TimeJoinNode timeJoinNode =
-        new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), OrderBy.TIMESTAMP_ASC);
+    TimeJoinNode timeJoinNode = new TimeJoinNode(new PlanNodeId("TestTimeJoinNode"), Ordering.ASC);
     timeJoinNode.addChild(seriesScanNode1);
     timeJoinNode.addChild(seriesScanNode2);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java
index 2eb810809b..4601f85aeb 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java
@@ -29,7 +29,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.SeriesAggregationSc
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.AggregationStep;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.operator.In;
@@ -61,7 +61,7 @@ public class SeriesAggregationScanNodeSerdeTest {
             new PlanNodeId("TestSeriesAggregateScanNode"),
             new MeasurementPath("root.sg.d1.s1", TSDataType.BOOLEAN),
             aggregationDescriptorList,
-            OrderBy.TIMESTAMP_ASC,
+            Ordering.ASC,
             new In<>(Sets.newHashSet("s1", "s2"), VALUE_FILTER, true),
             groupByTimeParameter,
             null);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesScanNodeSerdeTest.java
index cb65b1b7ad..bf179e7f16 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/source/SeriesScanNodeSerdeTest.java
@@ -24,7 +24,7 @@ import org.apache.iotdb.db.metadata.path.MeasurementPath;
 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.source.SeriesScanNode;
-import org.apache.iotdb.db.mpp.plan.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
 
@@ -42,7 +42,7 @@ public class SeriesScanNodeSerdeTest {
         new SeriesScanNode(
             new PlanNodeId("TestSeriesScanNode"),
             new MeasurementPath("root.sg.d1.s1", TSDataType.INT32),
-            OrderBy.TIMESTAMP_DESC,
+            Ordering.DESC,
             new GroupByFilter(1, 2, 3, 4),
             null,
             100,


[iotdb] 02/04: add semantic check

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

hui pushed a commit to branch lmh/orderBySensor
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 9024ee03380cd15704a033e8990d21bac05275ab
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Tue Jul 19 18:18:16 2022 +0800

    add semantic check
---
 .../plan/statement/component/OrderByComponent.java | 24 ++++++++++++++++
 .../db/mpp/plan/statement/crud/QueryStatement.java | 33 ++++++++++++++++++++++
 2 files changed, 57 insertions(+)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
index 1aa09bbcae..8fe478d918 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
@@ -27,15 +27,39 @@ public class OrderByComponent {
 
   private final List<SortItem> sortItemList;
 
+  private boolean orderByTime = false;
+  private boolean orderByTimeseries = false;
+  private boolean orderByDevice = false;
+
   public OrderByComponent() {
     this.sortItemList = new ArrayList<>();
   }
 
   public void addSortItem(SortItem sortItem) {
     this.sortItemList.add(sortItem);
+
+    if (sortItem.getSortKey() == SortItem.SortKey.TIME) {
+      orderByTime = true;
+    } else if (sortItem.getSortKey() == SortItem.SortKey.TIMESERIES) {
+      orderByTimeseries = true;
+    } else {
+      orderByDevice = true;
+    }
   }
 
   public List<SortItem> getSortItemList() {
     return sortItemList;
   }
+
+  public boolean isOrderByTime() {
+    return orderByTime;
+  }
+
+  public boolean isOrderByTimeseries() {
+    return orderByTimeseries;
+  }
+
+  public boolean isOrderByDevice() {
+    return orderByDevice;
+  }
 }
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 2da55d6f42..ea99bd3fae 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
@@ -215,6 +215,18 @@ public class QueryStatement extends Statement {
     return resultSetFormat == ResultSetFormat.DISABLE_ALIGN;
   }
 
+  public boolean isOrderByTime() {
+    return orderByComponent != null && orderByComponent.isOrderByTime();
+  }
+
+  public boolean isOrderByTimeseries() {
+    return orderByComponent != null && orderByComponent.isOrderByTimeseries();
+  }
+
+  public boolean isOrderByDevice() {
+    return orderByComponent != null && orderByComponent.isOrderByDevice();
+  }
+
   public void semanticCheck() {
     if (isAggregationQuery()) {
       if (disableAlign()) {
@@ -243,6 +255,13 @@ public class QueryStatement extends Statement {
       if (getWhereCondition() != null) {
         ExpressionAnalyzer.checkIsAllMeasurement(getWhereCondition().getPredicate());
       }
+      if (isOrderByTimeseries()) {
+        throw new SemanticException("Sorting by timeseries is only supported in last queries.");
+      }
+      if (isOrderByDevice()) {
+        // TODO support sort by device
+        throw new SemanticException("Sorting by device is not yet supported.");
+      }
     }
 
     if (isLastQuery()) {
@@ -258,6 +277,20 @@ public class QueryStatement extends Statement {
           throw new SemanticException("Last queries can only be applied on raw time series.");
         }
       }
+      if (isOrderByDevice()) {
+        throw new SemanticException(
+            "Sorting by device is only supported in ALIGN BY DEVICE queries.");
+      }
+    }
+
+    if (!isAlignByDevice() && !isLastQuery()) {
+      if (isOrderByTimeseries()) {
+        throw new SemanticException("Sorting by timeseries is only supported in last queries.");
+      }
+      if (isOrderByDevice()) {
+        throw new SemanticException(
+            "Sorting by device is only supported in ALIGN BY DEVICE queries.");
+      }
     }
   }
 


[iotdb] 01/04: refactor Order by in SQL parser & QueryStatement

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

hui pushed a commit to branch lmh/orderBySensor
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit c53a1d49ed29dfe73ff725152f2dc3e75fc2857d
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Tue Jul 19 17:51:01 2022 +0800

    refactor Order by in SQL parser & QueryStatement
---
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   | 24 +++++--
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       | 47 ++++++++----
 .../{OrderBy.java => OrderByComponent.java}        | 32 +++++----
 .../db/mpp/plan/statement/component/SortItem.java  | 83 ++++++++++++++++++++++
 .../db/mpp/plan/statement/crud/QueryStatement.java | 12 ++--
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    | 36 +++++++---
 6 files changed, 182 insertions(+), 52 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 fe0add2da7..03868d644a 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
@@ -379,11 +379,11 @@ intoPath
 
 specialClause
     : specialLimit #specialLimitStatement
-    | orderByTimeClause specialLimit? #orderByTimeStatement
-    | groupByTimeClause orderByTimeClause? specialLimit? #groupByTimeStatement
-    | groupByFillClause orderByTimeClause? specialLimit? #groupByFillStatement
-    | groupByLevelClause orderByTimeClause? specialLimit? #groupByLevelStatement
-    | fillClause orderByTimeClause? specialLimit? #fillStatement
+    | orderByClause specialLimit? #orderByTimeStatement
+    | groupByTimeClause orderByClause? specialLimit? #groupByTimeStatement
+    | groupByFillClause orderByClause? specialLimit? #groupByFillStatement
+    | groupByLevelClause orderByClause? specialLimit? #groupByLevelStatement
+    | fillClause orderByClause? specialLimit? #fillStatement
     ;
 
 specialLimit
@@ -407,8 +407,18 @@ disableAlign
     : DISABLE ALIGN
     ;
 
-orderByTimeClause
-    : ORDER BY TIME (DESC | ASC)?
+orderByClause
+    : ORDER BY orderByAttributeClause (COMMA orderByAttributeClause)*
+    ;
+
+orderByAttributeClause
+    : sortKey (DESC | ASC)
+    ;
+
+sortKey
+    : TIME
+    | TIMESERIES
+    | DEVICE
     ;
 
 groupByTimeClause
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 7c8f642936..8d5f002540 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
@@ -64,10 +64,11 @@ import org.apache.iotdb.db.mpp.plan.statement.component.FillPolicy;
 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.OrderByComponent;
 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.SortItem;
 import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition;
 import org.apache.iotdb.db.mpp.plan.statement.crud.DeleteDataStatement;
 import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement;
@@ -142,9 +143,11 @@ import java.net.URISyntaxException;
 import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.regex.Pattern;
 
 /** Parse AST to Statement. */
@@ -790,8 +793,8 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     parseGroupByTimeClause(ctx.groupByTimeClause());
 
     // parse order by time
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
 
     // parse limit & offset
@@ -808,8 +811,8 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     parseGroupByTimeClause(ctx.groupByFillClause());
 
     // parse order by time
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
 
     // parse limit & offset
@@ -938,8 +941,8 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     parseGroupByLevelClause(ctx.groupByLevelClause());
 
     // parse order by time
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
 
     // parse limit & offset
@@ -972,8 +975,8 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     parseFillClause(ctx.fillClause());
 
     // parse order by time
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
 
     // parse limit & offset
@@ -1182,7 +1185,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   @Override
   public Statement visitOrderByTimeStatement(IoTDBSqlParser.OrderByTimeStatementContext ctx) {
     // parse ORDER BY TIME
-    parseOrderByTimeClause(ctx.orderByTimeClause());
+    parseOrderByClause(ctx.orderByClause());
 
     // parse others
     if (ctx.specialLimit() != null) {
@@ -1192,10 +1195,28 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   }
 
   // parse ORDER BY TIME
-  private void parseOrderByTimeClause(IoTDBSqlParser.OrderByTimeClauseContext ctx) {
-    if (ctx.DESC() != null) {
-      queryStatement.setResultOrder(OrderBy.TIMESTAMP_DESC);
+  private void parseOrderByClause(IoTDBSqlParser.OrderByClauseContext ctx) {
+    OrderByComponent orderByComponent = new OrderByComponent();
+    Set<SortItem.SortKey> sortKeySet = new HashSet<>();
+    for (IoTDBSqlParser.OrderByAttributeClauseContext orderByAttributeClauseContext :
+        ctx.orderByAttributeClause()) {
+      SortItem sortItem = parseOrderByAttributeClause(orderByAttributeClauseContext);
+
+      SortItem.SortKey sortKey = sortItem.getSortKey();
+      if (sortKeySet.contains(sortKey)) {
+        throw new SemanticException(String.format("ORDER BY: duplicate sort key '%s'", sortKey));
+      } else {
+        sortKeySet.add(sortKey);
+        orderByComponent.addSortItem(sortItem);
+      }
     }
+    queryStatement.setOrderByComponent(orderByComponent);
+  }
+
+  private SortItem parseOrderByAttributeClause(IoTDBSqlParser.OrderByAttributeClauseContext ctx) {
+    return new SortItem(
+        SortItem.SortKey.valueOf(ctx.sortKey().getText().toUpperCase()),
+        ctx.DESC() != null ? SortItem.Ordering.DESC : SortItem.Ordering.ASC);
   }
 
   // ResultSetFormat Clause
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderBy.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
similarity index 68%
rename from server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderBy.java
rename to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
index e72029b7d4..1aa09bbcae 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderBy.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/OrderByComponent.java
@@ -19,21 +19,23 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
-/** The order of query result set by timestamp */
-public enum OrderBy {
-  TIMESTAMP_ASC,
-  TIMESTAMP_DESC,
-  DEVICE_ASC,
-  DEVICE_DESC;
+import java.util.ArrayList;
+import java.util.List;
 
-  public OrderBy reverse() {
-    switch (this) {
-      case TIMESTAMP_ASC:
-        return TIMESTAMP_DESC;
-      case TIMESTAMP_DESC:
-        return TIMESTAMP_ASC;
-      default:
-        throw new UnsupportedOperationException();
-    }
+/** The order of query result set */
+public class OrderByComponent {
+
+  private final List<SortItem> sortItemList;
+
+  public OrderByComponent() {
+    this.sortItemList = new ArrayList<>();
+  }
+
+  public void addSortItem(SortItem sortItem) {
+    this.sortItemList.add(sortItem);
+  }
+
+  public List<SortItem> getSortItemList() {
+    return sortItemList;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortItem.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortItem.java
new file mode 100644
index 0000000000..bbdff9f01b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SortItem.java
@@ -0,0 +1,83 @@
+/*
+ * 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.db.mpp.plan.statement.StatementNode;
+
+import java.util.Objects;
+
+public class SortItem extends StatementNode {
+
+  public enum SortKey {
+    TIME,
+    TIMESERIES,
+    DEVICE
+  }
+
+  public enum Ordering {
+    ASC,
+    DESC;
+
+    public Ordering reverse() {
+      if (this == ASC) {
+        return DESC;
+      } else {
+        return ASC;
+      }
+    }
+  }
+
+  private final SortKey sortKey;
+  private final Ordering ordering;
+
+  public SortItem(SortKey sortKey, Ordering ordering) {
+    this.sortKey = sortKey;
+    this.ordering = ordering;
+  }
+
+  public SortKey getSortKey() {
+    return sortKey;
+  }
+
+  public Ordering getOrdering() {
+    return ordering;
+  }
+
+  public SortItem reverse() {
+    return new SortItem(getSortKey(), getOrdering().reverse());
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    SortItem sortItem = (SortItem) o;
+    return sortKey == sortItem.sortKey && ordering == sortItem.ordering;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(sortKey, ordering);
+  }
+}
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 17f433a2e7..2da55d6f42 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
@@ -31,7 +31,7 @@ import org.apache.iotdb.db.mpp.plan.statement.component.FillComponent;
 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.OrderByComponent;
 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;
@@ -76,7 +76,7 @@ public class QueryStatement extends Statement {
 
   protected FillComponent fillComponent;
 
-  protected OrderBy resultOrder = OrderBy.TIMESTAMP_ASC;
+  protected OrderByComponent orderByComponent;
 
   protected ResultSetFormat resultSetFormat = ResultSetFormat.ALIGN_BY_TIME;
 
@@ -159,12 +159,12 @@ public class QueryStatement extends Statement {
     this.fillComponent = fillComponent;
   }
 
-  public OrderBy getResultOrder() {
-    return resultOrder;
+  public OrderByComponent getOrderByComponent() {
+    return orderByComponent;
   }
 
-  public void setResultOrder(OrderBy resultOrder) {
-    this.resultOrder = resultOrder;
+  public void setOrderByComponent(OrderByComponent orderByComponent) {
+    this.orderByComponent = orderByComponent;
   }
 
   public ResultSetFormat getResultSetFormat() {
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 79bb5d9db0..d4c58d998c 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
@@ -53,6 +53,7 @@ import org.apache.iotdb.db.mpp.plan.expression.unary.LikeExpression;
 import org.apache.iotdb.db.mpp.plan.expression.unary.LogicNotExpression;
 import org.apache.iotdb.db.mpp.plan.expression.unary.NegationExpression;
 import org.apache.iotdb.db.mpp.plan.expression.unary.RegularExpression;
+import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
 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;
@@ -1341,7 +1342,7 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   @Override
   public Operator visitOrderByTimeStatement(IoTDBSqlParser.OrderByTimeStatementContext ctx) {
     queryOp = new QueryOperator();
-    parseOrderByTimeClause(ctx.orderByTimeClause());
+    parseOrderByClause(ctx.orderByClause());
     if (ctx.specialLimit() != null) {
       return visit(ctx.specialLimit());
     }
@@ -1352,8 +1353,8 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   public Operator visitGroupByTimeStatement(IoTDBSqlParser.GroupByTimeStatementContext ctx) {
     queryOp = new GroupByQueryOperator();
     parseGroupByTimeClause(ctx.groupByTimeClause());
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
     if (ctx.specialLimit() != null) {
       return visit(ctx.specialLimit());
@@ -1365,8 +1366,8 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   public Operator visitGroupByFillStatement(IoTDBSqlParser.GroupByFillStatementContext ctx) {
     queryOp = new GroupByFillQueryOperator();
     parseGroupByFillClause(ctx.groupByFillClause());
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
     if (ctx.specialLimit() != null) {
       return visit(ctx.specialLimit());
@@ -1378,8 +1379,8 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   public Operator visitGroupByLevelStatement(IoTDBSqlParser.GroupByLevelStatementContext ctx) {
     queryOp = new AggregationQueryOperator();
     parseGroupByLevelClause(ctx.groupByLevelClause());
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
     if (ctx.specialLimit() != null) {
       return visit(ctx.specialLimit());
@@ -1391,8 +1392,8 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   public Operator visitFillStatement(IoTDBSqlParser.FillStatementContext ctx) {
     queryOp = new FillQueryOperator();
     parseFillClause(ctx.fillClause());
-    if (ctx.orderByTimeClause() != null) {
-      parseOrderByTimeClause(ctx.orderByTimeClause());
+    if (ctx.orderByClause() != null) {
+      parseOrderByClause(ctx.orderByClause());
     }
     if (ctx.specialLimit() != null) {
       return visit(ctx.specialLimit());
@@ -1480,8 +1481,15 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     specialClauseComponent.setAlignByTime(false);
   }
 
-  private void parseOrderByTimeClause(IoTDBSqlParser.OrderByTimeClauseContext ctx) {
-    if (ctx.DESC() != null) {
+  private void parseOrderByClause(IoTDBSqlParser.OrderByClauseContext ctx) {
+    if (ctx.orderByAttributeClause().size() > 0) {
+      throw new SQLParserException("Sorting by multiple fields is not supported.");
+    }
+    SortItem sortItem = parseOrderByAttributeClause(ctx.orderByAttributeClause(0));
+    if (sortItem.getSortKey() != SortItem.SortKey.TIME) {
+      throw new SQLParserException("It only supports sorting by time.");
+    }
+    if (sortItem.getOrdering() == SortItem.Ordering.DESC) {
       SpecialClauseComponent specialClauseComponent = queryOp.getSpecialClauseComponent();
       if (specialClauseComponent == null) {
         specialClauseComponent = new SpecialClauseComponent();
@@ -1491,6 +1499,12 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
     }
   }
 
+  private SortItem parseOrderByAttributeClause(IoTDBSqlParser.OrderByAttributeClauseContext ctx) {
+    return new SortItem(
+        SortItem.SortKey.valueOf(ctx.sortKey().getText().toUpperCase()),
+        ctx.DESC() != null ? SortItem.Ordering.DESC : SortItem.Ordering.ASC);
+  }
+
   private void parseGroupByTimeClause(IoTDBSqlParser.GroupByTimeClauseContext ctx) {
     GroupByClauseComponent groupByClauseComponent = new GroupByClauseComponent();
     groupByClauseComponent.setLeftCRightO(ctx.timeRange().LS_BRACKET() != null);


[iotdb] 04/04: add mergeOrders in LastQueryMergeNode

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

hui pushed a commit to branch lmh/orderBySensor
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 574153d8187b68fc30e82e46ddaf0aadd3fb5a8d
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Wed Jul 20 11:18:06 2022 +0800

    add mergeOrders in LastQueryMergeNode
---
 .../db/mpp/plan/planner/LogicalPlanBuilder.java    |  5 +-
 .../db/mpp/plan/planner/LogicalPlanVisitor.java    | 15 ++++--
 .../plan/planner/distribution/SourceRewriter.java  |  8 +++-
 .../planner/plan/node/process/DeviceMergeNode.java |  6 +--
 .../planner/plan/node/process/DeviceViewNode.java  |  2 +-
 .../plan/node/process/LastQueryMergeNode.java      | 53 ++++++++++++++++------
 .../db/mpp/plan/statement/crud/QueryStatement.java |  9 ++++
 .../db/mpp/plan/plan/QueryLogicalPlanUtil.java     |  8 +++-
 .../mpp/plan/plan/distribution/LastQueryTest.java  |  2 +-
 9 files changed, 78 insertions(+), 30 deletions(-)

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 cb86295df9..5557e06bb7 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
@@ -141,7 +141,8 @@ public class LogicalPlanBuilder {
     return this;
   }
 
-  public LogicalPlanBuilder planLast(Set<Expression> sourceExpressions, Filter globalTimeFilter) {
+  public LogicalPlanBuilder planLast(
+      Set<Expression> sourceExpressions, Filter globalTimeFilter, List<SortItem> mergeOrders) {
     List<PlanNode> sourceNodeList = new ArrayList<>();
     for (Expression sourceExpression : sourceExpressions) {
       MeasurementPath selectPath =
@@ -157,7 +158,7 @@ public class LogicalPlanBuilder {
 
     this.root =
         new LastQueryMergeNode(
-            context.getQueryId().genPlanNodeId(), sourceNodeList, globalTimeFilter);
+            context.getQueryId().genPlanNodeId(), sourceNodeList, globalTimeFilter, mergeOrders);
     return this;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
index f35f51bcb4..b49801d547 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
@@ -62,6 +62,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowDevicesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTimeSeriesStatement;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -92,7 +93,10 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
 
     if (queryStatement.isLastQuery()) {
       return planBuilder
-          .planLast(analysis.getSourceExpressions(), analysis.getGlobalTimeFilter())
+          .planLast(
+              analysis.getSourceExpressions(),
+              analysis.getGlobalTimeFilter(),
+              queryStatement.getSortItemList())
           .getRoot();
     }
 
@@ -304,7 +308,9 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   public PlanNode visitLastPointFetch(
       LastPointFetchStatement lastPointFetchStatement, MPPQueryContext context) {
     LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
-    return planBuilder.planLast(analysis.getSourceExpressions(), null).getRoot();
+    return planBuilder
+        .planLast(analysis.getSourceExpressions(), null, Collections.emptyList())
+        .getRoot();
   }
 
   @Override
@@ -456,7 +462,10 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
         && 0 != analysis.getDataPartitionInfo().getDataPartitionMap().size()) {
       PlanNode lastPlanNode =
           new LogicalPlanBuilder(context)
-              .planLast(analysis.getSourceExpressions(), analysis.getGlobalTimeFilter())
+              .planLast(
+                  analysis.getSourceExpressions(),
+                  analysis.getGlobalTimeFilter(),
+                  Collections.emptyList())
               .getRoot();
       planBuilder = planBuilder.planSchemaQueryOrderByHeat(lastPlanNode);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java
index 747b9933d1..3934659bb4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java
@@ -231,7 +231,9 @@ public class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanConte
   public PlanNode visitLastQueryScan(LastQueryScanNode node, DistributionPlanContext context) {
     LastQueryMergeNode mergeNode =
         new LastQueryMergeNode(
-            context.queryContext.getQueryId().genPlanNodeId(), node.getPartitionTimeFilter());
+            context.queryContext.getQueryId().genPlanNodeId(),
+            node.getPartitionTimeFilter(),
+            Collections.emptyList());
     return processRawSeriesScan(node, context, mergeNode);
   }
 
@@ -240,7 +242,9 @@ public class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanConte
       AlignedLastQueryScanNode node, DistributionPlanContext context) {
     LastQueryMergeNode mergeNode =
         new LastQueryMergeNode(
-            context.queryContext.getQueryId().genPlanNodeId(), node.getPartitionTimeFilter());
+            context.queryContext.getQueryId().genPlanNodeId(),
+            node.getPartitionTimeFilter(),
+            Collections.emptyList());
     return processRawSeriesScan(node, context, mergeNode);
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java
index 37a71fa58c..53318dfe21 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceMergeNode.java
@@ -37,7 +37,7 @@ import java.util.stream.Collectors;
 public class DeviceMergeNode extends MultiChildNode {
 
   // The result output order, which could sort by device and time.
-  // The size of this list is 2 and the first OrderBy in this list has higher priority.
+  // The size of this list is 2 and the first SortItem in this list has higher priority.
   private final List<SortItem> mergeOrders;
 
   // the list of selected devices
@@ -45,15 +45,13 @@ public class DeviceMergeNode extends MultiChildNode {
 
   public DeviceMergeNode(
       PlanNodeId id, List<PlanNode> children, List<SortItem> mergeOrders, List<String> devices) {
-    super(id);
-    this.children = children;
+    super(id, children);
     this.mergeOrders = mergeOrders;
     this.devices = devices;
   }
 
   public DeviceMergeNode(PlanNodeId id, List<SortItem> mergeOrders, List<String> devices) {
     super(id);
-    this.children = new ArrayList<>();
     this.mergeOrders = mergeOrders;
     this.devices = devices;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java
index fb074122ba..fe7716cfa8 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/DeviceViewNode.java
@@ -45,7 +45,7 @@ import java.util.Objects;
 public class DeviceViewNode extends MultiChildNode {
 
   // The result output order, which could sort by device and time.
-  // The size of this list is 2 and the first OrderBy in this list has higher priority.
+  // The size of this list is 2 and the first SortItem in this list has higher priority.
   private final List<SortItem> mergeOrders;
 
   // The size devices and children should be the same.
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/LastQueryMergeNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/LastQueryMergeNode.java
index 51a20ea238..6445fe88c4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/LastQueryMergeNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/process/LastQueryMergeNode.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 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.statement.component.SortItem;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -39,21 +40,23 @@ import static org.apache.iotdb.db.mpp.plan.planner.plan.node.source.LastQuerySca
 
 public class LastQueryMergeNode extends MultiChildNode {
 
-  // make sure child in list has been ordered by their sensor name
-  private List<PlanNode> children;
-
   private final Filter timeFilter;
 
-  public LastQueryMergeNode(PlanNodeId id, Filter timeFilter) {
+  // The result output order, which could sort by sensor and time.
+  // The size of this list is 2 and the first SortItem in this list has higher priority.
+  private final List<SortItem> mergeOrders;
+
+  public LastQueryMergeNode(PlanNodeId id, Filter timeFilter, List<SortItem> mergeOrders) {
     super(id);
-    this.children = new ArrayList<>();
     this.timeFilter = timeFilter;
+    this.mergeOrders = mergeOrders;
   }
 
-  public LastQueryMergeNode(PlanNodeId id, List<PlanNode> children, Filter timeFilter) {
-    super(id);
-    this.children = children;
+  public LastQueryMergeNode(
+      PlanNodeId id, List<PlanNode> children, Filter timeFilter, List<SortItem> mergeOrders) {
+    super(id, children);
     this.timeFilter = timeFilter;
+    this.mergeOrders = mergeOrders;
   }
 
   @Override
@@ -68,7 +71,7 @@ public class LastQueryMergeNode extends MultiChildNode {
 
   @Override
   public PlanNode clone() {
-    return new LastQueryMergeNode(getPlanNodeId(), timeFilter);
+    return new LastQueryMergeNode(getPlanNodeId(), timeFilter, mergeOrders);
   }
 
   @Override
@@ -89,16 +92,22 @@ public class LastQueryMergeNode extends MultiChildNode {
 
   @Override
   public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-    if (!super.equals(o)) return false;
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
     LastQueryMergeNode that = (LastQueryMergeNode) o;
-    return Objects.equals(children, that.children);
+    return Objects.equals(timeFilter, that.timeFilter) && mergeOrders.equals(that.mergeOrders);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(super.hashCode(), children);
+    return Objects.hash(super.hashCode(), timeFilter, mergeOrders);
   }
 
   @Override
@@ -115,6 +124,10 @@ public class LastQueryMergeNode extends MultiChildNode {
       ReadWriteIOUtils.write((byte) 1, byteBuffer);
       timeFilter.serialize(byteBuffer);
     }
+    ReadWriteIOUtils.write(mergeOrders.size(), byteBuffer);
+    for (SortItem mergeOrder : mergeOrders) {
+      mergeOrder.serialize(byteBuffer);
+    }
   }
 
   @Override
@@ -126,6 +139,10 @@ public class LastQueryMergeNode extends MultiChildNode {
       ReadWriteIOUtils.write((byte) 1, stream);
       timeFilter.serialize(stream);
     }
+    ReadWriteIOUtils.write(mergeOrders.size(), stream);
+    for (SortItem mergeOrder : mergeOrders) {
+      mergeOrder.serialize(stream);
+    }
   }
 
   public static LastQueryMergeNode deserialize(ByteBuffer byteBuffer) {
@@ -133,8 +150,14 @@ public class LastQueryMergeNode extends MultiChildNode {
     if (ReadWriteIOUtils.readByte(byteBuffer) == 1) {
       timeFilter = FilterFactory.deserialize(byteBuffer);
     }
+    int mergeOrdersSize = ReadWriteIOUtils.readInt(byteBuffer);
+    List<SortItem> mergeOrders = new ArrayList<>(mergeOrdersSize);
+    while (mergeOrdersSize > 0) {
+      mergeOrders.add(SortItem.deserialize(byteBuffer));
+      mergeOrdersSize--;
+    }
     PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
-    return new LastQueryMergeNode(planNodeId, timeFilter);
+    return new LastQueryMergeNode(planNodeId, timeFilter, mergeOrders);
   }
 
   public void setChildren(List<PlanNode> children) {
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 aa4f01cd68..9067f7d55e 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
@@ -36,8 +36,10 @@ import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 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.SortItem;
 import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition;
 
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -235,6 +237,13 @@ public class QueryStatement extends Statement {
     return orderByComponent.getTimeOrder();
   }
 
+  public List<SortItem> getSortItemList() {
+    if (orderByComponent == null) {
+      return Collections.emptyList();
+    }
+    return orderByComponent.getSortItemList();
+  }
+
   public void semanticCheck() {
     if (isAggregationQuery()) {
       if (disableAlign()) {
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java
index e098262c81..7f8ade3f5a 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/QueryLogicalPlanUtil.java
@@ -108,7 +108,7 @@ public class QueryLogicalPlanUtil {
 
   /* Last Query */
   static {
-    String sql = "SELECT last * FROM root.sg.** WHERE time > 100";
+    String sql = "SELECT last * FROM root.sg.** WHERE time > 100 ORDER BY timeseries ASC";
 
     QueryId queryId = new QueryId("test");
     List<PlanNode> sourceNodeList = new ArrayList<>();
@@ -140,7 +140,11 @@ public class QueryLogicalPlanUtil {
             queryId.genPlanNodeId(), (MeasurementPath) schemaMap.get("root.sg.d2.s2")));
 
     LastQueryMergeNode lastQueryMergeNode =
-        new LastQueryMergeNode(queryId.genPlanNodeId(), sourceNodeList, TimeFilter.gt(100));
+        new LastQueryMergeNode(
+            queryId.genPlanNodeId(),
+            sourceNodeList,
+            TimeFilter.gt(100),
+            Collections.singletonList(new SortItem(SortKey.TIMESERIES, Ordering.ASC)));
 
     querySQLs.add(sql);
     sqlToPlanMap.put(sql, lastQueryMergeNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java
index 2a3765dded..73079d21f9 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java
@@ -112,7 +112,7 @@ public class LastQueryTest {
     for (String path : paths) {
       expressions.add(new TimeSeriesOperand(new MeasurementPath(path)));
     }
-    PlanNode root = builder.planLast(expressions, null).getRoot();
+    PlanNode root = builder.planLast(expressions, null, Collections.emptyList()).getRoot();
     return new LogicalQueryPlan(context, root);
   }
 }