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

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


The following commit(s) were added to refs/heads/ty/TableModelGrammar by this 
push:
     new c308e14c36e Use QualifiedObjectName in TableScan, perfect 
getDataPartition interface in Metadata, perfect FilterScanCombine rule
c308e14c36e is described below

commit c308e14c36ee986f22053fd33773c904f0487357
Author: Beyyes <[email protected]>
AuthorDate: Thu Jun 27 11:34:41 2024 +0800

    Use QualifiedObjectName in TableScan, perfect getDataPartition interface in 
Metadata, perfect FilterScanCombine rule
---
 .../db/queryengine/common/MPPQueryContext.java     |  15 -
 .../TableModelStatementMemorySourceVisitor.java    |   3 +-
 .../plan/planner/plan/node/PlanGraphPrinter.java   |   2 +-
 .../plan/relational/analyzer/Analysis.java         |  38 ++-
 .../plan/relational/metadata/Metadata.java         |  22 ++
 .../relational/metadata/QualifiedObjectName.java   |   2 +
 .../relational/metadata/TableMetadataImpl.java     |  17 ++
 .../plan/relational/planner/LogicalPlanner.java    |   8 +-
 .../plan/relational/planner/RelationPlanner.java   |   9 +-
 .../planner/distribute/ExchangeNodeGenerator.java  |  61 +++-
 .../ir/GlobalTimePredicateExtractVisitor.java      | 104 ++++---
 .../relational/planner/node/TableScanNode.java     | 104 +++++--
 .../planner/optimizations/FilterScanCombine.java   | 226 --------------
 .../planner/optimizations/IndexScan.java           | 247 ---------------
 .../optimizations/PushPredicateIntoTableScan.java  | 337 +++++++++++++++++++++
 .../planner/optimizations/SimplifyExpressions.java |   4 +-
 .../plan/relational/sql/ast/QualifiedName.java     |   2 +-
 .../plan/relational/analyzer/AnalyzerTest.java     | 192 ++++++++++--
 .../plan/relational/analyzer/TestMatadata.java     |  15 +-
 .../iotdb/commons/partition/DataPartition.java     |  40 +++
 20 files changed, 830 insertions(+), 618 deletions(-)

diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java
index ec536eb3612..988224da729 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java
@@ -27,7 +27,6 @@ import org.apache.iotdb.db.queryengine.plan.analyze.QueryType;
 import org.apache.iotdb.db.queryengine.plan.analyze.TypeProvider;
 import org.apache.iotdb.db.queryengine.plan.analyze.lock.SchemaLockType;
 import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner;
-import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression;
 import org.apache.iotdb.db.queryengine.statistics.QueryPlanStatistics;
 
 import org.apache.tsfile.read.filter.basic.Filter;
