You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2022/10/24 01:53:13 UTC

[iotdb] branch master updated: [IOTDB-3562] Support Deactivate Template in cluster (#7674)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f9f9721969 [IOTDB-3562] Support Deactivate Template in cluster (#7674)
f9f9721969 is described below

commit f9f9721969fb7a6b2bb49b22b4550315c51870ba
Author: Marcos_Zyk <38...@users.noreply.github.com>
AuthorDate: Mon Oct 24 09:53:08 2022 +0800

    [IOTDB-3562] Support Deactivate Template in cluster (#7674)
---
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |   9 +-
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4  |   4 +
 .../confignode/client/DataNodeRequestType.java     |   8 +-
 .../client/async/AsyncDataNodeClientPool.java      |  38 +-
 .../client/async/handlers/AsyncClientHandler.java  |   9 +-
 ...RPCHandler.java => DeleteSchemaRPCHandler.java} |   6 +-
 .../consensus/request/ConfigPhysicalPlan.java      |   4 +
 .../consensus/request/ConfigPhysicalPlanType.java  |   3 +-
 .../read/template/GetTemplateSetInfoPlan.java      |  68 +++
 .../consensus/response/TemplateSetInfoResp.java    |  41 +-
 .../confignode/manager/ClusterSchemaManager.java   |   7 +
 .../iotdb/confignode/manager/ConfigManager.java    |  47 ++
 .../apache/iotdb/confignode/manager/IManager.java  |   4 +
 .../iotdb/confignode/manager/ProcedureManager.java |  66 ++-
 .../persistence/executor/ConfigPlanExecutor.java   |   3 +
 .../persistence/schema/ClusterSchemaInfo.java      |  48 ++
 .../persistence/schema/TemplateTable.java          |  16 +
 .../impl/schema/DataNodeRegionGroupUtil.java       |  81 ++++
 .../procedure/impl/schema/DataNodeRegionTask.java  | 195 +++++++++
 .../impl/schema/DeactivateTemplateProcedure.java   | 487 +++++++++++++++++++++
 .../DeleteStorageGroupProcedure.java               |   5 +-
 .../DeleteTimeSeriesProcedure.java                 | 333 +++++---------
 .../DeactivateTemplateState.java}                  |   6 +-
 .../{ => schema}/DeleteStorageGroupState.java      |   2 +-
 .../state/{ => schema}/DeleteTimeSeriesState.java  |   2 +-
 .../procedure/store/ProcedureFactory.java          |  13 +-
 .../thrift/ConfigNodeRPCServiceProcessor.java      |   6 +
 .../request/ConfigPhysicalPlanSerDeTest.java       |   2 +-
 .../impl/DeactivateTemplateProcedureTest.java      |  89 ++++
 .../impl/DeleteStorageGroupProcedureTest.java      |   2 +-
 .../impl/DeleteTimeSeriesProcedureTest.java        |   2 +-
 docs/UserGuide/Operate-Metadata/Template.md        |  38 +-
 docs/zh/UserGuide/Operate-Metadata/Template.md     |  40 +-
 .../db/it/schema/IoTDBDeactivateTemplateIT.java    | 231 ++++++++++
 .../org/apache/iotdb/commons/path/PartialPath.java |   4 +
 .../schemaregion/rocksdb/RSchemaRegion.java        |  23 +-
 .../schemaregion/rocksdb/mnode/REntityMNode.java   |  20 +
 .../apache/iotdb/db/client/ConfigNodeClient.java   |  17 +
 .../iotdb/db/metadata/mnode/EntityMNode.java       |  38 ++
 .../iotdb/db/metadata/mnode/IEntityMNode.java      |   8 +
 .../iotdb/db/metadata/mnode/InternalMNode.java     |   8 +-
 .../iotdb/db/metadata/mtree/ConfigMTree.java       |  39 ++
 .../db/metadata/mtree/MTreeBelowSGMemoryImpl.java  | 102 +++--
 .../db/metadata/mtree/traverser/Traverser.java     |  19 +-
 .../traverser/collector/MeasurementCollector.java  |   3 +-
 .../plan/schemaregion/SchemaRegionPlanType.java    |   6 +-
 .../plan/schemaregion/SchemaRegionPlanVisitor.java |  17 +
 .../impl/DeactivateTemplatePlanImpl.java}          |  34 +-
 .../impl/PreDeactivateTemplatePlanImpl.java}       |  34 +-
 .../RollbackPreDeactivateTemplatePlanImpl.java}    |  34 +-
 .../impl/SchemaRegionPlanDeserializer.java         |  43 ++
 .../schemaregion/impl/SchemaRegionPlanFactory.java |   6 +
 .../impl/SchemaRegionPlanSerializer.java           |  51 +++
 .../impl/SchemaRegionPlanTxtSerializer.java        |  35 ++
 .../write/IDeactivateTemplatePlan.java}            |  36 +-
 .../write/IPreDeactivateTemplatePlan.java}         |  36 +-
 .../write/IRollbackPreDeactivateTemplatePlan.java  |  45 ++
 .../db/metadata/schemaregion/ISchemaRegion.java    |  15 +-
 .../schemaregion/SchemaRegionMemoryImpl.java       | 122 +++++-
 .../schemaregion/SchemaRegionSchemaFileImpl.java   |  23 +-
 .../iotdb/db/metadata/template/Template.java       |   5 +-
 .../metadata/visitor/SchemaExecutionVisitor.java   |  38 ++
 .../schema/PathsUsingTemplateScanOperator.java     |  21 +-
 .../apache/iotdb/db/mpp/plan/analyze/Analysis.java |  12 +
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  |  22 +-
 .../iotdb/db/mpp/plan/constant/StatementType.java  |   2 +
 .../plan/execution/config/ConfigTaskVisitor.java   |   8 +
 .../config/executor/ClusterConfigTaskExecutor.java |  78 +++-
 .../config/executor/IConfigTaskExecutor.java       |   4 +
 .../executor/StandaloneConfigTaskExecutor.java     |  12 +
 .../template/DeactivateSchemaTemplateTask.java     |  45 ++
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |  25 +-
 .../db/mpp/plan/parser/StatementGenerator.java     |   4 +-
 .../db/mpp/plan/planner/LogicalPlanBuilder.java    |   7 +-
 .../db/mpp/plan/planner/LogicalPlanVisitor.java    |   4 +-
 .../db/mpp/plan/planner/OperatorTreeGenerator.java |   2 +-
 .../plan/planner/distribution/SourceRewriter.java  |  88 +++-
 .../mpp/plan/planner/plan/node/PlanNodeType.java   |  14 +-
 .../db/mpp/plan/planner/plan/node/PlanVisitor.java |  15 +
 .../metedata/read/PathsUsingTemplateScanNode.java  |  36 +-
 .../node/metedata/read/SchemaQueryScanNode.java    |  10 +
 .../metedata/write/DeactivateTemplateNode.java     | 128 ++++++
 .../metedata/write/PreDeactivateTemplateNode.java  | 128 ++++++
 .../write/RollbackPreDeactivateTemplateNode.java   | 130 ++++++
 .../db/mpp/plan/statement/StatementVisitor.java    |   6 +
 ...ement.java => DeactivateTemplateStatement.java} |  44 +-
 .../template/ShowPathsUsingTemplateStatement.java  |  22 +-
 .../impl/DataNodeInternalRPCServiceImpl.java       | 157 ++++++-
 .../plan/SchemaRegionPlanCompatibilityTest.java    |  22 +-
 .../mpp/execution/operator/OperatorMemoryTest.java |   6 +-
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   2 +-
 .../src/main/thrift/confignode.thrift              |   7 +
 thrift/src/main/thrift/datanode.thrift             |  41 +-
 93 files changed, 3327 insertions(+), 551 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 13371dcdf3..5ef759a569 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
@@ -39,7 +39,7 @@ ddlStatement
     : setStorageGroup | createStorageGroup | createTimeseries
     | createSchemaTemplate | createTimeseriesOfSchemaTemplate
     | createFunction | createTrigger | createContinuousQuery
-    | alterTimeseries | deleteStorageGroup | deleteTimeseries | deletePartition
+    | alterTimeseries | deleteStorageGroup | deleteTimeseries | deletePartition | deleteTimeseriesOfSchemaTemplate
     | dropFunction | dropTrigger | dropContinuousQuery | dropSchemaTemplate
     | setTTL | unsetTTL | startTrigger | stopTrigger | setSchemaTemplate | unsetSchemaTemplate
     | showStorageGroup | showDevices | showTimeseries | showChildPaths | showChildNodes
@@ -206,6 +206,11 @@ deletePartition
     : DELETE PARTITION prefixPath INTEGER_LITERAL(COMMA INTEGER_LITERAL)*
     ;
 
+// Delete Timeseries of Schema Template
+deleteTimeseriesOfSchemaTemplate
+    : (DELETE TIMESERIES OF | DEACTIVATE) SCHEMA? TEMPLATE (templateName=identifier) ? FROM prefixPath (COMMA prefixPath)*
+    ;
+
 // Drop Function
 dropFunction
     : DROP FUNCTION udfName=identifier
@@ -361,7 +366,7 @@ showPathsSetSchemaTemplate
 
 // Show Paths Using Schema Template
 showPathsUsingSchemaTemplate
-    : SHOW PATHS USING SCHEMA? TEMPLATE templateName=identifier
+    : SHOW PATHS prefixPath? USING SCHEMA? TEMPLATE templateName=identifier
     ;
 
 // Count Storage Group
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
index af9c837c52..f595fcf9b1 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
@@ -162,6 +162,10 @@ DATANODES
     : D A T A N O D E S
     ;
 
+DEACTIVATE
+    : D E A C T I V A T E
+    ;
+
 DEBUG
     : D E B U G
     ;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java
