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/27 07:04:10 UTC

[iotdb] branch master updated: Support Alter View (#9955)

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 b7aaaf04378 Support Alter View (#9955)
b7aaaf04378 is described below

commit b7aaaf0437861c489a3965f90f51e698836c0785
Author: Marcos_Zyk <38...@users.noreply.github.com>
AuthorDate: Sat May 27 15:04:03 2023 +0800

    Support Alter View (#9955)
---
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |  10 +-
 .../iotdb/db/metadata/view/ViewPathType.java       |  26 ++++
 .../apache/iotdb/db/metadata/view/ViewPaths.java   |  97 ++++++++++++
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  |  44 +++---
 .../plan/execution/config/ConfigTaskVisitor.java   |  16 ++
 .../config/executor/ClusterConfigTaskExecutor.java | 163 +++++++++++++++++++++
 .../config/executor/IConfigTaskExecutor.java       |   8 +
 .../config/metadata/view/AlterLogicalViewTask.java |  44 ++++++
 .../metadata/view/RenameLogicalViewTask.java       |  45 ++++++
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |  58 ++++++--
 .../db/mpp/plan/planner/LogicalPlanVisitor.java    |  11 +-
 .../iotdb/db/mpp/plan/statement/StatementType.java |   2 +
 .../db/mpp/plan/statement/StatementVisitor.java    |  11 ++
 .../metadata/view/AlterLogicalViewStatement.java   | 115 +++++++++++++++
 .../metadata/view/CreateLogicalViewStatement.java  | 120 +++++----------
 .../metadata/view/RenameLogicalViewStatement.java  |  73 +++++++++
 16 files changed, 723 insertions(+), 120 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
index 93ffd7a1c5a..b44f9268cd3 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
@@ -64,7 +64,7 @@ ddlStatement
     // Quota
     | setSpaceQuota | showSpaceQuota | setThrottleQuota | showThrottleQuota
     // View
-    | createLogicalView | dropLogicalView | showLogicalView
+    | createLogicalView | dropLogicalView | showLogicalView | renameLogicalView | alterLogicalView
     ;
 
 dmlStatement
@@ -571,6 +571,14 @@ dropLogicalView
     : (DELETE | DROP) VIEW prefixPath (COMMA prefixPath)*
     ;
 
+renameLogicalView
+    : ALTER VIEW prefixPath RENAME TO prefixPath
+    ;
+
+alterLogicalView
+    : ALTER VIEW viewTargetPaths AS viewSourcePaths
+    ;
+
 viewSuffixPaths
     : nodeNameWithoutWildcard (DOT nodeNameWithoutWildcard)*
     ;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/view/ViewPathType.java b/server/src/main/java/org/apache/iotdb/db/metadata/view/ViewPathType.java
new file mode 100644
index 00000000000..668adbb07d4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/view/ViewPathType.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.view;
+
+public enum ViewPathType {
+  FULL_PATH_LIST,
+  PATHS_GROUP,
+  QUERY_STATEMENT
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/view/ViewPaths.java b/server/src/main/java/org/apache/iotdb/db/metadata/view/ViewPaths.java
new file mode 100644
index 00000000000..a0d86d1496b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/view/ViewPaths.java
@@ -0,0 +1,97 @@
+/*
+ * 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.metadata.view;
+
+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;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A class to save all paths' info in targetPaths and sourcePaths except query statement.
+ *
+ * <p>fullPathList: CREATE VIEW root.db.device.temp AS root.ln.d.s01 PathGroup: CREATE VIEW
+ * root.db(device.temp, status) AS root.ln(d.s01, wf.abc.s02)
+ */
+public class ViewPaths {
+  public ViewPathType viewPathType = ViewPathType.FULL_PATH_LIST;
+  public List<PartialPath> fullPathList = null;
+  public PartialPath prefixOfPathsGroup = null;
+  public List<PartialPath> suffixOfPathsGroup = null;
+
+  public List<Expression> expressionsList = null;
+
+  public void addPath(PartialPath path) {
+    if (this.fullPathList == null) {
+      this.fullPathList = new ArrayList<>();
+      this.fullPathList.add(path);
+    } else {
+      this.fullPathList.add(path);
+    }
+  }
+
+  public void setFullPathList(List<PartialPath> pathList) {
+    this.fullPathList = pathList;
+  }
+
+  public void setPrefixOfPathsGroup(PartialPath path) {
+    this.prefixOfPathsGroup = path;
+  }
+
+  public void setSuffixOfPathsGroup(List<PartialPath> pathList) {
+    this.suffixOfPathsGroup = pathList;
+  }
+
+  public void setViewPathType(ViewPathType viewPathType) {
+    this.viewPathType = viewPathType;
+  }
+
+  public void generateFullPathsFromPathsGroup() {
+    if (prefixOfPathsGroup != null && suffixOfPathsGroup != null) {
+      this.fullPathList = new ArrayList<>();
+      for (PartialPath suffixPath : suffixOfPathsGroup) {
+        PartialPath pathToAdd = prefixOfPathsGroup.concatPath(suffixPath);
+        this.addPath(pathToAdd);
+      }
+    }
+  }
+
+  public void generateExpressionsIfNecessary() {
+    if (this.viewPathType == ViewPathType.FULL_PATH_LIST
+        || this.viewPathType == ViewPathType.PATHS_GROUP) {
+      if (this.fullPathList != null) {
+        this.expressionsList = new ArrayList<>();
+        for (PartialPath path : this.fullPathList) {
+          TimeSeriesOperand tsExpression = new TimeSeriesOperand(path);
+          this.expressionsList.add(tsExpression);
+        }
+      }
+    } else if (this.viewPathType == ViewPathType.QUERY_STATEMENT) {
+      // no nothing. expressions should be set by setExpressionsList
+    }
+  }
+
+  public void setExpressionsList(List<Expression> expressionsList) {
+    this.expressionsList = expressionsList;
+  }
+  // end of viewPaths
+}
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 a54e4157a28..10e8e380f6b 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
@@ -3323,31 +3323,33 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       CreateLogicalViewStatement createLogicalViewStatement, MPPQueryContext context) {
     Analysis analysis = new Analysis();
     context.setQueryType(QueryType.WRITE);
+    analysis.setStatement(createLogicalViewStatement);
 
-    // 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);
+    if (createLogicalViewStatement.getViewExpression() == null) {
+      // 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;
-    }
+      // 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;
+      // 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.
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
index e42d3ae5466..6b9aa677375 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
@@ -63,7 +63,9 @@ import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowNodes
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowPathSetTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowSchemaTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.UnsetSchemaTemplateTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.metadata.view.AlterLogicalViewTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.view.DeleteLogicalViewTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.metadata.view.RenameLogicalViewTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.sys.AuthorizerTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.sys.ClearCacheTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.sys.FlushTask;
@@ -129,7 +131,9 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchem
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.UnsetSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.AlterLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.DeleteLogicalViewStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.RenameLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ClearCacheStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.FlushStatement;
@@ -439,6 +443,18 @@ public class ConfigTaskVisitor
     return new DeleteLogicalViewTask(context.getQueryId(), deleteLogicalViewStatement);
   }
 
+  @Override
+  public IConfigTask visitRenameLogicalView(
+      RenameLogicalViewStatement renameLogicalViewStatement, TaskContext context) {
+    return new RenameLogicalViewTask(context.queryId, renameLogicalViewStatement);
+  }
+
+  @Override
+  public IConfigTask visitAlterLogicalView(
+      AlterLogicalViewStatement alterLogicalViewStatement, TaskContext context) {
+    return new AlterLogicalViewTask(context.queryId, alterLogicalViewStatement);
+  }
+
   @Override
   public IConfigTask visitGetRegionId(
       GetRegionIdStatement getRegionIdStatement, TaskContext context) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
index 6858e22e238..4bde95a1df8 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
@@ -35,10 +35,13 @@ import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.executable.ExecutableManager;
 import org.apache.iotdb.commons.executable.ExecutableResource;
+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.pipe.plugin.service.PipePluginClassLoader;
 import org.apache.iotdb.commons.pipe.plugin.service.PipePluginExecutableManager;
+import org.apache.iotdb.commons.schema.view.LogicalViewSchema;
+import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression;
 import org.apache.iotdb.commons.trigger.service.TriggerExecutableManager;
 import org.apache.iotdb.commons.udf.service.UDFClassLoader;
 import org.apache.iotdb.commons.udf.service.UDFExecutableManager;
@@ -103,14 +106,20 @@ import org.apache.iotdb.db.client.DataNodeClientPoolFactory;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.sql.SemanticException;
 import org.apache.iotdb.db.metadata.template.ClusterTemplateManager;
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.metadata.template.TemplateAlterOperationType;
 import org.apache.iotdb.db.metadata.template.alter.TemplateAlterOperationUtil;
 import org.apache.iotdb.db.metadata.template.alter.TemplateExtendInfo;
+import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
+import org.apache.iotdb.db.mpp.plan.Coordinator;
 import org.apache.iotdb.db.mpp.plan.analyze.Analysis;
 import org.apache.iotdb.db.mpp.plan.analyze.Analyzer;
+import org.apache.iotdb.db.mpp.plan.analyze.ClusterPartitionFetcher;
+import org.apache.iotdb.db.mpp.plan.analyze.schema.ClusterSchemaFetcher;
+import org.apache.iotdb.db.mpp.plan.execution.ExecutionResult;
 import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.CountDatabaseTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.CountTimeSlotListTask;
@@ -168,7 +177,10 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchem
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.UnsetSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.AlterLogicalViewStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.CreateLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.DeleteLogicalViewStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.RenameLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.KillQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.pipe.CreatePipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.pipe.DropPipeStatement;
@@ -1758,6 +1770,157 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     return future;
   }
 
+  @Override
+  public SettableFuture<ConfigTaskResult> renameLogicalView(
+      String queryId, RenameLogicalViewStatement renameLogicalViewStatement) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+
+    // check path
+    PartialPath oldName = renameLogicalViewStatement.getOldName();
+    if (oldName.hasWildcard()) {
+      future.setException(
+          new MetadataException("Rename logical view doesn't support path pattern with wildcard."));
+      return future;
+    }
+
+    // fetch viewExpression
+    PathPatternTree patternTree = new PathPatternTree();
+    patternTree.appendFullPath(oldName);
+    patternTree.constructTree();
+    ISchemaTree schemaTree = ClusterSchemaFetcher.getInstance().fetchSchema(patternTree, null);
+    List<MeasurementPath> measurementPathList = schemaTree.searchMeasurementPaths(oldName).left;
+    if (measurementPathList.isEmpty()) {
+      future.setException(new PathNotExistException(oldName.getFullPath()));
+      return future;
+    }
+    LogicalViewSchema logicalViewSchema =
+        (LogicalViewSchema) measurementPathList.get(0).getMeasurementSchema();
+    ViewExpression viewExpression = logicalViewSchema.getExpression();
+
+    // create new view
+    CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement();
+    createLogicalViewStatement.setTargetFullPaths(
+        Collections.singletonList(renameLogicalViewStatement.getNewName()));
+    createLogicalViewStatement.setViewExpression(viewExpression);
+    ExecutionResult executionResult =
+        Coordinator.getInstance()
+            .execute(
+                createLogicalViewStatement,
+                0,
+                null,
+                "",
+                ClusterPartitionFetcher.getInstance(),
+                ClusterSchemaFetcher.getInstance(),
+                IoTDBDescriptor.getInstance().getConfig().getQueryTimeoutThreshold());
+    if (executionResult.status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      future.setException(
+          new IoTDBException(
+              executionResult.status.getMessage(), executionResult.status.getCode()));
+      return future;
+    }
+
+    // delete old view
+    TDeleteLogicalViewReq req =
+        new TDeleteLogicalViewReq(
+            queryId, serializePatternListToByteBuffer(Collections.singletonList(oldName)));
+    try (ConfigNodeClient client =
+        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
+      TSStatus tsStatus;
+      do {
+        try {
+          tsStatus = client.deleteLogicalView(req);
+        } catch (TTransportException e) {
+          if (e.getType() == TTransportException.TIMED_OUT
+              || e.getCause() instanceof SocketTimeoutException) {
+            // time out mainly caused by slow execution, wait until
+            tsStatus = RpcUtils.getStatus(TSStatusCode.OVERLAP_WITH_EXISTING_TASK);
+          } else {
+            throw e;
+          }
+        }
+        // keep waiting until task ends
+      } while (TSStatusCode.OVERLAP_WITH_EXISTING_TASK.getStatusCode() == tsStatus.getCode());
+
+      if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
+        LOGGER.warn("Failed to execute delete view {}, status is {}.", oldName, tsStatus);
+        future.setException(new IoTDBException(tsStatus.getMessage(), tsStatus.getCode()));
+      } else {
+        future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+      }
+    } catch (ClientManagerException | TException e) {
+      future.setException(e);
+    }
+    return future;
+  }
+
+  @Override
+  public SettableFuture<ConfigTaskResult> alterLogicalView(
+      String queryId, AlterLogicalViewStatement alterLogicalViewStatement) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    // delete old view
+    TDeleteLogicalViewReq req =
+        new TDeleteLogicalViewReq(
+            queryId,
+            serializePatternListToByteBuffer(alterLogicalViewStatement.getTargetPathList()));
+    try (ConfigNodeClient client =
+        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
+      TSStatus tsStatus;
+      do {
+        try {
+          tsStatus = client.deleteLogicalView(req);
+        } catch (TTransportException e) {
+          if (e.getType() == TTransportException.TIMED_OUT
+              || e.getCause() instanceof SocketTimeoutException) {
+            // time out mainly caused by slow execution, wait until
+            tsStatus = RpcUtils.getStatus(TSStatusCode.OVERLAP_WITH_EXISTING_TASK);
+          } else {
+            throw e;
+          }
+        }
+        // keep waiting until task ends
+      } while (TSStatusCode.OVERLAP_WITH_EXISTING_TASK.getStatusCode() == tsStatus.getCode());
+
+      if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
+        LOGGER.warn(
+            "Failed to execute delete view {}, status is {}.",
+            alterLogicalViewStatement.getTargetPathList(),
+            tsStatus);
+        future.setException(new IoTDBException(tsStatus.getMessage(), tsStatus.getCode()));
+        return future;
+      }
+    } catch (ClientManagerException | TException e) {
+      future.setException(e);
+      return future;
+    }
+
+    // recreate the logical view
+    CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement();
+    createLogicalViewStatement.setTargetPaths(alterLogicalViewStatement.getTargetPaths());
+    createLogicalViewStatement.setSourcePaths(alterLogicalViewStatement.getSourcePaths());
+    createLogicalViewStatement.setSourceQueryStatement(
+        alterLogicalViewStatement.getQueryStatement());
+
+    ExecutionResult executionResult =
+        Coordinator.getInstance()
+            .execute(
+                createLogicalViewStatement,
+                0,
+                null,
+                "",
+                ClusterPartitionFetcher.getInstance(),
+                ClusterSchemaFetcher.getInstance(),
+                IoTDBDescriptor.getInstance().getConfig().getQueryTimeoutThreshold());
+    if (executionResult.status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      future.setException(
+          new IoTDBException(
+              executionResult.status.getMessage(), executionResult.status.getCode()));
+    } else {
+      future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    }
+
+    return future;
+  }
+
   @Override
   public SettableFuture<ConfigTaskResult> getRegionId(GetRegionIdStatement getRegionIdStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
index aadfc3f7a7c..e88e57962fa 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
@@ -53,7 +53,9 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchem
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.UnsetSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.AlterLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.DeleteLogicalViewStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.RenameLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.KillQueryStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.pipe.CreatePipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.pipe.DropPipeStatement;
@@ -175,6 +177,12 @@ public interface IConfigTaskExecutor {
   SettableFuture<ConfigTaskResult> deleteLogicalView(
       String queryId, DeleteLogicalViewStatement deleteLogicalViewStatement);
 
+  SettableFuture<ConfigTaskResult> renameLogicalView(
+      String queryId, RenameLogicalViewStatement renameLogicalViewStatement);
+
+  SettableFuture<ConfigTaskResult> alterLogicalView(
+      String queryId, AlterLogicalViewStatement alterLogicalViewStatement);
+
   SettableFuture<ConfigTaskResult> getRegionId(GetRegionIdStatement getRegionIdStatement);
 
   SettableFuture<ConfigTaskResult> getSeriesSlotList(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/view/AlterLogicalViewTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/view/AlterLogicalViewTask.java
new file mode 100644
index 00000000000..11ad8979d9f
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/view/AlterLogicalViewTask.java
@@ -0,0 +1,44 @@
+/*
+ * 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.execution.config.metadata.view;
+
+import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
+import org.apache.iotdb.db.mpp.plan.execution.config.IConfigTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.executor.IConfigTaskExecutor;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.AlterLogicalViewStatement;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public class AlterLogicalViewTask implements IConfigTask {
+
+  private final String queryId;
+  private final AlterLogicalViewStatement alterLogicalViewStatement;
+
+  public AlterLogicalViewTask(String queryId, AlterLogicalViewStatement alterLogicalViewStatement) {
+    this.queryId = queryId;
+    this.alterLogicalViewStatement = alterLogicalViewStatement;
+  }
+
+  @Override
+  public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
+      throws InterruptedException {
+    return configTaskExecutor.alterLogicalView(queryId, alterLogicalViewStatement);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/view/RenameLogicalViewTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/view/RenameLogicalViewTask.java
new file mode 100644
index 00000000000..843aea5b45a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/view/RenameLogicalViewTask.java
@@ -0,0 +1,45 @@
+/*
+ * 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.execution.config.metadata.view;
+
+import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
+import org.apache.iotdb.db.mpp.plan.execution.config.IConfigTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.executor.IConfigTaskExecutor;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.RenameLogicalViewStatement;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public class RenameLogicalViewTask implements IConfigTask {
+
+  private final String queryId;
+  private final RenameLogicalViewStatement renameLogicalViewStatement;
+
+  public RenameLogicalViewTask(
+      String queryId, RenameLogicalViewStatement renameLogicalViewStatement) {
+    this.queryId = queryId;
+    this.renameLogicalViewStatement = renameLogicalViewStatement;
+  }
+
+  @Override
+  public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
+      throws InterruptedException {
+    return configTaskExecutor.renameLogicalView(queryId, renameLogicalViewStatement);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index 2bd2080cff0..57bf1fcee8e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -163,8 +163,10 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTempl
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathsUsingTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.UnsetSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.AlterLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.CreateLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.DeleteLogicalViewStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.RenameLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.ShowLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ClearCacheStatement;
@@ -227,6 +229,8 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
 import static org.apache.iotdb.db.constant.SqlConstant.CAST_FUNCTION;
@@ -999,9 +1003,16 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   public Statement visitCreateLogicalView(IoTDBSqlParser.CreateLogicalViewContext ctx) {
     CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement();
     // parse target
-    parseViewTargetPaths(ctx.viewTargetPaths(), createLogicalViewStatement);
+    parseViewTargetPaths(
+        ctx.viewTargetPaths(),
+        createLogicalViewStatement::setTargetFullPaths,
+        createLogicalViewStatement::setTargetPathsGroup);
     // parse source
-    parseViewSourcePaths(ctx.viewSourcePaths(), createLogicalViewStatement);
+    parseViewSourcePaths(
+        ctx.viewSourcePaths(),
+        createLogicalViewStatement::setSourceFullPaths,
+        createLogicalViewStatement::setSourcePathsGroup,
+        createLogicalViewStatement::setSourceQueryStatement);
 
     return createLogicalViewStatement;
   }
@@ -1043,6 +1054,32 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     return showLogicalViewStatement;
   }
 
+  @Override
+  public Statement visitRenameLogicalView(IoTDBSqlParser.RenameLogicalViewContext ctx) {
+    RenameLogicalViewStatement renameLogicalViewStatement = new RenameLogicalViewStatement();
+    renameLogicalViewStatement.setOldName(parsePrefixPath(ctx.prefixPath(0)));
+    renameLogicalViewStatement.setNewName(parsePrefixPath(ctx.prefixPath(1)));
+    return renameLogicalViewStatement;
+  }
+
+  @Override
+  public Statement visitAlterLogicalView(IoTDBSqlParser.AlterLogicalViewContext ctx) {
+    AlterLogicalViewStatement alterLogicalViewStatement = new AlterLogicalViewStatement();
+    // parse target
+    parseViewTargetPaths(
+        ctx.viewTargetPaths(),
+        alterLogicalViewStatement::setTargetFullPaths,
+        alterLogicalViewStatement::setTargetPathsGroup);
+    // parse source
+    parseViewSourcePaths(
+        ctx.viewSourcePaths(),
+        alterLogicalViewStatement::setSourceFullPaths,
+        alterLogicalViewStatement::setSourcePathsGroup,
+        alterLogicalViewStatement::setSourceQueryStatement);
+
+    return alterLogicalViewStatement;
+  }
+
   // parse suffix paths in logical view
   private PartialPath parseViewSuffixPath(IoTDBSqlParser.ViewSuffixPathsContext ctx) {
     List<IoTDBSqlParser.NodeNameWithoutWildcardContext> nodeNamesWithoutStar =
@@ -1057,7 +1094,8 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   // parse target paths in CreateLogicalView statement
   private void parseViewTargetPaths(
       IoTDBSqlParser.ViewTargetPathsContext ctx,
-      CreateLogicalViewStatement createLogicalViewStatement) {
+      Consumer<List<PartialPath>> setTargetFullPaths,
+      BiConsumer<PartialPath, List<PartialPath>> setTargetPathsGroup) {
     // full paths
     if (ctx.fullPath() != null && ctx.fullPath().size() > 0) {
       List<IoTDBSqlParser.FullPathContext> fullPathContextList = ctx.fullPath();
@@ -1065,7 +1103,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       for (IoTDBSqlParser.FullPathContext pathContext : fullPathContextList) {
         pathList.add(parseFullPath(pathContext));
       }
-      createLogicalViewStatement.setTargetFullPaths(pathList);
+      setTargetFullPaths.accept(pathList);
     }
     // prefix path and suffix paths
     if (ctx.prefixPath() != null
@@ -1078,14 +1116,16 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       for (IoTDBSqlParser.ViewSuffixPathsContext suffixPathContext : suffixPathContextList) {
         suffixPathList.add(parseViewSuffixPath(suffixPathContext));
       }
-      createLogicalViewStatement.setTargetPathsGroup(prefixPath, suffixPathList);
+      setTargetPathsGroup.accept(prefixPath, suffixPathList);
     }
   }
 
   // parse source paths in CreateLogicalView statement
   private void parseViewSourcePaths(
       IoTDBSqlParser.ViewSourcePathsContext ctx,
-      CreateLogicalViewStatement createLogicalViewStatement) {
+      Consumer<List<PartialPath>> setSourceFullPaths,
+      BiConsumer<PartialPath, List<PartialPath>> setSourcePathsGroup,
+      Consumer<QueryStatement> setSourceQueryStatement) {
     // full paths
     if (ctx.fullPath() != null && ctx.fullPath().size() > 0) {
       List<IoTDBSqlParser.FullPathContext> fullPathContextList = ctx.fullPath();
@@ -1093,7 +1133,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       for (IoTDBSqlParser.FullPathContext pathContext : fullPathContextList) {
         pathList.add(parseFullPath(pathContext));
       }
-      createLogicalViewStatement.setSourceFullPaths(pathList);
+      setSourceFullPaths.accept(pathList);
     }
     // prefix path and suffix paths
     if (ctx.prefixPath() != null
@@ -1106,13 +1146,13 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       for (IoTDBSqlParser.ViewSuffixPathsContext suffixPathContext : suffixPathContextList) {
         suffixPathList.add(parseViewSuffixPath(suffixPathContext));
       }
-      createLogicalViewStatement.setSourcePathsGroup(prefixPath, suffixPathList);
+      setSourcePathsGroup.accept(prefixPath, suffixPathList);
     }
     if (ctx.selectClause() != null && ctx.fromClause() != null) {
       QueryStatement queryStatement = new QueryStatement();
       queryStatement.setSelectComponent(parseSelectClause(ctx.selectClause(), queryStatement));
       queryStatement.setFromComponent(parseFromClause(ctx.fromClause()));
-      createLogicalViewStatement.setSourceQueryStatement(queryStatement);
+      setSourceQueryStatement.accept(queryStatement);
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
index b3397e563b6..cdc17b753b9 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
@@ -836,10 +836,15 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
     TransformToViewExpressionVisitor transformToViewExpressionVisitor =
         new TransformToViewExpressionVisitor();
     List<ViewExpression> viewExpressionList = new ArrayList<>();
-    List<Expression> expressionList = createLogicalViewStatement.getSourceExpressionList();
-    for (Expression expression : expressionList) {
-      viewExpressionList.add(transformToViewExpressionVisitor.process(expression, null));
+    if (createLogicalViewStatement.getViewExpression() == null) {
+      List<Expression> expressionList = createLogicalViewStatement.getSourceExpressionList();
+      for (Expression expression : expressionList) {
+        viewExpressionList.add(transformToViewExpressionVisitor.process(expression, null));
+      }
+    } else {
+      viewExpressionList.add(createLogicalViewStatement.getViewExpression());
     }
+
     return new CreateLogicalViewNode(
         context.getQueryId().genPlanNodeId(),
         createLogicalViewStatement.getTargetPathList(),
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementType.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementType.java
index 04403e9295d..6cf4d20ff1f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementType.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementType.java
@@ -170,4 +170,6 @@ public enum StatementType {
 
   CREATE_LOGICAL_VIEW,
   DELETE_LOGICAL_VIEW,
+  RENAME_LOGICAL_VIEW,
+  ALTER_LOGICAL_VIEW,
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
index 8a70c0cf927..5345b4c8ba8 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
@@ -90,8 +90,10 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTempl
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathsUsingTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.UnsetSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.AlterLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.CreateLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.DeleteLogicalViewStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.view.RenameLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.view.ShowLogicalViewStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ClearCacheStatement;
@@ -258,6 +260,15 @@ public abstract class StatementVisitor<R, C> {
     return visitStatement(showLogicalViewStatement, context);
   }
 
+  public R visitRenameLogicalView(
+      RenameLogicalViewStatement renameLogicalViewStatement, C context) {
+    return visitStatement(renameLogicalViewStatement, context);
+  }
+
+  public R visitAlterLogicalView(AlterLogicalViewStatement alterLogicalViewStatement, C context) {
+    return visitStatement(alterLogicalViewStatement, context);
+  }
+
   // ML Model
   public R visitCreateModel(CreateModelStatement createModelStatement, C context) {
     return visitStatement(createModelStatement, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/AlterLogicalViewStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/AlterLogicalViewStatement.java
new file mode 100644
index 00000000000..899d4f0b3be
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/AlterLogicalViewStatement.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.plan.statement.metadata.view;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.metadata.view.ViewPathType;
+import org.apache.iotdb.db.metadata.view.ViewPaths;
+import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
+import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement;
+import org.apache.iotdb.db.mpp.plan.statement.Statement;
+import org.apache.iotdb.db.mpp.plan.statement.StatementType;
+import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
+import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
+
+import java.util.List;
+
+public class AlterLogicalViewStatement extends Statement implements IConfigStatement {
+
+  // the paths of this view
+  private ViewPaths targetPaths;
+
+  // the paths of sources
+  private ViewPaths sourcePaths;
+  private QueryStatement queryStatement;
+
+  public AlterLogicalViewStatement() {
+    super();
+    this.statementType = StatementType.ALTER_LOGICAL_VIEW;
+    this.sourcePaths = new ViewPaths();
+    this.targetPaths = new ViewPaths();
+  }
+
+  // region Interfaces about setting and getting
+
+  // get paths
+  @Override
+  public List<PartialPath> getPaths() {
+    return this.getTargetPathList();
+  }
+
+  public ViewPaths getTargetPaths() {
+    return targetPaths;
+  }
+
+  public ViewPaths getSourcePaths() {
+    return sourcePaths;
+  }
+
+  public List<PartialPath> getTargetPathList() {
+    return this.targetPaths.fullPathList;
+  }
+
+  public QueryStatement getQueryStatement() {
+    return this.queryStatement;
+  }
+
+  // set source paths
+  public void setSourceFullPaths(List<PartialPath> paths) {
+    this.sourcePaths.setViewPathType(ViewPathType.FULL_PATH_LIST);
+    this.sourcePaths.setFullPathList(paths);
+  }
+
+  public void setSourcePathsGroup(PartialPath prefixPath, List<PartialPath> suffixPaths) {
+    this.sourcePaths.setViewPathType(ViewPathType.PATHS_GROUP);
+    this.sourcePaths.setPrefixOfPathsGroup(prefixPath);
+    this.sourcePaths.setSuffixOfPathsGroup(suffixPaths);
+    this.sourcePaths.generateFullPathsFromPathsGroup();
+  }
+
+  public void setSourceQueryStatement(QueryStatement queryStatement) {
+    this.sourcePaths.setViewPathType(ViewPathType.QUERY_STATEMENT);
+    this.queryStatement = queryStatement;
+  }
+
+  // set target paths
+  public void setTargetFullPaths(List<PartialPath> paths) {
+    this.targetPaths.setViewPathType(ViewPathType.FULL_PATH_LIST);
+    this.targetPaths.setFullPathList(paths);
+  }
+
+  public void setTargetPathsGroup(PartialPath prefixPath, List<PartialPath> suffixPaths) {
+    this.targetPaths.setViewPathType(ViewPathType.PATHS_GROUP);
+    this.targetPaths.setPrefixOfPathsGroup(prefixPath);
+    this.targetPaths.setSuffixOfPathsGroup(suffixPaths);
+    this.targetPaths.generateFullPathsFromPathsGroup();
+  }
+  // endregion
+
+  @Override
+  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
+    return visitor.visitAlterLogicalView(this, context);
+  }
+
+  @Override
+  public QueryType getQueryType() {
+    return QueryType.WRITE;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/CreateLogicalViewStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/CreateLogicalViewStatement.java
index e208be48b3a..e70c5772345 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/CreateLogicalViewStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/CreateLogicalViewStatement.java
@@ -20,32 +20,36 @@
 package org.apache.iotdb.db.mpp.plan.statement.metadata.view;
 
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression;
+import org.apache.iotdb.db.metadata.view.ViewPathType;
+import org.apache.iotdb.db.metadata.view.ViewPaths;
 import org.apache.iotdb.db.mpp.plan.expression.Expression;
-import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.db.mpp.plan.statement.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
 import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement;
 import org.apache.iotdb.tsfile.utils.Pair;
 
-import java.util.ArrayList;
 import java.util.List;
 
 /** CREATE LOGICAL VIEW statement. */
 public class CreateLogicalViewStatement extends Statement {
 
   // the paths of this view
-  private viewPaths targetPaths;
+  private ViewPaths targetPaths;
 
   // the paths of sources
-  private viewPaths sourcePaths;
+  private ViewPaths sourcePaths;
   private QueryStatement queryStatement;
 
+  // if not null, all related check and generation will be skipped
+  private ViewExpression viewExpression;
+
   public CreateLogicalViewStatement() {
     super();
     this.statementType = StatementType.CREATE_LOGICAL_VIEW;
-    this.sourcePaths = new viewPaths();
-    this.targetPaths = new viewPaths();
+    this.sourcePaths = new ViewPaths();
+    this.targetPaths = new ViewPaths();
   }
 
   // region Interfaces about setting and getting
@@ -56,6 +60,14 @@ public class CreateLogicalViewStatement extends Statement {
     return this.getTargetPathList();
   }
 
+  public ViewPaths getTargetPaths() {
+    return targetPaths;
+  }
+
+  public ViewPaths getSourcePaths() {
+    return sourcePaths;
+  }
+
   public List<PartialPath> getTargetPathList() {
     return this.targetPaths.fullPathList;
   }
@@ -69,7 +81,15 @@ public class CreateLogicalViewStatement extends Statement {
     return this.queryStatement;
   }
 
+  public ViewExpression getViewExpression() {
+    return viewExpression;
+  }
+
   // set source paths
+  public void setSourcePaths(ViewPaths sourcePaths) {
+    this.sourcePaths = sourcePaths;
+  }
+
   public void setSourceFullPaths(List<PartialPath> paths) {
     this.sourcePaths.setViewPathType(ViewPathType.FULL_PATH_LIST);
     this.sourcePaths.setFullPathList(paths);
@@ -98,6 +118,10 @@ public class CreateLogicalViewStatement extends Statement {
   }
 
   // set target paths
+  public void setTargetPaths(ViewPaths targetPaths) {
+    this.targetPaths = targetPaths;
+  }
+
   public void setTargetFullPaths(List<PartialPath> paths) {
     this.targetPaths.setViewPathType(ViewPathType.FULL_PATH_LIST);
     this.targetPaths.setFullPathList(paths);
@@ -110,6 +134,10 @@ public class CreateLogicalViewStatement extends Statement {
     this.targetPaths.generateFullPathsFromPathsGroup();
   }
 
+  public void setViewExpression(ViewExpression viewExpression) {
+    this.viewExpression = viewExpression;
+  }
+
   // endregion
 
   // region Interfaces for checking
@@ -165,84 +193,4 @@ public class CreateLogicalViewStatement extends Statement {
   public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
     return visitor.visitCreateLogicalView(this, context);
   }
-
-  // region private classes
-
-  private enum ViewPathType {
-    FULL_PATH_LIST,
-    PATHS_GROUP,
-    QUERY_STATEMENT
-  }
-
-  /**
-   * A private class to save all paths' info in targetPaths and sourcePaths except query statement.
-   *
-   * <p>fullPathList: CREATE VIEW root.db.device.temp AS root.ln.d.s01 PathGroup: CREATE VIEW
-   * root.db(device.temp, status) AS root.ln(d.s01, wf.abc.s02)
-   */
-  private class viewPaths {
-    public ViewPathType viewPathType = ViewPathType.FULL_PATH_LIST;
-    public List<PartialPath> fullPathList = null;
-    public PartialPath prefixOfPathsGroup = null;
-    public List<PartialPath> suffixOfPathsGroup = null;
-
-    public List<Expression> expressionsList = null;
-
-    public void addPath(PartialPath path) {
-      if (this.fullPathList == null) {
-        this.fullPathList = new ArrayList<>();
-        this.fullPathList.add(path);
-      } else {
-        this.fullPathList.add(path);
-      }
-    }
-
-    public void setFullPathList(List<PartialPath> pathList) {
-      this.fullPathList = pathList;
-    }
-
-    public void setPrefixOfPathsGroup(PartialPath path) {
-      this.prefixOfPathsGroup = path;
-    }
-
-    public void setSuffixOfPathsGroup(List<PartialPath> pathList) {
-      this.suffixOfPathsGroup = pathList;
-    }
-
-    public void setViewPathType(ViewPathType viewPathType) {
-      this.viewPathType = viewPathType;
-    }
-
-    public void generateFullPathsFromPathsGroup() {
-      if (prefixOfPathsGroup != null && suffixOfPathsGroup != null) {
-        this.fullPathList = new ArrayList<>();
-        for (PartialPath suffixPath : suffixOfPathsGroup) {
-          PartialPath pathToAdd = prefixOfPathsGroup.concatPath(suffixPath);
-          this.addPath(pathToAdd);
-        }
-      }
-    }
-
-    public void generateExpressionsIfNecessary() {
-      if (this.viewPathType == ViewPathType.FULL_PATH_LIST
-          || this.viewPathType == ViewPathType.PATHS_GROUP) {
-        if (this.fullPathList != null) {
-          this.expressionsList = new ArrayList<>();
-          for (PartialPath path : this.fullPathList) {
-            TimeSeriesOperand tsExpression = new TimeSeriesOperand(path);
-            this.expressionsList.add(tsExpression);
-          }
-        }
-      } else if (this.viewPathType == ViewPathType.QUERY_STATEMENT) {
-        // no nothing. expressions should be set by setExpressionsList
-      }
-    }
-
-    public void setExpressionsList(List<Expression> expressionsList) {
-      this.expressionsList = expressionsList;
-    }
-    // end of viewPaths
-  }
-
-  // endregion
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/RenameLogicalViewStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/RenameLogicalViewStatement.java
new file mode 100644
index 00000000000..08eea629c09
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/view/RenameLogicalViewStatement.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.plan.statement.metadata.view;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
+import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement;
+import org.apache.iotdb.db.mpp.plan.statement.Statement;
+import org.apache.iotdb.db.mpp.plan.statement.StatementType;
+import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class RenameLogicalViewStatement extends Statement implements IConfigStatement {
+
+  private PartialPath oldName;
+
+  private PartialPath newName;
+
+  public RenameLogicalViewStatement() {
+    super();
+    this.statementType = StatementType.RENAME_LOGICAL_VIEW;
+  }
+
+  public PartialPath getOldName() {
+    return oldName;
+  }
+
+  public void setOldName(PartialPath oldName) {
+    this.oldName = oldName;
+  }
+
+  public PartialPath getNewName() {
+    return newName;
+  }
+
+  public void setNewName(PartialPath newName) {
+    this.newName = newName;
+  }
+
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Arrays.asList(oldName, newName);
+  }
+
+  @Override
+  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
+    return visitor.visitRenameLogicalView(this, context);
+  }
+
+  @Override
+  public QueryType getQueryType() {
+    return QueryType.WRITE;
+  }
+}