You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by zy...@apache.org on 2023/05/24 11:07:22 UTC

[iotdb] branch master updated: Support using logical view in query aligned by time. (#9874)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 26a795172c5 Support using logical view in query aligned by time.  (#9874)
26a795172c5 is described below

commit 26a795172c54d0f1c6defc803a040e1a63b69afe
Author: 橘子 <70...@users.noreply.github.com>
AuthorDate: Wed May 24 19:07:15 2023 +0800

    Support using logical view in query aligned by time.  (#9874)
---
 .../schema/view/viewExpression/ViewExpression.java |   2 +
 .../impl/write/CreateLogicalViewPlanImpl.java      |   6 +-
 .../impl/write/SchemaRegionWritePlanFactory.java   |   2 +
 .../schemaregion/SchemaRegionMemoryImpl.java       |  10 +
 .../mpp/common/schematree/ClusterSchemaTree.java   |  27 ++-
 .../db/mpp/common/schematree/ISchemaTree.java      |   7 +
 .../apache/iotdb/db/mpp/plan/analyze/Analysis.java |  27 +++
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  | 263 ++++++++++++++++++---
 .../db/mpp/plan/analyze/ExpressionAnalyzer.java    |  37 +++
 .../iotdb/db/mpp/plan/expression/Expression.java   |  20 ++
 .../plan/expression/binary/BinaryExpression.java   |  22 ++
 .../plan/expression/binary/WhenThenExpression.java |   7 +
 .../db/mpp/plan/expression/leaf/LeafOperand.java   |   5 +
 .../plan/expression/multi/FunctionExpression.java  |  21 ++
 .../expression/other/CaseWhenThenExpression.java   |  17 ++
 .../plan/expression/ternary/BetweenExpression.java |   9 +
 .../plan/expression/ternary/TernaryExpression.java |  18 +-
 .../db/mpp/plan/expression/unary/InExpression.java |  26 +-
 .../plan/expression/unary/IsNullExpression.java    |   5 +
 .../mpp/plan/expression/unary/LikeExpression.java  |   5 +
 .../plan/expression/unary/LogicNotExpression.java  |   9 +
 .../plan/expression/unary/NegationExpression.java  |  11 +
 .../plan/expression/unary/RegularExpression.java   |   5 +
 .../mpp/plan/expression/unary/UnaryExpression.java |   6 +-
 .../visitor/CompleteMeasurementSchemaVisitor.java  | 112 +++++++++
 .../RemoveWildcardAndViewInExpressionVisitor.java  |  95 ++++++++
 .../RemoveWildcardAndViewInFilterVisitor.java      | 109 +++++++++
 .../visitor/ReplaceLogicalViewVisitor.java         | 188 +++++++++++++++
 .../metadata/CreateLogicalViewStatement.java       |  26 +-
 .../iotdb/db/mpp/plan/analyze/AnalyzeTest.java     |   2 +-
 .../mpp/plan/analyze/ExpressionAnalyzerTest.java   |   4 +-
 31 files changed, 1043 insertions(+), 60 deletions(-)

diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/view/viewExpression/ViewExpression.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/view/viewExpression/ViewExpression.java
index 2961fbb6596..0c149023cb1 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/schema/view/viewExpression/ViewExpression.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/view/viewExpression/ViewExpression.java
@@ -126,6 +126,7 @@ public abstract class ViewExpression {
     switch (type) {
       case -4:
         expression = new ConstantViewOperand(byteBuffer);
+        break;
       case -3:
         expression = new TimestampViewOperand(byteBuffer);
         break;
@@ -223,6 +224,7 @@ public abstract class ViewExpression {
       switch (type) {
         case -4:
           expression = new ConstantViewOperand(inputStream);
+          break;
         case -3:
           expression = new TimestampViewOperand(inputStream);
           break;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/CreateLogicalViewPlanImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/CreateLogicalViewPlanImpl.java
index f0860fb20d8..8cdcfe6611c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/CreateLogicalViewPlanImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/CreateLogicalViewPlanImpl.java
@@ -30,8 +30,10 @@ import java.util.Map;
 
 public class CreateLogicalViewPlanImpl implements ICreateLogicalViewPlan {
 
-  private PartialPath targetPath;
-  private ViewExpression sourceExpression;
+  private PartialPath targetPath = null;
+  private ViewExpression sourceExpression = null;
+
+  public CreateLogicalViewPlanImpl() {}
 
   public CreateLogicalViewPlanImpl(PartialPath targetPath, ViewExpression sourceExpression) {
     this.targetPath = targetPath;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/SchemaRegionWritePlanFactory.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/SchemaRegionWritePlanFactory.java
index fa90207d79f..e8491905240 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/SchemaRegionWritePlanFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/write/SchemaRegionWritePlanFactory.java
@@ -71,6 +71,8 @@ public class SchemaRegionWritePlanFactory {
         return new RollbackPreDeactivateTemplatePlanImpl();
       case DEACTIVATE_TEMPLATE:
         return new DeactivateTemplatePlanImpl();
+      case CREATE_LOGICAL_VIEW:
+        return new CreateLogicalViewPlanImpl();
       default:
         throw new UnsupportedOperationException(
             String.format(
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
index 510102f0954..8b24cfc508c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
@@ -1384,5 +1384,15 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
         return new RecoverOperationResult(e);
       }
     }
+
+    public RecoverOperationResult visitCreateLogicalView(
+        ICreateLogicalViewPlan createLogicalViewPlan, SchemaRegionMemoryImpl context) {
+      try {
+        createLogicalView(createLogicalViewPlan);
+        return RecoverOperationResult.SUCCESS;
+      } catch (MetadataException e) {
+        return new RecoverOperationResult(e);
+      }
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java
index 3100acd82e5..5fbb080c400 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java
@@ -56,6 +56,9 @@ public class ClusterSchemaTree implements ISchemaTree {
 
   private final SchemaNode root;
 
+  /** a flag recording whether there is logical view in this schema tree. */
+  private boolean hasLogicalMeasurementPath = false;
+
   public ClusterSchemaTree() {
     root = new SchemaInternalNode(PATH_ROOT);
   }
@@ -221,6 +224,7 @@ public class ClusterSchemaTree implements ISchemaTree {
           }
           measurementNode.setTagMap(tagMap);
           child = measurementNode;
+          this.hasLogicalMeasurementPath = true;
         } else if (i == nodes.length - 2) {
           SchemaEntityNode entityNode = new SchemaEntityNode(nodes[i]);
           entityNode.setAligned(isAligned);
@@ -241,7 +245,21 @@ public class ClusterSchemaTree implements ISchemaTree {
     }
   }
 
+  @Override
+  public void mergeSchemaTree(ISchemaTree schemaTree) {
+    if (schemaTree instanceof ClusterSchemaTree) {
+      this.mergeSchemaTree((ClusterSchemaTree) schemaTree);
+    }
+  }
+
+  @Override
+  public boolean hasLogicalViewMeasurement() {
+    return this.hasLogicalMeasurementPath;
+  }
+
   public void mergeSchemaTree(ClusterSchemaTree schemaTree) {
+    this.hasLogicalMeasurementPath =
+        this.hasLogicalMeasurementPath || schemaTree.hasLogicalViewMeasurement();
     traverseAndMerge(this.root, null, schemaTree.root);
   }
 
@@ -285,12 +303,16 @@ public class ClusterSchemaTree implements ISchemaTree {
     int childNum;
     Deque<SchemaNode> stack = new ArrayDeque<>();
     SchemaNode child;
+    boolean hasLogicalView = false;
 
     while (inputStream.available() > 0) {
       nodeType = ReadWriteIOUtils.readByte(inputStream);
       if (nodeType == SCHEMA_MEASUREMENT_NODE) {
         SchemaMeasurementNode measurementNode = SchemaMeasurementNode.deserialize(inputStream);
         stack.push(measurementNode);
+        if (measurementNode.isLogicalView()) {
+          hasLogicalView = true;
+        }
       } else {
         SchemaInternalNode internalNode;
         if (nodeType == SCHEMA_ENTITY_NODE) {
@@ -316,7 +338,9 @@ public class ClusterSchemaTree implements ISchemaTree {
         stack.push(internalNode);
       }
     }
-    return new ClusterSchemaTree(stack.poll());
+    ClusterSchemaTree result = new ClusterSchemaTree(stack.poll());
+    result.hasLogicalMeasurementPath = hasLogicalView;
+    return result;
   }
 
   /**
@@ -347,6 +371,7 @@ public class ClusterSchemaTree implements ISchemaTree {
     return databases;
   }
 
+  @Override
   public void setDatabases(Set<String> databases) {
     this.databases = databases;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ISchemaTree.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ISchemaTree.java
index 61e9bf02266..a69e0b71c60 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ISchemaTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ISchemaTree.java
@@ -65,5 +65,12 @@ public interface ISchemaTree {
 
   Set<String> getDatabases();
 
+  void setDatabases(Set<String> databases);
+
   boolean isEmpty();
+
+  void mergeSchemaTree(ISchemaTree schemaTree);
+
+  /** @return if there is measurementPath in this schema tree, return true; else return false */
+  boolean hasLogicalViewMeasurement();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
index 3ff74d63987..db2ae86e9d6 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
@@ -230,6 +230,13 @@ public class Analysis {
   // generated by combine the input path pattern and template set path
   private List<PartialPath> specifiedTemplateRelatedPathPatternList;
 
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // Logical View Analysis
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  private boolean hasViewsInQuery = false;
+  private List<Pair<Expression, String>> outputExpressions = null;
+
   /////////////////////////////////////////////////////////////////////////////////////////////////
   // Show Queries Analysis
   /////////////////////////////////////////////////////////////////////////////////////////////////
@@ -668,4 +675,24 @@ public class Analysis {
   public void setDeviceToSortItems(Map<String, List<SortItem>> deviceToSortItems) {
     this.deviceToSortItems = deviceToSortItems;
   }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // Logical View Analysis
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public void setHasViewsInQuery(boolean hasViewsInQuery) {
+    this.hasViewsInQuery = hasViewsInQuery;
+  }
+
+  public boolean hasViewsInQuery() {
+    return this.hasViewsInQuery;
+  }
+
+  public void setOutputExpressions(List<Pair<Expression, String>> outputExpressions) {
+    this.outputExpressions = outputExpressions;
+  }
+
+  public List<Pair<Expression, String>> getOutputExpressions() {
+    return this.outputExpressions;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index 1f00cfa4104..909b3e24685 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -32,6 +32,8 @@ import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.view.LogicalViewSchema;
+import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression;
 import org.apache.iotdb.commons.service.metric.PerformanceOverviewMetrics;
 import org.apache.iotdb.confignode.rpc.thrift.TGetDataNodeLocationsResp;
 import org.apache.iotdb.db.client.ConfigNodeClient;
@@ -48,6 +50,7 @@ import org.apache.iotdb.db.exception.sql.MeasurementNotExistException;
 import org.apache.iotdb.db.exception.sql.SemanticException;
 import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
 import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.view.viewExpression.visitor.GetSourcePathsVisitor;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
@@ -263,6 +266,9 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
         return finishQuery(queryStatement, analysis);
       }
 
+      // make sure paths in logical view is fetched
+      findAllViewsInTreeThenReFetchAndMerge(schemaTree);
+
       // extract global time filter from query filter and determine if there is a value filter
       analyzeGlobalTimeFilter(analysis, queryStatement);
 
@@ -345,6 +351,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
 
         analyzeInto(analysis, queryStatement, outputExpressions);
       }
+      analysis.setOutputExpressions(outputExpressions);
 
       analyzeGroupByTime(analysis, queryStatement);
 
@@ -439,11 +446,47 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
 
     for (Expression selectExpression : selectExpressions) {
       sourceExpressions.addAll(
-          ExpressionAnalyzer.removeWildcardInExpression(selectExpression, schemaTree));
+          ExpressionAnalyzer.removeWildcardAndViewInExpression(
+              selectExpression, analysis, schemaTree));
     }
     analysis.setSourceExpressions(sourceExpressions);
   }
 
+  private ISchemaTree findAllViewsInTreeThenReFetchAndMerge(ISchemaTree originSchemaTree) {
+    if (originSchemaTree.hasLogicalViewMeasurement()) {
+      PathPatternTree patternTree = new PathPatternTree();
+      GetSourcePathsVisitor getSourcePathsVisitor = new GetSourcePathsVisitor();
+      boolean needToReFetch = false;
+      try {
+        Pair<List<MeasurementPath>, Integer> tempPair =
+            originSchemaTree.searchMeasurementPaths(new PartialPath("root.**"));
+        for (MeasurementPath measurementPath : tempPair.left) {
+          if (measurementPath.getMeasurementSchema().isLogicalView()) {
+            LogicalViewSchema logicalViewSchema =
+                (LogicalViewSchema) measurementPath.getMeasurementSchema();
+            ViewExpression viewExpression = logicalViewSchema.getExpression();
+            List<PartialPath> pathsNeedToReFetch =
+                getSourcePathsVisitor.process(viewExpression, null);
+            for (PartialPath path : pathsNeedToReFetch) {
+              patternTree.appendFullPath(path);
+              needToReFetch = true;
+            }
+          }
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+      if (needToReFetch) {
+        ISchemaTree viewSchemaTree = this.schemaFetcher.fetchSchema(patternTree, null);
+        originSchemaTree.mergeSchemaTree(viewSchemaTree);
+        Set<String> allDatabases = viewSchemaTree.getDatabases();
+        allDatabases.addAll(originSchemaTree.getDatabases());
+        originSchemaTree.setDatabases(allDatabases);
+      }
+    }
+    return originSchemaTree;
+  }
+
   private Map<Integer, List<Pair<Expression, String>>> analyzeSelect(
       Analysis analysis, QueryStatement queryStatement, ISchemaTree schemaTree) {
     Map<Integer, List<Pair<Expression, String>>> outputExpressionMap = new HashMap<>();
@@ -458,17 +501,16 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     Set<String> aliasSet = new HashSet<>();
 
     int columnIndex = 0;
+
     for (ResultColumn resultColumn : queryStatement.getSelectComponent().getResultColumns()) {
       List<Pair<Expression, String>> outputExpressions = new ArrayList<>();
 
       boolean hasAlias = resultColumn.hasAlias();
       List<Expression> resultExpressions =
-          ExpressionAnalyzer.removeWildcardInExpression(resultColumn.getExpression(), schemaTree);
-      for (Expression expression : resultExpressions) {
-        // TODO: CRTODO If this expression is a logical view, replace the original expression with
-        // the parsed View Expression.
-        // And the view expression should be converted into expression.
+          ExpressionAnalyzer.removeWildcardAndViewInExpression(
+              resultColumn.getExpression(), analysis, schemaTree);
 
+      for (Expression expression : resultExpressions) {
         if (paginationController.hasCurOffset()) {
           paginationController.consumeOffset();
           continue;
@@ -483,10 +525,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
           } else {
             Expression expressionWithoutAlias =
                 ExpressionAnalyzer.removeAliasFromExpression(expression);
-            String alias =
-                !Objects.equals(expressionWithoutAlias, expression)
-                    ? expression.getExpressionString()
-                    : null;
+            String alias = expression.getStringWithViewOfThisExpression();
             alias = hasAlias ? resultColumn.getAlias() : alias;
             if (hasAlias) {
               if (aliasSet.contains(alias)) {
@@ -626,7 +665,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
 
     // get removeWildcard Expressions in Having
     List<Expression> conJunctions =
-        ExpressionAnalyzer.removeWildcardInFilter(
+        ExpressionAnalyzer.removeWildcardAndViewInFilter(
             queryStatement.getHavingCondition().getPredicate(),
             queryStatement.getFromComponent().getPrefixPaths(),
             schemaTree,
@@ -1078,7 +1117,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       return;
     }
     List<Expression> conJunctions =
-        ExpressionAnalyzer.removeWildcardInFilter(
+        ExpressionAnalyzer.removeWildcardAndViewInFilter(
             queryStatement.getWhereCondition().getPredicate(),
             queryStatement.getFromComponent().getPrefixPaths(),
             schemaTree,
@@ -1247,7 +1286,8 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     for (Expression expressionForItem : queryStatement.getExpressionSortItemList()) {
       // Expression in a sortItem only indicates one column
       List<Expression> expressions =
-          ExpressionAnalyzer.removeWildcardInExpression(expressionForItem, schemaTree);
+          ExpressionAnalyzer.removeWildcardAndViewInExpression(
+              expressionForItem, analysis, schemaTree);
       if (expressions.size() != 1) {
         throw new SemanticException("One sort item in order by should only indicate one value");
       }
@@ -1401,7 +1441,8 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       groupByExpression = groupByComponent.getControlColumnExpression();
       // Expression in group by variation clause only indicates one column
       List<Expression> expressions =
-          ExpressionAnalyzer.removeWildcardInExpression(groupByExpression, schemaTree);
+          ExpressionAnalyzer.removeWildcardAndViewInExpression(
+              groupByExpression, analysis, schemaTree);
       if (expressions.size() != 1) {
         throw new SemanticException("Expression in group by should indicate one value");
       }
@@ -3092,39 +3133,187 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     analysis.setWhereExpression(whereExpression);
   }
 
+  /**
+   * Compute how many paths exist, get the schema tree and the number of existed paths.
+   *
+   * @param pathList the path you want to check
+   * @param context the context of your analyzer
+   * @return a pair of ISchemaTree, and the number of exist paths.
+   */
+  private Pair<ISchemaTree, Integer> fetchSchemaOfPathsAndCount(
+      List<PartialPath> pathList, MPPQueryContext context) {
+    PathPatternTree pathPatternTree = new PathPatternTree();
+    for (PartialPath path : pathList) {
+      pathPatternTree.appendPathPattern(path);
+    }
+    ISchemaTree schemaTree = this.schemaFetcher.fetchSchema(pathPatternTree, context);
+
+    // search each path, make sure they all exist.
+    int numOfExistPaths = 0;
+    for (PartialPath path : pathList) {
+      Pair<List<MeasurementPath>, Integer> pathPair = schemaTree.searchMeasurementPaths(path);
+      numOfExistPaths += pathPair.left.size() > 0 ? 1 : 0;
+    }
+    return new Pair<>(schemaTree, numOfExistPaths);
+  }
+
+  /**
+   * @param pathList the paths you want to check
+   * @param schemaTree the given schema tree
+   * @return if all paths you give can be found in schema tree, return a pair of view paths and
+   *     null; else return view paths and the non-exist path.
+   */
+  private Pair<List<PartialPath>, PartialPath> findAllViewsInPaths(
+      List<PartialPath> pathList, ISchemaTree schemaTree) {
+    List<PartialPath> result = new ArrayList<>();
+    for (PartialPath path : pathList) {
+      Pair<List<MeasurementPath>, Integer> measurementPathList =
+          schemaTree.searchMeasurementPaths(path);
+      if (measurementPathList.left.size() <= 0) {
+        return new Pair<>(result, path);
+      }
+      for (MeasurementPath measurementPath : measurementPathList.left) {
+        if (measurementPath.getMeasurementSchema().isLogicalView()) {
+          result.add(measurementPath);
+        }
+      }
+    }
+    return new Pair<>(result, null);
+  }
+
+  private Pair<List<Expression>, Analysis> analyzeQueryInLogicalViewStatement(
+      Analysis analysis, QueryStatement queryStatement, MPPQueryContext context) {
+    Analysis queryAnalysis = this.visitQuery(queryStatement, context);
+    // get all expression from resultColumns
+    List<Pair<Expression, String>> outputExpressions = queryAnalysis.getOutputExpressions();
+    if (queryAnalysis.isFailed()) {
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(queryAnalysis.getFailStatus());
+      return new Pair<>(null, analysis);
+    }
+    if (outputExpressions == null) {
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.UNSUPPORTED_OPERATION.getStatusCode(),
+              "Columns in the query statement is empty. Please check your SQL."));
+      return new Pair<>(null, analysis);
+    }
+    if (queryAnalysis.hasViewsInQuery()) {
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.UNSUPPORTED_OPERATION.getStatusCode(),
+              "Can not create a logical view based on existing views. Check the query in your SQL."));
+      return new Pair<>(null, analysis);
+    }
+    List<Expression> expressionList = new ArrayList<>();
+    for (Pair<Expression, String> thisPair : outputExpressions) {
+      expressionList.add(thisPair.left);
+    }
+    return new Pair<>(expressionList, analysis);
+  }
+
+  private Analysis checkViewsInSource(
+      Analysis analysis, List<Expression> sourceExpressionList, MPPQueryContext context) {
+    List<PartialPath> pathsNeedCheck = new ArrayList<>();
+    for (Expression expression : sourceExpressionList) {
+      if (expression instanceof TimeSeriesOperand) {
+        pathsNeedCheck.add(((TimeSeriesOperand) expression).getPath());
+      }
+    }
+    Pair<ISchemaTree, Integer> schemaOfNeedToCheck =
+        fetchSchemaOfPathsAndCount(pathsNeedCheck, context);
+    if (schemaOfNeedToCheck.right != pathsNeedCheck.size()) {
+      // some source paths is not exist, and could not fetch schema.
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.UNSUPPORTED_OPERATION.getStatusCode(),
+              "Can not create a logical view based on non-exist time series."));
+      return analysis;
+    }
+    Pair<List<PartialPath>, PartialPath> viewInSourceCheckResult =
+        findAllViewsInPaths(pathsNeedCheck, schemaOfNeedToCheck.left);
+    if (viewInSourceCheckResult.right != null) {
+      // some source paths is not exist
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.UNSUPPORTED_OPERATION.getStatusCode(),
+              "Path "
+                  + viewInSourceCheckResult.right.toString()
+                  + " does not exist! You can not create a logical view based on non-exist time series."));
+      return analysis;
+    }
+    if (viewInSourceCheckResult.left.size() > 0) {
+      // some source paths is logical view
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.UNSUPPORTED_OPERATION.getStatusCode(),
+              "Can not create a logical view based on existing views."));
+      return analysis;
+    }
+    return analysis;
+  }
+
+  private Analysis checkPathsInCreateLogicalView(
+      Analysis analysis, CreateLogicalViewStatement createLogicalViewStatement) {
+    Pair<Boolean, String> checkResult = createLogicalViewStatement.checkAllPaths();
+    if (!checkResult.left) {
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.ILLEGAL_PATH.getStatusCode(),
+              "The path " + checkResult.right + " is illegal."));
+      return analysis;
+    }
+    if (createLogicalViewStatement.getSourceExpressionList().size()
+        != createLogicalViewStatement.getTargetPathList().size()) {
+      analysis.setFinishQueryAfterAnalyze(true);
+      analysis.setFailStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.UNSUPPORTED_OPERATION.getStatusCode(),
+              "The number of target and source paths are miss matched! Please check your SQL."));
+      return analysis;
+    }
+    return analysis;
+  }
+
   // create Logical View
   @Override
   public Analysis visitCreateLogicalView(
       CreateLogicalViewStatement createLogicalViewStatement, MPPQueryContext context) {
     Analysis analysis = new Analysis();
-    // TODO: CRTODO: add more analyzing
     context.setQueryType(QueryType.WRITE);
 
-    // check target paths; check source expressions.
-    Pair<Boolean, Exception> checkResult = createLogicalViewStatement.checkAll();
-    if (checkResult.left == false) {
-      throw new RuntimeException(checkResult.right);
-    }
-
-    Analysis queryAnalysis = null;
-    if (createLogicalViewStatement.getQueryStatement() != null) {
-      // analysis query statement
-      //      AnalyzeVisitor queryVisitor = new AnalyzeVisitor(this.partitionFetcher,
-      // this.schemaFetcher);
-      //      queryAnalysis =
-      // queryVisitor.visitQuery(createLogicalViewStatement.getQueryStatement(), null);
-      this.visitQuery(createLogicalViewStatement.getQueryStatement(), null);
-      // get all expression from resultColumns
-      List<ResultColumn> resultColumns =
-          createLogicalViewStatement.getQueryStatement().getSelectComponent().getResultColumns();
-      List<Expression> expressionList = new ArrayList<>();
-      for (ResultColumn resultCol : resultColumns) {
-        expressionList.add(resultCol.getExpression());
-      }
-      createLogicalViewStatement.setSourceExpressions(expressionList);
+    // analyze query in statement
+    QueryStatement queryStatement = createLogicalViewStatement.getQueryStatement();
+    if (queryStatement != null) {
+      Pair<List<Expression>, Analysis> queryAnalysisPair =
+          this.analyzeQueryInLogicalViewStatement(analysis, queryStatement, context);
+      if (queryAnalysisPair.right.isFinishQueryAfterAnalyze()) {
+        return analysis;
+      } else if (queryAnalysisPair.left != null) {
+        createLogicalViewStatement.setSourceExpressions(queryAnalysisPair.left);
+      }
     }
     analysis.setStatement(createLogicalViewStatement);
 
+    // check target paths; check source expressions.
+    checkPathsInCreateLogicalView(analysis, createLogicalViewStatement);
+    if (analysis.isFinishQueryAfterAnalyze()) {
+      return analysis;
+    }
+
+    // make sure there is no view in source
+    List<Expression> sourceExpressionList = createLogicalViewStatement.getSourceExpressionList();
+    checkViewsInSource(analysis, sourceExpressionList, context);
+    if (analysis.isFinishQueryAfterAnalyze()) {
+      return analysis;
+    }
+
     // set schema partition info, this info will be used to split logical plan node.
     PathPatternTree patternTree = new PathPatternTree();
     for (PartialPath thisFullPath : createLogicalViewStatement.getTargetPathList()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
index 7b31aad6b9d..8525bedf97a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
@@ -50,6 +50,8 @@ import org.apache.iotdb.db.mpp.plan.expression.visitor.ConcatDeviceAndRemoveWild
 import org.apache.iotdb.db.mpp.plan.expression.visitor.ConcatExpressionWithSuffixPathsVisitor;
 import org.apache.iotdb.db.mpp.plan.expression.visitor.GetMeasurementExpressionVisitor;
 import org.apache.iotdb.db.mpp.plan.expression.visitor.RemoveAliasFromExpressionVisitor;
+import org.apache.iotdb.db.mpp.plan.expression.visitor.RemoveWildcardAndViewInExpressionVisitor;
+import org.apache.iotdb.db.mpp.plan.expression.visitor.RemoveWildcardAndViewInFilterVisitor;
 import org.apache.iotdb.db.mpp.plan.expression.visitor.RemoveWildcardInExpressionVisitor;
 import org.apache.iotdb.db.mpp.plan.expression.visitor.RemoveWildcardInFilterByDeviceVisitor;
 import org.apache.iotdb.db.mpp.plan.expression.visitor.RemoveWildcardInFilterVisitor;
@@ -414,6 +416,25 @@ public class ExpressionAnalyzer {
     return new RemoveWildcardInExpressionVisitor().process(expression, schemaTree);
   }
 
+  /**
+   * Bind schema ({@link PartialPath} -> {@link MeasurementPath}) and removes wildcards in
+   * Expression. And all logical view will be replaced.
+   *
+   * @param schemaTree interface for querying schema information
+   * @return the expression list after binding schema and whether there is logical view in
+   *     expressions
+   */
+  public static List<Expression> removeWildcardAndViewInExpression(
+      Expression expression, Analysis analysis, ISchemaTree schemaTree) {
+    RemoveWildcardAndViewInExpressionVisitor removeWildcardAndViewInExpressionVisitor =
+        new RemoveWildcardAndViewInExpressionVisitor();
+    List<Expression> expressions =
+        removeWildcardAndViewInExpressionVisitor.process(expression, schemaTree);
+    analysis.setHasViewsInQuery(
+        removeWildcardAndViewInExpressionVisitor.isHasProcessedLogicalView());
+    return expressions;
+  }
+
   /**
    * Concat suffix path in WHERE and HAVING clause with the prefix path in the FROM clause. And
    * then, bind schema ({@link PartialPath} -> {@link MeasurementPath}) and removes wildcards in
@@ -430,6 +451,22 @@ public class ExpressionAnalyzer {
             predicate, new RemoveWildcardInFilterVisitor.Context(prefixPaths, schemaTree, isRoot));
   }
 
+  /**
+   * Concat suffix path in WHERE and HAVING clause with the prefix path in the FROM clause. And
+   * then, bind schema ({@link PartialPath} -> {@link MeasurementPath}) and removes wildcards in
+   * Expression. Logical view will be replaced.
+   *
+   * @param prefixPaths prefix paths in the FROM clause
+   * @param schemaTree interface for querying schema information
+   * @return the expression list with full path and after binding schema
+   */
+  public static List<Expression> removeWildcardAndViewInFilter(
+      Expression predicate, List<PartialPath> prefixPaths, ISchemaTree schemaTree, boolean isRoot) {
+    return new RemoveWildcardAndViewInFilterVisitor()
+        .process(
+            predicate, new RemoveWildcardInFilterVisitor.Context(prefixPaths, schemaTree, isRoot));
+  }
+
   public static Expression replaceRawPathWithGroupedPath(
       Expression expression,
       GroupByLevelController.RawPathToGroupedPathMap rawPathToGroupedPathMap) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/Expression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/Expression.java
index ea2605288d1..519e1dc67c2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/Expression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/Expression.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.mpp.plan.expression;
 
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.NodeRef;
 import org.apache.iotdb.db.mpp.plan.expression.binary.AdditionExpression;
 import org.apache.iotdb.db.mpp.plan.expression.binary.DivisionExpression;
@@ -371,4 +372,23 @@ public abstract class Expression extends StatementNode {
       return current;
     }
   }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // For analyzing logical view
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  protected PartialPath viewPathOfThisExpression = null;
+
+  public void setViewPathOfThisExpression(PartialPath viewPathOfThisExpression) {
+    this.viewPathOfThisExpression = viewPathOfThisExpression;
+  }
+
+  public String getStringWithViewOfThisExpression() {
+    if (this.viewPathOfThisExpression == null) {
+      return this.getStringWithViewOfThisExpressionInternal();
+    }
+    return this.viewPathOfThisExpression.getFullPath();
+  }
+
+  public abstract String getStringWithViewOfThisExpressionInternal();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/BinaryExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/BinaryExpression.java
index 33a35e6b181..111e956fce5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/BinaryExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/BinaryExpression.java
@@ -145,4 +145,26 @@ public abstract class BinaryExpression extends Expression {
     Expression.serialize(leftExpression, stream);
     Expression.serialize(rightExpression, stream);
   }
+
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    String left = this.getLeftExpression().getStringWithViewOfThisExpression();
+    String right = this.getRightExpression().getStringWithViewOfThisExpression();
+
+    StringBuilder builder = new StringBuilder();
+    if (leftExpression.getExpressionType().getPriority() < this.getExpressionType().getPriority()) {
+      builder.append("(").append(left).append(")");
+    } else {
+      builder.append(left);
+    }
+    builder.append(" ").append(operator()).append(" ");
+    if (rightExpression.getExpressionType().getPriority()
+        < this.getExpressionType().getPriority()) {
+      builder.append("(").append(right).append(")");
+    } else {
+      builder.append(right);
+    }
+
+    return builder.toString();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/WhenThenExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/WhenThenExpression.java
index 53b08473d5c..c61ff03c3bd 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/WhenThenExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/binary/WhenThenExpression.java
@@ -66,6 +66,13 @@ public class WhenThenExpression extends BinaryExpression {
     return "WHEN " + this.getWhen().toString() + " THEN " + this.getThen().toString();
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    String when = this.getWhen().getStringWithViewOfThisExpression();
+    String then = this.getThen().getStringWithViewOfThisExpression();
+    return "WHEN " + when + " THEN " + then;
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitWhenThenExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/leaf/LeafOperand.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/leaf/LeafOperand.java
index c614c9fb9e2..cc3b068d4ba 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/leaf/LeafOperand.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/leaf/LeafOperand.java
@@ -46,4 +46,9 @@ public abstract class LeafOperand extends Expression {
   public boolean isMappable(Map<NodeRef<Expression>, TSDataType> expressionTypes) {
     return true;
   }
+
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    return this.getExpressionString();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java
index 6a36dc93c9a..39a15b6a723 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java
@@ -175,6 +175,27 @@ public class FunctionExpression extends Expression {
     return expressions;
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    StringBuilder builder = new StringBuilder();
+    if (!expressions.isEmpty()) {
+      builder.append(expressions.get(0).getStringWithViewOfThisExpression());
+      for (int i = 1; i < expressions.size(); ++i) {
+        builder.append(", ").append(expressions.get(i).getStringWithViewOfThisExpression());
+      }
+    }
+    if (!functionAttributes.isEmpty()) {
+      // Some built-in scalar functions may have different header.
+      if (BuiltinScalarFunction.contains(functionName)) {
+        BuiltInScalarFunctionHelperFactory.createHelper(functionName)
+            .appendFunctionAttributes(!expressions.isEmpty(), builder, functionAttributes);
+      } else {
+        appendAttributes(!expressions.isEmpty(), builder, functionAttributes);
+      }
+    }
+    return functionName + "(" + builder + ")";
+  }
+
   @Override
   public void constructUdfExecutors(
       Map<String, UDTFExecutor> expressionName2Executor, ZoneId zoneId) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/other/CaseWhenThenExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/other/CaseWhenThenExpression.java
index fa122ef87fc..616c2394e19 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/other/CaseWhenThenExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/other/CaseWhenThenExpression.java
@@ -165,6 +165,23 @@ public class CaseWhenThenExpression extends Expression {
     return result;
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("CASE ");
+    for (Expression expression : this.whenThenExpressions) {
+      builder.append(expression.getStringWithViewOfThisExpression()).append(" ");
+    }
+    if (!(this.elseExpression instanceof NullOperand)) {
+      builder
+          .append("ELSE ")
+          .append(this.elseExpression.getStringWithViewOfThisExpression())
+          .append(" ");
+    }
+    builder.append("END");
+    return builder.toString();
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitCaseWhenThenExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/BetweenExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/BetweenExpression.java
index 46463a2097a..392d9c2418c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/BetweenExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/BetweenExpression.java
@@ -84,6 +84,15 @@ public class BetweenExpression extends TernaryExpression {
     ReadWriteIOUtils.write(isNotBetween, stream);
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    return firstExpression.getStringWithViewOfThisExpression()
+        + " BETWEEN "
+        + secondExpression.getStringWithViewOfThisExpression()
+        + " AND "
+        + thirdExpression.getStringWithViewOfThisExpression();
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitBetweenExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/TernaryExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/TernaryExpression.java
index 1fc7906a1a7..2242e0f32e3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/TernaryExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/ternary/TernaryExpression.java
@@ -38,22 +38,34 @@ import java.util.List;
 import java.util.Map;
 
 public abstract class TernaryExpression extends Expression {
-  protected final Expression firstExpression;
-  protected final Expression secondExpression;
-  protected final Expression thirdExpression;
+  protected Expression firstExpression;
+  protected Expression secondExpression;
+  protected Expression thirdExpression;
 
   public Expression getFirstExpression() {
     return firstExpression;
   }
 
+  public void setFirstExpression(Expression expression) {
+    this.firstExpression = expression;
+  }
+
   public Expression getSecondExpression() {
     return secondExpression;
   }
 
+  public void setSecondExpression(Expression expression) {
+    this.secondExpression = expression;
+  }
+
   public Expression getThirdExpression() {
     return thirdExpression;
   }
 
+  public void setThirdExpression(Expression expression) {
+    this.thirdExpression = expression;
+  }
+
   protected TernaryExpression(
       Expression firstExpression, Expression secondExpression, Expression thirdExpression) {
     this.firstExpression = firstExpression;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/InExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/InExpression.java
index c9278f632eb..be7bb54977a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/InExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/InExpression.java
@@ -27,6 +27,8 @@ import org.apache.iotdb.db.mpp.plan.expression.multi.FunctionExpression;
 import org.apache.iotdb.db.mpp.plan.expression.visitor.ExpressionVisitor;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
+import org.jetbrains.annotations.NotNull;
+
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -72,6 +74,11 @@ public class InExpression extends UnaryExpression {
     } else {
       stringBuilder.append('(').append(expression).append(')').append(" IN (");
     }
+    return appendValuesToBuild(stringBuilder).toString();
+  }
+
+  @NotNull
+  private StringBuilder appendValuesToBuild(StringBuilder stringBuilder) {
     Iterator<String> iterator = values.iterator();
     if (iterator.hasNext()) {
       stringBuilder.append(iterator.next());
@@ -80,7 +87,7 @@ public class InExpression extends UnaryExpression {
       stringBuilder.append(',').append(iterator.next());
     }
     stringBuilder.append(')');
-    return stringBuilder.toString();
+    return stringBuilder;
   }
 
   @Override
@@ -113,6 +120,23 @@ public class InExpression extends UnaryExpression {
     }
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    StringBuilder stringBuilder = new StringBuilder();
+    if (expression instanceof FunctionExpression
+        || expression instanceof ConstantOperand
+        || expression instanceof TimeSeriesOperand) {
+      stringBuilder.append(expression.getStringWithViewOfThisExpression()).append(" IN (");
+    } else {
+      stringBuilder
+          .append('(')
+          .append(expression.getStringWithViewOfThisExpression())
+          .append(')')
+          .append(" IN (");
+    }
+    return appendValuesToBuild(stringBuilder).toString();
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitInExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/IsNullExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/IsNullExpression.java
index ba6e4a9fcb1..97a45e10b8f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/IsNullExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/IsNullExpression.java
@@ -72,6 +72,11 @@ public class IsNullExpression extends UnaryExpression {
     ReadWriteIOUtils.write(isNot, stream);
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    return expression.getStringWithViewOfThisExpression() + " IS " + (isNot ? "NOT " : "") + "NULL";
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitIsNullExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LikeExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LikeExpression.java
index 543af09d2bd..9ef4b420f44 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LikeExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LikeExpression.java
@@ -142,6 +142,11 @@ public class LikeExpression extends UnaryExpression {
     ReadWriteIOUtils.write(patternString, stream);
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    return expression.getStringWithViewOfThisExpression() + " LIKE '" + pattern + "'";
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitLikeExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LogicNotExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LogicNotExpression.java
index 00527cd785a..1186d0097d3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LogicNotExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/LogicNotExpression.java
@@ -52,6 +52,15 @@ public class LogicNotExpression extends UnaryExpression {
         : "!(" + expression + ")";
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    return expression instanceof FunctionExpression
+            || expression instanceof ConstantOperand
+            || expression instanceof TimeSeriesOperand
+        ? "!" + expression.getStringWithViewOfThisExpression()
+        : "!(" + expression.getStringWithViewOfThisExpression() + ")";
+  }
+
   @Override
   public ExpressionType getExpressionType() {
     return ExpressionType.LOGIC_NOT;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/NegationExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/NegationExpression.java
index b213d315284..70fb48958a5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/NegationExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/NegationExpression.java
@@ -55,6 +55,17 @@ public class NegationExpression extends UnaryExpression {
         : "-(" + expression + ")";
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    return expression instanceof TimeSeriesOperand
+            || expression instanceof FunctionExpression
+            || expression instanceof NullOperand
+            || (expression instanceof ConstantOperand
+                && !((ConstantOperand) expression).isNegativeNumber())
+        ? "-" + expression.getStringWithViewOfThisExpression()
+        : "-(" + expression.getStringWithViewOfThisExpression() + ")";
+  }
+
   @Override
   public ExpressionType getExpressionType() {
     return ExpressionType.NEGATION;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/RegularExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/RegularExpression.java
index a67dfb0340d..6afbf688747 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/RegularExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/RegularExpression.java
@@ -89,6 +89,11 @@ public class RegularExpression extends UnaryExpression {
     ReadWriteIOUtils.write(patternString, stream);
   }
 
+  @Override
+  public String getStringWithViewOfThisExpressionInternal() {
+    return expression.getStringWithViewOfThisExpression() + " REGEXP '" + patternString + "'";
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitRegularExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/UnaryExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/UnaryExpression.java
index 288354996f2..f697423f2fa 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/UnaryExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/unary/UnaryExpression.java
@@ -37,7 +37,7 @@ import java.util.Map;
 
 public abstract class UnaryExpression extends Expression {
 
-  protected final Expression expression;
+  protected Expression expression;
 
   protected UnaryExpression(Expression expression) {
     this.expression = expression;
@@ -47,6 +47,10 @@ public abstract class UnaryExpression extends Expression {
     return expression;
   }
 
+  public void setExpression(Expression expression) {
+    this.expression = expression;
+  }
+
   @Override
   public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
     return visitor.visitUnaryExpression(this, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/CompleteMeasurementSchemaVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/CompleteMeasurementSchemaVisitor.java
new file mode 100644
index 00000000000..fe9716f3b18
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/CompleteMeasurementSchemaVisitor.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.plan.expression.visitor;
+
+import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
+import org.apache.iotdb.db.mpp.plan.expression.Expression;
+import org.apache.iotdb.db.mpp.plan.expression.binary.BinaryExpression;
+import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.db.mpp.plan.expression.multi.FunctionExpression;
+import org.apache.iotdb.db.mpp.plan.expression.ternary.TernaryExpression;
+import org.apache.iotdb.db.mpp.plan.expression.unary.UnaryExpression;
+
+import java.rmi.UnexpectedException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class CompleteMeasurementSchemaVisitor extends ExpressionVisitor<Expression, ISchemaTree> {
+
+  @Override
+  public Expression process(Expression expression, ISchemaTree schemaTree) {
+    return expression.accept(this, schemaTree);
+  }
+
+  @Override
+  public Expression visitExpression(Expression expression, ISchemaTree schemaTree) {
+    return expression;
+  }
+
+  @Override
+  public Expression visitUnaryExpression(UnaryExpression unaryExpression, ISchemaTree schemaTree) {
+    unaryExpression.setExpression(this.process(unaryExpression.getExpression(), schemaTree));
+    return unaryExpression;
+  }
+
+  @Override
+  public Expression visitBinaryExpression(
+      BinaryExpression binaryExpression, ISchemaTree schemaTree) {
+    binaryExpression.setLeftExpression(
+        this.process(binaryExpression.getLeftExpression(), schemaTree));
+    binaryExpression.setRightExpression(
+        this.process(binaryExpression.getRightExpression(), schemaTree));
+    return binaryExpression;
+  }
+
+  @Override
+  public Expression visitTernaryExpression(
+      TernaryExpression ternaryExpression, ISchemaTree schemaTree) {
+    ternaryExpression.setFirstExpression(
+        this.process(ternaryExpression.getFirstExpression(), schemaTree));
+    ternaryExpression.setSecondExpression(
+        this.process(ternaryExpression.getSecondExpression(), schemaTree));
+    ternaryExpression.setThirdExpression(
+        this.process(ternaryExpression.getThirdExpression(), schemaTree));
+    return ternaryExpression;
+  }
+
+  @Override
+  public Expression visitFunctionExpression(
+      FunctionExpression functionExpression, ISchemaTree schemaTree) {
+    List<Expression> children = functionExpression.getExpressions();
+    List<Expression> replacedChildren = new ArrayList<>();
+    for (Expression child : children) {
+      replacedChildren.add(this.process(child, schemaTree));
+    }
+    functionExpression.setExpressions(replacedChildren);
+    return functionExpression;
+  }
+
+  @Override
+  public Expression visitTimeSeriesOperand(
+      TimeSeriesOperand timeSeriesOperand, ISchemaTree schemaTree) {
+    PartialPath path = timeSeriesOperand.getPath();
+    try {
+      try {
+        path.getMeasurementSchema();
+      } catch (Exception notAMeasurementPath) {
+        List<MeasurementPath> actualPaths = schemaTree.searchMeasurementPaths(path).left;
+        if (actualPaths.size() != 1) {
+          throw new UnexpectedException(
+              "given one path ["
+                  + path.toString()
+                  + "], "
+                  + "but got unmatched path(s) :"
+                  + actualPaths.toString());
+        }
+        return new TimeSeriesOperand(actualPaths.get(0));
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return timeSeriesOperand;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/RemoveWildcardAndViewInExpressionVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/RemoveWildcardAndViewInExpressionVisitor.java
new file mode 100644
index 00000000000..303df8ce695
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/RemoveWildcardAndViewInExpressionVisitor.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.mpp.plan.expression.visitor;
+
+import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.view.LogicalViewSchema;
+import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression;
+import org.apache.iotdb.db.metadata.view.viewExpression.visitor.TransformToExpressionVisitor;
+import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
+import org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils;
+import org.apache.iotdb.db.mpp.plan.expression.Expression;
+import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class RemoveWildcardAndViewInExpressionVisitor extends RemoveWildcardInExpressionVisitor {
+  private final TransformToExpressionVisitor transformToExpressionVisitor;
+
+  private final CompleteMeasurementSchemaVisitor completeMeasurementSchemaVisitor;
+
+  boolean hasProcessedLogicalView;
+
+  public RemoveWildcardAndViewInExpressionVisitor() {
+    super();
+    this.transformToExpressionVisitor = new TransformToExpressionVisitor();
+    this.completeMeasurementSchemaVisitor = new CompleteMeasurementSchemaVisitor();
+    this.hasProcessedLogicalView = false;
+  }
+
+  public boolean isHasProcessedLogicalView() {
+    return this.hasProcessedLogicalView;
+  }
+
+  private Expression transformViewPath(MeasurementPath measurementPath, ISchemaTree schemaTree) {
+    IMeasurementSchema measurementSchema = measurementPath.getMeasurementSchema();
+    if (measurementSchema.isLogicalView()) {
+      ViewExpression viewExpression = ((LogicalViewSchema) measurementSchema).getExpression();
+      // complete measurementPaths in expressions.
+      Expression expression = this.transformToExpressionVisitor.process(viewExpression, null);
+      expression = this.completeMeasurementSchemaVisitor.process(expression, schemaTree);
+      return expression;
+    } else {
+      throw new RuntimeException(
+          new UnsupportedOperationException(
+              "Can not construct expression using non view path in transformViewPath!"));
+    }
+  }
+
+  @Override
+  public List<Expression> visitTimeSeriesOperand(
+      TimeSeriesOperand timeSeriesOperand, ISchemaTree schemaTree) {
+    PartialPath timeSeriesOperandPath = timeSeriesOperand.getPath();
+    List<MeasurementPath> actualPaths =
+        schemaTree.searchMeasurementPaths(timeSeriesOperandPath).left;
+    // process logical view
+    List<MeasurementPath> nonViewActualPaths = new ArrayList<>();
+    List<MeasurementPath> viewPaths = new ArrayList<>();
+    for (MeasurementPath measurementPath : actualPaths) {
+      if (measurementPath.getMeasurementSchema().isLogicalView()) {
+        this.hasProcessedLogicalView = true;
+        viewPaths.add(measurementPath);
+      } else {
+        nonViewActualPaths.add(measurementPath);
+      }
+    }
+    List<Expression> reconstructTimeSeriesOperands =
+        ExpressionUtils.reconstructTimeSeriesOperands(nonViewActualPaths);
+    // handle logical views
+    for (MeasurementPath measurementPath : viewPaths) {
+      Expression replacedExpression = this.transformViewPath(measurementPath, schemaTree);
+      replacedExpression.setViewPathOfThisExpression(measurementPath);
+      reconstructTimeSeriesOperands.add(replacedExpression);
+    }
+    return reconstructTimeSeriesOperands;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/RemoveWildcardAndViewInFilterVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/RemoveWildcardAndViewInFilterVisitor.java
new file mode 100644
index 00000000000..550f5ac7299
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/RemoveWildcardAndViewInFilterVisitor.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.mpp.plan.expression.visitor;
+
+import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.view.LogicalViewSchema;
+import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression;
+import org.apache.iotdb.db.constant.SqlConstant;
+import org.apache.iotdb.db.metadata.view.viewExpression.visitor.TransformToExpressionVisitor;
+import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
+import org.apache.iotdb.db.mpp.plan.expression.Expression;
+import org.apache.iotdb.db.mpp.plan.expression.leaf.NullOperand;
+import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.iotdb.db.mpp.plan.analyze.ExpressionUtils.reconstructTimeSeriesOperands;
+
+public class RemoveWildcardAndViewInFilterVisitor extends RemoveWildcardInFilterVisitor {
+
+  private final TransformToExpressionVisitor transformToExpressionVisitor;
+
+  private final CompleteMeasurementSchemaVisitor completeMeasurementSchemaVisitor;
+
+  boolean hasProcessedLogicalView;
+
+  public RemoveWildcardAndViewInFilterVisitor() {
+    super();
+    this.transformToExpressionVisitor = new TransformToExpressionVisitor();
+    this.completeMeasurementSchemaVisitor = new CompleteMeasurementSchemaVisitor();
+    this.hasProcessedLogicalView = false;
+  }
+
+  public boolean isHasProcessedLogicalView() {
+    return this.hasProcessedLogicalView;
+  }
+
+  private Expression transformViewPath(MeasurementPath measurementPath, ISchemaTree schemaTree) {
+    IMeasurementSchema measurementSchema = measurementPath.getMeasurementSchema();
+    if (measurementSchema.isLogicalView()) {
+      ViewExpression viewExpression = ((LogicalViewSchema) measurementSchema).getExpression();
+      // complete measurementPaths in expressions.
+      Expression expression = this.transformToExpressionVisitor.process(viewExpression, null);
+      expression = this.completeMeasurementSchemaVisitor.process(expression, schemaTree);
+      return expression;
+    } else {
+      throw new RuntimeException(
+          new UnsupportedOperationException(
+              "Can not construct expression using non view path in transformViewPath!"));
+    }
+  }
+
+  @Override
+  public List<Expression> visitTimeSeriesOperand(TimeSeriesOperand predicate, Context context) {
+    PartialPath filterPath = predicate.getPath();
+    List<PartialPath> concatPaths = new ArrayList<>();
+    if (!filterPath.getFirstNode().equals(SqlConstant.ROOT)) {
+      context.getPrefixPaths().forEach(prefix -> concatPaths.add(prefix.concatPath(filterPath)));
+    } else {
+      // do nothing in the case of "where root.d1.s1 > 5"
+      concatPaths.add(filterPath);
+    }
+
+    List<MeasurementPath> nonViewPathList = new ArrayList<>();
+    List<MeasurementPath> viewPathList = new ArrayList<>();
+    for (PartialPath concatPath : concatPaths) {
+      List<MeasurementPath> actualPaths =
+          context.getSchemaTree().searchMeasurementPaths(concatPath).left;
+      if (actualPaths.isEmpty()) {
+        return Collections.singletonList(new NullOperand());
+      }
+      for (MeasurementPath measurementPath : actualPaths) {
+        if (measurementPath.getMeasurementSchema().isLogicalView()) {
+          viewPathList.add(measurementPath);
+        } else {
+          nonViewPathList.add(measurementPath);
+        }
+      }
+    }
+    List<Expression> reconstructTimeSeriesOperands = reconstructTimeSeriesOperands(nonViewPathList);
+    for (MeasurementPath measurementPath : viewPathList) {
+      Expression replacedExpression =
+          this.transformViewPath(measurementPath, context.getSchemaTree());
+      replacedExpression.setViewPathOfThisExpression(measurementPath);
+      reconstructTimeSeriesOperands.add(replacedExpression);
+    }
+    return reconstructTimeSeriesOperands;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/ReplaceLogicalViewVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/ReplaceLogicalViewVisitor.java
new file mode 100644
index 00000000000..9185f0aff45
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/ReplaceLogicalViewVisitor.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.plan.expression.visitor;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.view.LogicalViewSchema;
+import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression;
+import org.apache.iotdb.db.metadata.view.viewExpression.visitor.GetSourcePathsVisitor;
+import org.apache.iotdb.db.metadata.view.viewExpression.visitor.TransformToExpressionVisitor;
+import org.apache.iotdb.db.mpp.plan.expression.Expression;
+import org.apache.iotdb.db.mpp.plan.expression.binary.BinaryExpression;
+import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.db.mpp.plan.expression.multi.FunctionExpression;
+import org.apache.iotdb.db.mpp.plan.expression.ternary.TernaryExpression;
+import org.apache.iotdb.db.mpp.plan.expression.unary.UnaryExpression;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * step 1. check whether this expression contains logical view, that means check TimeSeriesOperand
+ * which has the LogicalViewSchema.
+ *
+ * <p>step 2. replace that TimeSeriesOperand with expression recorded in LogicalViewSchema (view
+ * expression).
+ *
+ * <p>step 3. record paths that appeared in view expression. They should be fetched later.
+ */
+public class ReplaceLogicalViewVisitor extends ExpressionVisitor<Expression, List<PartialPath>> {
+
+  private TransformToExpressionVisitor transformToExpressionVisitor = null;
+
+  private GetSourcePathsVisitor getSourcePathsVisitor = null;
+
+  private boolean hasProcessedAggregationFunction = false;
+
+  /** The paths that are new added, which should be re-fetched. */
+  private List<PartialPath> newAddedPathList = null;
+
+  public ReplaceLogicalViewVisitor() {
+    this.transformToExpressionVisitor = new TransformToExpressionVisitor();
+    this.getSourcePathsVisitor = new GetSourcePathsVisitor();
+    this.resetHadProcessedAggregationFunction();
+    this.newAddedPathList = new ArrayList<>();
+  }
+
+  /**
+   * This function will check the expression you put in, find the TimeSeriesOperand which has
+   * LogicalViewSchema. These TimeSeriesOperand will be replaced with logical view expression, and
+   * this function will record paths that appeared in view expression. The logical view you replaced
+   * have INCOMPLETE path information, and use PartialPath in TimeSeriesOperand. This may cause
+   * ERROR, therefore you should call completeMeasurementPathCausedByView() later, make sure the
+   * path info is complete and using MeasurementPath with full MeasurementSchema.
+   *
+   * @param expression the expression you want to check.
+   * @return pair of replaced expression and whether replacement was happened. 'True' means the
+   *     expression you put in contains logical view, and has been replaced. 'False' means there is
+   *     no need to modify the expression you put in.
+   */
+  public Pair<Expression, Boolean> replaceViewInThisExpression(Expression expression) {
+    // step 1. check whether this expression contains logical view, that means finding
+    // TimeSeriesOperand which has
+    // the LogicalViewSchema.
+    // step 2. replace that TimeSeriesOperand with expression recorded in LogicalViewSchema (view
+    // expression).
+    // step 3. record paths that appeared in view expression. They should be fetched, then you can
+    // use fetched schema
+    // to complete new added TimeSeriesOperand.
+    int oldSize = this.newAddedPathList.size();
+    Expression result = this.process(expression, this.newAddedPathList);
+    int newSize = this.newAddedPathList.size();
+    if (oldSize != newSize) {
+      return new Pair<>(result, true);
+    }
+    return new Pair<>(expression, false);
+  }
+
+  public List<PartialPath> getNewAddedPathList() {
+    return this.newAddedPathList;
+  }
+
+  private Expression transform(ViewExpression viewExpression) {
+    return this.transformToExpressionVisitor.process(viewExpression, null);
+  }
+
+  private List<PartialPath> collectSourcePaths(ViewExpression viewExpression) {
+    return this.getSourcePathsVisitor.process(viewExpression, null);
+  }
+
+  public boolean getHadProcessedAggregationFunction() {
+    return this.hasProcessedAggregationFunction;
+  }
+
+  public void resetHadProcessedAggregationFunction() {
+    this.hasProcessedAggregationFunction = false;
+  }
+
+  @Override
+  public Expression process(Expression expression, List<PartialPath> context) {
+    return expression.accept(this, context);
+  }
+
+  @Override
+  public Expression visitExpression(Expression expression, List<PartialPath> context) {
+    return expression;
+  }
+
+  @Override
+  public Expression visitUnaryExpression(
+      UnaryExpression unaryExpression, List<PartialPath> context) {
+    unaryExpression.setExpression(this.process(unaryExpression.getExpression(), context));
+    return unaryExpression;
+  }
+
+  @Override
+  public Expression visitBinaryExpression(
+      BinaryExpression binaryExpression, List<PartialPath> context) {
+    binaryExpression.setLeftExpression(this.process(binaryExpression.getLeftExpression(), context));
+    binaryExpression.setRightExpression(
+        this.process(binaryExpression.getRightExpression(), context));
+    return binaryExpression;
+  }
+
+  @Override
+  public Expression visitTernaryExpression(
+      TernaryExpression ternaryExpression, List<PartialPath> context) {
+    ternaryExpression.setFirstExpression(
+        this.process(ternaryExpression.getFirstExpression(), context));
+    ternaryExpression.setSecondExpression(
+        this.process(ternaryExpression.getSecondExpression(), context));
+    ternaryExpression.setThirdExpression(
+        this.process(ternaryExpression.getThirdExpression(), context));
+    return ternaryExpression;
+  }
+
+  @Override
+  public Expression visitFunctionExpression(
+      FunctionExpression functionExpression, List<PartialPath> context) {
+    List<Expression> children = functionExpression.getExpressions();
+    List<Expression> replacedChildren = new ArrayList<>();
+    for (Expression child : children) {
+      replacedChildren.add(this.process(child, context));
+    }
+    functionExpression.setExpressions(replacedChildren);
+    if (functionExpression.isBuiltInAggregationFunctionExpression()) {
+      this.hasProcessedAggregationFunction = true;
+    }
+    return functionExpression;
+  }
+
+  @Override
+  public Expression visitTimeSeriesOperand(
+      TimeSeriesOperand timeSeriesOperand, List<PartialPath> context) {
+    PartialPath path = timeSeriesOperand.getPath();
+    try {
+      IMeasurementSchema measurementSchema = path.getMeasurementSchema();
+      if (measurementSchema.isLogicalView()) {
+        ViewExpression viewExpression = ((LogicalViewSchema) measurementSchema).getExpression();
+        Expression result = this.transform(viewExpression);
+        // record paths in this viewExpression
+        context.addAll(this.collectSourcePaths(viewExpression));
+        return result;
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return timeSeriesOperand;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateLogicalViewStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateLogicalViewStatement.java
index b96aa42dd7e..66c60210305 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateLogicalViewStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateLogicalViewStatement.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.metadata;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.plan.expression.Expression;
 import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
@@ -117,13 +116,13 @@ public class CreateLogicalViewStatement extends Statement {
   /**
    * Check errors in targetPaths.
    *
-   * @return Pair<Boolean, Exception>. True: checks passed, exception is null; False: checks failed,
-   *     returns exception.
+   * @return Pair<Boolean, String>. True: checks passed; False: checks failed. if check failed,
+   *     return the string of illegal path.
    */
-  public Pair<Boolean, Exception> checkTargetPaths() {
+  public Pair<Boolean, String> checkTargetPaths() {
     for (PartialPath thisPath : this.getTargetPathList()) {
       if (thisPath.getNodeLength() < 3) {
-        return new Pair<>(false, new IllegalPathException(thisPath.getFullPath()));
+        return new Pair<>(false, thisPath.getFullPath());
       }
     }
     return new Pair<>(true, null);
@@ -133,24 +132,27 @@ public class CreateLogicalViewStatement extends Statement {
    * Check errors in sourcePaths. Only usable when not using query statement. If this statement is
    * generated with a query statement, check always pass; if not, check each full paths.
    *
-   * @return Pair<Boolean, Exception>. True: checks passed, exception is null; False: checks failed,
-   *     returns exception.
+   * @return Pair<Boolean, String>. True: checks passed; False: checks failed. if check failed,
+   *     return the string of illegal path.
    */
-  public Pair<Boolean, Exception> checkSourcePathsIfNotUsingQueryStatement() {
+  public Pair<Boolean, String> checkSourcePathsIfNotUsingQueryStatement() {
     if (this.sourcePaths.viewPathType == ViewPathType.PATHS_GROUP
         || this.sourcePaths.viewPathType == ViewPathType.FULL_PATH_LIST) {
       for (PartialPath thisPath : this.sourcePaths.fullPathList) {
         if (thisPath.getNodeLength() < 3) {
-          return new Pair<>(false, new IllegalPathException(thisPath.getFullPath()));
+          return new Pair<>(false, thisPath.getFullPath());
         }
       }
     }
     return new Pair<>(true, null);
   }
 
-  public Pair<Boolean, Exception> checkAll() {
-    Pair<Boolean, Exception> result = null;
-    result = this.checkTargetPaths();
+  /**
+   * @return return true if checks passed; else return false. if check failed, return the string of
+   *     illegal path.
+   */
+  public Pair<Boolean, String> checkAllPaths() {
+    Pair<Boolean, String> result = this.checkTargetPaths();
     if (result.left == false) return result;
     result = this.checkSourcePathsIfNotUsingQueryStatement();
     if (result.left == false) return result;
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeTest.java
index 670553df0f4..b907ec0523b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeTest.java
@@ -99,7 +99,7 @@ public class AnalyzeTest {
       expectedAnalysis.setRespDatasetHeader(
           new DatasetHeader(
               Arrays.asList(
-                  new ColumnHeader("root.sg.d1.s1", TSDataType.INT32),
+                  new ColumnHeader("root.sg.d1.s1", TSDataType.INT32, "root.sg.d1.s1"),
                   new ColumnHeader("root.sg.d1.s2", TSDataType.DOUBLE, "root.sg.d1.status"),
                   new ColumnHeader("root.sg.d1.s1 + 1", TSDataType.DOUBLE, "t")),
               false));
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzerTest.java
index 15a7a13c5df..1672ff9dbd4 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzerTest.java
@@ -51,7 +51,7 @@ public class ExpressionAnalyzerTest {
             gt(timeSeries("root.sg.d2.s1"), intValue("1")),
             gt(timeSeries("root.sg.d1.s2"), intValue("1")),
             gt(timeSeries("root.sg.d2.s2"), intValue("1"))),
-        ExpressionAnalyzer.removeWildcardInFilter(
+        ExpressionAnalyzer.removeWildcardAndViewInFilter(
             and(gt(timeSeries("s1"), intValue("1")), gt(timeSeries("s2"), intValue("1"))),
             prefixPaths,
             fakeSchemaTree,
@@ -75,7 +75,7 @@ public class ExpressionAnalyzerTest {
                 and(
                     gt(timeSeries("root.sg.d2.s1"), intValue("1")),
                     gt(timeSeries("root.sg.d2.s2"), intValue("1"))))),
-        ExpressionAnalyzer.removeWildcardInFilter(
+        ExpressionAnalyzer.removeWildcardAndViewInFilter(
             count(and(gt(timeSeries("s1"), intValue("1")), gt(timeSeries("s2"), intValue("1")))),
             prefixPaths,
             fakeSchemaTree,