index 5fa8b3b0fa..8e00389093 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java
@@ -75,6 +75,10 @@ public enum DataNodeRequestType {
   ROLLBACK_SCHEMA_BLACK_LIST,
   FETCH_SCHEMA_BLACK_LIST,
   INVALIDATE_MATCHED_SCHEMA_CACHE,
-  DELETE_DATA_FOR_DELETE_TIMESERIES,
-  DELETE_TIMESERIES
+  DELETE_DATA_FOR_DELETE_SCHEMA,
+  DELETE_TIMESERIES,
+
+  CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE,
+  ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE,
+  DEACTIVATE_TEMPLATE
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeClientPool.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeClientPool.java
index f2736fe121..79dcec7e37 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeClientPool.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeClientPool.java
@@ -28,16 +28,18 @@ import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient;
 import org.apache.iotdb.confignode.client.DataNodeRequestType;
 import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler;
 import org.apache.iotdb.confignode.client.async.handlers.rpc.AsyncTSStatusRPCHandler;
-import org.apache.iotdb.confignode.client.async.handlers.rpc.DeleteTimeSeriesRPCHandler;
+import org.apache.iotdb.confignode.client.async.handlers.rpc.DeleteSchemaRPCHandler;
 import org.apache.iotdb.confignode.client.async.handlers.rpc.FetchSchemaBlackListRPCHandler;
 import org.apache.iotdb.mpp.rpc.thrift.TActiveTriggerInstanceReq;
 import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListReq;
+import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListWithTemplateReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateFunctionRequest;
 import org.apache.iotdb.mpp.rpc.thrift.TCreatePipeOnDataNodeReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateTriggerInstanceReq;
-import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteTimeSeriesReq;
+import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq;
+import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDeleteTimeSeriesReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDropFunctionRequest;
 import org.apache.iotdb.mpp.rpc.thrift.TDropTriggerInstanceReq;
@@ -47,6 +49,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TInvalidateMatchedSchemaCacheReq;
 import org.apache.iotdb.mpp.rpc.thrift.TOperatePipeOnDataNodeReq;
 import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq;
 import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListReq;
+import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListWithTemplateReq;
 import org.apache.iotdb.mpp.rpc.thrift.TUpdateConfigNodeGroupReq;
 import org.apache.iotdb.mpp.rpc.thrift.TUpdateTriggerLocationReq;
 
@@ -220,13 +223,13 @@ public class AsyncDataNodeClientPool {
         case CONSTRUCT_SCHEMA_BLACK_LIST:
           client.constructSchemaBlackList(
               (TConstructSchemaBlackListReq) clientHandler.getRequest(requestId),
-              (DeleteTimeSeriesRPCHandler)
+              (DeleteSchemaRPCHandler)
                   clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
           break;
         case ROLLBACK_SCHEMA_BLACK_LIST:
           client.rollbackSchemaBlackList(
               (TRollbackSchemaBlackListReq) clientHandler.getRequest(requestId),
-              (DeleteTimeSeriesRPCHandler)
+              (DeleteSchemaRPCHandler)
                   clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
           break;
         case FETCH_SCHEMA_BLACK_LIST:
@@ -241,16 +244,16 @@ public class AsyncDataNodeClientPool {
               (AsyncTSStatusRPCHandler)
                   clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
           break;
-        case DELETE_DATA_FOR_DELETE_TIMESERIES:
-          client.deleteDataForDeleteTimeSeries(
-              (TDeleteDataForDeleteTimeSeriesReq) clientHandler.getRequest(requestId),
-              (DeleteTimeSeriesRPCHandler)
+        case DELETE_DATA_FOR_DELETE_SCHEMA:
+          client.deleteDataForDeleteSchema(
+              (TDeleteDataForDeleteSchemaReq) clientHandler.getRequest(requestId),
+              (DeleteSchemaRPCHandler)
                   clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
           break;
         case DELETE_TIMESERIES:
           client.deleteTimeSeries(
               (TDeleteTimeSeriesReq) clientHandler.getRequest(requestId),
-              (DeleteTimeSeriesRPCHandler)
+              (DeleteSchemaRPCHandler)
                   clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
           break;
         case PRE_CREATE_PIPE:
@@ -265,6 +268,23 @@ public class AsyncDataNodeClientPool {
               (AsyncTSStatusRPCHandler)
                   clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
           break;
+        case CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE:
+          client.constructSchemaBlackListWithTemplate(
+              (TConstructSchemaBlackListWithTemplateReq) clientHandler.getRequest(requestId),
+              (DeleteSchemaRPCHandler)
+                  clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
+          break;
+        case ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE:
+          client.rollbackSchemaBlackListWithTemplate(
+              (TRollbackSchemaBlackListWithTemplateReq) clientHandler.getRequest(requestId),
+              (DeleteSchemaRPCHandler)
+                  clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
+          break;
+        case DEACTIVATE_TEMPLATE:
+          client.deactivateTemplate(
+              (TDeactivateTemplateReq) clientHandler.getRequest(requestId),
+              (DeleteSchemaRPCHandler)
+                  clientHandler.createAsyncRPCHandler(requestId, targetDataNode));
         default:
           LOGGER.error(
               "Unexpected DataNode Request Type: {} when sendAsyncRequestToDataNode",
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/AsyncClientHandler.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/AsyncClientHandler.java
index d23bf29633..a013a15895 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/AsyncClientHandler.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/AsyncClientHandler.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.confignode.client.DataNodeRequestType;
 import org.apache.iotdb.confignode.client.async.handlers.rpc.AbstractAsyncRPCHandler;
 import org.apache.iotdb.confignode.client.async.handlers.rpc.AsyncTSStatusRPCHandler;
-import org.apache.iotdb.confignode.client.async.handlers.rpc.DeleteTimeSeriesRPCHandler;
+import org.apache.iotdb.confignode.client.async.handlers.rpc.DeleteSchemaRPCHandler;
 import org.apache.iotdb.confignode.client.async.handlers.rpc.FetchSchemaBlackListRPCHandler;
 import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListResp;
 
@@ -153,9 +153,12 @@ public class AsyncClientHandler<Q, R> {
     switch (requestType) {
       case CONSTRUCT_SCHEMA_BLACK_LIST:
       case ROLLBACK_SCHEMA_BLACK_LIST:
-      case DELETE_DATA_FOR_DELETE_TIMESERIES:
+      case DELETE_DATA_FOR_DELETE_SCHEMA:
       case DELETE_TIMESERIES:
-        return new DeleteTimeSeriesRPCHandler(
+      case CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE:
+      case ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE:
+      case DEACTIVATE_TEMPLATE:
+        return new DeleteSchemaRPCHandler(
             requestType,
             requestId,
             targetDataNode,
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DeleteTimeSeriesRPCHandler.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DeleteSchemaRPCHandler.java
similarity index 95%
rename from confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DeleteTimeSeriesRPCHandler.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DeleteSchemaRPCHandler.java
index c2d9074d6e..0b7fab3f40 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DeleteTimeSeriesRPCHandler.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DeleteSchemaRPCHandler.java
@@ -31,11 +31,11 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 
-public class DeleteTimeSeriesRPCHandler extends AsyncTSStatusRPCHandler {
+public class DeleteSchemaRPCHandler extends AsyncTSStatusRPCHandler {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(DeleteTimeSeriesRPCHandler.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(DeleteSchemaRPCHandler.class);
 
-  public DeleteTimeSeriesRPCHandler(
+  public DeleteSchemaRPCHandler(
       DataNodeRequestType requestType,
       int requestId,
       TDataNodeLocation targetDataNode,
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java
index 15257fbf94..63c0dcce60 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java
@@ -41,6 +41,7 @@ import org.apache.iotdb.confignode.consensus.request.read.template.GetAllSchemaT
 import org.apache.iotdb.confignode.consensus.request.read.template.GetAllTemplateSetInfoPlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetPathsSetTemplatePlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetSchemaTemplatePlan;
+import org.apache.iotdb.confignode.consensus.request.read.template.GetTemplateSetInfoPlan;
 import org.apache.iotdb.confignode.consensus.request.write.confignode.ApplyConfigNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.confignode.RemoveConfigNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.datanode.RegisterDataNodePlan;
@@ -272,6 +273,9 @@ public abstract class ConfigPhysicalPlan implements IConsensusRequest {
         case SetSchemaTemplate:
           req = new SetSchemaTemplatePlan();
           break;
+        case GetTemplateSetInfo:
+          req = new GetTemplateSetInfoPlan();
+          break;
         case GetNodePathsPartition:
           req = new GetNodePathsPartitionPlan();
           break;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java
index 8f5d6c1724..8699f91e22 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java
@@ -97,5 +97,6 @@ public enum ConfigPhysicalPlanType {
   UpdateTriggersOnTransferNodes,
   UpdateTriggerLocation,
   GetTransferringTriggers,
-  GetTriggerLocation
+  GetTriggerLocation,
+  GetTemplateSetInfo
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/template/GetTemplateSetInfoPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/template/GetTemplateSetInfoPlan.java
new file mode 100644
index 0000000000..e0e7707948
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/template/GetTemplateSetInfoPlan.java
@@ -0,0 +1,68 @@
+/*
+ * 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.confignode.consensus.request.read.template;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
+import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class GetTemplateSetInfoPlan extends ConfigPhysicalPlan {
+
+  private List<PartialPath> patternList;
+
+  public GetTemplateSetInfoPlan() {
+    super(ConfigPhysicalPlanType.GetTemplateSetInfo);
+  }
+
+  public GetTemplateSetInfoPlan(List<PartialPath> patternList) {
+    super(ConfigPhysicalPlanType.GetTemplateSetInfo);
+    this.patternList = patternList;
+  }
+
+  public List<PartialPath> getPatternList() {
+    return patternList;
+  }
+
+  @Override
+  protected void serializeImpl(DataOutputStream stream) throws IOException {
+    ReadWriteIOUtils.write(ConfigPhysicalPlanType.GetTemplateSetInfo.ordinal(), stream);
+    ReadWriteIOUtils.write(patternList.size(), stream);
+    for (PartialPath pattern : patternList) {
+      pattern.serialize(stream);
+    }
+  }
+
+  @Override
+  protected void deserializeImpl(ByteBuffer buffer) throws IOException {
+    int size = ReadWriteIOUtils.readInt(buffer);
+    patternList = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      patternList.add((PartialPath) PathDeserializeUtil.deserialize(buffer));
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/TemplateSetInfoResp.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
copy to confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/TemplateSetInfoResp.java
index a6780cb8f3..3f525c3005 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/TemplateSetInfoResp.java
@@ -16,32 +16,37 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+package org.apache.iotdb.confignode.consensus.response;
 
-import java.util.Map;
-
-public interface IEntityMNode extends IMNode {
-
-  boolean addAlias(String alias, IMeasurementMNode child);
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.consensus.common.DataSet;
+import org.apache.iotdb.db.metadata.template.Template;
 
-  void deleteAliasChild(String alias);
-
-  Map<String, IMeasurementMNode> getAliasChildren();
+import java.util.List;
+import java.util.Map;
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+public class TemplateSetInfoResp implements DataSet {
 
-  @Override
-  boolean isUseTemplate();
+  private TSStatus status;
+  private Map<PartialPath, List<Template>> patternTemplateMap;
 
-  void setUseTemplate(boolean useTemplate);
+  public TemplateSetInfoResp() {}
 
-  boolean isAligned();
+  public TSStatus getStatus() {
+    return status;
+  }
 
-  void setAligned(boolean isAligned);
+  public void setStatus(TSStatus status) {
+    this.status = status;
+  }
 
-  ILastCacheContainer getLastCacheContainer(String measurementId);
+  public Map<PartialPath, List<Template>> getPatternTemplateMap() {
+    return patternTemplateMap;
+  }
 
-  Map<String, ILastCacheContainer> getTemplateLastCaches();
+  public void setPatternTemplateMap(Map<PartialPath, List<Template>> patternTemplateMap) {
+    this.patternTemplateMap = patternTemplateMap;
+  }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java
index 7627405067..090736a582 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java
@@ -39,6 +39,7 @@ import org.apache.iotdb.confignode.consensus.request.read.template.GetAllSchemaT
 import org.apache.iotdb.confignode.consensus.request.read.template.GetAllTemplateSetInfoPlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetPathsSetTemplatePlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetSchemaTemplatePlan;
+import org.apache.iotdb.confignode.consensus.request.read.template.GetTemplateSetInfoPlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.AdjustMaxRegionGroupCountPlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.DeleteStorageGroupPlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.SetDataReplicationFactorPlan;
@@ -52,6 +53,7 @@ import org.apache.iotdb.confignode.consensus.response.AllTemplateSetInfoResp;
 import org.apache.iotdb.confignode.consensus.response.PathInfoResp;
 import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaResp;
 import org.apache.iotdb.confignode.consensus.response.TemplateInfoResp;
+import org.apache.iotdb.confignode.consensus.response.TemplateSetInfoResp;
 import org.apache.iotdb.confignode.exception.StorageGroupNotExistsException;
 import org.apache.iotdb.confignode.manager.node.NodeManager;
 import org.apache.iotdb.confignode.manager.partition.PartitionManager;
@@ -553,6 +555,11 @@ public class ClusterSchemaManager {
     return resp.getTemplateInfo();
   }
 
+  public TemplateSetInfoResp getTemplateSetInfo(List<PartialPath> patternList) {
+    return (TemplateSetInfoResp)
+        getConsensusManager().read(new GetTemplateSetInfoPlan(patternList)).getDataset();
+  }
+
   private NodeManager getNodeManager() {
     return configManager.getNodeManager();
   }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 73e2c79ca4..013494cb46 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -73,6 +73,7 @@ import org.apache.iotdb.confignode.consensus.response.RegionInfoListResp;
 import org.apache.iotdb.confignode.consensus.response.SchemaNodeManagementResp;
 import org.apache.iotdb.confignode.consensus.response.SchemaPartitionResp;
 import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaResp;
+import org.apache.iotdb.confignode.consensus.response.TemplateSetInfoResp;
 import org.apache.iotdb.confignode.consensus.statemachine.PartitionRegionStateMachine;
 import org.apache.iotdb.confignode.manager.load.LoadManager;
 import org.apache.iotdb.confignode.manager.node.NodeManager;
@@ -92,6 +93,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDeactivateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteTimeSeriesReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
@@ -120,6 +122,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.confignode.rpc.thrift.TShowStorageGroupResp;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.consensus.common.DataSet;
+import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 
@@ -127,6 +130,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -140,6 +144,7 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
 
 /** Entry of all management, AssignPartitionManager,AssignRegionManager. */
 public class ConfigManager implements IManager {
@@ -1016,6 +1021,48 @@ public class ConfigManager implements IManager {
     }
   }
 
+  @Override
+  public TSStatus deactivateSchemaTemplate(TDeactivateSchemaTemplateReq req) {
+    TSStatus status = confirmLeader();
+    if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      return status;
+    }
+
+    PathPatternTree patternTree =
+        PathPatternTree.deserialize(ByteBuffer.wrap(req.getPathPatternTree()));
+
+    List<PartialPath> patternList = patternTree.getAllPathPatterns();
+    TemplateSetInfoResp templateSetInfoResp = clusterSchemaManager.getTemplateSetInfo(patternList);
+    if (templateSetInfoResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      return templateSetInfoResp.getStatus();
+    }
+
+    Map<PartialPath, List<Template>> templateSetInfo = templateSetInfoResp.getPatternTemplateMap();
+    if (templateSetInfo.isEmpty()) {
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    }
+
+    if (!req.getTemplateName().equals(ONE_LEVEL_PATH_WILDCARD)) {
+      Map<PartialPath, List<Template>> filteredTemplateSetInfo = new HashMap<>();
+      for (Map.Entry<PartialPath, List<Template>> entry : templateSetInfo.entrySet()) {
+        for (Template template : entry.getValue()) {
+          if (template.getName().equals(req.getTemplateName())) {
+            filteredTemplateSetInfo.put(entry.getKey(), Collections.singletonList(template));
+            break;
+          }
+        }
+      }
+
+      if (filteredTemplateSetInfo.isEmpty()) {
+        return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+      }
+
+      templateSetInfo = filteredTemplateSetInfo;
+    }
+
+    return procedureManager.deactivateTemplate(req.getQueryId(), templateSetInfo);
+  }
+
   @Override
   public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) {
     TSStatus status = confirmLeader();
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index 8080014a44..ae682da23d 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -53,6 +53,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDeactivateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteTimeSeriesReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
@@ -400,6 +401,9 @@ public interface IManager {
    */
   TGetPathsSetTemplatesResp getPathsSetTemplate(String req);
 
+  /** deactivate schema template */
+  TSStatus deactivateSchemaTemplate(TDeactivateSchemaTemplateReq req);
+
   /*
    * delete timeseries
    *
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
index bf84a46e89..6683aa583c 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.exception.sync.PipeException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
 import org.apache.iotdb.commons.trigger.TriggerInformation;
 import org.apache.iotdb.commons.utils.StatusUtils;
@@ -41,9 +42,10 @@ import org.apache.iotdb.confignode.procedure.impl.DropTriggerProcedure;
 import org.apache.iotdb.confignode.procedure.impl.node.AddConfigNodeProcedure;
 import org.apache.iotdb.confignode.procedure.impl.node.RemoveConfigNodeProcedure;
 import org.apache.iotdb.confignode.procedure.impl.node.RemoveDataNodeProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteTimeSeriesProcedure;
 import org.apache.iotdb.confignode.procedure.impl.statemachine.CreateRegionGroupsProcedure;
-import org.apache.iotdb.confignode.procedure.impl.statemachine.DeleteStorageGroupProcedure;
-import org.apache.iotdb.confignode.procedure.impl.statemachine.DeleteTimeSeriesProcedure;
 import org.apache.iotdb.confignode.procedure.impl.statemachine.RegionMigrateProcedure;
 import org.apache.iotdb.confignode.procedure.impl.sync.CreatePipeProcedure;
 import org.apache.iotdb.confignode.procedure.impl.sync.DropPipeProcedure;
@@ -60,6 +62,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteTimeSeriesReq;
 import org.apache.iotdb.confignode.rpc.thrift.TRegionMigrateResultReportReq;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.tsfile.utils.Binary;
@@ -71,6 +74,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 public class ProcedureManager {
@@ -168,7 +172,7 @@ public class ProcedureManager {
       if (procedureId == -1) {
         if (hasOverlappedTask) {
           return RpcUtils.getStatus(
-              TSStatusCode.OVERLAP_WITH_EXISTING_DELETE_TIMESERIES_TASK,
+              TSStatusCode.OVERLAP_WITH_EXISTING_TASK,
               "Some other task is deleting some target timeseries.");
         }
         procedureId =
@@ -185,6 +189,62 @@ public class ProcedureManager {
     }
   }
 
+  public TSStatus deactivateTemplate(
+      String queryId, Map<PartialPath, List<Template>> templateSetInfo) {
+    long procedureId = -1;
+    synchronized (this) {
+      boolean hasOverlappedTask = false;
+      ProcedureFactory.ProcedureType type;
+      DeactivateTemplateProcedure deactivateTemplateProcedure;
+      for (Procedure<?> procedure : executor.getProcedures().values()) {
+        type = ProcedureFactory.getProcedureType(procedure);
+        if (type == null
+            || !type.equals(ProcedureFactory.ProcedureType.DEACTIVATE_TEMPLATE_PROCEDURE)) {
+          continue;
+        }
+        deactivateTemplateProcedure = (DeactivateTemplateProcedure) procedure;
+        if (queryId.equals(deactivateTemplateProcedure.getQueryId())) {
+          procedureId = deactivateTemplateProcedure.getProcId();
+          break;
+        }
+        for (PartialPath pattern : templateSetInfo.keySet()) {
+          for (PartialPath existingPattern :
+              deactivateTemplateProcedure.getTemplateSetInfo().keySet()) {
+            if (pattern.overlapWith(existingPattern)) {
+              hasOverlappedTask = true;
+              break;
+            }
+          }
+          if (hasOverlappedTask) {
+            break;
+          }
+        }
+        if (hasOverlappedTask) {
+          break;
+        }
+      }
+
+      if (procedureId == -1) {
+        if (hasOverlappedTask) {
+          return RpcUtils.getStatus(
+              TSStatusCode.OVERLAP_WITH_EXISTING_TASK,
+              "Some other task is deactivating some target template from target path.");
+        }
+        procedureId =
+            this.executor.submitProcedure(
+                new DeactivateTemplateProcedure(queryId, templateSetInfo));
+      }
+    }
+    List<TSStatus> procedureStatus = new ArrayList<>();
+    boolean isSucceed =
+        waitingProcedureFinished(Collections.singletonList(procedureId), procedureStatus);
+    if (isSucceed) {
+      return StatusUtils.OK;
+    } else {
+      return procedureStatus.get(0);
+    }
+  }
+
   /** Generate a AddConfigNodeProcedure, and serially execute all the AddConfigNodeProcedure */
   public void addConfigNode(TConfigNodeRegisterReq req) {
     AddConfigNodeProcedure addConfigNodeProcedure =
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
index 56fbf13318..477c075e50 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
@@ -42,6 +42,7 @@ import org.apache.iotdb.confignode.consensus.request.read.GetTriggerTablePlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.CheckTemplateSettablePlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetPathsSetTemplatePlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetSchemaTemplatePlan;
+import org.apache.iotdb.confignode.consensus.request.read.template.GetTemplateSetInfoPlan;
 import org.apache.iotdb.confignode.consensus.request.write.confignode.ApplyConfigNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.confignode.RemoveConfigNodePlan;
 import org.apache.iotdb.confignode.consensus.request.write.datanode.RegisterDataNodePlan;
@@ -182,6 +183,8 @@ public class ConfigPlanExecutor {
         return clusterSchemaInfo.getPathsSetTemplate((GetPathsSetTemplatePlan) req);
       case GetAllTemplateSetInfo:
         return clusterSchemaInfo.getAllTemplateSetInfo();
+      case GetTemplateSetInfo:
+        return clusterSchemaInfo.getTemplateSetInfo((GetTemplateSetInfoPlan) req);
       case GetPipeSink:
         return syncInfo.getPipeSink((GetPipeSinkPlan) req);
       case ShowPipe:
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
index 14700c298c..10888631af 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.confignode.consensus.request.read.GetStorageGroupPlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.CheckTemplateSettablePlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetPathsSetTemplatePlan;
 import org.apache.iotdb.confignode.consensus.request.read.template.GetSchemaTemplatePlan;
+import org.apache.iotdb.confignode.consensus.request.read.template.GetTemplateSetInfoPlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.AdjustMaxRegionGroupCountPlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.DeleteStorageGroupPlan;
 import org.apache.iotdb.confignode.consensus.request.write.storagegroup.SetDataReplicationFactorPlan;
@@ -45,6 +46,7 @@ import org.apache.iotdb.confignode.consensus.response.CountStorageGroupResp;
 import org.apache.iotdb.confignode.consensus.response.PathInfoResp;
 import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaResp;
 import org.apache.iotdb.confignode.consensus.response.TemplateInfoResp;
+import org.apache.iotdb.confignode.consensus.response.TemplateSetInfoResp;
 import org.apache.iotdb.confignode.exception.StorageGroupNotExistsException;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.db.metadata.mtree.ConfigMTree;
@@ -694,6 +696,52 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
     return new AllTemplateSetInfoResp(outputStream.toByteArray());
   }
 
+  /**
+   * Get the templateId set on paths covered by input path pattern. Resolve the input path patterns
+   * into specified path pattern start with template set path. The result set is organized as
+   * specified path pattern -> template id
+   */
+  public TemplateSetInfoResp getTemplateSetInfo(GetTemplateSetInfoPlan plan) {
+    TemplateSetInfoResp resp = new TemplateSetInfoResp();
+    try {
+
+      Map<PartialPath, Set<Integer>> allTemplateSetInfo = new HashMap<>();
+      for (PartialPath pattern : plan.getPatternList()) {
+        Map<Integer, Set<PartialPath>> templateSetInfo = mTree.getTemplateSetInfo(pattern);
+        if (templateSetInfo.isEmpty()) {
+          continue;
+        }
+        templateSetInfo.forEach(
+            (templateId, templateSetPathList) -> {
+              for (PartialPath templateSetPath : templateSetPathList) {
+                pattern
+                    .alterPrefixPath(templateSetPath)
+                    .forEach(
+                        path ->
+                            allTemplateSetInfo
+                                .computeIfAbsent(path, k -> new HashSet<>())
+                                .add(templateId));
+              }
+            });
+      }
+      Map<PartialPath, List<Template>> result = new HashMap<>();
+      for (Map.Entry<PartialPath, Set<Integer>> entry : allTemplateSetInfo.entrySet()) {
+        List<Template> templateList = new ArrayList<>(entry.getValue().size());
+        for (int templateId : entry.getValue()) {
+          templateList.add(templateTable.getTemplate(templateId));
+        }
+        result.put(entry.getKey(), templateList);
+      }
+      resp.setStatus(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS));
+      resp.setPatternTemplateMap(result);
+      return resp;
+    } catch (MetadataException e) {
+      LOGGER.error(e.getMessage(), e);
+      resp.setStatus(RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
+    }
+    return resp;
+  }
+
   public Map<String, TStorageGroupSchema> getMatchedStorageGroupSchemasByOneName(
       String[] storageGroupPathPattern) {
     Map<String, TStorageGroupSchema> schemaMap = new HashMap<>();
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/TemplateTable.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/TemplateTable.java
index cc50a9d2cd..864dd92af3 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/TemplateTable.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/TemplateTable.java
@@ -54,6 +54,7 @@ public class TemplateTable {
 
   private final AtomicInteger templateIdGenerator;
   private final Map<String, Template> templateMap = new ConcurrentHashMap<>();
+  private final Map<Integer, Template> templateIdMap = new ConcurrentHashMap<>();
 
   private final String snapshotFileName = "template_info.bin";
 
@@ -75,6 +76,19 @@ public class TemplateTable {
     }
   }
 
+  public Template getTemplate(int templateId) throws MetadataException {
+    try {
+      templateReadWriteLock.readLock().lock();
+      Template template = templateIdMap.get(templateId);
+      if (template == null) {
+        throw new MetadataException(String.format("Template with id=%s not exits", templateId));
+      }
+      return template;
+    } finally {
+      templateReadWriteLock.readLock().unlock();
+    }
+  }
+
   public List<Template> getAllTemplate() {
     try {
       templateReadWriteLock.readLock().lock();
@@ -95,6 +109,7 @@ public class TemplateTable {
       }
       template.setId(templateIdGenerator.getAndIncrement());
       this.templateMap.put(template.getName(), template);
+      templateIdMap.put(template.getId(), template);
     } finally {
       templateReadWriteLock.writeLock().unlock();
     }
@@ -123,6 +138,7 @@ public class TemplateTable {
     while (size > 0) {
       Template template = deserializeTemplate(byteBuffer);
       templateMap.put(template.getName(), template);
+      templateIdMap.put(template.getId(), template);
       size--;
     }
   }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionGroupUtil.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionGroupUtil.java
new file mode 100644
index 0000000000..a741d536ff
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionGroupUtil.java
@@ -0,0 +1,81 @@
+/*
+ * 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.confignode.procedure.impl.schema;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class DataNodeRegionGroupUtil {
+
+  private DataNodeRegionGroupUtil() {}
+
+  /**
+   * Try to get and execute request on consensus group leader as possible. If fail to get leader,
+   * select some other replica for execution.
+   */
+  static Map<TDataNodeLocation, List<TConsensusGroupId>> getLeaderDataNodeRegionGroupMap(
+      Map<TConsensusGroupId, Integer> leaderMap,
+      Map<TConsensusGroupId, TRegionReplicaSet> regionReplicaSetMap) {
+    Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap = new HashMap<>();
+    regionReplicaSetMap.forEach(
+        (consensusGroupId, regionReplicaSet) -> {
+          Integer leaderId = leaderMap.get(consensusGroupId);
+          TDataNodeLocation leaderDataNodeLocation = null;
+          if (leaderId == null || leaderId == -1) {
+            leaderDataNodeLocation = regionReplicaSet.getDataNodeLocations().get(0);
+          } else {
+            for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+              if (dataNodeLocation.getDataNodeId() == leaderId) {
+                leaderDataNodeLocation = dataNodeLocation;
+                break;
+              }
+            }
+          }
+          dataNodeConsensusGroupIdMap
+              .computeIfAbsent(leaderDataNodeLocation, k -> new ArrayList<>())
+              .add(regionReplicaSet.getRegionId());
+        });
+    return dataNodeConsensusGroupIdMap;
+  }
+
+  /**
+   * Try to execute request on all replica of one consensus group. If some replica failed, execute
+   * according request on some other replica and let consensus layer to sync it.
+   */
+  static Map<TDataNodeLocation, List<TConsensusGroupId>> getAllReplicaDataNodeRegionGroupMap(
+      Map<TConsensusGroupId, TRegionReplicaSet> regionReplicaSetMap) {
+    Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap = new HashMap<>();
+    regionReplicaSetMap.forEach(
+        (consensusGroupId, regionReplicaSet) -> {
+          for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+            dataNodeConsensusGroupIdMap
+                .computeIfAbsent(dataNodeLocation, k -> new ArrayList<>())
+                .add(regionReplicaSet.getRegionId());
+          }
+        });
+    return dataNodeConsensusGroupIdMap;
+  }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionTask.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionTask.java
new file mode 100644
index 0000000000..c18beba40c
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionTask.java
@@ -0,0 +1,195 @@
+/*
+ * 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.confignode.procedure.impl.schema;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.iotdb.confignode.procedure.impl.schema.DataNodeRegionGroupUtil.getAllReplicaDataNodeRegionGroupMap;
+import static org.apache.iotdb.confignode.procedure.impl.schema.DataNodeRegionGroupUtil.getLeaderDataNodeRegionGroupMap;
+
+/**
+ * This class takes the responsibility of sending region related requests to DataNode for execution,
+ * mainly focus on DataNode selection and retry on failure.
+ *
+ * @param <T>
+ */
+abstract class DataNodeRegionTask<T> {
+
+  protected final ConfigNodeProcedureEnv env;
+  protected final Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup;
+  protected final boolean executeOnAllReplicaset;
+
+  // all the success response of dataNodes
+  private final Map<Integer, List<T>> responseMap = new ConcurrentHashMap<>();
+
+  protected DataNodeRegionTask(
+      ConfigNodeProcedureEnv env,
+      Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup,
+      boolean executeOnAllReplicaset) {
+    this.env = env;
+    this.targetSchemaRegionGroup = targetSchemaRegionGroup;
+    this.executeOnAllReplicaset = executeOnAllReplicaset;
+  }
+
+  void execute() {
+    // organize region by dataNode
+    Set<TDataNodeLocation> allFailedDataNodeSet = new HashSet<>();
+    Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap =
+        executeOnAllReplicaset
+            ? getAllReplicaDataNodeRegionGroupMap(targetSchemaRegionGroup)
+            : getLeaderDataNodeRegionGroupMap(
+                env.getConfigManager().getPartitionManager().getAllLeadership(),
+                targetSchemaRegionGroup);
+    while (!dataNodeConsensusGroupIdMap.isEmpty()) {
+      Map<TDataNodeLocation, List<TConsensusGroupId>> currentFailedDataNodeMap =
+          sendRegionRequest(dataNodeConsensusGroupIdMap);
+      if (hasFailure()) {
+        // some dataNode execution failure
+        return;
+      }
+
+      if (currentFailedDataNodeMap.isEmpty()) {
+        // all succeeded
+        break;
+      }
+
+      // retry failed dataNode requests caused by unexpected error on other replicas on other
+      // dataNodes
+      currentFailedDataNodeMap.forEach(dataNodeConsensusGroupIdMap::remove);
+      // remove dataNodes that successfully executed request
+      allFailedDataNodeSet.removeAll(dataNodeConsensusGroupIdMap.keySet());
+      dataNodeConsensusGroupIdMap =
+          getAvailableDataNodeLocationForRetry(currentFailedDataNodeMap, allFailedDataNodeSet);
+      if (hasFailure()) {
+        // some consensus group has no available dataNode
+        return;
+      }
+    }
+  }
+
+  private Map<TDataNodeLocation, List<TConsensusGroupId>> sendRegionRequest(
+      Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap) {
+    // send request to each dataNode
+    Map<TDataNodeLocation, List<TConsensusGroupId>> failedDataNodeMap = new HashMap<>();
+    for (Map.Entry<TDataNodeLocation, List<TConsensusGroupId>> entry :
+        dataNodeConsensusGroupIdMap.entrySet()) {
+      // process response
+      Map<Integer, TSStatus> dataNodeResponseMap = sendRequest(entry.getKey(), entry.getValue());
+      TSStatus currentDataNodeResponse = dataNodeResponseMap.get(entry.getKey().getDataNodeId());
+      if (currentDataNodeResponse.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        if (currentDataNodeResponse.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
+          // dataNode execution error
+          onExecutionFailure(entry.getKey());
+          break;
+        } else {
+          // unexpected error, retry on other replicates on other dataNodes
+          failedDataNodeMap.put(entry.getKey(), entry.getValue());
+        }
+      }
+    }
+    return failedDataNodeMap;
+  }
+
+  private Map<TDataNodeLocation, List<TConsensusGroupId>> getAvailableDataNodeLocationForRetry(
+      Map<TDataNodeLocation, List<TConsensusGroupId>> failedDataNodeConsensusGroupIdMap,
+      Set<TDataNodeLocation> allFailedDataNodeSet) {
+    Map<TConsensusGroupId, Integer> leaderMap =
+        env.getConfigManager().getPartitionManager().getAllLeadership();
+    Map<TDataNodeLocation, List<TConsensusGroupId>> availableDataNodeLocation = new HashMap<>();
+    for (List<TConsensusGroupId> consensusGroupIdList :
+        failedDataNodeConsensusGroupIdMap.values()) {
+      for (TConsensusGroupId consensusGroupId : consensusGroupIdList) {
+        TRegionReplicaSet regionReplicaSet = targetSchemaRegionGroup.get(consensusGroupId);
+        TDataNodeLocation selectedDataNode = null;
+        Integer leaderId = leaderMap.get(consensusGroupId);
+        if (leaderId == null || leaderId == -1) {
+          for (TDataNodeLocation candidateDataNode : regionReplicaSet.getDataNodeLocations()) {
+            if (!allFailedDataNodeSet.contains(candidateDataNode)) {
+              // since leader of this group is unknown, take the first available one
+              selectedDataNode = candidateDataNode;
+              break;
+            }
+          }
+        } else {
+          for (TDataNodeLocation candidateDataNode : regionReplicaSet.getDataNodeLocations()) {
+            if (!allFailedDataNodeSet.contains(candidateDataNode)) {
+              if (leaderId == candidateDataNode.getDataNodeId()) {
+                // retry on the new leader as possible
+                selectedDataNode = candidateDataNode;
+                break;
+              }
+              if (selectedDataNode == null) {
+                selectedDataNode = candidateDataNode;
+              }
+            }
+          }
+        }
+
+        if (selectedDataNode == null) {
+          onAllReplicasetFailure(consensusGroupId);
+          return availableDataNodeLocation;
+        } else {
+          availableDataNodeLocation
+              .computeIfAbsent(selectedDataNode, k -> new ArrayList<>())
+              .add(consensusGroupId);
+        }
+      }
+    }
+    return availableDataNodeLocation;
+  }
+
+  protected void saveDataNodeResponse(Integer dataNodeId, T response) {
+    responseMap.computeIfAbsent(dataNodeId, k -> new ArrayList<>()).add(response);
+  }
+
+  Map<Integer, List<T>> getResponseMap() {
+    return responseMap;
+  }
+
+  /**
+   * This method should be implemented as constructing and send custom DataNode requests and collect
+   * response of each DataNode.
+   *
+   * @param dataNodeLocation the location of target DataNode
+   * @param consensusGroupIdList the target region group to execute request
+   * @return the execution result of each DataNode
+   */
+  protected abstract Map<Integer, TSStatus> sendRequest(
+      TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList);
+
+  protected abstract boolean hasFailure();
+
+  protected abstract void onExecutionFailure(TDataNodeLocation dataNodeLocation);
+
+  protected abstract void onAllReplicasetFailure(TConsensusGroupId consensusGroupId);
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java
new file mode 100644
index 0000000000..c51cb6349d
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java
@@ -0,0 +1,487 @@
+/*
+ * 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.confignode.procedure.impl.schema;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.confignode.client.DataNodeRequestType;
+import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool;
+import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureYieldException;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.schema.DeactivateTemplateState;
+import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListWithTemplateReq;
+import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq;
+import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidateMatchedSchemaCacheReq;
+import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListWithTemplateReq;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class DeactivateTemplateProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, DeactivateTemplateState> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DeactivateTemplateProcedure.class);
+
+  private String queryId;
+  private Map<PartialPath, List<Template>> templateSetInfo;
+
+  private transient String requestMessage;
+  // generate from templateSetInfo by concat pattern and measurement in template
+  private transient PathPatternTree timeSeriesPatternTree;
+  private transient ByteBuffer timeSeriesPatternTreeBytes;
+  private transient Map<String, List<Integer>> dataNodeRequest;
+
+  public DeactivateTemplateProcedure() {
+    super();
+  }
+
+  public DeactivateTemplateProcedure(
+      String queryId, Map<PartialPath, List<Template>> templateSetInfo) {
+    super();
+    this.queryId = queryId;
+    setTemplateSetInfo(templateSetInfo);
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, DeactivateTemplateState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    long startTime = System.currentTimeMillis();
+    try {
+      switch (state) {
+        case CONSTRUCT_BLACK_LIST:
+          LOGGER.info("Construct schema black list with template {}", requestMessage);
+          if (constructBlackList(env) > 0) {
+            setNextState(DeactivateTemplateState.CLEAN_DATANODE_SCHEMA_CACHE);
+            break;
+          } else {
+            return Flow.NO_MORE_STATE;
+          }
+        case CLEAN_DATANODE_SCHEMA_CACHE:
+          LOGGER.info("Invalidate cache of template timeseries {}", requestMessage);
+          invalidateCache(env);
+          break;
+        case DELETE_DATA:
+          LOGGER.info("Delete data of template timeseries {}", requestMessage);
+          deleteData(env);
+          break;
+        case DEACTIVATE_TEMPLATE:
+          LOGGER.info("Deactivate template of {}", requestMessage);
+          deactivateTemplate(env);
+          return Flow.NO_MORE_STATE;
+        default:
+          setFailure(new ProcedureException("Unrecognized state " + state.toString()));
+          return Flow.NO_MORE_STATE;
+      }
+      return Flow.HAS_MORE_STATE;
+    } finally {
+      LOGGER.info(
+          String.format(
+              "DeactivateTemplate-[%s] costs %sms",
+              state.toString(), (System.currentTimeMillis() - startTime)));
+    }
+  }
+
+  // return the total num of timeseries in schema black list
+  private int constructBlackList(ConfigNodeProcedureEnv env) {
+    Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup =
+        env.getConfigManager().getRelatedSchemaRegionGroup(timeSeriesPatternTree);
+    if (targetSchemaRegionGroup.isEmpty()) {
+      return 0;
+    }
+    DeactivateTemplateRegionTask<TSStatus> constructBlackListTask =
+        new DeactivateTemplateRegionTask<TSStatus>(
+            "construct schema black list", env, targetSchemaRegionGroup) {
+          @Override
+          protected Map<Integer, TSStatus> sendRequest(
+              TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
+            // construct request and send
+            Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
+            dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
+
+            AsyncClientHandler<TConstructSchemaBlackListWithTemplateReq, TSStatus> clientHandler =
+                new AsyncClientHandler<>(
+                    DataNodeRequestType.CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE,
+                    new TConstructSchemaBlackListWithTemplateReq(
+                        consensusGroupIdList, dataNodeRequest),
+                    dataNodeLocationMap);
+            AsyncDataNodeClientPool.getInstance()
+                .sendAsyncRequestToDataNodeWithRetry(clientHandler);
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
+            return clientHandler.getResponseMap();
+          }
+        };
+    constructBlackListTask.execute();
+
+    if (isFailed()) {
+      return 0;
+    }
+
+    int preDeletedNum = 0;
+    for (List<TSStatus> respList : constructBlackListTask.getResponseMap().values()) {
+      for (TSStatus resp : respList) {
+        preDeletedNum += Integer.parseInt(resp.getMessage());
+      }
+    }
+    return preDeletedNum;
+  }
+
+  private void invalidateCache(ConfigNodeProcedureEnv env) {
+    Map<Integer, TDataNodeLocation> dataNodeLocationMap =
+        env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations();
+    AsyncClientHandler<TInvalidateMatchedSchemaCacheReq, TSStatus> clientHandler =
+        new AsyncClientHandler<>(
+            DataNodeRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE,
+            new TInvalidateMatchedSchemaCacheReq(timeSeriesPatternTreeBytes),
+            dataNodeLocationMap);
+    AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler);
+    Map<Integer, TSStatus> statusMap = clientHandler.getResponseMap();
+    for (TSStatus status : statusMap.values()) {
+      // all dataNodes must clear the related schema cache
+      if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        LOGGER.error("Failed to invalidate schema cache of template timeseries {}", requestMessage);
+        setFailure(new ProcedureException(new MetadataException("Invalidate schema cache failed")));
+        return;
+      }
+    }
+
+    setNextState(DeactivateTemplateState.DELETE_DATA);
+  }
+
+  private void deleteData(ConfigNodeProcedureEnv env) {
+
+    Map<TConsensusGroupId, TRegionReplicaSet> relatedDataRegionGroup =
+        env.getConfigManager().getRelatedDataRegionGroup(timeSeriesPatternTree);
+
+    // target timeseries has no data
+    if (relatedDataRegionGroup.isEmpty()) {
+      return;
+    }
+
+    DeactivateTemplateRegionTask<TSStatus> deleteDataTask =
+        new DeactivateTemplateRegionTask<TSStatus>(
+            "delete data", env, relatedDataRegionGroup, true) {
+          @Override
+          protected Map<Integer, TSStatus> sendRequest(
+              TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
+            Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
+            dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
+            AsyncClientHandler<TDeleteDataForDeleteSchemaReq, TSStatus> clientHandler =
+                new AsyncClientHandler<>(
+                    DataNodeRequestType.DELETE_DATA_FOR_DELETE_SCHEMA,
+                    new TDeleteDataForDeleteSchemaReq(
+                        new ArrayList<>(consensusGroupIdList), timeSeriesPatternTreeBytes),
+                    dataNodeLocationMap);
+            AsyncDataNodeClientPool.getInstance()
+                .sendAsyncRequestToDataNodeWithRetry(clientHandler);
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
+            return clientHandler.getResponseMap();
+          }
+        };
+    deleteDataTask.execute();
+    setNextState(DeactivateTemplateState.DEACTIVATE_TEMPLATE);
+  }
+
+  private void deactivateTemplate(ConfigNodeProcedureEnv env) {
+    DeactivateTemplateRegionTask<TSStatus> deleteTimeSeriesTask =
+        new DeactivateTemplateRegionTask<TSStatus>(
+            "deactivate template schema",
+            env,
+            env.getConfigManager().getRelatedSchemaRegionGroup(timeSeriesPatternTree)) {
+          @Override
+          protected Map<Integer, TSStatus> sendRequest(
+              TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
+            Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
+            dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
+            AsyncClientHandler<TDeactivateTemplateReq, TSStatus> clientHandler =
+                new AsyncClientHandler<>(
+                    DataNodeRequestType.DEACTIVATE_TEMPLATE,
+                    new TDeactivateTemplateReq(consensusGroupIdList, dataNodeRequest),
+                    dataNodeLocationMap);
+            AsyncDataNodeClientPool.getInstance()
+                .sendAsyncRequestToDataNodeWithRetry(clientHandler);
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
+            return clientHandler.getResponseMap();
+          }
+        };
+    deleteTimeSeriesTask.execute();
+  }
+
+  @Override
+  protected void rollbackState(
+      ConfigNodeProcedureEnv env, DeactivateTemplateState deactivateTemplateState)
+      throws IOException, InterruptedException, ProcedureException {
+    DeactivateTemplateRegionTask<TSStatus> rollbackStateTask =
+        new DeactivateTemplateRegionTask<TSStatus>(
+            "roll back schema black list",
+            env,
+            env.getConfigManager().getRelatedSchemaRegionGroup(timeSeriesPatternTree)) {
+          @Override
+          protected Map<Integer, TSStatus> sendRequest(
+              TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
+            Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
+            dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
+            AsyncClientHandler<TRollbackSchemaBlackListWithTemplateReq, TSStatus> clientHandler =
+                new AsyncClientHandler<>(
+                    DataNodeRequestType.ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE,
+                    new TRollbackSchemaBlackListWithTemplateReq(
+                        consensusGroupIdList, dataNodeRequest),
+                    dataNodeLocationMap);
+            AsyncDataNodeClientPool.getInstance()
+                .sendAsyncRequestToDataNodeWithRetry(clientHandler);
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
+            return clientHandler.getResponseMap();
+          }
+        };
+    rollbackStateTask.execute();
+  }
+
+  @Override
+  protected boolean isRollbackSupported(DeactivateTemplateState deactivateTemplateState) {
+    return true;
+  }
+
+  @Override
+  protected DeactivateTemplateState getState(int stateId) {
+    return DeactivateTemplateState.values()[stateId];
+  }
+
+  @Override
+  protected int getStateId(DeactivateTemplateState deactivateTemplateState) {
+    return deactivateTemplateState.ordinal();
+  }
+
+  @Override
+  protected DeactivateTemplateState getInitialState() {
+    return DeactivateTemplateState.CONSTRUCT_BLACK_LIST;
+  }
+
+  public String getQueryId() {
+    return queryId;
+  }
+
+  public Map<PartialPath, List<Template>> getTemplateSetInfo() {
+    return templateSetInfo;
+  }
+
+  private void setTemplateSetInfo(Map<PartialPath, List<Template>> templateSetInfo) {
+    this.templateSetInfo = templateSetInfo;
+    prepareRequestMessage(templateSetInfo);
+    prepareTimeSeriesPatternTree(templateSetInfo);
+    prepareDataNodeRequest(templateSetInfo);
+  }
+
+  private void prepareRequestMessage(Map<PartialPath, List<Template>> templateSetInfo) {
+    StringBuilder stringBuilder = new StringBuilder();
+    stringBuilder.append("{");
+    for (Map.Entry<PartialPath, List<Template>> entry : templateSetInfo.entrySet()) {
+      stringBuilder
+          .append(entry.getKey())
+          .append(":")
+          .append(entry.getValue().stream().map(Template::getName).collect(Collectors.toList()))
+          .append(";");
+    }
+    stringBuilder.append("}");
+    this.requestMessage = stringBuilder.toString();
+  }
+
+  private void prepareTimeSeriesPatternTree(Map<PartialPath, List<Template>> templateSetInfo) {
+    PathPatternTree patternTree = new PathPatternTree();
+    for (Map.Entry<PartialPath, List<Template>> entry : templateSetInfo.entrySet()) {
+      for (Template template : entry.getValue()) {
+        for (String measurement : template.getSchemaMap().keySet()) {
+          patternTree.appendPathPattern(entry.getKey().concatNode(measurement));
+        }
+      }
+    }
+    patternTree.constructTree();
+    this.timeSeriesPatternTree = patternTree;
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+    try {
+      patternTree.serialize(dataOutputStream);
+    } catch (IOException ignored) {
+
+    }
+    this.timeSeriesPatternTreeBytes = ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  private void prepareDataNodeRequest(Map<PartialPath, List<Template>> templateSetInfo) {
+    Map<String, List<Integer>> dataNodeRequest = new HashMap<>();
+    templateSetInfo.forEach(
+        (k, v) ->
+            dataNodeRequest.put(
+                k.getFullPath(), v.stream().map(Template::getId).collect(Collectors.toList())));
+    this.dataNodeRequest = dataNodeRequest;
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    stream.writeInt(ProcedureFactory.ProcedureType.DEACTIVATE_TEMPLATE_PROCEDURE.ordinal());
+    super.serialize(stream);
+    ReadWriteIOUtils.write(queryId, stream);
+    ReadWriteIOUtils.write(templateSetInfo.size(), stream);
+    for (Map.Entry<PartialPath, List<Template>> entry : templateSetInfo.entrySet()) {
+      entry.getKey().serialize(stream);
+      ReadWriteIOUtils.write(entry.getValue().size(), stream);
+      for (Template template : entry.getValue()) {
+        template.serialize(stream);
+      }
+    }
+  }
+
+  @Override
+  public void deserialize(ByteBuffer byteBuffer) {
+    super.deserialize(byteBuffer);
+    queryId = ReadWriteIOUtils.readString(byteBuffer);
+    int size = ReadWriteIOUtils.readInt(byteBuffer);
+    Map<PartialPath, List<Template>> templateSetInfo = new HashMap<>();
+    for (int i = 0; i < size; i++) {
+      PartialPath pattern = (PartialPath) PathDeserializeUtil.deserialize(byteBuffer);
+      int templateNum = ReadWriteIOUtils.readInt(byteBuffer);
+      List<Template> templateList = new ArrayList<>(templateNum);
+      for (int j = 0; j < templateNum; j++) {
+        Template template = new Template();
+        template.deserialize(byteBuffer);
+        templateList.add(template);
+      }
+      templateSetInfo.put(pattern, templateList);
+    }
+    setTemplateSetInfo(templateSetInfo);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    DeactivateTemplateProcedure that = (DeactivateTemplateProcedure) o;
+    return Objects.equals(queryId, that.queryId)
+        && Objects.equals(templateSetInfo, that.templateSetInfo);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(queryId, templateSetInfo);
+  }
+
+  private abstract class DeactivateTemplateRegionTask<T> extends DataNodeRegionTask<T> {
+
+    private final String taskName;
+
+    DeactivateTemplateRegionTask(
+        String taskName,
+        ConfigNodeProcedureEnv env,
+        Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup) {
+      super(env, targetSchemaRegionGroup, false);
+      this.taskName = taskName;
+    }
+
+    DeactivateTemplateRegionTask(
+        String taskName,
+        ConfigNodeProcedureEnv env,
+        Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup,
+        boolean executeOnAllReplicaset) {
+      super(env, targetSchemaRegionGroup, executeOnAllReplicaset);
+      this.taskName = taskName;
+    }
+
+    @Override
+    protected boolean hasFailure() {
+      return isFailed();
+    }
+
+    @Override
+    protected void onExecutionFailure(TDataNodeLocation dataNodeLocation) {
+      LOGGER.error(
+          "Failed to execute [{}] of deactivate template {} on {}",
+          taskName,
+          requestMessage,
+          dataNodeLocation);
+      setFailure(
+          new ProcedureException(
+              new MetadataException(
+                  String.format(
+                      "Deactivate template of %s failed when [%s]", requestMessage, taskName))));
+    }
+
+    @Override
+    protected void onAllReplicasetFailure(TConsensusGroupId consensusGroupId) {
+      setFailure(
+          new ProcedureException(
+              new MetadataException(
+                  String.format(
+                      "Deactivate template of %s failed when [%s] because all replicaset of schemaRegion %s failed.",
+                      requestMessage, taskName, consensusGroupId.id))));
+    }
+  }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/statemachine/DeleteStorageGroupProcedure.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteStorageGroupProcedure.java
similarity index 97%
rename from confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/statemachine/DeleteStorageGroupProcedure.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteStorageGroupProcedure.java
index c80d6e3ae6..e6e22e3e73 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/statemachine/DeleteStorageGroupProcedure.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteStorageGroupProcedure.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.confignode.procedure.impl.statemachine;
+package org.apache.iotdb.confignode.procedure.impl.schema;
 
 import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
@@ -30,7 +30,8 @@ import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureYieldException;
-import org.apache.iotdb.confignode.procedure.state.DeleteStorageGroupState;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.schema.DeleteStorageGroupState;
 import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.rpc.TSStatusCode;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/statemachine/DeleteTimeSeriesProcedure.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java
similarity index 60%
rename from confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/statemachine/DeleteTimeSeriesProcedure.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java
index 645865efdc..2300839e67 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/statemachine/DeleteTimeSeriesProcedure.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.confignode.procedure.impl.statemachine;
+package org.apache.iotdb.confignode.procedure.impl.schema;
 
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
@@ -33,11 +33,12 @@ import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureYieldException;
-import org.apache.iotdb.confignode.procedure.state.DeleteTimeSeriesState;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.schema.DeleteTimeSeriesState;
 import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListReq;
-import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteTimeSeriesReq;
+import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDeleteTimeSeriesReq;
 import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListReq;
 import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListResp;
@@ -54,16 +55,14 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 
+import static org.apache.iotdb.confignode.procedure.impl.schema.DataNodeRegionGroupUtil.getLeaderDataNodeRegionGroupMap;
+
 public class DeleteTimeSeriesProcedure
     extends StateMachineProcedure<ConfigNodeProcedureEnv, DeleteTimeSeriesState> {
 
@@ -138,10 +137,11 @@ public class DeleteTimeSeriesProcedure
     if (targetSchemaRegionGroup.isEmpty()) {
       return 0;
     }
-    RegionTask<TSStatus> constructBlackListTask =
-        new RegionTask<TSStatus>("construct schema black list", env, targetSchemaRegionGroup) {
+    DeleteTimeSeriesRegionTask<TSStatus> constructBlackListTask =
+        new DeleteTimeSeriesRegionTask<TSStatus>(
+            "construct schema black list", env, targetSchemaRegionGroup) {
           @Override
-          Map<Integer, TSStatus> sendRequest(
+          protected Map<Integer, TSStatus> sendRequest(
               TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
             // construct request and send
             Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
@@ -154,7 +154,14 @@ public class DeleteTimeSeriesProcedure
                     dataNodeLocationMap);
             AsyncDataNodeClientPool.getInstance()
                 .sendAsyncRequestToDataNodeWithRetry(clientHandler);
-            responseMap.putAll(clientHandler.getResponseMap());
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
             return clientHandler.getResponseMap();
           }
         };
@@ -165,8 +172,10 @@ public class DeleteTimeSeriesProcedure
     }
 
     int preDeletedNum = 0;
-    for (TSStatus resp : constructBlackListTask.getResponseMap().values()) {
-      preDeletedNum += Integer.parseInt(resp.getMessage());
+    for (List<TSStatus> respList : constructBlackListTask.getResponseMap().values()) {
+      for (TSStatus resp : respList) {
+        preDeletedNum += Integer.parseInt(resp.getMessage());
+      }
     }
     return preDeletedNum;
   }
@@ -263,38 +272,44 @@ public class DeleteTimeSeriesProcedure
       return;
     }
 
-    RegionTask<TSStatus> deleteDataTask =
-        new RegionTask<TSStatus>("delete data", env, relatedDataRegionGroup) {
+    DeleteTimeSeriesRegionTask<TSStatus> deleteDataTask =
+        new DeleteTimeSeriesRegionTask<TSStatus>("delete data", env, relatedDataRegionGroup, true) {
           @Override
-          Map<Integer, TSStatus> sendRequest(
+          protected Map<Integer, TSStatus> sendRequest(
               TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
             Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
             dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
-            AsyncClientHandler<TDeleteDataForDeleteTimeSeriesReq, TSStatus> clientHandler =
+            AsyncClientHandler<TDeleteDataForDeleteSchemaReq, TSStatus> clientHandler =
                 new AsyncClientHandler<>(
-                    DataNodeRequestType.DELETE_DATA_FOR_DELETE_TIMESERIES,
-                    new TDeleteDataForDeleteTimeSeriesReq(
+                    DataNodeRequestType.DELETE_DATA_FOR_DELETE_SCHEMA,
+                    new TDeleteDataForDeleteSchemaReq(
                         new ArrayList<>(consensusGroupIdList),
                         preparePatternTreeBytesData(patternTree)),
                     dataNodeLocationMap);
             AsyncDataNodeClientPool.getInstance()
                 .sendAsyncRequestToDataNodeWithRetry(clientHandler);
-            responseMap.putAll(clientHandler.getResponseMap());
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
             return clientHandler.getResponseMap();
           }
         };
-    deleteDataTask.setExecuteOnAllReplicaset(true);
     deleteDataTask.execute();
   }
 
   private PathPatternTree fetchSchemaBlackListOnTargetDataNode(
       ConfigNodeProcedureEnv env,
       Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup) {
-    RegionTask<TFetchSchemaBlackListResp> fetchSchemaBlackListTask =
-        new RegionTask<TFetchSchemaBlackListResp>(
+    DeleteTimeSeriesRegionTask<TFetchSchemaBlackListResp> fetchSchemaBlackListTask =
+        new DeleteTimeSeriesRegionTask<TFetchSchemaBlackListResp>(
             "fetch schema black list", env, targetSchemaRegionGroup) {
           @Override
-          Map<Integer, TSStatus> sendRequest(
+          protected Map<Integer, TSStatus> sendRequest(
               TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
             Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
             dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
@@ -305,9 +320,16 @@ public class DeleteTimeSeriesProcedure
                     dataNodeLocationMap);
             AsyncDataNodeClientPool.getInstance()
                 .sendAsyncRequestToDataNodeWithRetry(clientHandler);
-            responseMap.putAll(clientHandler.getResponseMap());
             Map<Integer, TSStatus> statusMap = new HashMap<>();
-            clientHandler.getResponseMap().forEach((k, v) -> statusMap.put(k, v.getStatus()));
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                      statusMap.put(k, v.getStatus());
+                    });
             return statusMap;
           }
         };
@@ -316,13 +338,16 @@ public class DeleteTimeSeriesProcedure
       return null;
     }
 
-    Map<Integer, TFetchSchemaBlackListResp> respMap = fetchSchemaBlackListTask.getResponseMap();
+    Map<Integer, List<TFetchSchemaBlackListResp>> respMap =
+        fetchSchemaBlackListTask.getResponseMap();
     PathPatternTree patternTree = new PathPatternTree();
-    for (TFetchSchemaBlackListResp resp : respMap.values()) {
-      for (PartialPath path :
-          PathPatternTree.deserialize(ByteBuffer.wrap(resp.getPathPatternTree()))
-              .getAllPathPatterns()) {
-        patternTree.appendFullPath(path);
+    for (List<TFetchSchemaBlackListResp> respList : respMap.values()) {
+      for (TFetchSchemaBlackListResp resp : respList) {
+        for (PartialPath path :
+            PathPatternTree.deserialize(ByteBuffer.wrap(resp.getPathPatternTree()))
+                .getAllPathPatterns()) {
+          patternTree.appendFullPath(path);
+        }
       }
     }
     patternTree.constructTree();
@@ -330,13 +355,13 @@ public class DeleteTimeSeriesProcedure
   }
 
   private void deleteTimeSeriesSchema(ConfigNodeProcedureEnv env) {
-    RegionTask<TSStatus> deleteTimeSeriesTask =
-        new RegionTask<TSStatus>(
+    DeleteTimeSeriesRegionTask<TSStatus> deleteTimeSeriesTask =
+        new DeleteTimeSeriesRegionTask<TSStatus>(
             "delete timeseries schema",
             env,
             env.getConfigManager().getRelatedSchemaRegionGroup(patternTree)) {
           @Override
-          Map<Integer, TSStatus> sendRequest(
+          protected Map<Integer, TSStatus> sendRequest(
               TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
             Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
             dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
@@ -347,71 +372,31 @@ public class DeleteTimeSeriesProcedure
                     dataNodeLocationMap);
             AsyncDataNodeClientPool.getInstance()
                 .sendAsyncRequestToDataNodeWithRetry(clientHandler);
-            responseMap.putAll(clientHandler.getResponseMap());
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
             return clientHandler.getResponseMap();
           }
         };
     deleteTimeSeriesTask.execute();
   }
 
-  /**
-   * Try to get and execute request on consensus group leader as possible. If fail to get leader,
-   * select some other replica for execution.
-   */
-  private Map<TDataNodeLocation, List<TConsensusGroupId>> getLeaderDataNodeRegionGroupMap(
-      Map<TConsensusGroupId, Integer> leaderMap,
-      Map<TConsensusGroupId, TRegionReplicaSet> regionReplicaSetMap) {
-    Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap = new HashMap<>();
-    regionReplicaSetMap.forEach(
-        (consensusGroupId, regionReplicaSet) -> {
-          Integer leaderId = leaderMap.get(consensusGroupId);
-          TDataNodeLocation leaderDataNodeLocation = null;
-          if (leaderId == null || leaderId == -1) {
-            leaderDataNodeLocation = regionReplicaSet.getDataNodeLocations().get(0);
-          } else {
-            for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
-              if (dataNodeLocation.getDataNodeId() == leaderId) {
-                leaderDataNodeLocation = dataNodeLocation;
-                break;
-              }
-            }
-          }
-          dataNodeConsensusGroupIdMap
-              .computeIfAbsent(leaderDataNodeLocation, k -> new ArrayList<>())
-              .add(regionReplicaSet.getRegionId());
-        });
-    return dataNodeConsensusGroupIdMap;
-  }
-
-  /**
-   * Try to execute request on all replica of one consensus group. If some replica failed, execute
-   * according request on some other replica and let consensus layer to sync it.
-   */
-  private Map<TDataNodeLocation, List<TConsensusGroupId>> getAllReplicaDataNodeRegionGroupMap(
-      Map<TConsensusGroupId, TRegionReplicaSet> regionReplicaSetMap) {
-    Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap = new HashMap<>();
-    regionReplicaSetMap.forEach(
-        (consensusGroupId, regionReplicaSet) -> {
-          for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
-            dataNodeConsensusGroupIdMap
-                .computeIfAbsent(dataNodeLocation, k -> new ArrayList<>())
-                .add(regionReplicaSet.getRegionId());
-          }
-        });
-    return dataNodeConsensusGroupIdMap;
-  }
-
   @Override
   protected void rollbackState(
       ConfigNodeProcedureEnv env, DeleteTimeSeriesState deleteTimeSeriesState)
       throws IOException, InterruptedException, ProcedureException {
-    RegionTask<TSStatus> rollbackStateTask =
-        new RegionTask<TSStatus>(
+    DeleteTimeSeriesRegionTask<TSStatus> rollbackStateTask =
+        new DeleteTimeSeriesRegionTask<TSStatus>(
             "roll back schema black list",
             env,
             env.getConfigManager().getRelatedSchemaRegionGroup(patternTree)) {
           @Override
-          Map<Integer, TSStatus> sendRequest(
+          protected Map<Integer, TSStatus> sendRequest(
               TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList) {
             Map<Integer, TDataNodeLocation> dataNodeLocationMap = new HashMap<>();
             dataNodeLocationMap.put(dataNodeLocation.getDataNodeId(), dataNodeLocation);
@@ -422,7 +407,14 @@ public class DeleteTimeSeriesProcedure
                     dataNodeLocationMap);
             AsyncDataNodeClientPool.getInstance()
                 .sendAsyncRequestToDataNodeWithRetry(clientHandler);
-            responseMap.putAll(clientHandler.getResponseMap());
+            clientHandler
+                .getResponseMap()
+                .forEach(
+                    (k, v) -> {
+                      if (v.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+                        saveDataNodeResponse(k, v);
+                      }
+                    });
             return clientHandler.getResponseMap();
           }
         };
@@ -504,155 +496,54 @@ public class DeleteTimeSeriesProcedure
     return Objects.hash(getProcId(), getState(), patternTree);
   }
 
-  private abstract class RegionTask<T> {
+  private abstract class DeleteTimeSeriesRegionTask<T> extends DataNodeRegionTask<T> {
 
     private final String taskName;
-    private final ConfigNodeProcedureEnv env;
-    private final Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup;
-
-    private boolean executeOnAllReplicaset = false;
 
-    protected Map<Integer, T> responseMap = new ConcurrentHashMap<>();
-
-    RegionTask(
+    DeleteTimeSeriesRegionTask(
         String taskName,
         ConfigNodeProcedureEnv env,
         Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup) {
+      super(env, targetSchemaRegionGroup, false);
       this.taskName = taskName;
-      this.env = env;
-      this.targetSchemaRegionGroup = targetSchemaRegionGroup;
-    }
-
-    private void execute() {
-      // organize schema region by dataNode
-      Set<TDataNodeLocation> allFailedDataNodeSet = new HashSet<>();
-      Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap =
-          executeOnAllReplicaset
-              ? getAllReplicaDataNodeRegionGroupMap(targetSchemaRegionGroup)
-              : getLeaderDataNodeRegionGroupMap(
-                  env.getConfigManager().getPartitionManager().getAllLeadership(),
-                  targetSchemaRegionGroup);
-      while (!dataNodeConsensusGroupIdMap.isEmpty()) {
-        Map<TDataNodeLocation, List<TConsensusGroupId>> currentFailedDataNodeMap =
-            sendRegionRequest(dataNodeConsensusGroupIdMap);
-        if (isFailed()) {
-          // some dataNode execution failure
-          return;
-        }
-
-        if (currentFailedDataNodeMap.isEmpty()) {
-          // all succeeded
-          break;
-        }
-
-        // retry failed dataNode requests caused by unexpected error on other replicas on other
-        // dataNodes
-        currentFailedDataNodeMap.forEach(dataNodeConsensusGroupIdMap::remove);
-        // remove dataNodes that successfully executed request
-        allFailedDataNodeSet.removeAll(dataNodeConsensusGroupIdMap.keySet());
-        dataNodeConsensusGroupIdMap =
-            getAvailableDataNodeLocationForRetry(currentFailedDataNodeMap, allFailedDataNodeSet);
-        if (isFailed()) {
-          // some consensus group has no available dataNode
-          return;
-        }
-      }
     }
 
-    private Map<TDataNodeLocation, List<TConsensusGroupId>> sendRegionRequest(
-        Map<TDataNodeLocation, List<TConsensusGroupId>> dataNodeConsensusGroupIdMap) {
-      // send request to each dataNode
-      Map<TDataNodeLocation, List<TConsensusGroupId>> failedDataNodeMap = new HashMap<>();
-      for (Map.Entry<TDataNodeLocation, List<TConsensusGroupId>> entry :
-          dataNodeConsensusGroupIdMap.entrySet()) {
-        // process response
-        Map<Integer, TSStatus> dataNodeResponseMap = sendRequest(entry.getKey(), entry.getValue());
-        TSStatus currentDataNodeResponse = dataNodeResponseMap.get(entry.getKey().getDataNodeId());
-        if (currentDataNodeResponse.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-          if (currentDataNodeResponse.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) {
-            // dataNode execution error
-            LOGGER.error(
-                "Failed to execute [{}] of delete timeseries {} on {}",
-                taskName,
-                requestMessage,
-                entry.getKey());
-            setFailure(
-                new ProcedureException(
-                    new MetadataException(
-                        String.format(
-                            "Delete timeseries %s failed when [%s]", requestMessage, taskName))));
-            break;
-          } else {
-            // unexpected error, retry on other replicates on other dataNodes
-            failedDataNodeMap.put(entry.getKey(), entry.getValue());
-          }
-        }
-      }
-      return failedDataNodeMap;
+    DeleteTimeSeriesRegionTask(
+        String taskName,
+        ConfigNodeProcedureEnv env,
+        Map<TConsensusGroupId, TRegionReplicaSet> targetSchemaRegionGroup,
+        boolean executeOnAllReplicaset) {
+      super(env, targetSchemaRegionGroup, executeOnAllReplicaset);
+      this.taskName = taskName;
     }
 
-    private Map<TDataNodeLocation, List<TConsensusGroupId>> getAvailableDataNodeLocationForRetry(
-        Map<TDataNodeLocation, List<TConsensusGroupId>> failedDataNodeConsensusGroupIdMap,
-        Set<TDataNodeLocation> allFailedDataNodeSet) {
-      Map<TConsensusGroupId, Integer> leaderMap =
-          env.getConfigManager().getPartitionManager().getAllLeadership();
-      Map<TDataNodeLocation, List<TConsensusGroupId>> availableDataNodeLocation = new HashMap<>();
-      for (List<TConsensusGroupId> consensusGroupIdList :
-          failedDataNodeConsensusGroupIdMap.values()) {
-        for (TConsensusGroupId consensusGroupId : consensusGroupIdList) {
-          TRegionReplicaSet regionReplicaSet = targetSchemaRegionGroup.get(consensusGroupId);
-          TDataNodeLocation selectedDataNode = null;
-          Integer leaderId = leaderMap.get(consensusGroupId);
-          if (leaderId == null || leaderId == -1) {
-            for (TDataNodeLocation candidateDataNode : regionReplicaSet.getDataNodeLocations()) {
-              if (!allFailedDataNodeSet.contains(candidateDataNode)) {
-                // since leader of this group is unknown, take the first available one
-                selectedDataNode = candidateDataNode;
-                break;
-              }
-            }
-          } else {
-            for (TDataNodeLocation candidateDataNode : regionReplicaSet.getDataNodeLocations()) {
-              if (!allFailedDataNodeSet.contains(candidateDataNode)) {
-                if (leaderId == candidateDataNode.getDataNodeId()) {
-                  // retry on the new leader as possible
-                  selectedDataNode = candidateDataNode;
-                  break;
-                }
-                if (selectedDataNode == null) {
-                  selectedDataNode = candidateDataNode;
-                }
-              }
-            }
-          }
-
-          if (selectedDataNode == null) {
-            setFailure(
-                new ProcedureException(
-                    new MetadataException(
-                        String.format(
-                            "Delete timeseries %s failed when [%s] because all replicaset of schemaRegion %s failed.",
-                            requestMessage, taskName, consensusGroupId.id))));
-            return availableDataNodeLocation;
-          } else {
-            availableDataNodeLocation
-                .compute(selectedDataNode, (k, v) -> new ArrayList<>())
-                .add(consensusGroupId);
-          }
-        }
-      }
-      return availableDataNodeLocation;
+    @Override
+    protected boolean hasFailure() {
+      return isFailed();
     }
 
-    Map<Integer, T> getResponseMap() {
-      return responseMap == null ? Collections.emptyMap() : responseMap;
+    @Override
+    protected void onExecutionFailure(TDataNodeLocation dataNodeLocation) {
+      LOGGER.error(
+          "Failed to execute [{}] of delete timeseries {} on {}",
+          taskName,
+          requestMessage,
+          dataNodeLocation);
+      setFailure(
+          new ProcedureException(
+              new MetadataException(
+                  String.format(
+                      "Delete timeseries %s failed when [%s]", requestMessage, taskName))));
     }
 
-    void setExecuteOnAllReplicaset(boolean executeOnAllReplicaset) {
-      this.executeOnAllReplicaset = executeOnAllReplicaset;
+    @Override
+    protected void onAllReplicasetFailure(TConsensusGroupId consensusGroupId) {
+      setFailure(
+          new ProcedureException(
+              new MetadataException(
+                  String.format(
+                      "Delete timeseries %s failed when [%s] because all replicaset of schemaRegion %s failed.",
+                      requestMessage, taskName, consensusGroupId.id))));
     }
-
-    abstract Map<Integer, TSStatus> sendRequest(
-        TDataNodeLocation dataNodeLocation, List<TConsensusGroupId> consensusGroupIdList);
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteTimeSeriesState.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeactivateTemplateState.java
similarity index 88%
copy from confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteTimeSeriesState.java
copy to confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeactivateTemplateState.java
index 5f998f2975..3abbb2e266 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteTimeSeriesState.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeactivateTemplateState.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package org.apache.iotdb.confignode.procedure.state;
+package org.apache.iotdb.confignode.procedure.state.schema;
 
-public enum DeleteTimeSeriesState {
+public enum DeactivateTemplateState {
   CONSTRUCT_BLACK_LIST,
   CLEAN_DATANODE_SCHEMA_CACHE,
   DELETE_DATA,
-  DELETE_TIMESERIES_SCHEMA
+  DEACTIVATE_TEMPLATE
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteStorageGroupState.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java
similarity index 93%
rename from confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteStorageGroupState.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java
index 2ca63c67a8..50895cc926 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteStorageGroupState.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.confignode.procedure.state;
+package org.apache.iotdb.confignode.procedure.state.schema;
 
 public enum DeleteStorageGroupState {
   DELETE_STORAGE_GROUP_PREPARE,
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteTimeSeriesState.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteTimeSeriesState.java
similarity index 94%
rename from confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteTimeSeriesState.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteTimeSeriesState.java
index 5f998f2975..9fcf045e3a 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DeleteTimeSeriesState.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteTimeSeriesState.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.confignode.procedure.state;
+package org.apache.iotdb.confignode.procedure.state.schema;
 
 public enum DeleteTimeSeriesState {
   CONSTRUCT_BLACK_LIST,
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
index a28eff1c8d..15ab6b7909 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
@@ -25,9 +25,10 @@ import org.apache.iotdb.confignode.procedure.impl.DropTriggerProcedure;
 import org.apache.iotdb.confignode.procedure.impl.node.AddConfigNodeProcedure;
 import org.apache.iotdb.confignode.procedure.impl.node.RemoveConfigNodeProcedure;
 import org.apache.iotdb.confignode.procedure.impl.node.RemoveDataNodeProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteTimeSeriesProcedure;
 import org.apache.iotdb.confignode.procedure.impl.statemachine.CreateRegionGroupsProcedure;
-import org.apache.iotdb.confignode.procedure.impl.statemachine.DeleteStorageGroupProcedure;
-import org.apache.iotdb.confignode.procedure.impl.statemachine.DeleteTimeSeriesProcedure;
 import org.apache.iotdb.confignode.procedure.impl.statemachine.RegionMigrateProcedure;
 import org.apache.iotdb.confignode.procedure.impl.sync.CreatePipeProcedure;
 import org.apache.iotdb.confignode.procedure.impl.sync.DropPipeProcedure;
@@ -93,6 +94,9 @@ public class ProcedureFactory implements IProcedureFactory {
       case DROP_PIPE_PROCEDURE:
         procedure = new DropPipeProcedure();
         break;
+      case DEACTIVATE_TEMPLATE_PROCEDURE:
+        procedure = new DeactivateTemplateProcedure();
+        break;
       default:
         LOGGER.error("unknown Procedure type: " + typeNum);
         throw new IOException("unknown Procedure type: " + typeNum);
@@ -128,6 +132,8 @@ public class ProcedureFactory implements IProcedureFactory {
       return ProcedureType.STOP_PIPE_PROCEDURE;
     } else if (procedure instanceof DropPipeProcedure) {
       return ProcedureType.DROP_PIPE_PROCEDURE;
+    } else if (procedure instanceof DeactivateTemplateProcedure) {
+      return ProcedureType.DEACTIVATE_TEMPLATE_PROCEDURE;
     }
     return null;
   }
@@ -145,7 +151,8 @@ public class ProcedureFactory implements IProcedureFactory {
     CREATE_PIPE_PROCEDURE,
     START_PIPE_PROCEDURE,
     STOP_PIPE_PROCEDURE,
-    DROP_PIPE_PROCEDURE
+    DROP_PIPE_PROCEDURE,
+    DEACTIVATE_TEMPLATE_PROCEDURE
   }
 
   private static class ProcedureFactoryHolder {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 73c7357ae2..3269a1252f 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -84,6 +84,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDeactivateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupsReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteTimeSeriesReq;
@@ -624,6 +625,11 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
     return configManager.getPathsSetTemplate(req);
   }
 
+  @Override
+  public TSStatus deactivateSchemaTemplate(TDeactivateSchemaTemplateReq req) throws TException {
+    return configManager.deactivateSchemaTemplate(req);
+  }
+
   @Override
   public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) {
     return configManager.deleteTimeSeries(req);
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
index a25e20ccf8..c95e47bdae 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
@@ -101,8 +101,8 @@ import org.apache.iotdb.confignode.persistence.partition.maintainer.RegionDelete
 import org.apache.iotdb.confignode.persistence.partition.statistics.RegionGroupStatistics;
 import org.apache.iotdb.confignode.persistence.partition.statistics.RegionStatistics;
 import org.apache.iotdb.confignode.procedure.Procedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
 import org.apache.iotdb.confignode.procedure.impl.statemachine.CreateRegionGroupsProcedure;
-import org.apache.iotdb.confignode.procedure.impl.statemachine.DeleteStorageGroupProcedure;
 import org.apache.iotdb.confignode.rpc.thrift.TPipeSinkInfo;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeactivateTemplateProcedureTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeactivateTemplateProcedureTest.java
new file mode 100644
index 0000000000..2f9e918fbb
--- /dev/null
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeactivateTemplateProcedureTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.confignode.procedure.impl;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure;
+import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class DeactivateTemplateProcedureTest {
+
+  @Test
+  public void serializeDeserializeTest() throws IllegalPathException, IOException {
+    String queryId = "1";
+    Map<PartialPath, List<Template>> templateSetInfo = new HashMap<>();
+    Template t1 = new Template();
+    t1.setId(0);
+    t1.setName("t1");
+    t1.addUnalignedMeasurements(
+        new String[] {"s1", "s2"},
+        new TSDataType[] {TSDataType.INT32, TSDataType.FLOAT},
+        new TSEncoding[] {TSEncoding.PLAIN, TSEncoding.BITMAP},
+        new CompressionType[] {CompressionType.UNCOMPRESSED, CompressionType.GZIP});
+
+    Template t2 = new Template();
+    t2.setId(0);
+    t2.setName("t2");
+    t2.addUnalignedMeasurements(
+        new String[] {"s3", "s4"},
+        new TSDataType[] {TSDataType.FLOAT, TSDataType.INT32},
+        new TSEncoding[] {TSEncoding.BITMAP, TSEncoding.PLAIN},
+        new CompressionType[] {CompressionType.GZIP, CompressionType.UNCOMPRESSED});
+
+    templateSetInfo.put(new PartialPath("root.sg1.**"), Arrays.asList(t1, t2));
+    templateSetInfo.put(new PartialPath("root.sg2.**"), Arrays.asList(t2, t1));
+
+    DeactivateTemplateProcedure deactivateTemplateProcedure =
+        new DeactivateTemplateProcedure(queryId, templateSetInfo);
+
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+    deactivateTemplateProcedure.serialize(dataOutputStream);
+
+    ByteBuffer byteBuffer = ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+
+    Assert.assertEquals(
+        ProcedureFactory.ProcedureType.DEACTIVATE_TEMPLATE_PROCEDURE.ordinal(),
+        byteBuffer.getInt());
+
+    DeactivateTemplateProcedure deserializedProcedure = new DeactivateTemplateProcedure();
+    deserializedProcedure.deserialize(byteBuffer);
+
+    Assert.assertEquals(queryId, deserializedProcedure.getQueryId());
+    Assert.assertEquals(templateSetInfo, deserializedProcedure.getTemplateSetInfo());
+  }
+}
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteStorageGroupProcedureTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteStorageGroupProcedureTest.java
index db37e59002..199e6202e6 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteStorageGroupProcedureTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteStorageGroupProcedureTest.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.confignode.procedure.impl;
 
-import org.apache.iotdb.confignode.procedure.impl.statemachine.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
 import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.tsfile.utils.PublicBAOS;
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteTimeSeriesProcedureTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteTimeSeriesProcedureTest.java
index 983893f3d0..5674946d9e 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteTimeSeriesProcedureTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteTimeSeriesProcedureTest.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.confignode.procedure.impl;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
-import org.apache.iotdb.confignode.procedure.impl.statemachine.DeleteTimeSeriesProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteTimeSeriesProcedure;
 import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
 
 import org.junit.Assert;
diff --git a/docs/UserGuide/Operate-Metadata/Template.md b/docs/UserGuide/Operate-Metadata/Template.md
index a18b1f8e20..128385ab8f 100644
--- a/docs/UserGuide/Operate-Metadata/Template.md
+++ b/docs/UserGuide/Operate-Metadata/Template.md
@@ -49,17 +49,23 @@ The` lat` and `lon` measurements are aligned.
 
 ## Set Schema Template
 
+After a schema template is created, it should be set to specific path before creating related timeseries or insert data.
+
+**It is recommended to set schema template to storage group path. It is not suggested to set schema template to some path above storage group**
+
+
 The SQL Statement for setting schema template is as follow:
 
 ```shell
 IoTDB> set schema template t1 to root.sg1.d1
 ```
 
-After setting the schema template, you can insert data into the timeseries. For example, suppose there's a storage group root.sg1 and t1 has been set to root.sg1.d1, then timeseries like root.sg1.d1.temperature and root.sg1.d1.status are available and data points can be inserted.
+## Activate Schema Template
+
+After setting the schema template, with the system enabled to auto create schema, you can insert data into the timeseries. For example, suppose there's a storage group root.sg1 and t1 has been set to root.sg1.d1, then timeseries like root.sg1.d1.temperature and root.sg1.d1.status are available and data points can be inserted.
 
-**Please notice that, we strongly recommend NOT setting templates on the nodes above the storage group to accommodate future updates and collaboration between modules.**
 
-**Attention**: Before inserting data, timeseries defined by the schema template will not be created. You can use the following SQL statement to create the timeseries before inserting data:
+**Attention**: Before inserting data or the system not enabled to auto create schema, timeseries defined by the schema template will not be created. You can use the following SQL statement to create the timeseries or activate the schema template, act before inserting data:
 
 ```shell
 IoTDB> create timeseries of schema template on root.sg1.d1
@@ -67,10 +73,10 @@ IoTDB> create timeseries of schema template on root.sg1.d1
 
 **Example:** Execute the following statement
 ```shell
-set schema template t1 to root.sg1.d1
-set schema template t2 to root.sg1.d2
-create timeseries of schema template on root.sg1.d1
-create timeseries of schema template on root.sg1.d2
+IoTDB> set schema template t1 to root.sg1.d1
+IoTDB> set schema template t2 to root.sg1.d2
+IoTDB> create timeseries of schema template on root.sg1.d1
+IoTDB> create timeseries of schema template on root.sg1.d2
 ```
 
 Show the time series:
@@ -171,7 +177,21 @@ The execution result is as follows:
 +-----------+
 ```
 
-## Uset Schema Template
+## Deactivate Schema Template
+
+To delete a group of timeseries represented by schema template, namely deactivate the schema template, use the following SQL statement:
+
+```shell
+IoTDB> delete timeseries of schema template t1 from root.sg1.d1
+```
+
+The deactivation supports batch process. 
+
+```shell
+IoTDB> delete timeseries of schema template t1 from root.sg1.*, root.sg2.*
+```
+
+## Unset Schema Template
 
 The SQL Statement for unsetting schema template is as follow:
 
@@ -179,7 +199,7 @@ The SQL Statement for unsetting schema template is as follow:
 IoTDB> unset schema template t1 from root.sg1.d1
 ```
 
-**Attention**: Unsetting the template from entities, which have already inserted records using the template, is not supported.
+**Attention**: It should be guaranteed that none of the timeseries represented by the target schema template exists, before unset it. It can be achieved by deactivation operation.
 
 ## Drop Schema Template
 
diff --git a/docs/zh/UserGuide/Operate-Metadata/Template.md b/docs/zh/UserGuide/Operate-Metadata/Template.md
index 90d58332a3..5d7a417fd7 100644
--- a/docs/zh/UserGuide/Operate-Metadata/Template.md
+++ b/docs/zh/UserGuide/Operate-Metadata/Template.md
@@ -49,7 +49,9 @@ IoTDB> create schema template t2 aligned (lat FLOAT encoding=Gorilla, lon FLOAT
 
 ## 挂载元数据模板
 
-**为了更好地适配未来版本的更新及各模块的协作,我们强烈建议您将模板设置在存储组及存储组下层的节点中。**
+元数据模板在创建后,需执行挂载操作,方可用于相应路径下的序列创建与数据写入。
+
+**推荐将模板挂载在存储组节点上,不建议将模板挂载到存储组上层的节点上。**
 
 挂载元数据模板的 SQL 语句如下所示:
 
@@ -57,9 +59,11 @@ IoTDB> create schema template t2 aligned (lat FLOAT encoding=Gorilla, lon FLOAT
 IoTDB> set schema template t1 to root.sg1.d1
 ```
 
-挂载好元数据模板后,即可进行数据的写入。例如存储组为 root.sg1,模板 t1 被挂载到了节点 root.sg1.d1,那么可直接向时间序列(如 root.sg1.d1.temperature 和 root.sg1.d1.status)写入时间序列数据,该时间序列已可被当作正常创建的序列使用。
+## 激活元数据模板
+
+挂载好元数据模板后,且系统开启自动注册序列功能的情况下,即可直接进行数据的写入。例如存储组为 root.sg1,模板 t1 被挂载到了节点 root.sg1.d1,那么可直接向时间序列(如 root.sg1.d1.temperature 和 root.sg1.d1.status)写入时间序列数据,该时间序列已可被当作正常创建的序列使用。
 
-**注意**:在插入数据之前,模板定义的时间序列不会被创建。可以使用如下SQL语句在插入数据前创建时间序列:
+**注意**:在插入数据之前或系统未开启自动注册序列功能,模板定义的时间序列不会被创建。可以使用如下SQL语句在插入数据前创建时间序列即激活模板:
 
 ```shell
 IoTDB> create timeseries of schema template on root.sg1.d1
@@ -67,10 +71,10 @@ IoTDB> create timeseries of schema template on root.sg1.d1
 
 **示例:** 执行以下语句
 ```shell
-set schema template t1 to root.sg1.d1
-set schema template t2 to root.sg1.d2
-create timeseries of schema template on root.sg1.d1
-create timeseries of schema template on root.sg1.d2
+IoTDB> set schema template t1 to root.sg1.d1
+IoTDB> set schema template t2 to root.sg1.d2
+IoTDB> create timeseries of schema template on root.sg1.d1
+IoTDB> create timeseries of schema template on root.sg1.d2
 ```
 
 查看此时的时间序列:
@@ -141,7 +145,7 @@ IoTDB> show nodes in schema template t1
 +-----------+--------+--------+-----------+
 ```
 
-- 查看挂载了某个元数据模板的路径前缀
+- 查看挂载了某个元数据模板的路径
 
 ```shell
 IoTDB> show paths set schema template t1
@@ -156,7 +160,7 @@ IoTDB> show paths set schema template t1
 +-----------+
 ```
 
-- 查看使用了某个元数据模板(即序列已创建)的路径前缀
+- 查看使用了某个元数据模板的路径(即模板在该路径上已激活,序列已创建)
 
 ```shell
 IoTDB> show paths using schema template t1
@@ -171,6 +175,20 @@ IoTDB> show paths using schema template t1
 +-----------+
 ```
 
+## 解除元数据模板
+
+若需删除模板表示的某一组时间序列,可采用解除模板操作,SQL语句如下所示:
+
+```shell
+IoTDB> delete timeseries of schema template t1 from root.sg1.d1
+```
+
+解除操作支持批量处理,SQL语句如下所示:
+
+```shell
+IoTDB> delete timeseries of schema template t1 from root.sg1.*, root.sg2.*
+```
+
 ## 卸载元数据模板
 
 卸载元数据模板的 SQL 语句如下所示:
@@ -179,7 +197,7 @@ IoTDB> show paths using schema template t1
 IoTDB> unset schema template t1 from root.sg1.d1
 ```
 
-**注意**:目前不支持从曾经使用模板插入数据后(即使数据已被删除)的实体中卸载模板。
+**注意**:不支持卸载仍处于激活状态的模板,需保证执行卸载操作前解除对该模板的所有使用,即删除所有该模板表示的序列。
 
 ## 删除元数据模板
 
@@ -189,4 +207,4 @@ IoTDB> unset schema template t1 from root.sg1.d1
 IoTDB> drop schema template t1
 ```
 
-**注意**:不支持删除已经挂载的模板。
+**注意**:不支持删除已经挂载的模板,需在删除操作前保证该模板卸载成功。
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java
new file mode 100644
index 0000000000..ed20ce2d85
--- /dev/null
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java
@@ -0,0 +1,231 @@
+/*
+ * 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.it.schema;
+
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.ClusterIT;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({ClusterIT.class})
+public class IoTDBDeactivateTemplateIT {
+
+  private Statement statement;
+  private Connection connection;
+
+  @Before
+  public void setUp() throws Exception {
+    EnvFactory.getEnv().initBeforeTest();
+    connection = EnvFactory.getEnv().getConnection();
+    statement = connection.createStatement();
+
+    prepareTemplate();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    statement.close();
+    connection.close();
+    EnvFactory.getEnv().cleanAfterTest();
+  }
+
+  private void prepareTemplate() throws SQLException {
+    // create storage group
+    statement.execute("CREATE STORAGE GROUP root.sg1");
+    statement.execute("CREATE STORAGE GROUP root.sg2");
+    statement.execute("CREATE STORAGE GROUP root.sg3");
+    statement.execute("CREATE STORAGE GROUP root.sg4");
+
+    // create schema template
+    statement.execute("CREATE SCHEMA TEMPLATE t1 (s1 INT64, s2 DOUBLE)");
+    statement.execute("CREATE SCHEMA TEMPLATE t2 (s1 INT64, s2 DOUBLE)");
+
+    // set schema template
+    statement.execute("SET SCHEMA TEMPLATE t1 TO root.sg1");
+    statement.execute("SET SCHEMA TEMPLATE t1 TO root.sg2");
+    statement.execute("SET SCHEMA TEMPLATE t2 TO root.sg3");
+    statement.execute("SET SCHEMA TEMPLATE t2 TO root.sg4");
+
+    String insertSql = "insert into root.sg%d.d1(time, s1, s2) values(%d, %d, %d)";
+    for (int i = 1; i <= 4; i++) {
+      for (int j = 1; j <= 4; j++) {
+        statement.execute(String.format(insertSql, j, i, i, i));
+      }
+    }
+  }
+
+  @Test
+  public void deactivateTemplateAndReactivateTest() throws Exception {
+    statement.execute("DEACTIVATE SCHEMA TEMPLATE t1 FROM root.sg1.d1");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg1.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+
+    statement.execute("CREATE TIMESERIES OF SCHEMA TEMPLATE ON root.sg1.d1");
+
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg1.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(3, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+
+    statement.execute("insert into root.sg1.d1(time, s1, s2) values(1, 1, 1)");
+
+    String[] retArray = new String[] {"1,1,1.0,"};
+    int cnt = 0;
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg1.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(3, resultSetMetaData.getColumnCount());
+      while (resultSet.next()) {
+        StringBuilder builder = new StringBuilder();
+        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+          builder.append(resultSet.getString(i)).append(",");
+        }
+        Assert.assertEquals(retArray[cnt], builder.toString());
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+    }
+  }
+
+  @Test
+  public void deactivateTemplateAndAutoDeleteDeviceTest() throws Exception {
+    statement.execute("DEACTIVATE SCHEMA TEMPLATE t1 FROM root.sg1.d1");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg1.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+    try (ResultSet resultSet = statement.executeQuery("SHOW DEVICES root.sg1.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertFalse(resultSet.next());
+    }
+  }
+
+  @Test
+  public void deactivateTemplateCrossSchemaRegionTest() throws Exception {
+    String insertSql = "insert into root.sg1.d%d(time, s1, s2) values(%d, %d, %d)";
+    for (int i = 1; i <= 4; i++) {
+      for (int j = 1; j <= 4; j++) {
+        statement.execute(String.format(insertSql, j, i, i, i));
+      }
+    }
+
+    statement.execute("DEACTIVATE TEMPLATE FROM root.sg1.*");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg1.**")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+  }
+
+  @Test
+  public void deactivateTemplateCrossStorageGroupTest() throws Exception {
+    String insertSql = "insert into root.sg%d.d2(time, s1, s2) values(%d, %d, %d)";
+    for (int i = 1; i <= 4; i++) {
+      for (int j = 1; j <= 2; j++) {
+        statement.execute(String.format(insertSql, j, i, i, i));
+      }
+    }
+
+    statement.execute("DEACTIVATE TEMPLATE FROM root.*.d1");
+    String[] retArray =
+        new String[] {"1,1,1.0,1,1.0,", "2,2,2.0,2,2.0,", "3,3,3.0,3,3.0,", "4,4,4.0,4,4.0,"};
+    int cnt = 0;
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.**")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(5, resultSetMetaData.getColumnCount());
+      while (resultSet.next()) {
+        StringBuilder builder = new StringBuilder();
+        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+          builder.append(resultSet.getString(i)).append(",");
+        }
+        Assert.assertEquals(retArray[cnt], builder.toString());
+        cnt++;
+      }
+      Assert.assertEquals(retArray.length, cnt);
+    }
+
+    statement.execute("DEACTIVATE TEMPLATE FROM root.**, root.sg1.*");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.**")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+  }
+
+  @Test
+  public void deactivateTemplateWithMultiPatternTest() throws Exception {
+    statement.execute("DEACTIVATE SCHEMA TEMPLATE t1 FROM root.sg1.d1, root.sg2.*");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg1.*, root.sg2.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+  }
+
+  @Test
+  public void deactivateNoneUsageTemplateTest() throws Exception {
+    statement.execute("DEACTIVATE SCHEMA TEMPLATE t1 FROM root.sg5.d1");
+
+    statement.execute("DEACTIVATE SCHEMA TEMPLATE t1 FROM root.sg1.d1");
+
+    statement.execute("DEACTIVATE SCHEMA TEMPLATE t1 FROM root.sg1.d1");
+  }
+
+  @Test
+  public void multiSyntaxTest() throws Exception {
+    statement.execute("DELETE TIMESERIES OF SCHEMA TEMPLATE t1 FROM root.sg1.d1");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg1.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+
+    statement.execute("DEACTIVATE SCHEMA TEMPLATE t2 FROM root.sg3.d1");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.sg3.*")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+
+    statement.execute("DEACTIVATE TEMPLATE FROM root.**");
+    try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.**")) {
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Assert.assertEquals(1, resultSetMetaData.getColumnCount());
+      Assert.assertFalse(resultSet.next());
+    }
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java
index 8d47504e3d..85d8b10d1c 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java
@@ -172,6 +172,10 @@ public class PartialPath extends Path implements Comparable<Path>, Cloneable {
    * <p>The goal of this method is to reduce the search space when querying a storage group with a
    * path with wildcard.
    *
+   * <p>If this path or path pattern doesn't start with given prefix, return empty list. For
+   * example, "root.a.b.c".alterPrefixPath("root.b") or "root.a.**".alterPrefixPath("root.b")
+   * returns [].
+   *
    * @param prefix The prefix. Cannot be null and cannot contain any wildcard.
    */
   public List<PartialPath> alterPrefixPath(PartialPath prefix) {
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
index f35c01a161..2b3458785c 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
@@ -53,6 +53,9 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IActivateTemplatePla
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IAutoCreateDeviceMNodePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
 import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
@@ -2026,7 +2029,25 @@ public class RSchemaRegion implements ISchemaRegion {
   }
 
   @Override
-  public List<String> getPathsUsingTemplate(int templateId) throws MetadataException {
+  public List<String> getPathsUsingTemplate(PartialPath pathPattern, int templateId)
+      throws MetadataException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int constructSchemaBlackListWithTemplate(IPreDeactivateTemplatePlan plan)
+      throws MetadataException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void rollbackSchemaBlackListWithTemplate(IRollbackPreDeactivateTemplatePlan plan)
+      throws MetadataException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deactivateTemplateInBlackList(IDeactivateTemplatePlan plan) throws MetadataException {
     throw new UnsupportedOperationException();
   }
 
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/REntityMNode.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/REntityMNode.java
index 15e397f926..22cd4e1045 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/REntityMNode.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/REntityMNode.java
@@ -88,6 +88,26 @@ public class REntityMNode extends RInternalMNode implements IEntityMNode {
     throw new UnsupportedOperationException();
   }
 
+  @Override
+  public boolean isPreDeactivateTemplate() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void preDeactivateTemplate() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void rollbackPreDeactivateTemplate() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deactivateTemplate() {
+    throw new UnsupportedOperationException();
+  }
+
   @Override
   public boolean isAligned() {
     return isAligned;
diff --git a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
index 5baf7350d4..f08e3ba926 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
@@ -51,6 +51,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveResp;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDeactivateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupsReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteTimeSeriesReq;
@@ -1065,6 +1066,22 @@ public class ConfigNodeClient
     throw new TException(MSG_RECONNECTION_FAIL);
   }
 
+  @Override
+  public TSStatus deactivateSchemaTemplate(TDeactivateSchemaTemplateReq req) throws TException {
+    for (int i = 0; i < RETRY_NUM; i++) {
+      try {
+        TSStatus status = client.deactivateSchemaTemplate(req);
+        if (!updateConfigNodeLeader(status)) {
+          return status;
+        }
+      } catch (TException e) {
+        configLeader = null;
+      }
+      reconnect();
+    }
+    throw new TException(MSG_RECONNECTION_FAIL);
+  }
+
   @Override
   public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) throws TException {
     for (int i = 0; i < RETRY_NUM; i++) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/EntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/EntityMNode.java
index 76dc8cda2d..a3e11676c2 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/EntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/EntityMNode.java
@@ -114,6 +114,44 @@ public class EntityMNode extends InternalMNode implements IEntityMNode {
     this.aliasChildren = aliasChildren;
   }
 
+  /**
+   * In EntityMNode(device node), schemaTemplateId represents the template activated on this node.
+   * The pre deactivation mechanism is implemented by making this value negative. Since value 0 and
+   * -1 are all occupied, the available negative value range is [Int.MIN_VALUE, -2]. The value of a
+   * pre deactivated case equals the negative normal value minus 2. For example, if the id of
+   * activated template is 0, then - 0 - 2 = -2 represents the pre deactivation of this template on
+   * this node.
+   */
+  @Override
+  public int getSchemaTemplateId() {
+    return schemaTemplateId >= -1 ? schemaTemplateId : -schemaTemplateId - 2;
+  }
+
+  @Override
+  public boolean isPreDeactivateTemplate() {
+    return schemaTemplateId < -1;
+  }
+
+  @Override
+  public void preDeactivateTemplate() {
+    if (schemaTemplateId > -1) {
+      schemaTemplateId = -schemaTemplateId - 2;
+    }
+  }
+
+  @Override
+  public void rollbackPreDeactivateTemplate() {
+    if (schemaTemplateId < -1) {
+      schemaTemplateId = -schemaTemplateId - 2;
+    }
+  }
+
+  @Override
+  public void deactivateTemplate() {
+    schemaTemplateId = -1;
+    setUseTemplate(false);
+  }
+
   @Override
   public boolean isAligned() {
     return isAligned;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
index a6780cb8f3..fc9b8d0f60 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
@@ -37,6 +37,14 @@ public interface IEntityMNode extends IMNode {
 
   void setUseTemplate(boolean useTemplate);
 
+  boolean isPreDeactivateTemplate();
+
+  void preDeactivateTemplate();
+
+  void rollbackPreDeactivateTemplate();
+
+  void deactivateTemplate();
+
   boolean isAligned();
 
   void setAligned(boolean isAligned);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
index b7e8376764..f9410f6ea0 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
@@ -46,7 +46,13 @@ public class InternalMNode extends MNode {
   @SuppressWarnings("squid:S3077")
   protected transient volatile IMNodeContainer children = null;
 
-  // schema template
+  /**
+   * This field is mainly used in cluster schema template features. In InternalMNode of ConfigMTree,
+   * this field represents the template set on this node. In EntityMNode of MTree in SchemaRegion,
+   * this field represents the template activated on this node. The value range is [0,
+   * Int.MaxValue], since this is implemented as auto inc id. The default value -1 means
+   * NON_TEMPLATE
+   */
   protected int schemaTemplateId = NON_TEMPLATE;
 
   protected Template schemaTemplate = null;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java
index 8cb797e4e5..166f5e2529 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java
@@ -54,6 +54,7 @@ import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Deque;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -668,6 +669,44 @@ public class ConfigMTree {
     return resSet;
   }
 
+  /** This method returns the templateId set on paths covered by input path pattern. */
+  public Map<Integer, Set<PartialPath>> getTemplateSetInfo(PartialPath pathPattern)
+      throws MetadataException {
+    Map<Integer, Set<PartialPath>> result = new HashMap<>();
+    CollectorTraverser<List<Integer>> collector =
+        new CollectorTraverser<List<Integer>>(root, pathPattern, store) {
+          @Override
+          protected boolean processInternalMatchedMNode(IMNode node, int idx, int level)
+              throws MetadataException {
+            if (node.getSchemaTemplateId() != NON_TEMPLATE) {
+              // node set template
+              result
+                  .computeIfAbsent(node.getSchemaTemplateId(), k -> new HashSet<>())
+                  .add(getCurrentPartialPath(node));
+              // descendants of the node cannot set another template, exit from this branch
+              return true;
+            }
+            return false;
+          }
+
+          @Override
+          protected boolean processFullMatchedMNode(IMNode node, int idx, int level)
+              throws MetadataException {
+            if (node.getSchemaTemplateId() != NON_TEMPLATE) {
+              // node set template
+              result
+                  .computeIfAbsent(node.getSchemaTemplateId(), k -> new HashSet<>())
+                  .add(getCurrentPartialPath(node));
+              // descendants of the node cannot set another template, exit from this branch
+              return true;
+            }
+            return false;
+          }
+        };
+    collector.traverse();
+    return result;
+  }
+
   // endregion
 
   // region Serialization and Deserialization
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
index e338665e45..b08e8280e7 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
@@ -90,8 +90,6 @@ import static java.util.stream.Collectors.toList;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_SEPARATOR;
 import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_RESULT_NODES;
-import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_TEMPLATE;
-import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
 import static org.apache.iotdb.db.metadata.lastCache.LastCacheManager.getLastTimeStamp;
 
 /**
@@ -465,11 +463,16 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
     if (deletedNode.getAlias() != null) {
       parent.addAlias(deletedNode.getAlias(), deletedNode);
     }
-    IMNode curNode = parent;
-    if (!parent.isUseTemplate()) {
+    return new Pair<>(deleteEmptyInternalMNodeAndReturnEmptyStorageGroup(parent), deletedNode);
+  }
+
+  /** Used when delete timeseries or deactivate template */
+  public PartialPath deleteEmptyInternalMNodeAndReturnEmptyStorageGroup(IEntityMNode entityMNode) {
+    IMNode curNode = entityMNode;
+    if (!entityMNode.isUseTemplate()) {
       boolean hasMeasurement = false;
       IMNode child;
-      IMNodeIterator iterator = store.getChildrenIterator(parent);
+      IMNodeIterator iterator = store.getChildrenIterator(entityMNode);
       while (iterator.hasNext()) {
         child = iterator.next();
         if (child.isMeasurement()) {
@@ -480,7 +483,7 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
 
       if (!hasMeasurement) {
         synchronized (this) {
-          curNode = store.setToInternal(parent);
+          curNode = store.setToInternal(entityMNode);
           if (curNode.isStorageGroup()) {
             this.storageGroupMNode = curNode.getAsStorageGroupMNode();
           }
@@ -492,12 +495,12 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
     while (isEmptyInternalMNode(curNode)) {
       // if current storage group has no time series, return the storage group name
       if (curNode.isStorageGroup()) {
-        return new Pair<>(curNode.getPartialPath(), deletedNode);
+        return curNode.getPartialPath();
       }
       store.deleteChild(curNode.getParent(), curNode.getName());
       curNode = curNode.getParent();
     }
-    return new Pair<>(null, deletedNode);
+    return null;
   }
 
   @Override
@@ -785,9 +788,6 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
         new MeasurementCollector<List<PartialPath>>(storageGroupMNode, pathPattern, store) {
           @Override
           protected void collectMeasurement(IMeasurementMNode node) {
-            if (node.isPreDeleted()) {
-              return;
-            }
             MeasurementPath path = getCurrentMeasurementPathInTraverse(node);
             if (nodes[nodes.length - 1].equals(node.getAlias())) {
               // only when user query with alias, the alias in path will be set
@@ -800,6 +800,7 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
           }
         };
     collector.setTemplateMap(templateMap);
+    collector.setSkipPreDeletedSchema(true);
     collector.traverse();
     return result;
   }
@@ -1589,23 +1590,12 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
     return null;
   }
 
-  public void activateTemplate(PartialPath activatePath, int templateSetLevel, Template template)
+  public void activateTemplate(PartialPath activatePath, Template template)
       throws MetadataException {
-    if (templateSetLevel <= levelOfSG) {
-      IMNode ancestor = storageGroupMNode;
-      for (int i = levelOfSG; i > templateSetLevel; i--) {
-        ancestor = ancestor.getParent();
-      }
-      ancestor.setSchemaTemplateId(template.getId());
-    }
-
     String[] nodes = activatePath.getNodes();
     IMNode cur = storageGroupMNode;
     for (int i = levelOfSG + 1; i < nodes.length; i++) {
       cur = cur.getChild(nodes[i]);
-      if (i == templateSetLevel) {
-        cur.setSchemaTemplateId(template.getId());
-      }
     }
 
     IEntityMNode entityMNode;
@@ -1636,17 +1626,15 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
       entityMNode.setAligned(template.isDirectAligned());
     }
     entityMNode.setUseTemplate(true);
+    entityMNode.setSchemaTemplateId(template.getId());
   }
 
   public void activateTemplateWithoutCheck(
-      PartialPath activatePath, int templateSetLevel, int templateId, boolean isAligned) {
+      PartialPath activatePath, int templateId, boolean isAligned) {
     String[] nodes = activatePath.getNodes();
     IMNode cur = storageGroupMNode;
     for (int i = levelOfSG + 1; i < nodes.length; i++) {
       cur = cur.getChild(nodes[i]);
-      if (i == templateSetLevel) {
-        cur.setSchemaTemplateId(templateId);
-      }
     }
 
     IEntityMNode entityMNode;
@@ -1663,38 +1651,56 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
       entityMNode.setAligned(isAligned);
     }
     entityMNode.setUseTemplate(true);
+    entityMNode.setSchemaTemplateId(templateId);
   }
 
-  public List<String> getPathsUsingTemplate(int templateId) throws MetadataException {
-    List<String> result = new ArrayList<>();
+  public List<String> getPathsUsingTemplate(PartialPath pathPattern, int templateId)
+      throws MetadataException {
+    Set<String> result = new HashSet<>();
 
-    CollectorTraverser<Set<String>> usingTemplatePaths =
-        new CollectorTraverser<Set<String>>(
-            storageGroupMNode, new PartialPath(ALL_RESULT_NODES), store) {
+    EntityCollector<Set<String>> collector =
+        new EntityCollector<Set<String>>(storageGroupMNode, pathPattern, store) {
           @Override
-          protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
-            return false;
+          protected void collectEntity(IEntityMNode node) throws MetadataException {
+            if (node.getSchemaTemplateId() == templateId) {
+              result.add(node.getFullPath());
+            }
           }
+        };
+    collector.traverse();
+    return new ArrayList<>(result);
+  }
 
+  public List<IEntityMNode> getDeviceMNodeUsingTargetTemplate(
+      PartialPath pathPattern, List<Integer> templateIdList) throws MetadataException {
+    List<IEntityMNode> result = new ArrayList<>();
+    EntityCollector<List<IEntityMNode>> collector =
+        new EntityCollector<List<IEntityMNode>>(storageGroupMNode, pathPattern, store) {
           @Override
-          protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
-            if (node.isMeasurement()) {
-              return true;
+          protected void collectEntity(IEntityMNode node) throws MetadataException {
+            if (templateIdList.contains(node.getSchemaTemplateId())) {
+              result.add(node);
             }
-            if (node.getSchemaTemplateId() != NON_TEMPLATE
-                && templateId != ALL_TEMPLATE
-                && node.getSchemaTemplateId() != templateId) {
-              // skip subTree
-              return true;
-            }
-            if (node.isUseTemplate()) {
-              result.add(node.getFullPath());
-            }
-            return false;
           }
         };
+    collector.traverse();
+    return result;
+  }
 
-    usingTemplatePaths.traverse();
+  public List<IEntityMNode> getPreDeactivatedDeviceMNode(
+      PartialPath pathPattern, List<Integer> templateIdList) throws MetadataException {
+    List<IEntityMNode> result = new ArrayList<>();
+    EntityCollector<List<IEntityMNode>> collector =
+        new EntityCollector<List<IEntityMNode>>(storageGroupMNode, pathPattern, store) {
+          @Override
+          protected void collectEntity(IEntityMNode node) throws MetadataException {
+            if (templateIdList.contains(node.getSchemaTemplateId())
+                && node.isPreDeactivateTemplate()) {
+              result.add(node);
+            }
+          }
+        };
+    collector.traverse();
     return result;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
index 55d139ab23..a10d9b06ed 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
@@ -69,6 +69,9 @@ public abstract class Traverser {
   protected boolean shouldTraverseTemplate = false;
   protected Map<Integer, Template> templateMap;
 
+  // if true, the pre deleted measurement or pre deactivated template won't be processed
+  protected boolean skipPreDeletedSchema = false;
+
   // default false means fullPath pattern match
   protected boolean isPrefixMatch = false;
 
@@ -469,15 +472,13 @@ public abstract class Traverser {
         }
       }
     } else {
-      // new cluster
+      // new cluster, the used template is directly recorded as template id in device mnode
       if (node.getSchemaTemplateId() != NON_TEMPLATE) {
-        return templateMap.get(node.getSchemaTemplateId());
-      }
-      while (iterator.hasNext()) {
-        ancestor = iterator.next();
-        if (ancestor.getSchemaTemplateId() != NON_TEMPLATE) {
-          return templateMap.get(ancestor.getSchemaTemplateId());
+        if (skipPreDeletedSchema && node.getAsEntityMNode().isPreDeactivateTemplate()) {
+          // skip this pre deactivated template, the invoker will skip this
+          return null;
         }
+        return templateMap.get(node.getSchemaTemplateId());
       }
     }
     // if the node is usingTemplate, the upperTemplate won't be null or the upperTemplateId won't be
@@ -498,6 +499,10 @@ public abstract class Traverser {
     this.shouldTraverseTemplate = shouldTraverseTemplate;
   }
 
+  public void setSkipPreDeletedSchema(boolean skipPreDeletedSchema) {
+    this.skipPreDeletedSchema = skipPreDeletedSchema;
+  }
+
   /**
    * @param currentNode the node need to get the full path of
    * @return full path from traverse start node to the current node
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
index f85965bc59..0610bd1f35 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
@@ -50,7 +50,8 @@ public abstract class MeasurementCollector<T> extends CollectorTraverser<T> {
   @Override
   protected boolean processFullMatchedMNode(IMNode node, int idx, int level)
       throws MetadataException {
-    if (!node.isMeasurement()) {
+    if (!node.isMeasurement()
+        || (skipPreDeletedSchema && node.getAsMeasurementMNode().isPreDeleted())) {
       return false;
     }
     if (hasLimit) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanType.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanType.java
index 469b867ee8..fadf837323 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanType.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanType.java
@@ -38,9 +38,13 @@ public enum SchemaRegionPlanType {
   UNSET_TEMPLATE((byte) 57),
   ACTIVATE_TEMPLATE_IN_CLUSTER((byte) 63),
   PRE_DELETE_TIMESERIES_IN_CLUSTER((byte) 64),
-  ROLLBACK_PRE_DELETE_TIMESERIES((byte) 65);
+  ROLLBACK_PRE_DELETE_TIMESERIES((byte) 65),
   // endregion
 
+  PRE_DEACTIVATE_TEMPLATE((byte) 0),
+  ROLLBACK_PRE_DEACTIVATE_TEMPLATE((byte) 1),
+  DEACTIVATE_TEMPLATE((byte) 2);
+
   public static final int MAX_NUM = Byte.MAX_VALUE;
   private static final SchemaRegionPlanType[] PLAN_TYPE_TABLE = new SchemaRegionPlanType[MAX_NUM];
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanVisitor.java
index 2789493ad2..40fae83f6d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/SchemaRegionPlanVisitor.java
@@ -26,8 +26,11 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeAliasPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeTagOffsetPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeleteTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
@@ -87,4 +90,18 @@ public abstract class SchemaRegionPlanVisitor<R, C> {
   public R visitUnsetTemplate(IUnsetTemplatePlan unsetTemplatePlan, C context) {
     return visitSchemaRegionPlan(unsetTemplatePlan, context);
   }
+
+  public R visitPreDeactivateTemplate(
+      IPreDeactivateTemplatePlan preDeactivateTemplatePlan, C context) {
+    return visitSchemaRegionPlan(preDeactivateTemplatePlan, context);
+  }
+
+  public R visitRollbackPreDeactivateTemplate(
+      IRollbackPreDeactivateTemplatePlan rollbackPreDeactivateTemplatePlan, C context) {
+    return visitSchemaRegionPlan(rollbackPreDeactivateTemplatePlan, context);
+  }
+
+  public R visitDeactivateTemplate(IDeactivateTemplatePlan deactivateTemplatePlan, C context) {
+    return visitSchemaRegionPlan(deactivateTemplatePlan, context);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/DeactivateTemplatePlanImpl.java
similarity index 56%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/DeactivateTemplatePlanImpl.java
index a6780cb8f3..986f5c9dc3 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/DeactivateTemplatePlanImpl.java
@@ -16,32 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+package org.apache.iotdb.db.metadata.plan.schemaregion.impl;
 
-import java.util.Map;
-
-public interface IEntityMNode extends IMNode {
-
-  boolean addAlias(String alias, IMeasurementMNode child);
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
 
-  void deleteAliasChild(String alias);
+import java.util.List;
+import java.util.Map;
 
-  Map<String, IMeasurementMNode> getAliasChildren();
+public class DeactivateTemplatePlanImpl implements IDeactivateTemplatePlan {
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+  private Map<PartialPath, List<Integer>> templateSetInfo;
 
   @Override
-  boolean isUseTemplate();
-
-  void setUseTemplate(boolean useTemplate);
+  public Map<PartialPath, List<Integer>> getTemplateSetInfo() {
+    return templateSetInfo;
+  }
 
-  boolean isAligned();
-
-  void setAligned(boolean isAligned);
-
-  ILastCacheContainer getLastCacheContainer(String measurementId);
-
-  Map<String, ILastCacheContainer> getTemplateLastCaches();
+  @Override
+  public void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo) {
+    this.templateSetInfo = templateSetInfo;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/PreDeactivateTemplatePlanImpl.java
similarity index 56%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/PreDeactivateTemplatePlanImpl.java
index a6780cb8f3..539bbfcd90 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/PreDeactivateTemplatePlanImpl.java
@@ -16,32 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+package org.apache.iotdb.db.metadata.plan.schemaregion.impl;
 
-import java.util.Map;
-
-public interface IEntityMNode extends IMNode {
-
-  boolean addAlias(String alias, IMeasurementMNode child);
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
 
-  void deleteAliasChild(String alias);
+import java.util.List;
+import java.util.Map;
 
-  Map<String, IMeasurementMNode> getAliasChildren();
+public class PreDeactivateTemplatePlanImpl implements IPreDeactivateTemplatePlan {
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+  private Map<PartialPath, List<Integer>> templateSetInfo;
 
   @Override
-  boolean isUseTemplate();
-
-  void setUseTemplate(boolean useTemplate);
+  public Map<PartialPath, List<Integer>> getTemplateSetInfo() {
+    return templateSetInfo;
+  }
 
-  boolean isAligned();
-
-  void setAligned(boolean isAligned);
-
-  ILastCacheContainer getLastCacheContainer(String measurementId);
-
-  Map<String, ILastCacheContainer> getTemplateLastCaches();
+  @Override
+  public void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo) {
+    this.templateSetInfo = templateSetInfo;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/RollbackPreDeactivateTemplatePlanImpl.java
similarity index 56%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/RollbackPreDeactivateTemplatePlanImpl.java
index a6780cb8f3..65a442f473 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/RollbackPreDeactivateTemplatePlanImpl.java
@@ -16,32 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+package org.apache.iotdb.db.metadata.plan.schemaregion.impl;
 
-import java.util.Map;
-
-public interface IEntityMNode extends IMNode {
-
-  boolean addAlias(String alias, IMeasurementMNode child);
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 
-  void deleteAliasChild(String alias);
+import java.util.List;
+import java.util.Map;
 
-  Map<String, IMeasurementMNode> getAliasChildren();
+public class RollbackPreDeactivateTemplatePlanImpl implements IRollbackPreDeactivateTemplatePlan {
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+  private Map<PartialPath, List<Integer>> templateSetInfo;
 
   @Override
-  boolean isUseTemplate();
-
-  void setUseTemplate(boolean useTemplate);
+  public Map<PartialPath, List<Integer>> getTemplateSetInfo() {
+    return templateSetInfo;
+  }
 
-  boolean isAligned();
-
-  void setAligned(boolean isAligned);
-
-  ILastCacheContainer getLastCacheContainer(String measurementId);
-
-  Map<String, ILastCacheContainer> getTemplateLastCaches();
+  @Override
+  public void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo) {
+    this.templateSetInfo = templateSetInfo;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanDeserializer.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanDeserializer.java
index f77568ad09..95d9c76396 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanDeserializer.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanDeserializer.java
@@ -33,8 +33,11 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeAliasPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeTagOffsetPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeleteTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
@@ -48,6 +51,7 @@ import org.slf4j.LoggerFactory;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -333,5 +337,44 @@ public class SchemaRegionPlanDeserializer implements IDeserializer<ISchemaRegion
 
       return unsetTemplatePlan;
     }
+
+    @Override
+    public ISchemaRegionPlan visitPreDeactivateTemplate(
+        IPreDeactivateTemplatePlan preDeactivateTemplatePlan, ByteBuffer buffer) {
+      preDeactivateTemplatePlan.setTemplateSetInfo(deserializeTemplateSetInfo(buffer));
+      return preDeactivateTemplatePlan;
+    }
+
+    @Override
+    public ISchemaRegionPlan visitRollbackPreDeactivateTemplate(
+        IRollbackPreDeactivateTemplatePlan rollbackPreDeactivateTemplatePlan, ByteBuffer buffer) {
+      rollbackPreDeactivateTemplatePlan.setTemplateSetInfo(deserializeTemplateSetInfo(buffer));
+      return rollbackPreDeactivateTemplatePlan;
+    }
+
+    @Override
+    public ISchemaRegionPlan visitDeactivateTemplate(
+        IDeactivateTemplatePlan deactivateTemplatePlan, ByteBuffer buffer) {
+      deactivateTemplatePlan.setTemplateSetInfo(deserializeTemplateSetInfo(buffer));
+      return deactivateTemplatePlan;
+    }
+
+    private Map<PartialPath, List<Integer>> deserializeTemplateSetInfo(ByteBuffer buffer) {
+      int size = buffer.getInt();
+      Map<PartialPath, List<Integer>> result = new HashMap<>(size);
+      PartialPath pattern;
+      int templateNum;
+      List<Integer> templateIdList;
+      for (int i = 0; i < size; i++) {
+        pattern = (PartialPath) PathDeserializeUtil.deserialize(buffer);
+        templateNum = buffer.getInt();
+        templateIdList = new ArrayList<>(templateNum);
+        for (int j = 0; j < templateNum; j++) {
+          templateIdList.add(buffer.getInt());
+        }
+        result.put(pattern, templateIdList);
+      }
+      return result;
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanFactory.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanFactory.java
index 7a2ccb0e84..eeacf36177 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanFactory.java
@@ -68,6 +68,12 @@ public class SchemaRegionPlanFactory {
         return new PreDeleteTimeSeriesPlanImpl();
       case ROLLBACK_PRE_DELETE_TIMESERIES:
         return new RollbackPreDeleteTimeSeriesPlanImpl();
+      case PRE_DEACTIVATE_TEMPLATE:
+        return new PreDeactivateTemplatePlanImpl();
+      case ROLLBACK_PRE_DEACTIVATE_TEMPLATE:
+        return new RollbackPreDeactivateTemplatePlanImpl();
+      case DEACTIVATE_TEMPLATE:
+        return new DeactivateTemplatePlanImpl();
       default:
         throw new UnsupportedOperationException(
             String.format(
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanSerializer.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanSerializer.java
index 9bed473b8f..4f4b4167a4 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanSerializer.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanSerializer.java
@@ -30,8 +30,11 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeAliasPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeTagOffsetPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeleteTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
@@ -395,5 +398,53 @@ public class SchemaRegionPlanSerializer implements ISerializer<ISchemaRegionPlan
         return new SchemaRegionPlanSerializationResult(e);
       }
     }
+
+    @Override
+    public SchemaRegionPlanSerializationResult visitPreDeactivateTemplate(
+        IPreDeactivateTemplatePlan preDeactivateTemplatePlan, DataOutputStream dataOutputStream) {
+      try {
+        serializeTemplateSetInfo(preDeactivateTemplatePlan.getTemplateSetInfo(), dataOutputStream);
+        return SchemaRegionPlanSerializationResult.SUCCESS;
+      } catch (IOException e) {
+        return new SchemaRegionPlanSerializationResult(e);
+      }
+    }
+
+    @Override
+    public SchemaRegionPlanSerializationResult visitRollbackPreDeactivateTemplate(
+        IRollbackPreDeactivateTemplatePlan rollbackPreDeactivateTemplatePlan,
+        DataOutputStream dataOutputStream) {
+      try {
+        serializeTemplateSetInfo(
+            rollbackPreDeactivateTemplatePlan.getTemplateSetInfo(), dataOutputStream);
+        return SchemaRegionPlanSerializationResult.SUCCESS;
+      } catch (IOException e) {
+        return new SchemaRegionPlanSerializationResult(e);
+      }
+    }
+
+    @Override
+    public SchemaRegionPlanSerializationResult visitDeactivateTemplate(
+        IDeactivateTemplatePlan deactivateTemplatePlan, DataOutputStream dataOutputStream) {
+      try {
+        serializeTemplateSetInfo(deactivateTemplatePlan.getTemplateSetInfo(), dataOutputStream);
+        return SchemaRegionPlanSerializationResult.SUCCESS;
+      } catch (IOException e) {
+        return new SchemaRegionPlanSerializationResult(e);
+      }
+    }
+
+    private void serializeTemplateSetInfo(
+        Map<PartialPath, List<Integer>> templateSetInfo, DataOutputStream dataOutputStream)
+        throws IOException {
+      dataOutputStream.writeInt(templateSetInfo.size());
+      for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
+        entry.getKey().serialize(dataOutputStream);
+        dataOutputStream.writeInt(entry.getValue().size());
+        for (int templateId : entry.getValue()) {
+          dataOutputStream.writeInt(templateId);
+        }
+      }
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanTxtSerializer.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanTxtSerializer.java
index 775e77d241..7d8cfad2a0 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanTxtSerializer.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/impl/SchemaRegionPlanTxtSerializer.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.metadata.plan.schemaregion.impl;
 
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.metadata.logfile.ISerializer;
 import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
@@ -29,8 +30,11 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeAliasPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeTagOffsetPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeleteTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
@@ -41,6 +45,8 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 /**
@@ -234,5 +240,34 @@ public class SchemaRegionPlanTxtSerializer implements ISerializer<ISchemaRegionP
           .append(unsetTemplatePlan.getTemplateName());
       return null;
     }
+
+    @Override
+    public Void visitPreDeactivateTemplate(
+        IPreDeactivateTemplatePlan preDeactivateTemplatePlan, StringBuilder stringBuilder) {
+      parseTemplateSetInfo(preDeactivateTemplatePlan.getTemplateSetInfo(), stringBuilder);
+      return null;
+    }
+
+    @Override
+    public Void visitRollbackPreDeactivateTemplate(
+        IRollbackPreDeactivateTemplatePlan rollbackPreDeactivateTemplatePlan,
+        StringBuilder stringBuilder) {
+      parseTemplateSetInfo(rollbackPreDeactivateTemplatePlan.getTemplateSetInfo(), stringBuilder);
+      return null;
+    }
+
+    @Override
+    public Void visitDeactivateTemplate(
+        IDeactivateTemplatePlan deactivateTemplatePlan, StringBuilder stringBuilder) {
+      parseTemplateSetInfo(deactivateTemplatePlan.getTemplateSetInfo(), stringBuilder);
+      return null;
+    }
+
+    private void parseTemplateSetInfo(
+        Map<PartialPath, List<Integer>> templateSetInfo, StringBuilder stringBuilder) {
+      stringBuilder.append("{");
+      templateSetInfo.forEach((k, v) -> stringBuilder.append(k).append(": ").append(v).append(";"));
+      stringBuilder.append("}");
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IDeactivateTemplatePlan.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IDeactivateTemplatePlan.java
index a6780cb8f3..39a20e32e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IDeactivateTemplatePlan.java
@@ -16,32 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+package org.apache.iotdb.db.metadata.plan.schemaregion.write;
 
-import java.util.Map;
-
-public interface IEntityMNode extends IMNode {
-
-  boolean addAlias(String alias, IMeasurementMNode child);
-
-  void deleteAliasChild(String alias);
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanType;
+import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
 
-  Map<String, IMeasurementMNode> getAliasChildren();
+import java.util.List;
+import java.util.Map;
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+public interface IDeactivateTemplatePlan extends ISchemaRegionPlan {
 
   @Override
-  boolean isUseTemplate();
-
-  void setUseTemplate(boolean useTemplate);
+  default SchemaRegionPlanType getPlanType() {
+    return SchemaRegionPlanType.DEACTIVATE_TEMPLATE;
+  }
 
-  boolean isAligned();
-
-  void setAligned(boolean isAligned);
+  @Override
+  default <R, C> R accept(SchemaRegionPlanVisitor<R, C> visitor, C context) {
+    return visitor.visitDeactivateTemplate(this, context);
+  }
 
-  ILastCacheContainer getLastCacheContainer(String measurementId);
+  Map<PartialPath, List<Integer>> getTemplateSetInfo();
 
-  Map<String, ILastCacheContainer> getTemplateLastCaches();
+  void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IPreDeactivateTemplatePlan.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IPreDeactivateTemplatePlan.java
index a6780cb8f3..5f85d6df4c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IPreDeactivateTemplatePlan.java
@@ -16,32 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+package org.apache.iotdb.db.metadata.plan.schemaregion.write;
 
-import java.util.Map;
-
-public interface IEntityMNode extends IMNode {
-
-  boolean addAlias(String alias, IMeasurementMNode child);
-
-  void deleteAliasChild(String alias);
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanType;
+import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
 
-  Map<String, IMeasurementMNode> getAliasChildren();
+import java.util.List;
+import java.util.Map;
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+public interface IPreDeactivateTemplatePlan extends ISchemaRegionPlan {
 
   @Override
-  boolean isUseTemplate();
-
-  void setUseTemplate(boolean useTemplate);
+  default SchemaRegionPlanType getPlanType() {
+    return SchemaRegionPlanType.PRE_DEACTIVATE_TEMPLATE;
+  }
 
-  boolean isAligned();
-
-  void setAligned(boolean isAligned);
+  @Override
+  default <R, C> R accept(SchemaRegionPlanVisitor<R, C> visitor, C context) {
+    return visitor.visitPreDeactivateTemplate(this, context);
+  }
 
-  ILastCacheContainer getLastCacheContainer(String measurementId);
+  Map<PartialPath, List<Integer>> getTemplateSetInfo();
 
-  Map<String, ILastCacheContainer> getTemplateLastCaches();
+  void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IRollbackPreDeactivateTemplatePlan.java b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IRollbackPreDeactivateTemplatePlan.java
new file mode 100644
index 0000000000..e37cd63adf
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/plan/schemaregion/write/IRollbackPreDeactivateTemplatePlan.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.metadata.plan.schemaregion.write;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanType;
+import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
+
+import java.util.List;
+import java.util.Map;
+
+public interface IRollbackPreDeactivateTemplatePlan extends ISchemaRegionPlan {
+
+  @Override
+  default SchemaRegionPlanType getPlanType() {
+    return SchemaRegionPlanType.ROLLBACK_PRE_DEACTIVATE_TEMPLATE;
+  }
+
+  @Override
+  default <R, C> R accept(SchemaRegionPlanVisitor<R, C> visitor, C context) {
+    return visitor.visitRollbackPreDeactivateTemplate(this, context);
+  }
+
+  Map<PartialPath, List<Integer>> getTemplateSetInfo();
+
+  void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
index eccdde71de..3a54be10dc 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
@@ -34,6 +34,9 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IActivateTemplatePla
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IAutoCreateDeviceMNodePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
 import org.apache.iotdb.db.metadata.template.Template;
@@ -429,7 +432,17 @@ public interface ISchemaRegion {
   void activateSchemaTemplate(IActivateTemplateInClusterPlan plan, Template template)
       throws MetadataException;
 
-  List<String> getPathsUsingTemplate(int templateId) throws MetadataException;
+  List<String> getPathsUsingTemplate(PartialPath pathPattern, int templateId)
+      throws MetadataException;
+
+  int constructSchemaBlackListWithTemplate(IPreDeactivateTemplatePlan plan)
+      throws MetadataException;
+
+  void rollbackSchemaBlackListWithTemplate(IRollbackPreDeactivateTemplatePlan plan)
+      throws MetadataException;
+
+  void deactivateTemplateInBlackList(IDeactivateTemplatePlan plan) throws MetadataException;
+
   // endregion
 
   // region Interfaces for Trigger
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 b55db20d7b..4ac6d44c41 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
@@ -51,6 +51,7 @@ import org.apache.iotdb.db.metadata.logfile.FakeCRC32Deserializer;
 import org.apache.iotdb.db.metadata.logfile.FakeCRC32Serializer;
 import org.apache.iotdb.db.metadata.logfile.SchemaLogReader;
 import org.apache.iotdb.db.metadata.logfile.SchemaLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
@@ -68,8 +69,11 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeAliasPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeTagOffsetPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeleteTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
@@ -120,8 +124,6 @@ import java.util.function.Function;
 import java.util.stream.Stream;
 
 import static java.util.stream.Collectors.toList;
-import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_TEMPLATE;
-import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
 import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
 import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
 
@@ -172,8 +174,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
   private String storageGroupFullPath;
   private SchemaRegionId schemaRegionId;
 
-  private int templateId = NON_TEMPLATE;
-
   // the log file writer
   private boolean usingMLog = true;
   private SchemaLogWriter<ISchemaRegionPlan> logWriter;
@@ -241,8 +241,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
       return;
     }
 
-    templateId = NON_TEMPLATE;
-
     initDir();
 
     try {
@@ -404,8 +402,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
   public synchronized void clear() {
     try {
 
-      templateId = NON_TEMPLATE;
-
       if (this.mNodeCache != null) {
         this.mNodeCache.invalidateAll();
       }
@@ -2069,13 +2065,9 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
   public void activateSchemaTemplate(IActivateTemplateInClusterPlan plan, Template template)
       throws MetadataException {
     try {
-      if (plan.getPathSetTemplate().getFullPath().length() <= storageGroupFullPath.length()) {
-        templateId = plan.getTemplateId();
-      }
-
       getDeviceNodeWithAutoCreate(plan.getActivatePath());
 
-      mtree.activateTemplate(plan.getActivatePath(), plan.getTemplateSetLevel(), template);
+      mtree.activateTemplate(plan.getActivatePath(), template);
       writeToMLog(plan);
     } catch (IOException e) {
       logger.error(e.getMessage(), e);
@@ -2084,21 +2076,71 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
   }
 
   private void recoverActivatingSchemaTemplate(IActivateTemplateInClusterPlan plan) {
-    if (plan.getPathSetTemplate().getFullPath().length() <= storageGroupFullPath.length()) {
-      templateId = plan.getTemplateId();
-    }
     mtree.activateTemplateWithoutCheck(
-        plan.getActivatePath(), plan.getTemplateSetLevel(), plan.getTemplateId(), plan.isAligned());
+        plan.getActivatePath(), plan.getTemplateId(), plan.isAligned());
+  }
+
+  @Override
+  public List<String> getPathsUsingTemplate(PartialPath pathPattern, int templateId)
+      throws MetadataException {
+    return mtree.getPathsUsingTemplate(pathPattern, templateId);
   }
 
   @Override
-  public List<String> getPathsUsingTemplate(int templateId) throws MetadataException {
-    if (this.templateId != NON_TEMPLATE
-        && templateId != ALL_TEMPLATE
-        && this.templateId != templateId) {
-      return Collections.emptyList();
+  public int constructSchemaBlackListWithTemplate(IPreDeactivateTemplatePlan plan)
+      throws MetadataException {
+    int preDeactivateNum = 0;
+    Map<PartialPath, List<Integer>> templateSetInfo = plan.getTemplateSetInfo();
+    for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
+      for (IEntityMNode entityMNode :
+          mtree.getDeviceMNodeUsingTargetTemplate(entry.getKey(), entry.getValue())) {
+        entityMNode.preDeactivateTemplate();
+        preDeactivateNum++;
+        try {
+          writeToMLog(plan);
+        } catch (IOException e) {
+          throw new MetadataException(e);
+        }
+      }
+    }
+    return preDeactivateNum;
+  }
+
+  @Override
+  public void rollbackSchemaBlackListWithTemplate(IRollbackPreDeactivateTemplatePlan plan)
+      throws MetadataException {
+    Map<PartialPath, List<Integer>> templateSetInfo = plan.getTemplateSetInfo();
+    for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
+      for (IEntityMNode entityMNode :
+          mtree.getPreDeactivatedDeviceMNode(entry.getKey(), entry.getValue())) {
+        if (!entityMNode.isPreDeactivateTemplate()) {
+          continue;
+        }
+        entityMNode.rollbackPreDeactivateTemplate();
+        try {
+          writeToMLog(plan);
+        } catch (IOException e) {
+          throw new MetadataException(e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void deactivateTemplateInBlackList(IDeactivateTemplatePlan plan) throws MetadataException {
+    Map<PartialPath, List<Integer>> templateSetInfo = plan.getTemplateSetInfo();
+    for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
+      for (IEntityMNode entityMNode :
+          mtree.getPreDeactivatedDeviceMNode(entry.getKey(), entry.getValue())) {
+        entityMNode.deactivateTemplate();
+        mtree.deleteEmptyInternalMNodeAndReturnEmptyStorageGroup(entityMNode);
+        try {
+          writeToMLog(plan);
+        } catch (IOException e) {
+          throw new MetadataException(e);
+        }
+      }
     }
-    return mtree.getPathsUsingTemplate(templateId);
   }
 
   // endregion
@@ -2279,5 +2321,39 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
         return new RecoverOperationResult(e);
       }
     }
+
+    @Override
+    public RecoverOperationResult visitPreDeactivateTemplate(
+        IPreDeactivateTemplatePlan preDeactivateTemplatePlan, SchemaRegionMemoryImpl context) {
+      try {
+        constructSchemaBlackListWithTemplate(preDeactivateTemplatePlan);
+        return RecoverOperationResult.SUCCESS;
+      } catch (MetadataException e) {
+        return new RecoverOperationResult(e);
+      }
+    }
+
+    @Override
+    public RecoverOperationResult visitRollbackPreDeactivateTemplate(
+        IRollbackPreDeactivateTemplatePlan rollbackPreDeactivateTemplatePlan,
+        SchemaRegionMemoryImpl context) {
+      try {
+        rollbackSchemaBlackListWithTemplate(rollbackPreDeactivateTemplatePlan);
+        return RecoverOperationResult.SUCCESS;
+      } catch (MetadataException e) {
+        return new RecoverOperationResult(e);
+      }
+    }
+
+    @Override
+    public RecoverOperationResult visitDeactivateTemplate(
+        IDeactivateTemplatePlan deactivateTemplatePlan, SchemaRegionMemoryImpl context) {
+      try {
+        deactivateTemplateInBlackList(deactivateTemplatePlan);
+        return RecoverOperationResult.SUCCESS;
+      } catch (MetadataException e) {
+        return new RecoverOperationResult(e);
+      }
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
index df28bc01da..b91d05a54b 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
@@ -66,7 +66,10 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeAliasPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IChangeTagOffsetPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeleteTimeSeriesPlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
 import org.apache.iotdb.db.metadata.rescon.MemoryStatistics;
@@ -1917,7 +1920,25 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
   }
 
   @Override
-  public List<String> getPathsUsingTemplate(int templateId) throws MetadataException {
+  public List<String> getPathsUsingTemplate(PartialPath pathPattern, int templateId)
+      throws MetadataException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int constructSchemaBlackListWithTemplate(IPreDeactivateTemplatePlan plan)
+      throws MetadataException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void rollbackSchemaBlackListWithTemplate(IRollbackPreDeactivateTemplatePlan plan)
+      throws MetadataException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deactivateTemplateInBlackList(IDeactivateTemplatePlan plan) throws MetadataException {
     throw new UnsupportedOperationException();
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java b/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
index 58ee117458..9b6c0961cb 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
@@ -76,7 +76,10 @@ public class Template implements Serializable {
   // since order of CreateTemplatePlan is fixed, this code shall be fixed as well
   private int rehashCode;
 
-  public Template() {}
+  public Template() {
+    schemaMap = new HashMap<>();
+    directNodes = new HashMap<>();
+  }
 
   /**
    * build a template from a createTemplatePlan
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/visitor/SchemaExecutionVisitor.java b/server/src/main/java/org/apache/iotdb/db/metadata/visitor/SchemaExecutionVisitor.java
index cf3ab351f4..e67be55539 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/visitor/SchemaExecutionVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/visitor/SchemaExecutionVisitor.java
@@ -39,9 +39,12 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.ConstructSc
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateMultiTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateTimeSeriesNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeleteTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.InternalCreateTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.MeasurementGroup;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.PreDeactivateTemplateNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackPreDeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackSchemaBlackListNode;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
@@ -321,6 +324,41 @@ public class SchemaExecutionVisitor extends PlanVisitor<TSStatus, ISchemaRegion>
     }
   }
 
+  @Override
+  public TSStatus visitPreDeactivateTemplate(
+      PreDeactivateTemplateNode node, ISchemaRegion schemaRegion) {
+    try {
+      int preDeactivateNum = schemaRegion.constructSchemaBlackListWithTemplate(node);
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, String.valueOf(preDeactivateNum));
+    } catch (MetadataException e) {
+      logger.error(e.getMessage(), e);
+      return RpcUtils.getStatus(e.getErrorCode(), e.getMessage());
+    }
+  }
+
+  @Override
+  public TSStatus visitRollbackPreDeactivateTemplate(
+      RollbackPreDeactivateTemplateNode node, ISchemaRegion schemaRegion) {
+    try {
+      schemaRegion.rollbackSchemaBlackListWithTemplate(node);
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    } catch (MetadataException e) {
+      logger.error(e.getMessage(), e);
+      return RpcUtils.getStatus(e.getErrorCode(), e.getMessage());
+    }
+  }
+
+  @Override
+  public TSStatus visitDeactivateTemplate(DeactivateTemplateNode node, ISchemaRegion schemaRegion) {
+    try {
+      schemaRegion.deactivateTemplateInBlackList(node);
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    } catch (MetadataException e) {
+      logger.error(e.getMessage(), e);
+      return RpcUtils.getStatus(e.getErrorCode(), e.getMessage());
+    }
+  }
+
   @Override
   public TSStatus visitPlan(PlanNode node, ISchemaRegion context) {
     return null;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/PathsUsingTemplateScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/PathsUsingTemplateScanOperator.java
index 0c8a9a2b4e..a89346aa23 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/PathsUsingTemplateScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/PathsUsingTemplateScanOperator.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.db.mpp.execution.operator.schema;
 
 import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
 import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
@@ -30,18 +31,25 @@ import org.apache.iotdb.tsfile.read.common.block.TsBlock;
 import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
 import org.apache.iotdb.tsfile.utils.Binary;
 
+import java.util.LinkedList;
 import java.util.List;
 import java.util.stream.Collectors;
 
 public class PathsUsingTemplateScanOperator extends SchemaQueryScanOperator {
 
+  private final List<PartialPath> pathPatternList;
+
   private final int templateId;
 
   private final List<TSDataType> outputDataTypes;
 
   public PathsUsingTemplateScanOperator(
-      PlanNodeId planNodeId, OperatorContext context, int templateId) {
+      PlanNodeId planNodeId,
+      OperatorContext context,
+      List<PartialPath> pathPatternList,
+      int templateId) {
     super(planNodeId, context, 0, 0, null, false);
+    this.pathPatternList = pathPatternList;
     this.templateId = templateId;
     this.outputDataTypes =
         ColumnHeaderConstant.showPathsUsingTemplateHeaders.stream()
@@ -52,10 +60,13 @@ public class PathsUsingTemplateScanOperator extends SchemaQueryScanOperator {
   @Override
   protected List<TsBlock> createTsBlockList() {
     try {
-      List<String> schemaRegionResult =
-          ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
-              .getSchemaRegion()
-              .getPathsUsingTemplate(templateId);
+      List<String> schemaRegionResult = new LinkedList<>();
+      for (PartialPath pathPattern : pathPatternList) {
+        schemaRegionResult.addAll(
+            ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
+                .getSchemaRegion()
+                .getPathsUsingTemplate(pathPattern, templateId));
+      }
       return SchemaTsBlockUtil.transferSchemaResultToTsBlockList(
           schemaRegionResult.iterator(), outputDataTypes, this::setColumns);
     } catch (MetadataException e) {
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 5cec1e6d04..dec1ba3ba8 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
@@ -182,6 +182,9 @@ public class Analysis {
   // potential template used in timeseries query or fetch
   private Map<Integer, Template> relatedTemplateInfo;
 
+  // generated by combine the input path pattern and template set path
+  private List<PartialPath> specifiedTemplateRelatedPathPatternList;
+
   public Analysis() {
     this.finishQueryAfterAnalyze = false;
   }
@@ -437,6 +440,15 @@ public class Analysis {
     this.relatedTemplateInfo = relatedTemplateInfo;
   }
 
+  public List<PartialPath> getSpecifiedTemplateRelatedPathPatternList() {
+    return specifiedTemplateRelatedPathPatternList;
+  }
+
+  public void setSpecifiedTemplateRelatedPathPatternList(
+      List<PartialPath> specifiedTemplateRelatedPathPatternList) {
+    this.specifiedTemplateRelatedPathPatternList = specifiedTemplateRelatedPathPatternList;
+  }
+
   public void addTypes(Map<NodeRef<Expression>, TSDataType> types) {
     this.expressionTypes.putAll(types);
   }
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 77e086f4eb..925d5558ec 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
@@ -146,7 +146,6 @@ import java.util.TimeZone;
 import java.util.stream.Collectors;
 
 import static com.google.common.base.Preconditions.checkState;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
 import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_RESULT_NODES;
 import static org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant.COLUMN_DEVICE;
@@ -2260,7 +2259,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
 
     Pair<Template, List<PartialPath>> templateSetInfo =
         schemaFetcher.getAllPathsSetTemplate(showPathsUsingTemplateStatement.getTemplateName());
-    analysis.setTemplateSetInfo(templateSetInfo);
+
     if (templateSetInfo == null
         || templateSetInfo.right == null
         || templateSetInfo.right.isEmpty()) {
@@ -2268,13 +2267,26 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       return analysis;
     }
 
+    analysis.setTemplateSetInfo(templateSetInfo);
+
     PathPatternTree patternTree = new PathPatternTree();
+    PartialPath rawPathPattern = showPathsUsingTemplateStatement.getPathPattern();
+    List<PartialPath> specifiedPatternList = new ArrayList<>();
     templateSetInfo.right.forEach(
-        path -> {
-          patternTree.appendPathPattern(path);
-          patternTree.appendPathPattern(path.concatNode(MULTI_LEVEL_PATH_WILDCARD));
+        setPath -> {
+          for (PartialPath specifiedPattern : rawPathPattern.alterPrefixPath(setPath)) {
+            patternTree.appendPathPattern(specifiedPattern);
+            specifiedPatternList.add(specifiedPattern);
+          }
         });
 
+    if (specifiedPatternList.isEmpty()) {
+      analysis.setFinishQueryAfterAnalyze(true);
+      return analysis;
+    }
+
+    analysis.setSpecifiedTemplateRelatedPathPatternList(specifiedPatternList);
+
     SchemaPartition partition = partitionFetcher.getOrCreateSchemaPartition(patternTree);
     analysis.setSchemaPartitionInfo(partition);
     if (partition.isEmpty()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java
index 680d922e28..411fbc5bbd 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java
@@ -150,4 +150,6 @@ public enum StatementType {
   DROP_PIPE,
 
   SHOW_TRIGGERS,
+
+  DEACTIVATE_TEMPLATE,
 }
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 7fc68c0d46..db163bbac9 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
@@ -41,6 +41,7 @@ import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowTriggersTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.UnSetTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.CreateSchemaTemplateTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.DeactivateSchemaTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.SetSchemaTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowNodesInSchemaTemplateTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.template.ShowPathSetTemplateTask;
@@ -84,6 +85,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTriggersStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.UnSetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.template.DeactivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.SetSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
@@ -265,6 +267,12 @@ public class ConfigTaskVisitor
     return new ShowPathSetTemplateTask(showPathSetTemplateStatement);
   }
 
+  @Override
+  public IConfigTask visitDeactivateTemplate(
+      DeactivateTemplateStatement deactivateTemplateStatement, TaskContext context) {
+    return new DeactivateSchemaTemplateTask(context.getQueryId(), deactivateTemplateStatement);
+  }
+
   @Override
   public IConfigTask visitShowDataNodes(
       ShowDataNodesStatement showDataNodesStatement, 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 fff5a5545b..4cab93e460 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,6 +35,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCountStorageGroupResp;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateFunctionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
 import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq;
+import org.apache.iotdb.confignode.rpc.thrift.TDeactivateSchemaTemplateReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupsReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteTimeSeriesReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDropFunctionReq;
@@ -100,6 +101,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowRegionStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.template.DeactivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.SetSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
@@ -815,6 +817,66 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     return future;
   }
 
+  @Override
+  public SettableFuture<ConfigTaskResult> deactivateSchemaTemplate(
+      String queryId, DeactivateTemplateStatement deactivateTemplateStatement) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    TDeactivateSchemaTemplateReq req = new TDeactivateSchemaTemplateReq();
+    req.setQueryId(queryId);
+    req.setTemplateName(deactivateTemplateStatement.getTemplateName());
+    req.setPathPatternTree(
+        serializePatternListToByteBuffer(deactivateTemplateStatement.getPathPatternList()));
+    try (ConfigNodeClient client =
+        CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(
+            ConfigNodeInfo.partitionRegionId)) {
+      TSStatus tsStatus;
+      do {
+        try {
+          tsStatus = client.deactivateSchemaTemplate(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.STILL_EXECUTING_STATUS);
+          } else {
+            throw e;
+          }
+        }
+        // keep waiting until task ends
+      } while (TSStatusCode.STILL_EXECUTING_STATUS.getStatusCode() == tsStatus.getCode());
+
+      if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
+        LOGGER.error(
+            "Failed to execute deactivate schema template {} from {} in config node, status is {}.",
+            deactivateTemplateStatement.getPathPatternList(),
+            deactivateTemplateStatement.getTemplateName(),
+            tsStatus);
+        future.setException(new IoTDBException(tsStatus.getMessage(), tsStatus.getCode()));
+      } else {
+        future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+      }
+    } catch (TException | IOException e) {
+      future.setException(e);
+    }
+    return future;
+  }
+
+  private ByteBuffer serializePatternListToByteBuffer(List<PartialPath> patternList) {
+    PathPatternTree patternTree = new PathPatternTree();
+    for (PartialPath pathPattern : patternList) {
+      patternTree.appendPathPattern(pathPattern);
+    }
+    patternTree.constructTree();
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+    try {
+      patternTree.serialize(dataOutputStream);
+    } catch (IOException ignored) {
+      // memory operation, won't happen
+    }
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
   @Override
   public SettableFuture<ConfigTaskResult> createPipeSink(
       CreatePipeSinkStatement createPipeSinkStatement) {
@@ -989,20 +1051,10 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> deleteTimeSeries(
       String queryId, DeleteTimeSeriesStatement deleteTimeSeriesStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    PathPatternTree patternTree = new PathPatternTree();
-    for (PartialPath pathPattern : deleteTimeSeriesStatement.getPathPatternList()) {
-      patternTree.appendPathPattern(pathPattern);
-    }
-    patternTree.constructTree();
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
-    try {
-      patternTree.serialize(dataOutputStream);
-    } catch (IOException ignored) {
-      // memory operation, won't happen
-    }
     TDeleteTimeSeriesReq req =
-        new TDeleteTimeSeriesReq(queryId, ByteBuffer.wrap(byteArrayOutputStream.toByteArray()));
+        new TDeleteTimeSeriesReq(
+            queryId,
+            serializePatternListToByteBuffer(deleteTimeSeriesStatement.getPathPatternList()));
     try (ConfigNodeClient client =
         CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(
             ConfigNodeInfo.partitionRegionId)) {
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 9cbae3e4f6..1881fbefd1 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
@@ -36,6 +36,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowRegionStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.template.DeactivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.SetSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
@@ -115,6 +116,9 @@ public interface IConfigTaskExecutor {
   SettableFuture<ConfigTaskResult> showPathSetTemplate(
       ShowPathSetTemplateStatement showPathSetTemplateStatement);
 
+  SettableFuture<ConfigTaskResult> deactivateSchemaTemplate(
+      String queryId, DeactivateTemplateStatement deactivateTemplateStatement);
+
   SettableFuture<ConfigTaskResult> createPipeSink(CreatePipeSinkStatement createPipeSinkStatement);
 
   SettableFuture<ConfigTaskResult> dropPipeSink(DropPipeSinkStatement dropPipeSinkStatement);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
index a934b815c7..b3d48aadf0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
@@ -54,6 +54,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowRegionStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.template.DeactivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.SetSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
@@ -480,6 +481,17 @@ public class StandaloneConfigTaskExecutor implements IConfigTaskExecutor {
     return future;
   }
 
+  @Override
+  public SettableFuture<ConfigTaskResult> deactivateSchemaTemplate(
+      String queryId, DeactivateTemplateStatement deactivateTemplateStatement) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    future.setException(
+        new IoTDBException(
+            "Executing deactivate schema template is not supported",
+            TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()));
+    return future;
+  }
+
   @Override
   public SettableFuture<ConfigTaskResult> createPipeSink(
       CreatePipeSinkStatement createPipeSinkStatement) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/template/DeactivateSchemaTemplateTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/template/DeactivateSchemaTemplateTask.java
new file mode 100644
index 0000000000..f4be211ad2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/template/DeactivateSchemaTemplateTask.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.template;
+
+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.template.DeactivateTemplateStatement;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public class DeactivateSchemaTemplateTask implements IConfigTask {
+
+  private final String queryId;
+
+  private final DeactivateTemplateStatement statement;
+
+  public DeactivateSchemaTemplateTask(String queryId, DeactivateTemplateStatement statement) {
+    this.queryId = queryId;
+    this.statement = statement;
+  }
+
+  @Override
+  public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
+      throws InterruptedException {
+    return configTaskExecutor.deactivateSchemaTemplate(queryId, statement);
+  }
+}
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 d599436580..14bf81bd82 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
@@ -125,6 +125,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTriggersStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.UnSetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ActivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.template.DeactivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.SetSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
@@ -2870,7 +2871,29 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   @Override
   public Statement visitShowPathsUsingSchemaTemplate(
       IoTDBSqlParser.ShowPathsUsingSchemaTemplateContext ctx) {
-    return new ShowPathsUsingTemplateStatement(parseIdentifier(ctx.templateName.getText()));
+    PartialPath pathPattern;
+    if (ctx.prefixPath() == null) {
+      pathPattern = new PartialPath(SQLConstant.getSingleRootArray());
+    } else {
+      pathPattern = parsePrefixPath(ctx.prefixPath());
+    }
+    return new ShowPathsUsingTemplateStatement(
+        pathPattern, parseIdentifier(ctx.templateName.getText()));
+  }
+
+  @Override
+  public Statement visitDeleteTimeseriesOfSchemaTemplate(
+      IoTDBSqlParser.DeleteTimeseriesOfSchemaTemplateContext ctx) {
+    DeactivateTemplateStatement statement = new DeactivateTemplateStatement();
+    if (ctx.templateName != null) {
+      statement.setTemplateName(parseIdentifier(ctx.templateName.getText()));
+    }
+    List<PartialPath> pathPatternList = new ArrayList<>();
+    for (IoTDBSqlParser.PrefixPathContext prefixPathContext : ctx.prefixPath()) {
+      pathPatternList.add(parsePrefixPath(prefixPathContext));
+    }
+    statement.setPathPatternList(pathPatternList);
+    return statement;
   }
 
   public Map<String, String> parseSyncAttributeClauses(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
index 97a945bcc7..f478d7af33 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
@@ -54,6 +54,7 @@ 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.ShowPathsUsingTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.sql.IoTDBSqlParser;
 import org.apache.iotdb.db.qp.sql.SqlLexer;
 import org.apache.iotdb.db.qp.strategy.SQLParseError;
@@ -613,7 +614,8 @@ public class StatementGenerator {
       case SHOW_SET_TEMPLATES:
         return new ShowPathSetTemplateStatement(req.getName());
       case SHOW_USING_TEMPLATES:
-        return new ShowPathsUsingTemplateStatement(req.getName());
+        return new ShowPathsUsingTemplateStatement(
+            new PartialPath(SQLConstant.getSingleRootArray()), req.getName());
       default:
         return null;
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
index 965c4bd477..31643cd550 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
@@ -1013,8 +1013,11 @@ public class LogicalPlanBuilder {
     return this;
   }
 
-  public LogicalPlanBuilder planPathsUsingTemplateSource(int templateId) {
-    this.root = new PathsUsingTemplateScanNode(context.getQueryId().genPlanNodeId(), templateId);
+  public LogicalPlanBuilder planPathsUsingTemplateSource(
+      List<PartialPath> pathPatternList, int templateId) {
+    this.root =
+        new PathsUsingTemplateScanNode(
+            context.getQueryId().genPlanNodeId(), pathPatternList, templateId);
     return this;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
index 2f2f954520..f98475912f 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
@@ -688,7 +688,9 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
     LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(analysis, context);
     planBuilder =
         planBuilder
-            .planPathsUsingTemplateSource(analysis.getTemplateSetInfo().left.getId())
+            .planPathsUsingTemplateSource(
+                analysis.getSpecifiedTemplateRelatedPathPatternList(),
+                analysis.getTemplateSetInfo().left.getId())
             .planSchemaQueryMerge(false);
     return planBuilder.getRoot();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java
index f3de5e72d4..d83990cde3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java
@@ -1824,6 +1824,6 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
                 PathsUsingTemplateScanNode.class.getSimpleName());
     context.getTimeSliceAllocator().recordExecutionWeight(operatorContext, 1);
     return new PathsUsingTemplateScanOperator(
-        node.getPlanNodeId(), operatorContext, node.getTemplateId());
+        node.getPlanNodeId(), operatorContext, node.getPathPatternList(), node.getTemplateId());
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java
index 8df3a16804..fd65b24a12 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/distribution/SourceRewriter.java
@@ -20,7 +20,9 @@
 package org.apache.iotdb.db.mpp.plan.planner.distribution;
 
 import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.plan.analyze.Analysis;
 import org.apache.iotdb.db.mpp.plan.expression.Expression;
@@ -70,6 +72,7 @@ import java.util.TreeSet;
 import java.util.stream.Collectors;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
 
 public class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
 
@@ -171,24 +174,73 @@ public class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanConte
       SchemaQueryMergeNode node, DistributionPlanContext context) {
     SchemaQueryMergeNode root = (SchemaQueryMergeNode) node.clone();
     SchemaQueryScanNode seed = (SchemaQueryScanNode) node.getChildren().get(0);
-    TreeSet<TRegionReplicaSet> schemaRegions =
-        new TreeSet<>(Comparator.comparingInt(region -> region.getRegionId().getId()));
-    analysis
-        .getSchemaPartitionInfo()
-        .getSchemaPartitionMap()
-        .forEach(
-            (storageGroup, deviceGroup) -> {
-              deviceGroup.forEach(
-                  (deviceGroupId, schemaRegionReplicaSet) ->
-                      schemaRegions.add(schemaRegionReplicaSet));
-            });
-    schemaRegions.forEach(
-        region -> {
-          SchemaQueryScanNode schemaQueryScanNode = (SchemaQueryScanNode) seed.clone();
-          schemaQueryScanNode.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
-          schemaQueryScanNode.setRegionReplicaSet(region);
-          root.addChild(schemaQueryScanNode);
-        });
+    List<PartialPath> pathPatternList = seed.getPathPatternList();
+    if (pathPatternList.size() == 1) {
+      // the path pattern overlaps with all storageGroup or storageGroup.**
+      TreeSet<TRegionReplicaSet> schemaRegions =
+          new TreeSet<>(Comparator.comparingInt(region -> region.getRegionId().getId()));
+      analysis
+          .getSchemaPartitionInfo()
+          .getSchemaPartitionMap()
+          .forEach(
+              (storageGroup, deviceGroup) -> {
+                deviceGroup.forEach(
+                    (deviceGroupId, schemaRegionReplicaSet) ->
+                        schemaRegions.add(schemaRegionReplicaSet));
+              });
+      schemaRegions.forEach(
+          region -> {
+            SchemaQueryScanNode schemaQueryScanNode = (SchemaQueryScanNode) seed.clone();
+            schemaQueryScanNode.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
+            schemaQueryScanNode.setRegionReplicaSet(region);
+            root.addChild(schemaQueryScanNode);
+          });
+    } else {
+      // the path pattern may only overlap with part of storageGroup or storageGroup.**, need filter
+      PathPatternTree patternTree = new PathPatternTree();
+      for (PartialPath pathPattern : pathPatternList) {
+        patternTree.appendPathPattern(pathPattern);
+      }
+      Map<String, Set<TRegionReplicaSet>> storageGroupSchemaRegionMap = new HashMap<>();
+      analysis
+          .getSchemaPartitionInfo()
+          .getSchemaPartitionMap()
+          .forEach(
+              (storageGroup, deviceGroup) -> {
+                deviceGroup.forEach(
+                    (deviceGroupId, schemaRegionReplicaSet) ->
+                        storageGroupSchemaRegionMap
+                            .computeIfAbsent(storageGroup, k -> new HashSet<>())
+                            .add(schemaRegionReplicaSet));
+              });
+
+      storageGroupSchemaRegionMap.forEach(
+          (storageGroup, schemaRegionSet) -> {
+            // extract the patterns overlap with current storage group
+            Set<PartialPath> filteredPathPatternSet = new HashSet<>();
+            try {
+              PartialPath storageGroupPath = new PartialPath(storageGroup);
+              filteredPathPatternSet.addAll(
+                  patternTree.getOverlappedPathPatterns(storageGroupPath));
+              filteredPathPatternSet.addAll(
+                  patternTree.getOverlappedPathPatterns(
+                      storageGroupPath.concatNode(MULTI_LEVEL_PATH_WILDCARD)));
+            } catch (IllegalPathException ignored) {
+              // won't reach here
+            }
+            List<PartialPath> filteredPathPatternList = new ArrayList<>(filteredPathPatternSet);
+
+            schemaRegionSet.forEach(
+                region -> {
+                  SchemaQueryScanNode schemaQueryScanNode = (SchemaQueryScanNode) seed.clone();
+                  schemaQueryScanNode.setPlanNodeId(
+                      context.queryContext.getQueryId().genPlanNodeId());
+                  schemaQueryScanNode.setRegionReplicaSet(region);
+                  schemaQueryScanNode.setPathPatternList(filteredPathPatternList);
+                  root.addChild(schemaQueryScanNode);
+                });
+          });
+    }
     return root;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java
index dd102898d5..9ecf8d5b3e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java
@@ -40,9 +40,12 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.ConstructSc
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateMultiTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateTimeSeriesNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeleteTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.InternalCreateTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.InvalidateSchemaCacheNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.PreDeactivateTemplateNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackPreDeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackSchemaBlackListNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.AggregationNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.DeviceMergeNode;
@@ -140,7 +143,10 @@ public enum PlanNodeType {
   LOAD_TSFILE((short) 55),
   CONSTRUCT_SCHEMA_BLACK_LIST_NODE((short) 56),
   ROLLBACK_SCHEMA_BLACK_LIST_NODE((short) 57),
-  GROUP_BY_TAG((short) 58);
+  GROUP_BY_TAG((short) 58),
+  PRE_DEACTIVATE_TEMPLATE_NODE((short) 59),
+  ROLLBACK_PRE_DEACTIVATE_TEMPLATE_NODE((short) 60),
+  DEACTIVATE_TEMPLATE_NODE((short) 61);
 
   public static final int BYTES = Short.BYTES;
 
@@ -307,6 +313,12 @@ public enum PlanNodeType {
         return RollbackSchemaBlackListNode.deserialize(buffer);
       case 58:
         return GroupByTagNode.deserialize(buffer);
+      case 59:
+        return PreDeactivateTemplateNode.deserialize(buffer);
+      case 60:
+        return RollbackPreDeactivateTemplateNode.deserialize(buffer);
+      case 61:
+        return DeactivateTemplateNode.deserialize(buffer);
       default:
         throw new IllegalArgumentException("Invalid node type: " + nodeType);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanVisitor.java
index 8b4155cb53..f0b0a1925d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanVisitor.java
@@ -39,8 +39,11 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.ConstructSc
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateMultiTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateTimeSeriesNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeleteTimeSeriesNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.InternalCreateTimeSeriesNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.PreDeactivateTemplateNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackPreDeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackSchemaBlackListNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.AggregationNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.DeviceMergeNode;
@@ -301,4 +304,16 @@ public abstract class PlanVisitor<R, C> {
   public R visitActivateTemplate(ActivateTemplateNode node, C context) {
     return visitPlan(node, context);
   }
+
+  public R visitPreDeactivateTemplate(PreDeactivateTemplateNode node, C context) {
+    return visitPlan(node, context);
+  }
+
+  public R visitRollbackPreDeactivateTemplate(RollbackPreDeactivateTemplateNode node, C context) {
+    return visitPlan(node, context);
+  }
+
+  public R visitDeactivateTemplate(DeactivateTemplateNode node, C context) {
+    return visitPlan(node, context);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java
index 235863687c..d265a92a73 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java
@@ -19,6 +19,8 @@
 
 package org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read;
 
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
@@ -29,25 +31,38 @@ import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 
 public class PathsUsingTemplateScanNode extends SchemaQueryScanNode {
 
+  private List<PartialPath> pathPatternList;
+
   private int templateId;
 
-  public PathsUsingTemplateScanNode(PlanNodeId id, int templateId) {
+  public PathsUsingTemplateScanNode(
+      PlanNodeId id, List<PartialPath> pathPatternList, int templateId) {
     super(id);
+    this.pathPatternList = pathPatternList;
     this.templateId = templateId;
   }
 
+  public List<PartialPath> getPathPatternList() {
+    return pathPatternList;
+  }
+
+  public void setPathPatternList(List<PartialPath> pathPatternList) {
+    this.pathPatternList = pathPatternList;
+  }
+
   public int getTemplateId() {
     return templateId;
   }
 
   @Override
   public PlanNode clone() {
-    return new PathsUsingTemplateScanNode(getPlanNodeId(), templateId);
+    return new PathsUsingTemplateScanNode(getPlanNodeId(), pathPatternList, templateId);
   }
 
   @Override
@@ -60,20 +75,31 @@ public class PathsUsingTemplateScanNode extends SchemaQueryScanNode {
   @Override
   protected void serializeAttributes(ByteBuffer byteBuffer) {
     PlanNodeType.PATHS_USING_TEMPLATE_SCAN.serialize(byteBuffer);
-
+    ReadWriteIOUtils.write(pathPatternList.size(), byteBuffer);
+    for (PartialPath pathPattern : pathPatternList) {
+      pathPattern.serialize(byteBuffer);
+    }
     ReadWriteIOUtils.write(templateId, byteBuffer);
   }
 
   @Override
   protected void serializeAttributes(DataOutputStream stream) throws IOException {
     PlanNodeType.PATHS_USING_TEMPLATE_SCAN.serialize(stream);
-
+    ReadWriteIOUtils.write(pathPatternList.size(), stream);
+    for (PartialPath pathPattern : pathPatternList) {
+      pathPattern.serialize(stream);
+    }
     ReadWriteIOUtils.write(templateId, stream);
   }
 
   public static PathsUsingTemplateScanNode deserialize(ByteBuffer buffer) {
+    int size = ReadWriteIOUtils.readInt(buffer);
+    List<PartialPath> pathPatternList = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      pathPatternList.add((PartialPath) PathDeserializeUtil.deserialize(buffer));
+    }
     int templateId = ReadWriteIOUtils.readInt(buffer);
     PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
-    return new PathsUsingTemplateScanNode(planNodeId, templateId);
+    return new PathsUsingTemplateScanNode(planNodeId, pathPatternList, templateId);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java
index 88c6db1e20..dd2b355e9e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java
@@ -75,6 +75,16 @@ public abstract class SchemaQueryScanNode extends SourceNode {
   @Override
   public void close() throws Exception {}
 
+  public List<PartialPath> getPathPatternList() {
+    return Collections.singletonList(path);
+  }
+
+  public void setPathPatternList(List<PartialPath> pathPatternList) {
+    if (pathPatternList.size() == 1) {
+      this.path = pathPatternList.get(0);
+    }
+  }
+
   public boolean isPrefixPath() {
     return isPrefixPath;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/DeactivateTemplateNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/DeactivateTemplateNode.java
new file mode 100644
index 0000000000..fb02cda592
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/DeactivateTemplateNode.java
@@ -0,0 +1,128 @@
+/*
+ * 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.planner.plan.node.metedata.write;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IDeactivateTemplatePlan;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class DeactivateTemplateNode extends PlanNode implements IDeactivateTemplatePlan {
+
+  private Map<PartialPath, List<Integer>> templateSetInfo;
+
+  public DeactivateTemplateNode(PlanNodeId id, Map<PartialPath, List<Integer>> templateSetInfo) {
+    super(id);
+    this.templateSetInfo = templateSetInfo;
+  }
+
+  @Override
+  public List<PlanNode> getChildren() {
+    return null;
+  }
+
+  @Override
+  public void addChild(PlanNode child) {}
+
+  @Override
+  public PlanNode clone() {
+    return new DeactivateTemplateNode(getPlanNodeId(), templateSetInfo);
+  }
+
+  @Override
+  public int allowedChildCount() {
+    return 0;
+  }
+
+  @Override
+  public List<String> getOutputColumnNames() {
+    return null;
+  }
+
+  @Override
+  public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+    return visitor.visitDeactivateTemplate(this, context);
+  }
+
+  @Override
+  protected void serializeAttributes(ByteBuffer byteBuffer) {
+    PlanNodeType.DEACTIVATE_TEMPLATE_NODE.serialize(byteBuffer);
+    ReadWriteIOUtils.write(templateSetInfo.size(), byteBuffer);
+    templateSetInfo.forEach(
+        (k, v) -> {
+          k.serialize(byteBuffer);
+          ReadWriteIOUtils.write(v.size(), byteBuffer);
+          for (int templateId : v) {
+            ReadWriteIOUtils.write(templateId, byteBuffer);
+          }
+        });
+  }
+
+  @Override
+  protected void serializeAttributes(DataOutputStream stream) throws IOException {
+    PlanNodeType.DEACTIVATE_TEMPLATE_NODE.serialize(stream);
+    ReadWriteIOUtils.write(templateSetInfo.size(), stream);
+    for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
+      entry.getKey().serialize(stream);
+      ReadWriteIOUtils.write(entry.getValue().size(), stream);
+      for (int templateId : entry.getValue()) {
+        ReadWriteIOUtils.write(templateId, stream);
+      }
+    }
+  }
+
+  public static DeactivateTemplateNode deserialize(ByteBuffer buffer) {
+    int size = ReadWriteIOUtils.readInt(buffer);
+    Map<PartialPath, List<Integer>> templateSetInfo = new HashMap<>(size);
+    for (int i = 0; i < size; i++) {
+      PartialPath pattern = (PartialPath) PathDeserializeUtil.deserialize(buffer);
+      int templateNum = ReadWriteIOUtils.readInt(buffer);
+      List<Integer> templateIdList = new ArrayList<>(templateNum);
+      for (int j = 0; j < templateNum; j++) {
+        templateIdList.add(ReadWriteIOUtils.readInt(buffer));
+      }
+      templateSetInfo.put(pattern, templateIdList);
+    }
+    PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
+    return new DeactivateTemplateNode(planNodeId, templateSetInfo);
+  }
+
+  @Override
+  public Map<PartialPath, List<Integer>> getTemplateSetInfo() {
+    return templateSetInfo;
+  }
+
+  @Override
+  public void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo) {
+    this.templateSetInfo = templateSetInfo;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/PreDeactivateTemplateNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/PreDeactivateTemplateNode.java
new file mode 100644
index 0000000000..5e49291a86
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/PreDeactivateTemplateNode.java
@@ -0,0 +1,128 @@
+/*
+ * 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.planner.plan.node.metedata.write;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IPreDeactivateTemplatePlan;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PreDeactivateTemplateNode extends PlanNode implements IPreDeactivateTemplatePlan {
+
+  private Map<PartialPath, List<Integer>> templateSetInfo;
+
+  public PreDeactivateTemplateNode(PlanNodeId id, Map<PartialPath, List<Integer>> templateSetInfo) {
+    super(id);
+    this.templateSetInfo = templateSetInfo;
+  }
+
+  @Override
+  public List<PlanNode> getChildren() {
+    return null;
+  }
+
+  @Override
+  public void addChild(PlanNode child) {}
+
+  @Override
+  public PlanNode clone() {
+    return new PreDeactivateTemplateNode(getPlanNodeId(), templateSetInfo);
+  }
+
+  @Override
+  public int allowedChildCount() {
+    return 0;
+  }
+
+  @Override
+  public List<String> getOutputColumnNames() {
+    return null;
+  }
+
+  @Override
+  public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+    return visitor.visitPreDeactivateTemplate(this, context);
+  }
+
+  @Override
+  protected void serializeAttributes(ByteBuffer byteBuffer) {
+    PlanNodeType.PRE_DEACTIVATE_TEMPLATE_NODE.serialize(byteBuffer);
+    ReadWriteIOUtils.write(templateSetInfo.size(), byteBuffer);
+    templateSetInfo.forEach(
+        (k, v) -> {
+          k.serialize(byteBuffer);
+          ReadWriteIOUtils.write(v.size(), byteBuffer);
+          for (int templateId : v) {
+            ReadWriteIOUtils.write(templateId, byteBuffer);
+          }
+        });
+  }
+
+  @Override
+  protected void serializeAttributes(DataOutputStream stream) throws IOException {
+    PlanNodeType.PRE_DEACTIVATE_TEMPLATE_NODE.serialize(stream);
+    ReadWriteIOUtils.write(templateSetInfo.size(), stream);
+    for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
+      entry.getKey().serialize(stream);
+      ReadWriteIOUtils.write(entry.getValue().size(), stream);
+      for (int templateId : entry.getValue()) {
+        ReadWriteIOUtils.write(templateId, stream);
+      }
+    }
+  }
+
+  public static PreDeactivateTemplateNode deserialize(ByteBuffer buffer) {
+    int size = ReadWriteIOUtils.readInt(buffer);
+    Map<PartialPath, List<Integer>> templateSetInfo = new HashMap<>(size);
+    for (int i = 0; i < size; i++) {
+      PartialPath pattern = (PartialPath) PathDeserializeUtil.deserialize(buffer);
+      int templateNum = ReadWriteIOUtils.readInt(buffer);
+      List<Integer> templateIdList = new ArrayList<>(templateNum);
+      for (int j = 0; j < templateNum; j++) {
+        templateIdList.add(ReadWriteIOUtils.readInt(buffer));
+      }
+      templateSetInfo.put(pattern, templateIdList);
+    }
+    PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
+    return new PreDeactivateTemplateNode(planNodeId, templateSetInfo);
+  }
+
+  @Override
+  public Map<PartialPath, List<Integer>> getTemplateSetInfo() {
+    return templateSetInfo;
+  }
+
+  @Override
+  public void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo) {
+    this.templateSetInfo = templateSetInfo;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/RollbackPreDeactivateTemplateNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/RollbackPreDeactivateTemplateNode.java
new file mode 100644
index 0000000000..369489c661
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/metedata/write/RollbackPreDeactivateTemplateNode.java
@@ -0,0 +1,130 @@
+/*
+ * 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.planner.plan.node.metedata.write;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.db.metadata.plan.schemaregion.write.IRollbackPreDeactivateTemplatePlan;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class RollbackPreDeactivateTemplateNode extends PlanNode
+    implements IRollbackPreDeactivateTemplatePlan {
+
+  private Map<PartialPath, List<Integer>> templateSetInfo;
+
+  public RollbackPreDeactivateTemplateNode(
+      PlanNodeId id, Map<PartialPath, List<Integer>> templateSetInfo) {
+    super(id);
+    this.templateSetInfo = templateSetInfo;
+  }
+
+  @Override
+  public List<PlanNode> getChildren() {
+    return null;
+  }
+
+  @Override
+  public void addChild(PlanNode child) {}
+
+  @Override
+  public PlanNode clone() {
+    return new RollbackPreDeactivateTemplateNode(getPlanNodeId(), templateSetInfo);
+  }
+
+  @Override
+  public int allowedChildCount() {
+    return 0;
+  }
+
+  @Override
+  public List<String> getOutputColumnNames() {
+    return null;
+  }
+
+  @Override
+  public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+    return visitor.visitRollbackPreDeactivateTemplate(this, context);
+  }
+
+  @Override
+  protected void serializeAttributes(ByteBuffer byteBuffer) {
+    PlanNodeType.ROLLBACK_PRE_DEACTIVATE_TEMPLATE_NODE.serialize(byteBuffer);
+    ReadWriteIOUtils.write(templateSetInfo.size(), byteBuffer);
+    templateSetInfo.forEach(
+        (k, v) -> {
+          k.serialize(byteBuffer);
+          ReadWriteIOUtils.write(v.size(), byteBuffer);
+          for (int templateId : v) {
+            ReadWriteIOUtils.write(templateId, byteBuffer);
+          }
+        });
+  }
+
+  @Override
+  protected void serializeAttributes(DataOutputStream stream) throws IOException {
+    PlanNodeType.ROLLBACK_PRE_DEACTIVATE_TEMPLATE_NODE.serialize(stream);
+    ReadWriteIOUtils.write(templateSetInfo.size(), stream);
+    for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
+      entry.getKey().serialize(stream);
+      ReadWriteIOUtils.write(entry.getValue().size(), stream);
+      for (int templateId : entry.getValue()) {
+        ReadWriteIOUtils.write(templateId, stream);
+      }
+    }
+  }
+
+  public static RollbackPreDeactivateTemplateNode deserialize(ByteBuffer buffer) {
+    int size = ReadWriteIOUtils.readInt(buffer);
+    Map<PartialPath, List<Integer>> templateSetInfo = new HashMap<>(size);
+    for (int i = 0; i < size; i++) {
+      PartialPath pattern = (PartialPath) PathDeserializeUtil.deserialize(buffer);
+      int templateNum = ReadWriteIOUtils.readInt(buffer);
+      List<Integer> templateIdList = new ArrayList<>(templateNum);
+      for (int j = 0; j < templateNum; j++) {
+        templateIdList.add(ReadWriteIOUtils.readInt(buffer));
+      }
+      templateSetInfo.put(pattern, templateIdList);
+    }
+    PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
+    return new RollbackPreDeactivateTemplateNode(planNodeId, templateSetInfo);
+  }
+
+  @Override
+  public Map<PartialPath, List<Integer>> getTemplateSetInfo() {
+    return templateSetInfo;
+  }
+
+  @Override
+  public void setTemplateSetInfo(Map<PartialPath, List<Integer>> templateSetInfo) {
+    this.templateSetInfo = templateSetInfo;
+  }
+}
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 be99227d27..0c74684373 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
@@ -65,6 +65,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTriggersStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.UnSetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ActivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.CreateSchemaTemplateStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.template.DeactivateTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.SetSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
@@ -409,4 +410,9 @@ public abstract class StatementVisitor<R, C> {
   public R visitGetTimeSlotList(GetTimeSlotListStatement getTimeSlotListStatement, C context) {
     return visitStatement(getTimeSlotListStatement, context);
   }
+
+  public R visitDeactivateTemplate(
+      DeactivateTemplateStatement deactivateTemplateStatement, C context) {
+    return visitStatement(deactivateTemplateStatement, context);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/DeactivateTemplateStatement.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/DeactivateTemplateStatement.java
index 8af1725412..8f54876f29 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/DeactivateTemplateStatement.java
@@ -19,30 +19,62 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.metadata.template;
 
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
 import org.apache.iotdb.db.mpp.plan.constant.StatementType;
+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.StatementVisitor;
-import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStatement;
 
-public class ShowPathsUsingTemplateStatement extends ShowStatement {
+import java.util.List;
+
+import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class DeactivateTemplateStatement extends Statement implements IConfigStatement {
 
   private String templateName;
 
-  public ShowPathsUsingTemplateStatement(String templateName) {
+  private List<PartialPath> pathPatternList;
+
+  public DeactivateTemplateStatement() {
     super();
-    statementType = StatementType.SHOW_PATH_USING_SCHEMA_TEMPLATE;
+    statementType = StatementType.DEACTIVATE_TEMPLATE;
+  }
+
+  public DeactivateTemplateStatement(String templateName, List<PartialPath> pathPatternList) {
     this.templateName = templateName;
+    this.pathPatternList = pathPatternList;
+    statementType = StatementType.DEACTIVATE_TEMPLATE;
+  }
+
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return pathPatternList;
   }
 
   public String getTemplateName() {
-    return templateName;
+    return templateName == null ? ONE_LEVEL_PATH_WILDCARD : templateName;
   }
 
   public void setTemplateName(String templateName) {
     this.templateName = templateName;
   }
 
+  public List<PartialPath> getPathPatternList() {
+    return pathPatternList;
+  }
+
+  public void setPathPatternList(List<PartialPath> pathPatternList) {
+    this.pathPatternList = pathPatternList;
+  }
+
   @Override
   public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitShowPathsUsingTemplate(this, context);
+    return visitor.visitDeactivateTemplate(this, context);
+  }
+
+  @Override
+  public QueryType getQueryType() {
+    return QueryType.WRITE;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
index 8af1725412..e3d95a646a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
@@ -19,20 +19,40 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.metadata.template;
 
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowStatement;
 
+import java.util.Collections;
+import java.util.List;
+
 public class ShowPathsUsingTemplateStatement extends ShowStatement {
 
+  private PartialPath pathPattern;
+
   private String templateName;
 
-  public ShowPathsUsingTemplateStatement(String templateName) {
+  public ShowPathsUsingTemplateStatement(PartialPath pathPattern, String templateName) {
     super();
     statementType = StatementType.SHOW_PATH_USING_SCHEMA_TEMPLATE;
+    this.pathPattern = pathPattern;
     this.templateName = templateName;
   }
 
+  @Override
+  public List<PartialPath> getPaths() {
+    return Collections.singletonList(pathPattern);
+  }
+
+  public PartialPath getPathPattern() {
+    return pathPattern;
+  }
+
+  public void setPathPattern(PartialPath pathPattern) {
+    this.pathPattern = pathPattern;
+  }
+
   public String getTemplateName() {
     return templateName;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
index 30a2eeb644..c8dc6a8da4 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
@@ -76,7 +76,10 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.ConstructSchemaBlackListNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeleteTimeSeriesNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.PreDeactivateTemplateNode;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackPreDeactivateTemplateNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackSchemaBlackListNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.scheduler.load.LoadTsFileScheduler;
@@ -100,13 +103,15 @@ import org.apache.iotdb.mpp.rpc.thrift.TCancelPlanFragmentReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCancelQueryReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCancelResp;
 import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListReq;
+import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListWithTemplateReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateFunctionRequest;
 import org.apache.iotdb.mpp.rpc.thrift.TCreatePeerReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreatePipeOnDataNodeReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq;
 import org.apache.iotdb.mpp.rpc.thrift.TCreateTriggerInstanceReq;
-import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteTimeSeriesReq;
+import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq;
+import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDeleteTimeSeriesReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDisableDataNodeReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDropFunctionRequest;
@@ -131,6 +136,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TOperatePipeOnDataNodeReq;
 import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq;
 import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq;
 import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListReq;
+import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListWithTemplateReq;
 import org.apache.iotdb.mpp.rpc.thrift.TSchemaFetchRequest;
 import org.apache.iotdb.mpp.rpc.thrift.TSchemaFetchResponse;
 import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceReq;
@@ -458,8 +464,7 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface
   }
 
   @Override
-  public TSStatus deleteDataForDeleteTimeSeries(TDeleteDataForDeleteTimeSeriesReq req)
-      throws TException {
+  public TSStatus deleteDataForDeleteSchema(TDeleteDataForDeleteSchemaReq req) throws TException {
     PathPatternTree patternTree =
         PathPatternTree.deserialize(ByteBuffer.wrap(req.getPathPatternTree()));
     List<PartialPath> pathList = patternTree.getAllPathPatterns();
@@ -519,6 +524,152 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface
     return RpcUtils.SUCCESS_STATUS;
   }
 
+  @Override
+  public TSStatus constructSchemaBlackListWithTemplate(TConstructSchemaBlackListWithTemplateReq req)
+      throws TException {
+    Map<PartialPath, List<Integer>> templateSetInfo =
+        transformTemplateSetInfo(req.getTemplateSetInfo());
+    List<TSStatus> failureList = new ArrayList<>();
+    TSStatus status;
+    int preDeactivateTemplateNum = 0;
+    for (TConsensusGroupId consensusGroupId : req.getSchemaRegionIdList()) {
+      Map<PartialPath, List<Integer>> filteredTemplateSetInfo =
+          filterTemplateSetInfo(templateSetInfo, consensusGroupId);
+      if (filteredTemplateSetInfo.isEmpty()) {
+        continue;
+      }
+
+      RegionWriteExecutor executor = new RegionWriteExecutor();
+      status =
+          executor
+              .execute(
+                  new SchemaRegionId(consensusGroupId.getId()),
+                  new PreDeactivateTemplateNode(new PlanNodeId(""), filteredTemplateSetInfo))
+              .getStatus();
+      if (status.code == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        preDeactivateTemplateNum += Integer.parseInt(status.getMessage());
+      } else {
+        failureList.add(status);
+      }
+    }
+
+    if (!failureList.isEmpty()) {
+      return RpcUtils.getStatus(failureList);
+    }
+
+    return RpcUtils.getStatus(
+        TSStatusCode.SUCCESS_STATUS, String.valueOf(preDeactivateTemplateNum));
+  }
+
+  private Map<PartialPath, List<Integer>> transformTemplateSetInfo(
+      Map<String, List<Integer>> rawTemplateSetInfo) {
+    Map<PartialPath, List<Integer>> result = new HashMap<>();
+    rawTemplateSetInfo.forEach(
+        (k, v) -> {
+          try {
+            result.put(new PartialPath(k), v);
+          } catch (IllegalPathException ignored) {
+            // won't reach here
+          }
+        });
+    return result;
+  }
+
+  private Map<PartialPath, List<Integer>> filterTemplateSetInfo(
+      Map<PartialPath, List<Integer>> templateSetInfo, TConsensusGroupId consensusGroupId) {
+
+    PartialPath storageGroupPath = getStorageGroupPath(consensusGroupId);
+    PartialPath storageGroupPattern = storageGroupPath.concatNode(MULTI_LEVEL_PATH_WILDCARD);
+    Map<PartialPath, List<Integer>> result = new HashMap<>();
+    templateSetInfo.forEach(
+        (k, v) -> {
+          if (storageGroupPattern.overlapWith(k) || storageGroupPath.overlapWith(k)) {
+            result.put(k, v);
+          }
+        });
+    return result;
+  }
+
+  private PartialPath getStorageGroupPath(TConsensusGroupId consensusGroupId) {
+    PartialPath storageGroupPath = null;
+    try {
+      storageGroupPath =
+          new PartialPath(
+              schemaEngine
+                  .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId()))
+                  .getStorageGroupFullPath());
+    } catch (IllegalPathException ignored) {
+      // won't reach here
+    }
+    return storageGroupPath;
+  }
+
+  @Override
+  public TSStatus rollbackSchemaBlackListWithTemplate(TRollbackSchemaBlackListWithTemplateReq req)
+      throws TException {
+    Map<PartialPath, List<Integer>> templateSetInfo =
+        transformTemplateSetInfo(req.getTemplateSetInfo());
+    List<TSStatus> failureList = new ArrayList<>();
+    TSStatus status;
+    for (TConsensusGroupId consensusGroupId : req.getSchemaRegionIdList()) {
+      Map<PartialPath, List<Integer>> filteredTemplateSetInfo =
+          filterTemplateSetInfo(templateSetInfo, consensusGroupId);
+      if (filteredTemplateSetInfo.isEmpty()) {
+        continue;
+      }
+
+      RegionWriteExecutor executor = new RegionWriteExecutor();
+      status =
+          executor
+              .execute(
+                  new SchemaRegionId(consensusGroupId.getId()),
+                  new RollbackPreDeactivateTemplateNode(
+                      new PlanNodeId(""), filteredTemplateSetInfo))
+              .getStatus();
+      if (status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        failureList.add(status);
+      }
+    }
+
+    if (!failureList.isEmpty()) {
+      return RpcUtils.getStatus(failureList);
+    }
+
+    return RpcUtils.SUCCESS_STATUS;
+  }
+
+  @Override
+  public TSStatus deactivateTemplate(TDeactivateTemplateReq req) throws TException {
+    Map<PartialPath, List<Integer>> templateSetInfo =
+        transformTemplateSetInfo(req.getTemplateSetInfo());
+    List<TSStatus> failureList = new ArrayList<>();
+    TSStatus status;
+    for (TConsensusGroupId consensusGroupId : req.getSchemaRegionIdList()) {
+      Map<PartialPath, List<Integer>> filteredTemplateSetInfo =
+          filterTemplateSetInfo(templateSetInfo, consensusGroupId);
+      if (filteredTemplateSetInfo.isEmpty()) {
+        continue;
+      }
+
+      RegionWriteExecutor executor = new RegionWriteExecutor();
+      status =
+          executor
+              .execute(
+                  new SchemaRegionId(consensusGroupId.getId()),
+                  new DeactivateTemplateNode(new PlanNodeId(""), filteredTemplateSetInfo))
+              .getStatus();
+      if (status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        failureList.add(status);
+      }
+    }
+
+    if (!failureList.isEmpty()) {
+      return RpcUtils.getStatus(failureList);
+    }
+
+    return RpcUtils.SUCCESS_STATUS;
+  }
+
   @Override
   public TSStatus createPipeOnDataNode(TCreatePipeOnDataNodeReq req) throws TException {
     try {
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/plan/SchemaRegionPlanCompatibilityTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/plan/SchemaRegionPlanCompatibilityTest.java
index 708a890fd9..ee917b240b 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/plan/SchemaRegionPlanCompatibilityTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/plan/SchemaRegionPlanCompatibilityTest.java
@@ -84,10 +84,26 @@ public class SchemaRegionPlanCompatibilityTest {
 
   @Test
   public void testPlanTypeCompatibility() {
-    for (SchemaRegionPlanType schemaRegionPlanType : SchemaRegionPlanType.values()) {
+
+    String[] typeNames =
+        new String[] {
+          "CREATE_TIMESERIES",
+          "DELETE_TIMESERIES",
+          "CHANGE_TAG_OFFSET",
+          "CHANGE_ALIAS",
+          "SET_TEMPLATE",
+          "ACTIVATE_TEMPLATE",
+          "AUTO_CREATE_DEVICE_MNODE",
+          "CREATE_ALIGNED_TIMESERIES",
+          "UNSET_TEMPLATE",
+          "ACTIVATE_TEMPLATE_IN_CLUSTER",
+          "PRE_DELETE_TIMESERIES_IN_CLUSTER",
+          "ROLLBACK_PRE_DELETE_TIMESERIES"
+        };
+    for (String typeName : typeNames) {
       Assert.assertEquals(
-          schemaRegionPlanType.getPlanType(),
-          PhysicalPlan.PhysicalPlanType.valueOf(schemaRegionPlanType.name()).ordinal());
+          SchemaRegionPlanType.valueOf(typeName).getPlanType(),
+          PhysicalPlan.PhysicalPlanType.valueOf(typeName).ordinal());
     }
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
index 50490364f7..c1d7728804 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.AlignedPath;
 import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.aggregation.AccumulatorFactory;
 import org.apache.iotdb.db.mpp.aggregation.Aggregator;
 import org.apache.iotdb.db.mpp.aggregation.timerangeiterator.ITimeRangeIterator;
@@ -744,7 +745,10 @@ public class OperatorMemoryTest {
 
       PathsUsingTemplateScanOperator operator =
           new PathsUsingTemplateScanOperator(
-              planNodeId, fragmentInstanceContext.getOperatorContexts().get(0), 0);
+              planNodeId,
+              fragmentInstanceContext.getOperatorContexts().get(0),
+              Collections.singletonList(new PartialPath(new String[] {"root", "**"})),
+              0);
 
       assertEquals(DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, operator.calculateMaxPeekMemory());
       assertEquals(DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, operator.calculateMaxReturnSize());
diff --git a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
index c5d56e571a..9a2e566325 100644
--- a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
+++ b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
@@ -166,7 +166,7 @@ public enum TSStatusCode {
   DATANODE_STOP_ERROR(917),
   REGION_LEADER_CHANGE_FAILED(918),
   REMOVE_DATANODE_FAILED(919),
-  OVERLAP_WITH_EXISTING_DELETE_TIMESERIES_TASK(920),
+  OVERLAP_WITH_EXISTING_TASK(920),
   NOT_AVAILABLE_REGION_GROUP(921),
   CREATE_TRIGGER_ERROR(922),
   DROP_TRIGGER_ERROR(923);
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index cb7d3d1d08..503341ae05 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -526,6 +526,12 @@ struct TDeleteTimeSeriesReq{
   2: required binary pathPatternTree
 }
 
+struct TDeactivateSchemaTemplateReq{
+  1: required string queryId
+  2: required binary pathPatternTree
+  3: optional string templateName
+}
+
 service IConfigNodeRPCService {
 
   // ======================================================
@@ -869,6 +875,7 @@ service IConfigNodeRPCService {
 
   TGetPathsSetTemplatesResp getPathsSetTemplate(string req)
 
+  common.TSStatus deactivateSchemaTemplate(TDeactivateSchemaTemplateReq req)
 
   /**
    * Generate a set of DeleteTimeSeriesProcedure to delete some specific TimeSeries
diff --git a/thrift/src/main/thrift/datanode.thrift b/thrift/src/main/thrift/datanode.thrift
index aa43b7a300..898bf2f7a1 100644
--- a/thrift/src/main/thrift/datanode.thrift
+++ b/thrift/src/main/thrift/datanode.thrift
@@ -286,7 +286,7 @@ struct TFetchSchemaBlackListResp{
   2: required binary pathPatternTree
 }
 
-struct TDeleteDataForDeleteTimeSeriesReq{
+struct TDeleteDataForDeleteSchemaReq{
   1: required list<common.TConsensusGroupId> dataRegionIdList
   2: required binary pathPatternTree
 }
@@ -296,6 +296,21 @@ struct TDeleteTimeSeriesReq{
   2: required binary pathPatternTree
 }
 
+struct TConstructSchemaBlackListWithTemplateReq{
+  1: required list<common.TConsensusGroupId> schemaRegionIdList
+  2: required map<string, list<i32>> templateSetInfo
+}
+
+struct TRollbackSchemaBlackListWithTemplateReq{
+  1: required list<common.TConsensusGroupId> schemaRegionIdList
+  2: required map<string, list<i32>> templateSetInfo
+}
+
+struct TDeactivateTemplateReq{
+  1: required list<common.TConsensusGroupId> schemaRegionIdList
+  2: required map<string, list<i32>> templateSetInfo
+}
+
 struct TCreatePipeOnDataNodeReq{
   1: required binary pipeInfo
 }
@@ -553,13 +568,29 @@ service IDataNodeRPCService {
   /**
    * Config node inform this dataNode to execute a distribution data deleion mpp task
    */
-  common.TSStatus deleteDataForDeleteTimeSeries(TDeleteDataForDeleteTimeSeriesReq req)
+  common.TSStatus deleteDataForDeleteSchema(TDeleteDataForDeleteSchemaReq req)
 
- /**
-  * Delete matched timeseries and remove according schema black list in target schemRegion
-  */
+  /**
+   * Delete matched timeseries and remove according schema black list in target schemRegion
+   */
   common.TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req)
 
+  /**
+   * Construct schema black list in target schemaRegion to block R/W on matched timeseries represent by template
+   */
+  common.TSStatus constructSchemaBlackListWithTemplate(TConstructSchemaBlackListWithTemplateReq req)
+
+  /**
+   * Remove the schema black list to recover R/W on matched timeseries represent by template
+   */
+  common.TSStatus rollbackSchemaBlackListWithTemplate(TRollbackSchemaBlackListWithTemplateReq req)
+
+  /**
+   * Deactivate template on device matched by input path pattern
+   * and remove according template schema black list in target schemRegion
+   */
+  common.TSStatus deactivateTemplate(TDeactivateTemplateReq req)
+
  /**
   * Create PIPE on DataNode
   */