@@ -90,11 +89,6 @@ public class MPPQueryContext {
 
   private final LocalExecutionPlanner LOCAL_EXECUTION_PLANNER = 
LocalExecutionPlanner.getInstance();
 
-  // splits predicate expression in table model into three parts,
-  // index 0 represents metadataExpressions, index 1 represents 
expressionsCanPushDownToOperator,
-  // index 2 represents expressionsCannotPushDownToOperator
-  private List<List<Expression>> tableModelPredicateExpressions;
-
   public MPPQueryContext(QueryId queryId) {
     this.queryId = queryId;
     this.endPointBlackList = new LinkedList<>();
@@ -318,15 +312,6 @@ public class MPPQueryContext {
     queryPlanStatistics.setLogicalOptimizationCost(logicalOptimizeCost);
   }
 
-  public List<List<Expression>> getTableModelPredicateExpressions() {
-    return tableModelPredicateExpressions;
-  }
-
-  public void setTableModelPredicateExpressions(
-      List<List<Expression>> tableModelPredicateExpressions) {
-    this.tableModelPredicateExpressions = tableModelPredicateExpressions;
-  }
-
   // region =========== FE memory related, make sure its not called 
concurrently ===========
 
   /**
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java
index d0a90a7ede3..7b74850287c 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java
@@ -68,8 +68,7 @@ public class TableModelStatementMemorySourceVisitor
                 context.getQueryContext().getSession(),
                 WarningCollector.NOOP)
             .plan(context.getAnalysis());
-    if (context.getAnalysis().getDataPartition() == null
-        || context.getAnalysis().getDataPartition().isEmpty()) {
+    if (context.getAnalysis().isEmptyDataSource()) {
       return new StatementMemorySource(new TsBlock(0), header);
     }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java
index 7f94c4ba316..1e43a247942 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java
@@ -604,7 +604,7 @@ public class PlanGraphPrinter extends 
PlanVisitor<List<String>, PlanGraphPrinter
   public List<String> visitTableScan(TableScanNode node, GraphContext context) 
{
     List<String> boxValue = new ArrayList<>();
     boxValue.add(String.format("TableScan-%s", node.getPlanNodeId().getId()));
-    boxValue.add(String.format("QualifiedTableName: %s", 
node.getQualifiedTableName()));
+    boxValue.add(String.format("QualifiedTableName: %s", 
node.getQualifiedObjectName().toString()));
     boxValue.add(String.format("OutputSymbols: %s", node.getOutputSymbols()));
     boxValue.add(String.format("DeviceEntriesSize: %s", 
node.getDeviceEntries().size()));
     boxValue.add(String.format("ScanOrder: %s", node.getScanOrder()));
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
index 065260c9899..25a8e6edc7f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
@@ -154,7 +154,6 @@ public class Analysis implements IAnalysis {
 
   private final Set<NodeRef<Relation>> aliasedRelations = new 
LinkedHashSet<>();
 
-  // only be used in write plan and won't be used in query
   private DataPartition dataPartition;
 
   // only be used in write plan and won't be used in query
@@ -167,13 +166,8 @@ public class Analysis implements IAnalysis {
   // indicate is there a value filter
   private boolean hasValueFilter = false;
 
-  public DataPartition getDataPartition() {
-    return dataPartition;
-  }
-
-  public void setDataPartition(DataPartition dataPartition) {
-    this.dataPartition = dataPartition;
-  }
+  // if emptyDataSource, there is no need to execute the query in BE
+  private boolean emptyDataSource = false;
 
   public Analysis(@Nullable Statement root, Map<NodeRef<Parameter>, 
Expression> parameters) {
     this.root = root;
@@ -580,6 +574,14 @@ public class Analysis implements IAnalysis {
     this.hasValueFilter = hasValueFilter;
   }
 
+  public boolean isEmptyDataSource() {
+    return emptyDataSource;
+  }
+
+  public void setEmptyDataSource(boolean emptyDataSource) {
+    this.emptyDataSource = emptyDataSource;
+  }
+
   @Override
   public boolean isFailed() {
     return false;
@@ -603,14 +605,6 @@ public class Analysis implements IAnalysis {
     return finishQueryAfterAnalyze;
   }
 
-  private boolean hasDataSource() {
-    return (dataPartition != null && !dataPartition.isEmpty());
-    //            || (schemaPartition != null && !schemaPartition.isEmpty())
-    //            || statement instanceof ShowQueriesStatement
-    //            || (statement instanceof QueryStatement
-    //            && ((QueryStatement) statement).isAggregationQuery());
-  }
-
   @Override
   public TsBlock constructResultForMemorySource(MPPQueryContext context) {
     requireNonNull(getStatement(), "root statement is analysis is null");
@@ -660,6 +654,18 @@ public class Analysis implements IAnalysis {
     return dataPartition;
   }
 
+  public void setDataPartition(DataPartition dataPartition) {
+    this.dataPartition = dataPartition;
+  }
+
+  public void upsertDataPartition(DataPartition targetDataPartition) {
+    if (this.dataPartition == null) {
+      this.dataPartition = targetDataPartition;
+    } else {
+      this.dataPartition.upsertDataPartition(targetDataPartition);
+    }
+  }
+
   @Override
   public void setRedirectNodeList(List<TEndPoint> redirectNodeList) {
     throw new UnsupportedOperationException();
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/Metadata.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/Metadata.java
index 8e7fddf677d..1acfdff1dbe 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/Metadata.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/Metadata.java
@@ -19,6 +19,8 @@
 
 package org.apache.iotdb.db.queryengine.plan.relational.metadata;
 
+import org.apache.iotdb.commons.partition.DataPartition;
+import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.db.queryengine.common.MPPQueryContext;
 import org.apache.iotdb.db.queryengine.common.SessionInfo;
@@ -132,4 +134,24 @@ public interface Metadata {
    * <p>The device id shall be [table, seg1, ....]
    */
   SchemaPartition getSchemaPartition(String database);
+
+  // ======================== Table Model Data Partition Interface 
========================
+  /**
+   * Get data partition, used in query scenarios.
+   *
+   * @param database a user-provided db name, the database shall start with 
"root.".
+   * @param sgNameToQueryParamsMap database name -> the list of 
DataPartitionQueryParams
+   */
+  DataPartition getDataPartition(
+      String database, List<DataPartitionQueryParam> sgNameToQueryParamsMap);
+
+  /**
+   * Get data partition, used in query scenarios which contains time filter 
like: time < XX or time
+   * > XX
+   *
+   * @param database a user-provided db name, the database shall start with 
"root.".
+   * @return sgNameToQueryParamsMap database name -> the list of 
DataPartitionQueryParams
+   */
+  DataPartition getDataPartitionWithUnclosedTimeRange(
+      String database, List<DataPartitionQueryParam> sgNameToQueryParamsMap);
 }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/QualifiedObjectName.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/QualifiedObjectName.java
index 834170b27d6..76446cdf648 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/QualifiedObjectName.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/QualifiedObjectName.java
@@ -45,6 +45,8 @@ public class QualifiedObjectName {
   }
 
   private final String dbName;
+
+  // objectName represents tableName
   private final String objectName;
 
   public QualifiedObjectName(String dbName, String objectName) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
index dcf64372cfa..16a1d2df8ee 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
@@ -19,6 +19,8 @@
 
 package org.apache.iotdb.db.queryengine.plan.relational.metadata;
 
+import org.apache.iotdb.commons.partition.DataPartition;
+import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.commons.schema.table.TsTable;
 import org.apache.iotdb.commons.udf.builtin.BuiltinAggregationFunction;
@@ -45,6 +47,7 @@ import org.apache.tsfile.file.metadata.IDeviceID;
 import org.apache.tsfile.read.common.type.Type;
 import org.apache.tsfile.read.common.type.TypeFactory;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
 import java.util.Optional;
@@ -337,6 +340,20 @@ public class TableMetadataImpl implements Metadata {
     return partitionFetcher.getSchemaPartition(PATH_ROOT + PATH_SEPARATOR + 
database);
   }
 
+  @Override
+  public DataPartition getDataPartition(
+      String database, List<DataPartitionQueryParam> sgNameToQueryParamsMap) {
+    return partitionFetcher.getDataPartition(
+        Collections.singletonMap(database, sgNameToQueryParamsMap));
+  }
+
+  @Override
+  public DataPartition getDataPartitionWithUnclosedTimeRange(
+      String database, List<DataPartitionQueryParam> sgNameToQueryParamsMap) {
+    return partitionFetcher.getDataPartitionWithUnclosedTimeRange(
+        Collections.singletonMap(database, sgNameToQueryParamsMap));
+  }
+
   public static boolean isTwoNumericType(List<? extends Type> argumentTypes) {
     return argumentTypes.size() == 2
         && isNumericType(argumentTypes.get(0))
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/LogicalPlanner.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/LogicalPlanner.java
index 7758a6437f9..987afe90162 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/LogicalPlanner.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/LogicalPlanner.java
@@ -34,9 +34,8 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.RelationType;
 import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.CreateTableDeviceNode;
 import org.apache.iotdb.db.queryengine.plan.relational.planner.node.OutputNode;
-import 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.FilterScanCombine;
-import 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.IndexScan;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.PruneUnUsedColumns;
+import 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.PushPredicateIntoTableScan;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.RelationalPlanOptimizer;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.RemoveRedundantIdentityProjections;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.SimplifyExpressions;
@@ -87,9 +86,8 @@ public class LogicalPlanner {
         Arrays.asList(
             new SimplifyExpressions(),
             new PruneUnUsedColumns(),
-            new FilterScanCombine(),
-            new RemoveRedundantIdentityProjections(),
-            new IndexScan());
+            new PushPredicateIntoTableScan(),
+            new RemoveRedundantIdentityProjections());
   }
 
   public LogicalQueryPlan plan(Analysis analysis) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
index cedde9533c9..fca9a8af675 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
@@ -21,6 +21,7 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.Field;
 import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef;
 import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Scope;
 import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema;
+import 
org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AliasedRelation;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AstVisitor;
@@ -28,6 +29,7 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Except;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Intersect;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Join;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Node;
+import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QualifiedName;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query;
 import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QuerySpecification;
 import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SubqueryExpression;
@@ -107,10 +109,15 @@ public class RelationPlanner extends 
AstVisitor<RelationPlan, Void> {
     }
 
     List<Symbol> outputSymbols = outputSymbolsBuilder.build();
+    QualifiedName qualifiedName = analysis.getRelationName(table);
+    if (!qualifiedName.getPrefix().isPresent()) {
+      throw new IllegalStateException("Table " + table.getName() + " has no 
prefix!");
+    }
     TableScanNode tableScanNode =
         new TableScanNode(
             idAllocator.genPlanNodeId(),
-            table.getName().toString(),
+            new QualifiedObjectName(
+                qualifiedName.getPrefix().get().toString(), 
qualifiedName.getSuffix()),
             outputSymbols,
             symbolToColumnSchema.build());
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/ExchangeNodeGenerator.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/ExchangeNodeGenerator.java
index 0cd3d7ba492..809d1398aee 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/ExchangeNodeGenerator.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/ExchangeNodeGenerator.java
@@ -25,12 +25,14 @@ import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read.Tabl
 import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ExchangeNode;
 import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SourceNode;
 import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis;
+import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry;
 import org.apache.iotdb.db.queryengine.plan.relational.planner.OrderingScheme;
 import org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder;
 import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.MergeSortNode;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -45,7 +47,42 @@ public class ExchangeNodeGenerator extends 
SimplePlanRewriter<ExchangeNodeGenera
   @Override
   public List<PlanNode> visitTableScan(TableScanNode node, PlanContext 
context) {
 
-    if (node.getRegionReplicaSetList().size() > 1) {
+    Map<TRegionReplicaSet, TableScanNode> tableScanNodeMap = new HashMap<>();
+
+    for (DeviceEntry deviceEntry : node.getDeviceEntries()) {
+      List<TRegionReplicaSet> regionReplicaSets =
+          context
+              .analysis
+              .getDataPartitionInfo()
+              .getDataRegionReplicaSetWithTimeFilter(
+                  node.getQualifiedObjectName().getDatabaseName(),
+                  deviceEntry.getDeviceID(),
+                  node.getTimeFilter());
+      for (TRegionReplicaSet regionReplicaSet : regionReplicaSets) {
+        TableScanNode tableScanNode =
+            tableScanNodeMap.computeIfAbsent(
+                regionReplicaSet,
+                k -> {
+                  TableScanNode scanNode =
+                      new TableScanNode(
+                          context.queryContext.getQueryId().genPlanNodeId(),
+                          node.getQualifiedObjectName(),
+                          node.getOutputSymbols(),
+                          node.getAssignments(),
+                          new ArrayList<>(),
+                          node.getIdAndAttributeIndexMap(),
+                          node.getScanOrder(),
+                          node.getTimePredicate().orElse(null),
+                          node.getPushDownPredicate());
+                  scanNode.setRegionReplicaSet(regionReplicaSet);
+                  return scanNode;
+                });
+        tableScanNode.appendDeviceEntry(deviceEntry);
+      }
+    }
+
+    int i = 0;
+    if (tableScanNodeMap.size() > 1) {
       context.hasExchangeNode = true;
       List<Symbol> orderBy = node.getOutputSymbols().subList(0, 1);
       Map<Symbol, SortOrder> orderings =
@@ -57,9 +94,9 @@ public class ExchangeNodeGenerator extends 
SimplePlanRewriter<ExchangeNodeGenera
               orderingScheme,
               node.getOutputSymbols());
 
-      for (int i = 0; i < node.getRegionReplicaSetList().size(); i++) {
-        TRegionReplicaSet regionReplicaSet = 
node.getRegionReplicaSetList().get(i);
-        TableScanNode subTableScanNode = node.clone();
+      for (Map.Entry<TRegionReplicaSet, TableScanNode> entry : 
tableScanNodeMap.entrySet()) {
+        TRegionReplicaSet regionReplicaSet = entry.getKey();
+        TableScanNode subTableScanNode = entry.getValue();
         
subTableScanNode.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
         subTableScanNode.setRegionReplicaSet(regionReplicaSet);
         context.nodeDistributionMap.put(
@@ -78,11 +115,11 @@ public class ExchangeNodeGenerator extends 
SimplePlanRewriter<ExchangeNodeGenera
           exchangeNode.addChild(subTableScanNode);
           mergeSortNode.addChild(exchangeNode);
         }
+        i++;
       }
       return Collections.singletonList(mergeSortNode);
     } else {
-      node.setRegionReplicaSet(node.getRegionReplicaSetList().get(0));
-      return Collections.singletonList(node);
+      return 
Collections.singletonList(tableScanNodeMap.entrySet().iterator().next().getValue());
     }
   }
 
@@ -121,12 +158,12 @@ public class ExchangeNodeGenerator extends 
SimplePlanRewriter<ExchangeNodeGenera
       AbstractSchemaMergeNode node, PlanContext context) {
     node.getChildren()
         .forEach(
-            child -> {
-              context.putNodeDistribution(
-                  child.getPlanNodeId(),
-                  new NodeDistribution(
-                      NodeDistributionType.NO_CHILD, ((SourceNode) 
child).getRegionReplicaSet()));
-            });
+            child ->
+                context.putNodeDistribution(
+                    child.getPlanNodeId(),
+                    new NodeDistribution(
+                        NodeDistributionType.NO_CHILD,
+                        ((SourceNode) child).getRegionReplicaSet())));
     NodeDistribution nodeDistribution =
         new NodeDistribution(NodeDistributionType.DIFFERENT_FROM_ALL_CHILDREN);
     PlanNode newNode = node.clone();
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/GlobalTimePredicateExtractVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/GlobalTimePredicateExtractVisitor.java
index 71efd34ffda..acc614ce879 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/GlobalTimePredicateExtractVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/GlobalTimePredicateExtractVisitor.java
@@ -25,7 +25,6 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BetweenPredicate;
 import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall;
-import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.IfExpression;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.InPredicate;
 import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.IsNotNullPredicate;
@@ -37,17 +36,18 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NotExpression;
 import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NullIfExpression;
 import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SearchedCaseExpression;
 import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SimpleCaseExpression;
+import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference;
 
 import org.apache.tsfile.utils.Pair;
 
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
 import static org.apache.iotdb.commons.conf.IoTDBConstant.TIME;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral.TRUE_LITERAL;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression.Operator.AND;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression.Operator.OR;
-import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression.and;
-import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression.or;
 
 public class GlobalTimePredicateExtractVisitor
     extends IrVisitor<Pair<Expression, Boolean>, 
GlobalTimePredicateExtractVisitor.Context> {
@@ -76,61 +76,83 @@ public class GlobalTimePredicateExtractVisitor
   protected Pair<Expression, Boolean> visitLogicalExpression(
       LogicalExpression node, Context context) {
     if (node.getOperator() == AND) {
-      Pair<Expression, Boolean> leftResultPair = 
process(node.getTerms().get(0), context);
-      Pair<Expression, Boolean> rightResultPair = 
process(node.getTerms().get(1), context);
+      List<Pair<Expression, Boolean>> resultPairs = new ArrayList<>();
+      for (Expression term : node.getTerms()) {
+        resultPairs.add(process(term, context));
+      }
 
+      List<Expression> newTimeFilterTerms = new ArrayList<>();
+      List<Expression> newValueFilterTerms = new ArrayList<>();
       // rewrite predicate to avoid duplicate calculation on time filter
       // If Left-child or Right-child does not contain value filter
       // We can set it to true in Predicate Tree
       if (context.canRewrite) {
-        Expression newLeftExpression = null, newRightExpression = null;
-        if (leftResultPair.left != null && !leftResultPair.right) {
-          newLeftExpression = TRUE_LITERAL;
-        }
-        if (rightResultPair.left != null && !rightResultPair.right) {
-          newRightExpression = TRUE_LITERAL;
-        }
-        if (newLeftExpression != null || newRightExpression != null) {
-          node.setTerms(
-              Arrays.asList(
-                  newLeftExpression != null ? newLeftExpression : 
node.getTerms().get(0),
-                  newRightExpression != null ? newRightExpression : 
node.getTerms().get(1)));
-        }
+        getNewTimeValueExpressions(node, resultPairs, newTimeFilterTerms, 
newValueFilterTerms);
       }
 
-      if (leftResultPair.left != null && rightResultPair.left != null) {
+      if (!newTimeFilterTerms.isEmpty()) {
+        node.setTerms(newValueFilterTerms);
+
         return new Pair<>(
-            and(leftResultPair.left, rightResultPair.left),
-            leftResultPair.right || rightResultPair.right);
-      } else if (leftResultPair.left != null) {
-        return new Pair<>(leftResultPair.left, true);
-      } else if (rightResultPair.left != null) {
-        return new Pair<>(rightResultPair.left, true);
+            newTimeFilterTerms.size() == 1
+                ? newTimeFilterTerms.get(0)
+                : new LogicalExpression(AND, newTimeFilterTerms),
+            !newValueFilterTerms.isEmpty());
       }
 
       return new Pair<>(null, true);
     } else if (node.getOperator() == OR) {
-      Pair<Expression, Boolean> leftResultPair =
-          process(node.getTerms().get(0), new Context(false, false));
-      Pair<Expression, Boolean> rightResultPair =
-          process(node.getTerms().get(1), new Context(false, false));
-
-      if (leftResultPair.left != null && rightResultPair.left != null) {
-        if (Boolean.TRUE.equals(
-            context.isFirstOr && !leftResultPair.right && 
!rightResultPair.right)) {
-          node.getTerms().set(0, TRUE_LITERAL);
-          node.getTerms().set(0, TRUE_LITERAL);
+
+      List<Pair<Expression, Boolean>> resultPairs = new ArrayList<>();
+      for (Expression term : node.getTerms()) {
+        resultPairs.add(process(term, new Context(false, false)));
+      }
+
+      List<Expression> newTimeFilterTerms = new ArrayList<>();
+      List<Expression> newValueFilterTerms = new ArrayList<>();
+
+      getNewTimeValueExpressions(node, resultPairs, newTimeFilterTerms, 
newValueFilterTerms);
+
+      // for example, `(t1 and s1) or t2`, `t1 or t2` meets this condition
+      if (newTimeFilterTerms.size() == node.getTerms().size()) {
+        if (context.isFirstOr && newValueFilterTerms.isEmpty()) {
+          node.setTerms(Collections.singletonList(TRUE_LITERAL));
         }
         return new Pair<>(
-            or(leftResultPair.left, rightResultPair.left),
-            leftResultPair.right || rightResultPair.right);
+            newTimeFilterTerms.size() == 1
+                ? newTimeFilterTerms.get(0)
+                : new LogicalExpression(OR, newTimeFilterTerms),
+            !newValueFilterTerms.isEmpty());
       }
+
       return new Pair<>(null, true);
     } else {
       throw new IllegalStateException("Illegal state in 
visitLogicalExpression");
     }
   }
 
+  private void getNewTimeValueExpressions(
+      LogicalExpression node,
+      List<Pair<Expression, Boolean>> resultPairs,
+      List<Expression> newTimeFilterTerms,
+      List<Expression> newValueFilterTerms) {
+    for (int i = 0; i < resultPairs.size(); i++) {
+      Pair<Expression, Boolean> pair = resultPairs.get(i);
+
+      if (pair.left != null) {
+        newTimeFilterTerms.add(pair.left);
+
+        // has time filter, also has value filter
+        if (pair.right) {
+          newValueFilterTerms.add(node.getTerms().get(i));
+        }
+      } else {
+        // only has value filter
+        newValueFilterTerms.add(node.getTerms().get(i));
+      }
+    }
+  }
+
   @Override
   protected Pair<Expression, Boolean> visitComparisonExpression(
       ComparisonExpression node, Context context) {
@@ -237,12 +259,12 @@ public class GlobalTimePredicateExtractVisitor
   }
 
   private static boolean isTimeIdentifier(Expression e) {
-    return e instanceof Identifier && TIME.equalsIgnoreCase(((Identifier) 
e).getValue());
+    return e instanceof SymbolReference && 
TIME.equalsIgnoreCase(((SymbolReference) e).getName());
   }
 
   private static boolean checkIsTimeFilter(Expression timeExpression, 
Expression valueExpression) {
-    return timeExpression instanceof Identifier
-        && ((Identifier) timeExpression).getValue().equalsIgnoreCase(TIME)
+    return timeExpression instanceof SymbolReference
+        && TIME.equalsIgnoreCase(((SymbolReference) timeExpression).getName())
         && valueExpression instanceof LongLiteral;
   }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/TableScanNode.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/TableScanNode.java
index 80b96c48ea5..d90f5af29a1 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/TableScanNode.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/TableScanNode.java
@@ -22,11 +22,13 @@ import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor;
 import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SourceNode;
 import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema;
 import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry;
+import 
org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName;
 import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression;
 import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.tsfile.read.filter.basic.Filter;
 import org.apache.tsfile.utils.ReadWriteIOUtils;
 
 import javax.annotation.Nullable;
@@ -44,8 +46,7 @@ import java.util.stream.Collectors;
 
 public class TableScanNode extends SourceNode {
 
-  // db.tablename
-  private final String qualifiedTableName;
+  private final QualifiedObjectName qualifiedObjectName;
   private List<Symbol> outputSymbols;
   private Map<Symbol, ColumnSchema> assignments;
 
@@ -64,6 +65,8 @@ public class TableScanNode extends SourceNode {
   // 1 or time < 10
   @Nullable private Expression timePredicate;
 
+  private Filter timeFilter;
+
   // push down predicate for current series, could be null if it doesn't exist
   @Nullable private Expression pushDownPredicate;
 
@@ -78,35 +81,35 @@ public class TableScanNode extends SourceNode {
   // The id of DataRegion where the node will run
   private TRegionReplicaSet regionReplicaSet;
 
-  private List<TRegionReplicaSet> regionReplicaSetList;
-
   public TableScanNode(
       PlanNodeId id,
-      String qualifiedTableName,
+      QualifiedObjectName qualifiedObjectName,
       List<Symbol> outputSymbols,
       Map<Symbol, ColumnSchema> assignments) {
     super(id);
-    this.qualifiedTableName = qualifiedTableName;
+    this.qualifiedObjectName = qualifiedObjectName;
     this.outputSymbols = outputSymbols;
     this.assignments = assignments;
   }
 
   public TableScanNode(
       PlanNodeId id,
-      String qualifiedTableName,
+      QualifiedObjectName qualifiedObjectName,
       List<Symbol> outputSymbols,
       Map<Symbol, ColumnSchema> assignments,
       List<DeviceEntry> deviceEntries,
       Map<Symbol, Integer> idAndAttributeIndexMap,
       Ordering scanOrder,
+      Expression timePredicate,
       Expression pushDownPredicate) {
     super(id);
-    this.qualifiedTableName = qualifiedTableName;
+    this.qualifiedObjectName = qualifiedObjectName;
     this.outputSymbols = outputSymbols;
     this.assignments = assignments;
     this.deviceEntries = deviceEntries;
     this.idAndAttributeIndexMap = idAndAttributeIndexMap;
     this.scanOrder = scanOrder;
+    this.timePredicate = timePredicate;
     this.pushDownPredicate = pushDownPredicate;
   }
 
@@ -127,12 +130,13 @@ public class TableScanNode extends SourceNode {
   public TableScanNode clone() {
     return new TableScanNode(
         getPlanNodeId(),
-        qualifiedTableName,
+        qualifiedObjectName,
         outputSymbols,
         assignments,
         deviceEntries,
         idAndAttributeIndexMap,
         scanOrder,
+        timePredicate,
         pushDownPredicate);
   }
 
@@ -149,7 +153,14 @@ public class TableScanNode extends SourceNode {
   @Override
   protected void serializeAttributes(ByteBuffer byteBuffer) {
     PlanNodeType.TABLE_SCAN_NODE.serialize(byteBuffer);
-    ReadWriteIOUtils.write(qualifiedTableName, byteBuffer);
+
+    if (qualifiedObjectName.getDatabaseName() != null) {
+      ReadWriteIOUtils.write(true, byteBuffer);
+      ReadWriteIOUtils.write(qualifiedObjectName.getDatabaseName(), 
byteBuffer);
+    } else {
+      ReadWriteIOUtils.write(false, byteBuffer);
+    }
+    ReadWriteIOUtils.write(qualifiedObjectName.getObjectName(), byteBuffer);
 
     ReadWriteIOUtils.write(outputSymbols.size(), byteBuffer);
     outputSymbols.forEach(symbol -> ReadWriteIOUtils.write(symbol.getName(), 
byteBuffer));
@@ -173,6 +184,13 @@ public class TableScanNode extends SourceNode {
 
     ReadWriteIOUtils.write(scanOrder.ordinal(), byteBuffer);
 
+    if (timePredicate != null) {
+      ReadWriteIOUtils.write(true, byteBuffer);
+      Expression.serialize(timePredicate, byteBuffer);
+    } else {
+      ReadWriteIOUtils.write(false, byteBuffer);
+    }
+
     if (pushDownPredicate != null) {
       ReadWriteIOUtils.write(true, byteBuffer);
       Expression.serialize(pushDownPredicate, byteBuffer);
@@ -184,7 +202,13 @@ public class TableScanNode extends SourceNode {
   @Override
   protected void serializeAttributes(DataOutputStream stream) throws 
IOException {
     PlanNodeType.TABLE_SCAN_NODE.serialize(stream);
-    ReadWriteIOUtils.write(qualifiedTableName, stream);
+    if (qualifiedObjectName.getDatabaseName() != null) {
+      ReadWriteIOUtils.write(true, stream);
+      ReadWriteIOUtils.write(qualifiedObjectName.getDatabaseName(), stream);
+    } else {
+      ReadWriteIOUtils.write(false, stream);
+    }
+    ReadWriteIOUtils.write(qualifiedObjectName.getObjectName(), stream);
 
     ReadWriteIOUtils.write(outputSymbols.size(), stream);
     for (Symbol symbol : outputSymbols) {
@@ -210,6 +234,13 @@ public class TableScanNode extends SourceNode {
 
     ReadWriteIOUtils.write(scanOrder.ordinal(), stream);
 
+    if (timePredicate != null) {
+      ReadWriteIOUtils.write(true, stream);
+      Expression.serialize(timePredicate, stream);
+    } else {
+      ReadWriteIOUtils.write(false, stream);
+    }
+
     if (pushDownPredicate != null) {
       ReadWriteIOUtils.write(true, stream);
       Expression.serialize(pushDownPredicate, stream);
@@ -219,9 +250,15 @@ public class TableScanNode extends SourceNode {
   }
 
   public static TableScanNode deserialize(ByteBuffer byteBuffer) {
-    String qualifiedTableName = ReadWriteIOUtils.readString(byteBuffer);
-    int size = ReadWriteIOUtils.readInt(byteBuffer);
+    boolean hasDatabaseName = ReadWriteIOUtils.readBool(byteBuffer);
+    String databaseName = null;
+    if (hasDatabaseName) {
+      databaseName = ReadWriteIOUtils.readString(byteBuffer);
+    }
+    String tableName = ReadWriteIOUtils.readString(byteBuffer);
+    QualifiedObjectName qualifiedObjectName = new 
QualifiedObjectName(databaseName, tableName);
 
+    int size = ReadWriteIOUtils.readInt(byteBuffer);
     List<Symbol> outputSymbols = new ArrayList<>(size);
     for (int i = 0; i < size; i++) {
       outputSymbols.add(Symbol.deserialize(byteBuffer));
@@ -248,6 +285,12 @@ public class TableScanNode extends SourceNode {
 
     Ordering scanOrder = 
Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)];
 
+    Expression timePredicate = null;
+    boolean hasTimePredicate = ReadWriteIOUtils.readBool(byteBuffer);
+    if (hasTimePredicate) {
+      timePredicate = Expression.deserialize(byteBuffer);
+    }
+
     Expression pushDownPredicate = null;
     boolean hasPushDownPredicate = ReadWriteIOUtils.readBool(byteBuffer);
     if (hasPushDownPredicate) {
@@ -258,12 +301,13 @@ public class TableScanNode extends SourceNode {
 
     return new TableScanNode(
         planNodeId,
-        qualifiedTableName,
+        qualifiedObjectName,
         outputSymbols,
         assignments,
         deviceEntries,
         idAndAttributeIndexMap,
         scanOrder,
+        timePredicate,
         pushDownPredicate);
   }
 
@@ -290,18 +334,18 @@ public class TableScanNode extends SourceNode {
       return false;
     }
     TableScanNode that = (TableScanNode) o;
-    return Objects.equals(qualifiedTableName, that.qualifiedTableName)
+    return Objects.equals(qualifiedObjectName, that.qualifiedObjectName)
         && Objects.equals(outputSymbols, that.outputSymbols)
         && Objects.equals(regionReplicaSet, that.regionReplicaSet);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(super.hashCode(), qualifiedTableName, outputSymbols, 
regionReplicaSet);
+    return Objects.hash(super.hashCode(), qualifiedObjectName, outputSymbols, 
regionReplicaSet);
   }
 
-  public String getQualifiedTableName() {
-    return this.qualifiedTableName;
+  public QualifiedObjectName getQualifiedObjectName() {
+    return this.qualifiedObjectName;
   }
 
   public void setOutputSymbols(List<Symbol> outputSymbols) {
@@ -336,6 +380,10 @@ public class TableScanNode extends SourceNode {
     return deviceEntries;
   }
 
+  public void appendDeviceEntry(DeviceEntry deviceEntry) {
+    this.deviceEntries.add(deviceEntry);
+  }
+
   public long getPushDownLimit() {
     return this.pushDownLimit;
   }
@@ -360,14 +408,6 @@ public class TableScanNode extends SourceNode {
     return this.regionReplicaSet;
   }
 
-  public List<TRegionReplicaSet> getRegionReplicaSetList() {
-    return regionReplicaSetList;
-  }
-
-  public void setRegionReplicaSetList(List<TRegionReplicaSet> 
regionReplicaSetList) {
-    this.regionReplicaSetList = regionReplicaSetList;
-  }
-
   public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
     this.regionReplicaSet = regionReplicaSet;
   }
@@ -375,4 +415,16 @@ public class TableScanNode extends SourceNode {
   public Optional<Expression> getTimePredicate() {
     return Optional.ofNullable(timePredicate);
   }
+
+  public void setTimePredicate(@Nullable Expression timePredicate) {
+    this.timePredicate = timePredicate;
+  }
+
+  public Filter getTimeFilter() {
+    return timeFilter;
+  }
+
+  public void setTimeFilter(Filter timeFilter) {
+    this.timeFilter = timeFilter;
+  }
 }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/FilterScanCombine.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/FilterScanCombine.java
deleted file mode 100644
index 5cf01374ef5..00000000000
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/FilterScanCombine.java
+++ /dev/null
@@ -1,226 +0,0 @@
-/*
- * Licensed 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.queryengine.plan.relational.planner.optimizations;
-
-import org.apache.iotdb.db.queryengine.common.MPPQueryContext;
-import org.apache.iotdb.db.queryengine.common.SessionInfo;
-import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode;
-import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor;
-import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.MultiChildProcessNode;
-import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.SingleChildProcessNode;
-import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis;
-import 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicateCombineIntoTableScanChecker;
-import 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicatePushIntoMetadataChecker;
-import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata;
-import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol;
-import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode;
-import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode;
-import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression;
-import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall;
-import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression;
-import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Node;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static 
org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.MEASUREMENT;
-
-/**
- * After the optimized rule {@link SimplifyExpressions} finished, predicate 
expression in FilterNode
- * has been transformed to conjunctive normal forms(CNF).
- *
- * <p>In this class, we examine each expression in CNFs, determine how to use 
it, in metadata query,
- * or pushed down into ScanOperators, or it can only be used in FilterNode 
above with TableScanNode.
- *
- * <p>Notice that, when aggregation, multi-table, join are introduced, this 
optimization rule need
- * to be adapted.
- */
-public class FilterScanCombine implements RelationalPlanOptimizer {
-
-  @Override
-  public PlanNode optimize(
-      PlanNode planNode,
-      Analysis analysis,
-      Metadata metadata,
-      SessionInfo sessionInfo,
-      MPPQueryContext queryContext) {
-
-    // TODO change back after Gaofei accomplish ScanTimeAndValueFilterSplitRule
-    //    if (!analysis.hasValueFilter()) {
-    //      return planNode;
-    //    }
-
-    return planNode.accept(new Rewriter(), new RewriterContext(queryContext));
-  }
-
-  private static class Rewriter extends PlanVisitor<PlanNode, RewriterContext> 
{
-
-    @Override
-    public PlanNode visitPlan(PlanNode node, RewriterContext context) {
-      throw new IllegalArgumentException(
-          String.format("Unexpected plan node: %s in TableModel 
PredicatePushDown", node));
-    }
-
-    @Override
-    public PlanNode visitSingleChildProcess(SingleChildProcessNode node, 
RewriterContext context) {
-      PlanNode rewrittenChild = node.getChild().accept(this, context);
-      node.setChild(rewrittenChild);
-      return node;
-    }
-
-    @Override
-    public PlanNode visitMultiChildProcess(MultiChildProcessNode node, 
RewriterContext context) {
-      List<PlanNode> rewrittenChildren = new ArrayList<>();
-      for (PlanNode child : node.getChildren()) {
-        rewrittenChildren.add(child.accept(this, context));
-      }
-      node.setChildren(rewrittenChildren);
-      return node;
-    }
-
-    @Override
-    public PlanNode visitFilter(FilterNode node, RewriterContext context) {
-      context.filterNode = node;
-
-      if (node.getPredicate() != null) {
-        // when exist diff function, predicate can not be pushed down
-        if (containsDiffFunction(node.getPredicate())) {
-          context.pushDownPredicate = null;
-          return node;
-        }
-
-        context.pushDownPredicate = node.getPredicate();
-        return node.getChild().accept(this, context);
-      } else {
-        throw new IllegalStateException(
-            "Filter node has no predicate, node: " + node.getPlanNodeId());
-      }
-    }
-
-    @Override
-    public PlanNode visitTableScan(TableScanNode node, RewriterContext 
context) {
-      // has diff in FilterNode
-      if (context.pushDownPredicate == null) {
-        node.setPushDownPredicate(null);
-        return node;
-      }
-
-      context.queryContext.setTableModelPredicateExpressions(
-          splitConjunctionExpressions(context, node));
-
-      // exist expressions can push down to scan operator
-      if 
(!context.queryContext.getTableModelPredicateExpressions().get(1).isEmpty()) {
-        List<Expression> expressions =
-            context.queryContext.getTableModelPredicateExpressions().get(1);
-        node.setPushDownPredicate(
-            expressions.size() == 1
-                ? expressions.get(0)
-                : new LogicalExpression(LogicalExpression.Operator.AND, 
expressions));
-      } else {
-        node.setPushDownPredicate(null);
-      }
-
-      // exist expressions can not push down to scan operator
-      if 
(!context.queryContext.getTableModelPredicateExpressions().get(2).isEmpty()) {
-        List<Expression> expressions =
-            context.queryContext.getTableModelPredicateExpressions().get(2);
-        return new FilterNode(
-            context.queryContext.getQueryId().genPlanNodeId(),
-            node,
-            expressions.size() == 1
-                ? expressions.get(0)
-                : new LogicalExpression(LogicalExpression.Operator.AND, 
expressions));
-      }
-
-      return node;
-    }
-  }
-
-  private static List<List<Expression>> splitConjunctionExpressions(
-      RewriterContext context, TableScanNode node) {
-    Expression predicate = context.pushDownPredicate;
-
-    Set<String> idOrAttributeColumnNames =
-        node.getIdAndAttributeIndexMap().keySet().stream()
-            .map(Symbol::getName)
-            .collect(Collectors.toSet());
-
-    Set<String> measurementColumnNames =
-        node.getAssignments().entrySet().stream()
-            .filter(e -> MEASUREMENT.equals(e.getValue().getColumnCategory()))
-            .map(e -> e.getKey().getName())
-            .collect(Collectors.toSet());
-
-    List<Expression> metadataExpressions = new ArrayList<>();
-    List<Expression> expressionsCanPushDown = new ArrayList<>();
-    List<Expression> expressionsCannotPushDown = new ArrayList<>();
-
-    if (predicate instanceof LogicalExpression
-        && ((LogicalExpression) predicate).getOperator() == 
LogicalExpression.Operator.AND) {
-
-      for (Expression expression : ((LogicalExpression) predicate).getTerms()) 
{
-        if (PredicatePushIntoMetadataChecker.check(idOrAttributeColumnNames, 
expression)) {
-          metadataExpressions.add(expression);
-        } else if 
(PredicateCombineIntoTableScanChecker.check(measurementColumnNames, 
expression)) {
-          expressionsCanPushDown.add(expression);
-        } else {
-          expressionsCannotPushDown.add(expression);
-        }
-      }
-
-      return Arrays.asList(metadataExpressions, expressionsCanPushDown, 
expressionsCannotPushDown);
-    }
-
-    if (PredicatePushIntoMetadataChecker.check(idOrAttributeColumnNames, 
predicate)) {
-      metadataExpressions.add(predicate);
-    } else if 
(PredicateCombineIntoTableScanChecker.check(measurementColumnNames, predicate)) 
{
-      expressionsCanPushDown.add(predicate);
-    } else {
-      expressionsCannotPushDown.add(predicate);
-    }
-
-    return Arrays.asList(metadataExpressions, expressionsCanPushDown, 
expressionsCannotPushDown);
-  }
-
-  static boolean containsDiffFunction(Expression expression) {
-    if (expression instanceof FunctionCall
-        && "diff".equalsIgnoreCase(((FunctionCall) 
expression).getName().toString())) {
-      return true;
-    }
-
-    if (!expression.getChildren().isEmpty()) {
-      for (Node node : expression.getChildren()) {
-        if (containsDiffFunction((Expression) node)) {
-          return true;
-        }
-      }
-    }
-
-    return false;
-  }
-
-  private static class RewriterContext {
-    Expression pushDownPredicate;
-    MPPQueryContext queryContext;
-    FilterNode filterNode;
-
-    public RewriterContext(MPPQueryContext queryContext) {
-      this.queryContext = queryContext;
-    }
-  }
-}
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/IndexScan.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/IndexScan.java
deleted file mode 100644
index ab337c3c30f..00000000000
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/IndexScan.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * Licensed 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.queryengine.plan.relational.planner.optimizations;
-
-import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
-import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
-import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
-import org.apache.iotdb.commons.partition.DataPartition;
-import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.queryengine.common.MPPQueryContext;
-import org.apache.iotdb.db.queryengine.common.SessionInfo;
-import org.apache.iotdb.db.queryengine.plan.analyze.IPartitionFetcher;
-import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode;
-import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor;
-import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis;
-import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry;
-import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata;
-import 
org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName;
-import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol;
-import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode;
-import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode;
-import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression;
-
-import org.apache.tsfile.read.filter.basic.Filter;
-import org.apache.tsfile.utils.Pair;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static 
org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ATTRIBUTE;
-import static 
org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeVisitor.getTimePartitionSlotList;
-
-/** Extract IDeviceID */
-public class IndexScan implements RelationalPlanOptimizer {
-
-  static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
-
-  @Override
-  public PlanNode optimize(
-      PlanNode planNode,
-      Analysis analysis,
-      Metadata metadata,
-      SessionInfo sessionInfo,
-      MPPQueryContext queryContext) {
-
-    return planNode.accept(
-        new Rewriter(), new RewriterContext(null, metadata, sessionInfo, 
analysis, queryContext));
-  }
-
-  private static class Rewriter extends PlanVisitor<PlanNode, RewriterContext> 
{
-
-    @Override
-    public PlanNode visitPlan(PlanNode node, RewriterContext context) {
-      for (PlanNode child : node.getChildren()) {
-        child.accept(this, context);
-      }
-      return node;
-    }
-
-    @Override
-    public PlanNode visitFilter(FilterNode node, RewriterContext context) {
-      context.setPredicate(node.getPredicate());
-      context.setFilterNode(node);
-      node.getChild().accept(this, context);
-      return node;
-    }
-
-    @Override
-    public PlanNode visitTableScan(TableScanNode node, RewriterContext 
context) {
-
-      // only when exist diff predicate in FilterNode, context.predicate will 
not equal null
-      if (context.predicate == null) {
-        context.predicate = node.getPushDownPredicate();
-      }
-
-      List<Expression> metadataExpressions =
-          context.queryContext.getTableModelPredicateExpressions() == null
-                  || 
context.queryContext.getTableModelPredicateExpressions().get(0).isEmpty()
-              ? Collections.emptyList()
-              : 
context.queryContext.getTableModelPredicateExpressions().get(0);
-      String dbName = context.getSessionInfo().getDatabaseName().get();
-      List<String> attributeColumns =
-          node.getOutputSymbols().stream()
-              .filter(
-                  symbol -> 
ATTRIBUTE.equals(node.getAssignments().get(symbol).getColumnCategory()))
-              .map(Symbol::getName)
-              .collect(Collectors.toList());
-      List<DeviceEntry> deviceEntries =
-          context
-              .getMetadata()
-              .indexScan(
-                  new QualifiedObjectName(dbName, 
node.getQualifiedTableName()),
-                  metadataExpressions,
-                  attributeColumns);
-      node.setDeviceEntries(deviceEntries);
-      if (deviceEntries.isEmpty()) {
-        context.getAnalysis().setFinishQueryAfterAnalyze();
-      } else {
-        String treeModelDatabase = "root." + dbName;
-        DataPartition dataPartition =
-            fetchDataPartitionByDevices(
-                deviceEntries,
-                treeModelDatabase,
-                context.getQueryContext().getGlobalTimeFilter(),
-                context.getMetadata().getPartitionFetcher(),
-                context.getQueryContext());
-        context.getAnalysis().setDataPartition(dataPartition);
-
-        if (dataPartition.getDataPartitionMap().size() > 1) {
-          throw new IllegalStateException(
-              "Table model can only process data only in one database yet!");
-        }
-
-        if (dataPartition.getDataPartitionMap().isEmpty()) {
-          context.getAnalysis().setFinishQueryAfterAnalyze();
-        } else {
-          Set<TRegionReplicaSet> regionReplicaSet = new HashSet<>();
-          for (Map.Entry<
-                  String,
-                  Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, 
List<TRegionReplicaSet>>>>
-              e1 : dataPartition.getDataPartitionMap().entrySet()) {
-            for (Map.Entry<TSeriesPartitionSlot, Map<TTimePartitionSlot, 
List<TRegionReplicaSet>>>
-                e2 : e1.getValue().entrySet()) {
-              for (Map.Entry<TTimePartitionSlot, List<TRegionReplicaSet>> e3 :
-                  e2.getValue().entrySet()) {
-                regionReplicaSet.addAll(e3.getValue());
-              }
-            }
-          }
-          node.setRegionReplicaSetList(new ArrayList<>(regionReplicaSet));
-        }
-      }
-
-      return node;
-    }
-  }
-
-  private static DataPartition fetchDataPartitionByDevices(
-      List<DeviceEntry> deviceEntries,
-      String database,
-      Filter globalTimeFilter,
-      IPartitionFetcher partitionFetcher,
-      MPPQueryContext context) {
-    Pair<List<TTimePartitionSlot>, Pair<Boolean, Boolean>> res =
-        getTimePartitionSlotList(globalTimeFilter, context);
-
-    // there is no satisfied time range
-    if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) {
-      return new DataPartition(
-          Collections.emptyMap(),
-          CONFIG.getSeriesPartitionExecutorClass(),
-          CONFIG.getSeriesPartitionSlotNum());
-    }
-
-    Map<String, List<DataPartitionQueryParam>> sgNameToQueryParamsMap = new 
HashMap<>();
-    for (DeviceEntry deviceEntry : deviceEntries) {
-      DataPartitionQueryParam queryParam =
-          new DataPartitionQueryParam(
-              deviceEntry.getDeviceID(), res.left, res.right.left, 
res.right.right);
-      sgNameToQueryParamsMap.computeIfAbsent(database, key -> new 
ArrayList<>()).add(queryParam);
-    }
-
-    if (res.right.left || res.right.right) {
-      return 
partitionFetcher.getDataPartitionWithUnclosedTimeRange(sgNameToQueryParamsMap);
-    } else {
-      return partitionFetcher.getDataPartition(sgNameToQueryParamsMap);
-    }
-  }
-
-  private static class RewriterContext {
-    private Expression predicate;
-    private Metadata metadata;
-    private final SessionInfo sessionInfo;
-    private final Analysis analysis;
-    private final MPPQueryContext queryContext;
-    private FilterNode filterNode;
-
-    RewriterContext(
-        Expression predicate,
-        Metadata metadata,
-        SessionInfo sessionInfo,
-        Analysis analysis,
-        MPPQueryContext queryContext) {
-      this.predicate = predicate;
-      this.metadata = metadata;
-      this.sessionInfo = sessionInfo;
-      this.analysis = analysis;
-      this.queryContext = queryContext;
-    }
-
-    public Expression getPredicate() {
-      return this.predicate;
-    }
-
-    public void setPredicate(Expression predicate) {
-      this.predicate = predicate;
-    }
-
-    public Metadata getMetadata() {
-      return this.metadata;
-    }
-
-    public void setMetadata(Metadata metadata) {
-      this.metadata = metadata;
-    }
-
-    public SessionInfo getSessionInfo() {
-      return this.sessionInfo;
-    }
-
-    public Analysis getAnalysis() {
-      return this.analysis;
-    }
-
-    public MPPQueryContext getQueryContext() {
-      return queryContext;
-    }
-
-    public FilterNode getFilterNode() {
-      return filterNode;
-    }
-
-    public void setFilterNode(FilterNode filterNode) {
-      this.filterNode = filterNode;
-    }
-  }
-}
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java
new file mode 100644
index 00000000000..e9fb6ead873
--- /dev/null
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java
@@ -0,0 +1,337 @@
+/*
+ * Licensed 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.queryengine.plan.relational.planner.optimizations;
+
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.partition.DataPartition;
+import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.queryengine.common.MPPQueryContext;
+import org.apache.iotdb.db.queryengine.common.SessionInfo;
+import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode;
+import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor;
+import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.MultiChildProcessNode;
+import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.SingleChildProcessNode;
+import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis;
+import 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.ConvertPredicateToTimeFilterVisitor;
+import 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicateCombineIntoTableScanChecker;
+import 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicatePushIntoMetadataChecker;
+import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry;
+import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata;
+import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol;
+import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode;
+import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode;
+import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression;
+import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall;
+import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression;
+import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Node;
+
+import org.apache.tsfile.read.filter.basic.Filter;
+import org.apache.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ATTRIBUTE;
+import static 
org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.MEASUREMENT;
+import static 
org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeVisitor.getTimePartitionSlotList;
+import static 
org.apache.iotdb.db.queryengine.plan.relational.planner.ir.GlobalTimePredicateExtractVisitor.extractGlobalTimeFilter;
+
+/**
+ * After the optimized rule {@link SimplifyExpressions} finished, predicate 
expression in FilterNode
+ * has been transformed to conjunctive normal forms(CNF).
+ *
+ * <p>1. In this class, we examine each expression in CNFs, determine how to 
use it, in metadata
+ * query, or pushed down into ScanOperators, or it can only be used in 
FilterNode above with
+ * TableScanNode.
+ * <li>For metadata query expressions, it will be used in {@code 
tableIndexScan} method to generate
+ *     the deviceEntries and DataPartition used for TableScanNode.
+ * <li>For expressions which can be pushed into TableScanNode, we will execute 
{@code
+ *     extractGlobalTimeFilter}, to extract the timePredicate and 
pushDownValuePredicate.
+ * <li>Expression which can not be pushed down into TableScanNode, will be 
used in the FilterNode
+ *     above of TableScanNode.
+ *
+ *     <p>Notice that, when aggregation, multi-table, join are introduced, 
this optimization rule
+ *     need to be adapted.
+ */
+public class PushPredicateIntoTableScan implements RelationalPlanOptimizer {
+
+  private static final IoTDBConfig CONFIG = 
IoTDBDescriptor.getInstance().getConfig();
+
+  @Override
+  public PlanNode optimize(
+      PlanNode planNode,
+      Analysis analysis,
+      Metadata metadata,
+      SessionInfo sessionInfo,
+      MPPQueryContext queryContext) {
+    return planNode.accept(new Rewriter(queryContext, analysis, metadata), new 
RewriterContext());
+  }
+
+  private static class Rewriter extends PlanVisitor<PlanNode, RewriterContext> 
{
+    private final MPPQueryContext queryContext;
+    private final Analysis analysis;
+    private final Metadata metadata;
+    private Expression predicate;
+
+    Rewriter(MPPQueryContext queryContext, Analysis analysis, Metadata 
metadata) {
+      this.queryContext = queryContext;
+      this.analysis = analysis;
+      this.metadata = metadata;
+    }
+
+    @Override
+    public PlanNode visitPlan(PlanNode node, RewriterContext context) {
+      throw new IllegalArgumentException(
+          String.format("Unexpected plan node: %s in rule 
PushPredicateIntoTableScan", node));
+    }
+
+    @Override
+    public PlanNode visitSingleChildProcess(SingleChildProcessNode node, 
RewriterContext context) {
+      PlanNode rewrittenChild = node.getChild().accept(this, context);
+      node.setChild(rewrittenChild);
+      return node;
+    }
+
+    @Override
+    public PlanNode visitMultiChildProcess(MultiChildProcessNode node, 
RewriterContext context) {
+      List<PlanNode> rewrittenChildren = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        rewrittenChildren.add(child.accept(this, context));
+      }
+      node.setChildren(rewrittenChildren);
+      return node;
+    }
+
+    @Override
+    public PlanNode visitFilter(FilterNode node, RewriterContext context) {
+
+      if (node.getPredicate() != null) {
+
+        // when exist diff function, predicate can not be pushed down into 
TableScanNode
+        if (containsDiffFunction(node.getPredicate())) {
+          node.setChild(node.getChild().accept(this, context));
+          return node;
+        }
+
+        predicate = node.getPredicate();
+
+        if (node.getChild() instanceof TableScanNode) {
+          // child of FilterNode is TableScanNode, means FilterNode must get 
from where clause
+          return combineFilterAndScan((TableScanNode) node.getChild());
+        } else {
+          // FilterNode may get from having or subquery
+          node.setChild(node.getChild().accept(this, context));
+          return node;
+        }
+
+      } else {
+        throw new IllegalStateException(
+            "Filter node has no predicate, node: " + node.getPlanNodeId());
+      }
+    }
+
+    public PlanNode combineFilterAndScan(TableScanNode tableScanNode) {
+      List<List<Expression>> splitPredicates = splitPredicate(tableScanNode);
+
+      // exist indexed metadata expressions
+      tableMetadataIndexScan(tableScanNode, splitPredicates.get(0));
+
+      // exist expressions can push down to scan operator
+      if (!splitPredicates.get(1).isEmpty()) {
+        List<Expression> expressions = splitPredicates.get(1);
+        Expression pushDownPredicate =
+            expressions.size() == 1
+                ? expressions.get(0)
+                : new LogicalExpression(LogicalExpression.Operator.AND, 
expressions);
+
+        // extract global time filter and set it to TableScanNode
+        Pair<Expression, Boolean> resultPair = 
extractGlobalTimeFilter(pushDownPredicate);
+        if (resultPair.left != null) {
+          tableScanNode.setTimePredicate(resultPair.left);
+        }
+        if (Boolean.TRUE.equals(resultPair.right)) {
+          tableScanNode.setPushDownPredicate(pushDownPredicate);
+        }
+      } else {
+        tableScanNode.setPushDownPredicate(null);
+      }
+
+      // exist expressions can not push down to scan operator
+      if (!splitPredicates.get(2).isEmpty()) {
+        List<Expression> expressions = splitPredicates.get(2);
+        return new FilterNode(
+            queryContext.getQueryId().genPlanNodeId(),
+            tableScanNode,
+            expressions.size() == 1
+                ? expressions.get(0)
+                : new LogicalExpression(LogicalExpression.Operator.AND, 
expressions));
+      }
+
+      return tableScanNode;
+    }
+
+    /**
+     * splits predicate expression in table model into three parts, index 0 
represents
+     * metadataExpressions, index 1 represents 
expressionsCanPushDownToOperator, index 2 represents
+     * expressionsCannotPushDownToOperator
+     */
+    private List<List<Expression>> splitPredicate(TableScanNode node) {
+
+      Set<String> idOrAttributeColumnNames =
+          node.getIdAndAttributeIndexMap().keySet().stream()
+              .map(Symbol::getName)
+              .collect(Collectors.toSet());
+
+      Set<String> measurementColumnNames =
+          node.getAssignments().entrySet().stream()
+              .filter(e -> 
MEASUREMENT.equals(e.getValue().getColumnCategory()))
+              .map(e -> e.getKey().getName())
+              .collect(Collectors.toSet());
+      measurementColumnNames.add("time");
+
+      List<Expression> metadataExpressions = new ArrayList<>();
+      List<Expression> expressionsCanPushDown = new ArrayList<>();
+      List<Expression> expressionsCannotPushDown = new ArrayList<>();
+
+      if (predicate instanceof LogicalExpression
+          && ((LogicalExpression) predicate).getOperator() == 
LogicalExpression.Operator.AND) {
+
+        for (Expression expression : ((LogicalExpression) 
predicate).getTerms()) {
+          if (PredicatePushIntoMetadataChecker.check(idOrAttributeColumnNames, 
expression)) {
+            metadataExpressions.add(expression);
+          } else if (PredicateCombineIntoTableScanChecker.check(
+              measurementColumnNames, expression)) {
+            expressionsCanPushDown.add(expression);
+          } else {
+            expressionsCannotPushDown.add(expression);
+          }
+        }
+
+        return Arrays.asList(
+            metadataExpressions, expressionsCanPushDown, 
expressionsCannotPushDown);
+      }
+
+      if (PredicatePushIntoMetadataChecker.check(idOrAttributeColumnNames, 
predicate)) {
+        metadataExpressions.add(predicate);
+      } else if 
(PredicateCombineIntoTableScanChecker.check(measurementColumnNames, predicate)) 
{
+        expressionsCanPushDown.add(predicate);
+      } else {
+        expressionsCannotPushDown.add(predicate);
+      }
+
+      return Arrays.asList(metadataExpressions, expressionsCanPushDown, 
expressionsCannotPushDown);
+    }
+
+    @Override
+    public PlanNode visitTableScan(TableScanNode node, RewriterContext 
context) {
+      tableMetadataIndexScan(node, Collections.emptyList());
+      return node;
+    }
+
+    /** Get deviceEntries and DataPartition used in TableScan. */
+    private void tableMetadataIndexScan(TableScanNode node, List<Expression> 
metadataExpressions) {
+      List<String> attributeColumns =
+          node.getOutputSymbols().stream()
+              .filter(
+                  symbol -> 
ATTRIBUTE.equals(node.getAssignments().get(symbol).getColumnCategory()))
+              .map(Symbol::getName)
+              .collect(Collectors.toList());
+      List<DeviceEntry> deviceEntries =
+          metadata.indexScan(node.getQualifiedObjectName(), 
metadataExpressions, attributeColumns);
+      node.setDeviceEntries(deviceEntries);
+
+      if (deviceEntries.isEmpty()) {
+        analysis.setFinishQueryAfterAnalyze();
+        analysis.setEmptyDataSource(true);
+      } else {
+        Filter timeFilter =
+            node.getTimePredicate().isPresent()
+                ? node.getTimePredicate()
+                    .get()
+                    .accept(new ConvertPredicateToTimeFilterVisitor(), null)
+                : null;
+        node.setTimeFilter(timeFilter);
+        String treeModelDatabase = "root." + 
node.getQualifiedObjectName().getDatabaseName();
+        DataPartition dataPartition =
+            fetchDataPartitionByDevices(treeModelDatabase, deviceEntries, 
timeFilter);
+
+        if (dataPartition.getDataPartitionMap().size() > 1) {
+          throw new IllegalStateException(
+              "Table model can only process data only in one database yet!");
+        }
+
+        if (dataPartition.getDataPartitionMap().isEmpty()) {
+          analysis.setFinishQueryAfterAnalyze();
+          analysis.setEmptyDataSource(true);
+        } else {
+          analysis.upsertDataPartition(dataPartition);
+        }
+      }
+    }
+
+    private DataPartition fetchDataPartitionByDevices(
+        String database, List<DeviceEntry> deviceEntries, Filter 
globalTimeFilter) {
+      Pair<List<TTimePartitionSlot>, Pair<Boolean, Boolean>> res =
+          getTimePartitionSlotList(globalTimeFilter, queryContext);
+
+      // there is no satisfied time range
+      if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) {
+        return new DataPartition(
+            Collections.emptyMap(),
+            CONFIG.getSeriesPartitionExecutorClass(),
+            CONFIG.getSeriesPartitionSlotNum());
+      }
+
+      List<DataPartitionQueryParam> dataPartitionQueryParams =
+          deviceEntries.stream()
+              .map(
+                  deviceEntry ->
+                      new DataPartitionQueryParam(
+                          deviceEntry.getDeviceID(), res.left, res.right.left, 
res.right.right))
+              .collect(Collectors.toList());
+
+      if (res.right.left || res.right.right) {
+        return metadata.getDataPartitionWithUnclosedTimeRange(database, 
dataPartitionQueryParams);
+      } else {
+        return metadata.getDataPartition(database, dataPartitionQueryParams);
+      }
+    }
+  }
+
+  static boolean containsDiffFunction(Expression expression) {
+    if (expression instanceof FunctionCall
+        && "diff".equalsIgnoreCase(((FunctionCall) 
expression).getName().toString())) {
+      return true;
+    }
+
+    if (!expression.getChildren().isEmpty()) {
+      for (Node node : expression.getChildren()) {
+        if (containsDiffFunction((Expression) node)) {
+          return true;
+        }
+      }
+    }
+
+    return false;
+  }
+
+  private static class RewriterContext {}
+}
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/SimplifyExpressions.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/SimplifyExpressions.java
index 7d308c2cf10..76c83a0aa0c 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/SimplifyExpressions.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/SimplifyExpressions.java
@@ -52,8 +52,8 @@ public class SimplifyExpressions implements 
RelationalPlanOptimizer {
 
     @Override
     public PlanNode visitFilter(FilterNode node, RewriterContext context) {
-      Expression predicate = normalizeOrExpression(node.getPredicate());
-      predicate = extractCommonPredicates(predicate);
+      Expression predicate = extractCommonPredicates(node.getPredicate());
+      predicate = normalizeOrExpression(predicate);
       node.setPredicate(predicate);
       return node;
     }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/QualifiedName.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/QualifiedName.java
index db004cf891b..6b687145b96 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/QualifiedName.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/QualifiedName.java
@@ -114,7 +114,7 @@ public class QualifiedName {
   }
 
   /**
-   * For an identifier of the form "a.b.c.d", returns "a.b.c" For an 
identifier of the form "a",
+   * For an identifier of the form "a.b.c.d", returns "a.b.c", For an 
identifier of the form "a",
    * returns absent
    */
   public Optional<QualifiedName> getPrefix() {
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java
index dbd057665ad..0369e96d111 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java
@@ -49,6 +49,7 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser;
 
+import org.junit.Ignore;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -67,7 +68,9 @@ import static 
org.apache.tsfile.read.common.type.DoubleType.DOUBLE;
 import static org.apache.tsfile.read.common.type.IntType.INT32;
 import static org.apache.tsfile.read.common.type.LongType.INT64;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.eq;
@@ -165,7 +168,7 @@ public class AnalyzerTest {
   @Test
   public void singleTableNoFilterTest() {
     // wildcard
-    sql = "SELECT * FROM table1";
+    sql = "SELECT * FROM testdb.table1";
     actualAnalysis = analyzeSQL(sql, metadata);
     assertNotNull(actualAnalysis);
     assertEquals(1, actualAnalysis.getTables().size());
@@ -177,7 +180,7 @@ public class AnalyzerTest {
     assertTrue(rootNode instanceof OutputNode);
     assertTrue(((OutputNode) rootNode).getChild() instanceof TableScanNode);
     tableScanNode = (TableScanNode) ((OutputNode) rootNode).getChild();
-    assertEquals("table1", tableScanNode.getQualifiedTableName());
+    assertEquals("testdb.table1", 
tableScanNode.getQualifiedObjectName().toString());
     assertEquals(
         Arrays.asList("time", "tag1", "tag2", "tag3", "attr1", "attr2", "s1", 
"s2", "s3"),
         tableScanNode.getOutputColumnNames());
@@ -188,13 +191,11 @@ public class AnalyzerTest {
     assertEquals(ASC, tableScanNode.getScanOrder());
 
     distributionPlanner = new TableDistributionPlanner(actualAnalysis, 
logicalQueryPlan, context);
-    DistributedQueryPlan distributedQueryPlan = distributionPlanner.plan();
-    assertEquals(4, distributedQueryPlan.getInstances().size());
   }
 
   @Test
-  public void singleTableWithFilterTest() {
-    // global time filter
+  public void singleTableWithFilterTest1() {
+    // only global time filter
     sql = "SELECT * FROM table1 where time > 1";
     actualAnalysis = analyzeSQL(sql, metadata);
     assertNotNull(actualAnalysis);
@@ -204,28 +205,24 @@ public class AnalyzerTest {
     logicalQueryPlan = logicalPlanner.plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
     assertTrue(rootNode instanceof OutputNode);
-
-    // TODO change back after Gaofei update FilterAndScanCombineRule
-    /*assertTrue(((OutputNode) rootNode).getChild() instanceof TableScanNode);
-    tableScanNode = (TableScanNode) ((OutputNode) rootNode).getChild();*/
-    assertTrue(((OutputNode) rootNode).getChild() instanceof FilterNode);
-    FilterNode filterNode = (FilterNode) ((OutputNode) rootNode).getChild();
-    assertEquals("(\"time\" > 1)", filterNode.getPredicate().toString());
-
-    tableScanNode = (TableScanNode) (filterNode).getChild();
-    assertEquals("table1", tableScanNode.getQualifiedTableName());
+    assertTrue(((OutputNode) rootNode).getChild() instanceof TableScanNode);
+    tableScanNode = (TableScanNode) ((OutputNode) rootNode).getChild();
+    assertEquals("testdb.table1", 
tableScanNode.getQualifiedObjectName().toString());
     assertEquals(
         Arrays.asList("time", "tag1", "tag2", "tag3", "attr1", "attr2", "s1", 
"s2", "s3"),
         tableScanNode.getOutputColumnNames());
     assertEquals(9, tableScanNode.getAssignments().size());
     assertEquals(1, tableScanNode.getDeviceEntries().size());
     assertEquals(5, tableScanNode.getIdAndAttributeIndexMap().size());
+    assertEquals("(\"time\" > 1)", 
tableScanNode.getTimePredicate().get().toString());
+    assertNull(tableScanNode.getPushDownPredicate());
     assertEquals(ASC, tableScanNode.getScanOrder());
     distributionPlanner = new TableDistributionPlanner(actualAnalysis, 
logicalQueryPlan, context);
-    distributedQueryPlan = distributionPlanner.plan();
-    assertEquals(4, distributedQueryPlan.getInstances().size());
+  }
 
-    // value filter which can be pushed down
+  @Test
+  public void singleTableWithFilterTest2() {
+    // measurement value filter, which can be pushed down to TableScanNode
     sql = "SELECT tag1, attr1, s2 FROM table1 where s1 > 1";
     actualAnalysis = analyzeSQL(sql, metadata);
     assertNotNull(actualAnalysis);
@@ -237,9 +234,91 @@ public class AnalyzerTest {
     assertTrue(rootNode instanceof OutputNode);
     assertTrue(rootNode.getChildren().get(0) instanceof TableScanNode);
     tableScanNode = (TableScanNode) rootNode.getChildren().get(0);
+    assertNotNull(tableScanNode.getPushDownPredicate());
+    assertEquals("(\"s1\" > 1)", 
tableScanNode.getPushDownPredicate().toString());
+    assertFalse(tableScanNode.getTimePredicate().isPresent());
+    assertEquals(
+        Arrays.asList("time", "tag1", "attr1", "s1", "s2"), 
tableScanNode.getOutputColumnNames());
+  }
+
+  @Test
+  public void singleTableWithFilterTest3() {
+    // measurement value filter with time filter, take apart into 
pushDownPredicate and
+    // timePredicate of TableScanNode
+    sql =
+        "SELECT tag1, attr1, s2 FROM table1 where s1 > 1 and s2>2 and tag1='A' 
and time > 1 and time < 10";
+    actualAnalysis = analyzeSQL(sql, metadata);
+    assertNotNull(actualAnalysis);
+    assertEquals(1, actualAnalysis.getTables().size());
+    context = new MPPQueryContext(sql, queryId, sessionInfo, null, null);
+    logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP);
+    logicalQueryPlan = logicalPlanner.plan(actualAnalysis);
+    rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode instanceof OutputNode);
+    assertTrue(rootNode.getChildren().get(0) instanceof TableScanNode);
+    tableScanNode = (TableScanNode) rootNode.getChildren().get(0);
+    assertNotNull(tableScanNode.getPushDownPredicate());
+    assertEquals(
+        "((\"s1\" > 1) AND (\"s2\" > 2))", 
tableScanNode.getPushDownPredicate().toString());
+    assertTrue(tableScanNode.getTimePredicate().isPresent());
+    assertEquals(
+        "((\"time\" > 1) AND (\"time\" < 10))", 
tableScanNode.getTimePredicate().get().toString());
+    assertEquals(
+        Arrays.asList("time", "tag1", "attr1", "s1", "s2"), 
tableScanNode.getOutputColumnNames());
+  }
+
+  @Test
+  public void singleTableWithFilterTest4() {
+    // measurement value filter with time filter
+    // transfer to : ((("time" > 1) OR ("s1" > 1)) AND (("time" > 1) OR ("s2" 
> 2)) AND (("time" >
+    // 1) OR ("time" < 10)))
+    sql = "SELECT tag1, attr1, s2 FROM table1 where time > 1 or s1 > 1 and s2 
> 2 and time < 10";
+    actualAnalysis = analyzeSQL(sql, metadata);
+    assertNotNull(actualAnalysis);
+    assertEquals(1, actualAnalysis.getTables().size());
+    context = new MPPQueryContext(sql, queryId, sessionInfo, null, null);
+    logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP);
+    logicalQueryPlan = logicalPlanner.plan(actualAnalysis);
+    rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode instanceof OutputNode);
+    assertTrue(rootNode.getChildren().get(0) instanceof TableScanNode);
+    tableScanNode = (TableScanNode) rootNode.getChildren().get(0);
+    assertTrue(tableScanNode.getTimePredicate().isPresent());
+    assertEquals(
+        "((\"time\" > 1) OR (\"time\" < 10))", 
tableScanNode.getTimePredicate().get().toString());
+    assertNotNull(tableScanNode.getPushDownPredicate());
+    assertEquals(
+        "(((\"time\" > 1) OR (\"s1\" > 1)) AND ((\"time\" > 1) OR (\"s2\" > 
2)))",
+        tableScanNode.getPushDownPredicate().toString());
+    assertEquals(
+        Arrays.asList("time", "tag1", "attr1", "s1", "s2"), 
tableScanNode.getOutputColumnNames());
+  }
+
+  @Test
+  public void singleTableWithFilterTest5() {
+    // measurement value filter with time filter
+    sql = "SELECT tag1, attr1, s2 FROM table1 where time > 1 or s1 > 1 or time 
< 10 or s2 > 2";
+    actualAnalysis = analyzeSQL(sql, metadata);
+    assertNotNull(actualAnalysis);
+    assertEquals(1, actualAnalysis.getTables().size());
+    context = new MPPQueryContext(sql, queryId, sessionInfo, null, null);
+    logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP);
+    logicalQueryPlan = logicalPlanner.plan(actualAnalysis);
+    rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode instanceof OutputNode);
+    assertTrue(rootNode.getChildren().get(0) instanceof TableScanNode);
+    tableScanNode = (TableScanNode) rootNode.getChildren().get(0);
+    assertNotNull(tableScanNode.getPushDownPredicate());
+    assertEquals(
+        "((\"time\" > 1) OR (\"s1\" > 1) OR (\"time\" < 10) OR (\"s2\" > 2))",
+        tableScanNode.getPushDownPredicate().toString());
+    assertFalse(tableScanNode.getTimePredicate().isPresent());
     assertEquals(
         Arrays.asList("time", "tag1", "attr1", "s1", "s2"), 
tableScanNode.getOutputColumnNames());
+  }
 
+  @Test
+  public void singleTableWithFilterTest6() {
     // value filter which can not be pushed down
     sql = "SELECT tag1, attr1, s2 FROM table1 where diff(s1) > 1";
     actualAnalysis = analyzeSQL(sql, metadata);
@@ -264,12 +343,23 @@ public class AnalyzerTest {
     rootNode = logicalQueryPlan.getRootNode();
     assertTrue(rootNode instanceof OutputNode);
     assertTrue(rootNode.getChildren().get(0) instanceof FilterNode);
+    FilterNode filterNode = (FilterNode) rootNode.getChildren().get(0);
+    assertEquals("((diff(\"s1\") + 1) > 1)", 
filterNode.getPredicate().toString());
     assertTrue(rootNode.getChildren().get(0).getChildren().get(0) instanceof 
TableScanNode);
     tableScanNode = (TableScanNode) 
rootNode.getChildren().get(0).getChildren().get(0);
     assertEquals(
         Arrays.asList("time", "tag1", "attr1", "s1", "s2"), 
tableScanNode.getOutputColumnNames());
   }
 
+  @Ignore
+  @Test
+  public void singleTableWithFilterTest00() {
+    // TODO(beyyes) fix the CNFs parse error
+    sql =
+        "SELECT tag1, attr1, s2 FROM table1 where (time > 1 and s1 > 1 or s2 < 
7) or (time < 10 and s1 > 4)";
+    actualAnalysis = analyzeSQL(sql, metadata);
+  }
+
   @Test
   public void singleTableProjectTest() {
     // 1. project without filter
@@ -298,6 +388,9 @@ public class AnalyzerTest {
     rootNode = logicalQueryPlan.getRootNode();
     assertTrue(rootNode instanceof OutputNode);
     assertTrue(rootNode.getChildren().get(0) instanceof TableScanNode);
+    TableScanNode tableScanNode = (TableScanNode) 
rootNode.getChildren().get(0);
+    assertFalse(tableScanNode.getTimePredicate().isPresent());
+    assertEquals("(\"s2\" = 8)", 
tableScanNode.getPushDownPredicate().toString());
     tableScanNode = (TableScanNode) rootNode.getChildren().get(0);
     assertEquals(
         Arrays.asList("time", "tag1", "tag2", "attr1", "s1", "s2"),
@@ -312,9 +405,17 @@ public class AnalyzerTest {
         new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP)
             .plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode);
     assertTrue(rootNode.getChildren().get(0).getChildren().get(0) instanceof 
FilterNode);
+    FilterNode filterNode = (FilterNode) 
rootNode.getChildren().get(0).getChildren().get(0);
+    assertEquals("(REPLACE(\"tag1\", 'low', '!') = '!')", 
filterNode.getPredicate().toString());
+    assertTrue(
+        rootNode.getChildren().get(0).getChildren().get(0).getChildren().get(0)
+            instanceof TableScanNode);
     tableScanNode =
         (TableScanNode) 
rootNode.getChildren().get(0).getChildren().get(0).getChildren().get(0);
+    assertFalse(tableScanNode.getTimePredicate().isPresent());
+    assertNull(tableScanNode.getPushDownPredicate());
     assertEquals(
         Arrays.asList("time", "tag1", "attr2", "s1", "s2", "s3"),
         tableScanNode.getOutputColumnNames());
@@ -345,6 +446,15 @@ public class AnalyzerTest {
         new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP)
             .plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode.getChildren().get(0) instanceof FilterNode);
+    FilterNode filterNode = (FilterNode) rootNode.getChildren().get(0);
+    assertEquals(
+        "((\"tag1\" IS NOT NULL) AND (\"s1\" IS NULL))", 
filterNode.getPredicate().toString());
+    assertTrue(rootNode.getChildren().get(0).getChildren().get(0) instanceof 
TableScanNode);
+    TableScanNode tableScanNode =
+        (TableScanNode) rootNode.getChildren().get(0).getChildren().get(0);
+    assertNull(tableScanNode.getPushDownPredicate());
+    assertFalse(tableScanNode.getTimePredicate().isPresent());
 
     // 2. like
     sql = "SELECT * FROM table1 WHERE tag1 like '%m'";
@@ -354,6 +464,13 @@ public class AnalyzerTest {
         new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP)
             .plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode.getChildren().get(0) instanceof FilterNode);
+    filterNode = (FilterNode) rootNode.getChildren().get(0);
+    assertEquals("(\"tag1\" LIKE '%m')", filterNode.getPredicate().toString());
+    assertTrue(rootNode.getChildren().get(0).getChildren().get(0) instanceof 
TableScanNode);
+    tableScanNode = (TableScanNode) 
rootNode.getChildren().get(0).getChildren().get(0);
+    assertNull(tableScanNode.getPushDownPredicate());
+    assertFalse(tableScanNode.getTimePredicate().isPresent());
 
     // 3. in / not in
     sql =
@@ -364,6 +481,19 @@ public class AnalyzerTest {
         new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP)
             .plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode);
+    assertTrue(rootNode.getChildren().get(0).getChildren().get(0) instanceof 
FilterNode);
+    filterNode = (FilterNode) 
rootNode.getChildren().get(0).getChildren().get(0);
+    assertEquals(
+        "((\"tag1\" IN \"tag1\") AND (NOT (\"tag2\" IN \"tag2\")))",
+        filterNode.getPredicate().toString());
+    assertTrue(
+        rootNode.getChildren().get(0).getChildren().get(0).getChildren().get(0)
+            instanceof TableScanNode);
+    tableScanNode =
+        (TableScanNode) 
rootNode.getChildren().get(0).getChildren().get(0).getChildren().get(0);
+    assertNull(tableScanNode.getPushDownPredicate());
+    assertFalse(tableScanNode.getTimePredicate().isPresent());
 
     // 4. not
     sql = "SELECT * FROM table1 WHERE tag1 not like '%m'";
@@ -409,14 +539,34 @@ public class AnalyzerTest {
     logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP);
     logicalQueryPlan = logicalPlanner.plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
+  }
 
-    // 5. diff
+  @Test
+  public void diffTest() {
+    // 1. only diff
     sql = "SELECT DIFF(s1) FROM table1 WHERE DIFF(s2) > 0";
     context = new MPPQueryContext(sql, queryId, sessionInfo, null, null);
     actualAnalysis = analyzeSQL(sql, metadata);
     logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP);
     logicalQueryPlan = logicalPlanner.plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode);
+    assertTrue(rootNode.getChildren().get(0).getChildren().get(0) instanceof 
FilterNode);
+    FilterNode filterNode = (FilterNode) 
rootNode.getChildren().get(0).getChildren().get(0);
+    assertEquals("(DIFF(\"s2\") > 0)", filterNode.getPredicate().toString());
+
+    // 2. diff with time filter, tag filter and measurement filter
+    sql = "SELECT s1 FROM table1 WHERE DIFF(s2) > 0 and time > 5 and tag1 = 
'A' and s1 = 1";
+    context = new MPPQueryContext(sql, queryId, sessionInfo, null, null);
+    actualAnalysis = analyzeSQL(sql, metadata);
+    logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP);
+    logicalQueryPlan = logicalPlanner.plan(actualAnalysis);
+    rootNode = logicalQueryPlan.getRootNode();
+    assertTrue(rootNode.getChildren().get(0) instanceof FilterNode);
+    filterNode = (FilterNode) rootNode.getChildren().get(0);
+    assertEquals(
+        "((DIFF(\"s2\") > 0) AND (\"time\" > 5) AND (\"tag1\" = 'A') AND 
(\"s1\" = 1))",
+        filterNode.getPredicate().toString());
   }
 
   @Test
@@ -490,8 +640,6 @@ public class AnalyzerTest {
         new LogicalPlanner(context, metadata, sessionInfo, 
WarningCollector.NOOP)
             .plan(actualAnalysis);
     rootNode = logicalQueryPlan.getRootNode();
-    distributedQueryPlan =
-        new TableDistributionPlanner(actualAnalysis, logicalQueryPlan, 
context).plan();
   }
 
   public static Analysis analyzeSQL(String sql, Metadata metadata) {
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
index f0425d91dc7..55fbeacb86c 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
@@ -47,6 +47,7 @@ import org.apache.tsfile.read.common.type.BinaryType;
 import org.apache.tsfile.read.common.type.Type;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -194,7 +195,7 @@ public class TestMatadata implements Metadata {
       QualifiedObjectName tableName,
       List<Expression> expressionList,
       List<String> attributeColumns) {
-    return Arrays.asList(
+    return Collections.singletonList(
         new DeviceEntry(
             new StringArrayDeviceID("root.testdb", "table1", "t1", "t2", "t3"),
             Arrays.asList("a1", "a2")));
@@ -228,6 +229,18 @@ public class TestMatadata implements Metadata {
     return null;
   }
 
+  @Override
+  public DataPartition getDataPartition(
+      String database, List<DataPartitionQueryParam> sgNameToQueryParamsMap) {
+    return DATA_PARTITION;
+  }
+
+  @Override
+  public DataPartition getDataPartitionWithUnclosedTimeRange(
+      String database, List<DataPartitionQueryParam> sgNameToQueryParamsMap) {
+    return DATA_PARTITION;
+  }
+
   public static boolean isTwoNumericType(List<? extends Type> argumentTypes) {
     return argumentTypes.size() == 2
         && isNumericType(argumentTypes.get(0))
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
index 926dca39697..edeed746f13 100644
--- 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
@@ -34,6 +34,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static java.util.Objects.requireNonNull;
 import static java.util.stream.Collectors.toList;
 
 public class DataPartition extends Partition {
@@ -131,6 +132,19 @@ public class DataPartition extends Partition {
         .collect(toList());
   }
 
+  /**
+   * For table model usage.
+   *
+   * <p>The database shall start with "root.". Concat this to a user-provided 
db name if necessary.
+   *
+   * <p>The device id shall be [table, seg1, ....]
+   */
+  public List<TRegionReplicaSet> getDataRegionReplicaSetWithTimeFilter(
+      String database, IDeviceID deviceID, Filter timeFilter) {
+    // TODO implement this interface, @Potato
+    throw new UnsupportedOperationException();
+  }
+
   public List<TRegionReplicaSet> getDataRegionReplicaSet(
       IDeviceID deviceID, TTimePartitionSlot tTimePartitionSlot) {
     String storageGroup = getStorageGroupByDevice(deviceID);
@@ -231,4 +245,30 @@ public class DataPartition extends Partition {
         });
     return new ArrayList<>(distributionMap.values());
   }
+
+  public void upsertDataPartition(DataPartition targetDataPartition) {
+    requireNonNull(this.dataPartitionMap, "dataPartitionMap is null");
+
+    for (Map.Entry<
+            String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, 
List<TRegionReplicaSet>>>>
+        dbEntry : targetDataPartition.getDataPartitionMap().entrySet()) {
+      String database = dbEntry.getKey();
+      if (dataPartitionMap.containsKey(database)) {
+        Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, 
List<TRegionReplicaSet>>> innerMap1 =
+            dataPartitionMap.get(database);
+        for (Map.Entry<TSeriesPartitionSlot, Map<TTimePartitionSlot, 
List<TRegionReplicaSet>>>
+            seriesSlotEntry : dbEntry.getValue().entrySet()) {
+          TSeriesPartitionSlot seriesSlot = seriesSlotEntry.getKey();
+          if (innerMap1.containsKey(seriesSlot)) {
+            Map<TTimePartitionSlot, List<TRegionReplicaSet>> innerMap2 = 
innerMap1.get(seriesSlot);
+            innerMap2.putAll(seriesSlotEntry.getValue());
+          } else {
+            innerMap1.put(seriesSlot, seriesSlotEntry.getValue());
+          }
+        }
+      } else {
+        dataPartitionMap.put(database, dbEntry.getValue());
+      }
+    }
+  }
 }

Reply via email to