You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2021/11/21 10:25:56 UTC

[iotdb] branch master updated: [IOTDB-1931] Adapt tree structrued Template with MManager (#4391)

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

qiaojialin 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 441d6b7  [IOTDB-1931] Adapt tree structrued Template with MManager (#4391)
441d6b7 is described below

commit 441d6b7c27f337b0a314d2135e9a0a98af9b1bb5
Author: ZhaoXin <x_...@163.com>
AuthorDate: Sun Nov 21 18:25:13 2021 +0800

    [IOTDB-1931] Adapt tree structrued Template with MManager (#4391)
---
 docs/UserGuide/API/Programming-Java-Native-API.md  |  38 +--
 .../UserGuide/API/Programming-Java-Native-API.md   |  23 +-
 .../iotdb/AlignedTimeseriesSessionExample.java     |  46 ++--
 .../org/apache/iotdb/SessionConcurrentExample.java |  46 ++--
 .../main/java/org/apache/iotdb/SessionExample.java |  46 ++--
 .../metadata/TemplateImcompatibeException.java     |  40 ++++
 .../org/apache/iotdb/db/metadata/MManager.java     |  55 ++++-
 .../iotdb/db/metadata/mnode/EntityMNode.java       |  12 -
 .../org/apache/iotdb/db/metadata/mnode/IMNode.java |   2 +
 .../iotdb/db/metadata/mnode/InternalMNode.java     |  12 +
 .../iotdb/db/metadata/mnode/MeasurementMNode.java  |   3 +
 .../org/apache/iotdb/db/metadata/mtree/MTree.java  | 259 ++++++++++++++++-----
 .../db/metadata/mtree/traverser/Traverser.java     | 115 ++++++---
 .../traverser/collector/MeasurementCollector.java  |  29 +++
 .../iotdb/db/metadata/template/Template.java       | 109 +++++----
 .../db/metadata/template/TemplateManager.java      |   2 +-
 .../apache/iotdb/db/metadata/utils/MetaUtils.java  |   2 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  42 +---
 .../iotdb/db/metadata/MManagerBasicTest.java       | 218 ++++++++++++++---
 .../org/apache/iotdb/db/metadata/TemplateTest.java |   4 +-
 .../iotdb/db/qp/physical/InsertRowPlanTest.java    |  77 +++++-
 .../iotdb/db/qp/physical/InsertTabletPlanTest.java | 137 +++++++++++
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   1 +
 .../java/org/apache/iotdb/session/Session.java     |  96 +-------
 .../org/apache/iotdb/session/pool/SessionPool.java |  51 ----
 .../apache/iotdb/session/IoTDBSessionSimpleIT.java | 200 ++++++++++++++++
 .../java/org/apache/iotdb/session/SessionTest.java |  93 +++-----
 .../apache/iotdb/session/template/TemplateUT.java  |  56 +++--
 thrift/src/main/thrift/rpc.thrift                  |   7 +-
 29 files changed, 1212 insertions(+), 609 deletions(-)

diff --git a/docs/UserGuide/API/Programming-Java-Native-API.md b/docs/UserGuide/API/Programming-Java-Native-API.md
index bd3a9d4..1b3ca83 100644
--- a/docs/UserGuide/API/Programming-Java-Native-API.md
+++ b/docs/UserGuide/API/Programming-Java-Native-API.md
@@ -390,31 +390,11 @@ public Session(String host, int rpcPort, String username, String password,
 
 Open a session and specifies whether the Leader cache is enabled. Note that this interface improves performance for distributed IoTDB, but adds less cost to the client for stand-alone IoTDB.
 
+```java
+void createSchemaTemplate(Template template)
 ```
-* name: template name
-* measurements: List of measurements, if it is a single measurement, just put it's name
-*     into a list and add to measurements if it is a vector measurement, put all measurements of
-*     the vector into a list and add to measurements
-* dataTypes: List of datatypes, if it is a single measurement, just put it's type into a
-*     list and add to dataTypes if it is a vector measurement, put all types of the vector
-*     into a list and add to dataTypes
-* encodings: List of encodings, if it is a single measurement, just put it's encoding into
-*     a list and add to encodings if it is a vector measurement, put all encodings of the
-*     vector into a list and add to encodings
-* compressors: List of compressors, if it is a single measurement, just put it's 
-*     compressor into a list and add to encodings if it is a vector measurement, put all 
-*     encodings of the vector into a list and add to encodings
-void createSchemaTemplate(
-      String templateName,
-      List<List<String>> measurements,
-      List<List<TSDataType>> dataTypes,
-      List<List<TSEncoding>> encodings,
-      List<CompressionType> compressors)
-```
-
-Create a measurement template, the param description as above. As measurement templates could be tree-structured, parameter 'measurement' in above method could be a path to the measurement, be like: 'vehicle.GPS.x' . 
 
-You can also create instances of Template, InternalNode and MeasurementNode to depict the structure of the template, and use belowed interface to create it.
+Create a schema template for massive identical subtree will help to improve memory performance. You can use the API above to create a template at server side, and use Template, InternalNode and MeasurementNode to depict the structure of the template, and use belowed interface to create it inside session.
 
 ```java
 public void createSchemaTemplate(Template template);
@@ -479,7 +459,13 @@ createSchemaTemplate(template);
 
 After measurement template created, you can edit the template with belowed APIs.
 
-**Attention: templates had been set could not be pruned.**
+**Attention: **
+
+**1. templates had been set could not be pruned**
+
+**2. templates will be activated until data points insert into correspoding measurements**
+
+**3. templates will not be shown by showtimeseries before activating**
 
 ```java
 // Add aligned measurements to a template
@@ -536,14 +522,14 @@ public List<String> showMeasurementsInTemplate(String templateName, String patte
 
 Set the measurement template named 'templateName' at path 'prefixPath'. 
 
-``` 
+``` java
 void setSchemaTemplate(String templateName, String prefixPath)
 ```
 
 Before setting template, you should firstly create the template using
 
 ```java
-void createSchemaTemplate
+void createSchemaTemplate(Template template)
 ```
 ```java
 void unsetSchemaTemplate(String prefixPath, String templateName)
diff --git a/docs/zh/UserGuide/API/Programming-Java-Native-API.md b/docs/zh/UserGuide/API/Programming-Java-Native-API.md
index 14440dd..534aca5 100644
--- a/docs/zh/UserGuide/API/Programming-Java-Native-API.md
+++ b/docs/zh/UserGuide/API/Programming-Java-Native-API.md
@@ -211,27 +211,7 @@ SessionDataSet executeQueryStatement(String sql)
 void executeNonQueryStatement(String sql)
 ```
 
-* 创建一个物理量模板
-
-```java
-* name: 物理量模板名称
-* measurements: 工况名称列表,如果该工况是非对齐的,直接将其名称放入一个 list 中再放入 measurements 中,
-*               如果该工况是对齐的,将所有对齐工况名称放入一个 list 再放入 measurements 中
-* dataTypes: 数据类型名称列表,如果该工况是非对齐的,直接将其数据类型放入一个 list 中再放入 dataTypes 中,
-             如果该工况是对齐的,将所有对齐工况的数据类型放入一个 list 再放入 dataTypes 中
-* encodings: 编码类型名称列表,如果该工况是非对齐的,直接将其数据类型放入一个 list 中再放入 encodings 中,
-             如果该工况是对齐的,将所有对齐工况的编码类型放入一个 list 再放入 encodings 中
-* compressors: 压缩方式列表
-             如果该工况是对齐的,将所有对齐工况的压缩方式放入一个 list 再放入 compressors 中
-void createSchemaTemplate(
-      String templateName,
-      List<List<String>> measurements,
-      List<List<TSDataType>> dataTypes,
-      List<List<TSEncoding>> encodings,
-      List<List<CompressionType>> compressors)
-```
-
-* 物理量模板内部支持树状结构,因此也可以通过先后创建 Template、InternalNode、MeasurementNode 三类的对象,并通过以下接口创建模板
+* 物理量模板内部支持树状结构,可以通过先后创建 Template、InternalNode、MeasurementNode 三类的对象,并通过以下接口创建模板
 
 ```java
 public void createSchemaTemplate(Template template);
@@ -359,6 +339,7 @@ void setSchemaTemplate(String templateName, String prefixPath)
 void unsetSchemaTemplate(String prefixPath, String templateName)
 ```
 
+* 请注意,如果一个子树中有多个孩子节点需要使用模板,可以在其共同父母节点上使用 setSchemaTemplate 。而只有在已有数据点插入模板对应的物理量时,模板才会被设置为激活状态,进而被 show timeseries 等查询检测到。
 * 卸载'prefixPath'路径下的名为'templateName'的物理量模板。你需要保证给定的路径'prefixPath'下需要有名为'templateName'的物理量模板。
 
 注意:目前不支持从曾经在'prefixPath'路径及其后代节点使用模板插入数据后(即使数据已被删除)卸载模板。
diff --git a/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java b/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
index a953363..17c16e2 100644
--- a/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
+++ b/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
@@ -23,6 +23,9 @@ import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.session.Session;
 import org.apache.iotdb.session.SessionDataSet;
+import org.apache.iotdb.session.template.InternalNode;
+import org.apache.iotdb.session.template.MeasurementNode;
+import org.apache.iotdb.session.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;
@@ -31,9 +34,9 @@ import org.apache.iotdb.tsfile.write.record.Tablet;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.UnaryMeasurementSchema;
 
+import java.io.IOException;
 import java.security.SecureRandom;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -274,39 +277,20 @@ public class AlignedTimeseriesSessionExample {
 
   // be sure template is coordinate with tablet
   private static void createTemplate()
-      throws StatementExecutionException, IoTDBConnectionException {
-    List<List<String>> measurementList = new ArrayList<>();
-    List<String> vectorMeasurement = new ArrayList<>();
-    for (int i = 1; i <= 2; i++) {
-      vectorMeasurement.add("s" + i);
-    }
-    measurementList.add(vectorMeasurement);
-
-    List<List<TSDataType>> dataTypeList = new ArrayList<>();
-    List<TSDataType> vectorDatatype = new ArrayList<>();
-    vectorDatatype.add(TSDataType.INT64);
-    vectorDatatype.add(TSDataType.INT32);
-    dataTypeList.add(vectorDatatype);
-
-    List<List<TSEncoding>> encodingList = new ArrayList<>();
-    List<TSEncoding> vectorEncoding = new ArrayList<>();
-    for (int i = 1; i <= 2; i++) {
-      vectorEncoding.add(TSEncoding.RLE);
-    }
-    encodingList.add(vectorEncoding);
+      throws StatementExecutionException, IoTDBConnectionException, IOException {
+    Template template = new Template("template1");
+    InternalNode iNodeVector = new InternalNode("vector", true);
+    MeasurementNode mNodeS1 =
+        new MeasurementNode("s1", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+    MeasurementNode mNodeS2 =
+        new MeasurementNode("s2", TSDataType.INT32, TSEncoding.RLE, CompressionType.SNAPPY);
 
-    List<List<CompressionType>> compressionTypeList = new ArrayList<>();
-    List<CompressionType> vectorCompressions = new ArrayList<>();
-    vectorCompressions.add(CompressionType.SNAPPY);
-    vectorCompressions.add(CompressionType.SNAPPY);
-    compressionTypeList.add(Collections.singletonList(CompressionType.SNAPPY));
-    compressionTypeList.add(vectorCompressions);
+    iNodeVector.addChild(mNodeS1);
+    iNodeVector.addChild(mNodeS2);
 
-    List<String> schemaList = new ArrayList<>();
-    schemaList.add("vector");
+    template.addToTemplate(iNodeVector);
 
-    session.createSchemaTemplate(
-        "template1", schemaList, measurementList, dataTypeList, encodingList, compressionTypeList);
+    session.createSchemaTemplate(template);
     session.setSchemaTemplate("template1", "root.sg_1");
   }
 
diff --git a/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java b/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java
index 4ba05ab..7516c66 100644
--- a/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java
+++ b/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java
@@ -22,6 +22,8 @@ package org.apache.iotdb;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.session.Session;
+import org.apache.iotdb.session.template.MeasurementNode;
+import org.apache.iotdb.session.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;
@@ -29,8 +31,8 @@ import org.apache.iotdb.tsfile.write.record.Tablet;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.UnaryMeasurementSchema;
 
+import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.CountDownLatch;
@@ -44,7 +46,7 @@ public class SessionConcurrentExample {
   private static final int parallelDegreeForOneSG = 3;
 
   public static void main(String[] args)
-      throws IoTDBConnectionException, StatementExecutionException {
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
 
     Session session = new Session("127.0.0.1", 6667, "root", "root");
     session.open(false);
@@ -96,33 +98,21 @@ public class SessionConcurrentExample {
   }
 
   private static void createTemplate(Session session)
-      throws IoTDBConnectionException, StatementExecutionException {
-    List<List<String>> measurementList = new ArrayList<>();
-    measurementList.add(Collections.singletonList("s1"));
-    measurementList.add(Collections.singletonList("s2"));
-    measurementList.add(Collections.singletonList("s3"));
-
-    List<List<TSDataType>> dataTypeList = new ArrayList<>();
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-
-    List<List<TSEncoding>> encodingList = new ArrayList<>();
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-
-    List<List<CompressionType>> compressionTypes = new ArrayList<>();
-    for (int i = 0; i < 3; i++) {
-      compressionTypes.add(Collections.singletonList(CompressionType.SNAPPY));
-    }
-    List<String> schemaNames = new ArrayList<>();
-    schemaNames.add("s1");
-    schemaNames.add("s2");
-    schemaNames.add("s3");
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+
+    Template template = new Template("template1", false);
+    MeasurementNode mNodeS1 =
+        new MeasurementNode("s1", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+    MeasurementNode mNodeS2 =
+        new MeasurementNode("s2", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+    MeasurementNode mNodeS3 =
+        new MeasurementNode("s3", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+
+    template.addToTemplate(mNodeS1);
+    template.addToTemplate(mNodeS2);
+    template.addToTemplate(mNodeS3);
 
-    session.createSchemaTemplate(
-        "template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
+    session.createSchemaTemplate(template);
     for (int i = 0; i < sgNum; i++) {
       session.setSchemaTemplate("template1", "root.sg_" + i);
     }
diff --git a/example/session/src/main/java/org/apache/iotdb/SessionExample.java b/example/session/src/main/java/org/apache/iotdb/SessionExample.java
index cab736d..1f1c164 100644
--- a/example/session/src/main/java/org/apache/iotdb/SessionExample.java
+++ b/example/session/src/main/java/org/apache/iotdb/SessionExample.java
@@ -25,6 +25,8 @@ import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.session.Session;
 import org.apache.iotdb.session.SessionDataSet;
 import org.apache.iotdb.session.SessionDataSet.DataIterator;
+import org.apache.iotdb.session.template.MeasurementNode;
+import org.apache.iotdb.session.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;
@@ -33,8 +35,8 @@ import org.apache.iotdb.tsfile.write.record.Tablet;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.UnaryMeasurementSchema;
 
+import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -218,33 +220,21 @@ public class SessionExample {
   }
 
   private static void createTemplate()
-      throws IoTDBConnectionException, StatementExecutionException {
-    List<List<String>> measurementList = new ArrayList<>();
-    measurementList.add(Collections.singletonList("s1"));
-    measurementList.add(Collections.singletonList("s2"));
-    measurementList.add(Collections.singletonList("s3"));
-
-    List<List<TSDataType>> dataTypeList = new ArrayList<>();
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-
-    List<List<TSEncoding>> encodingList = new ArrayList<>();
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-
-    List<List<CompressionType>> compressionTypes = new ArrayList<>();
-    for (int i = 0; i < 3; i++) {
-      compressionTypes.add(Collections.singletonList(CompressionType.SNAPPY));
-    }
-    List<String> schemaNames = new ArrayList<>();
-    schemaNames.add("s1");
-    schemaNames.add("s2");
-    schemaNames.add("s3");
-
-    session.createSchemaTemplate(
-        "template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+
+    Template template = new Template("template1", false);
+    MeasurementNode mNodeS1 =
+        new MeasurementNode("s1", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+    MeasurementNode mNodeS2 =
+        new MeasurementNode("s2", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+    MeasurementNode mNodeS3 =
+        new MeasurementNode("s3", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+
+    template.addToTemplate(mNodeS1);
+    template.addToTemplate(mNodeS2);
+    template.addToTemplate(mNodeS3);
+
+    session.createSchemaTemplate(template);
     session.setSchemaTemplate("template1", "root.sg1");
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/TemplateImcompatibeException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/TemplateImcompatibeException.java
new file mode 100644
index 0000000..5aee1f9
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/TemplateImcompatibeException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.exception.metadata;
+
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class TemplateImcompatibeException extends MetadataException {
+
+  public TemplateImcompatibeException(String path, String templateName) {
+    super(
+        String.format("Path [%s] already exists in [%s]", path, templateName),
+        TSStatusCode.TEMPLATE_IMCOMPATIBLE.getStatusCode());
+    this.isUserException = true;
+  }
+
+  public TemplateImcompatibeException(String path, String templateName, String overlapNodeName) {
+    super(
+        String.format("Path [%s] overlaps with [%s] on [%s]", path, templateName, overlapNodeName),
+        TSStatusCode.TEMPLATE_IMCOMPATIBLE.getStatusCode());
+    this.isUserException = true;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 6c6bf4f..d7c2eb6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -40,7 +40,6 @@ import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
 import org.apache.iotdb.db.metadata.lastCache.LastCacheManager;
 import org.apache.iotdb.db.metadata.logfile.MLogReader;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
-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;
@@ -1725,11 +1724,30 @@ public class MManager {
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
       throws MetadataException, IOException {
+    // devicePath is a logical path which is parent of measurement, whether in template or not
     PartialPath devicePath = plan.getDeviceId();
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
 
-    // 1. get device node
+    // 1. get device node, set using template if accessed.
+    boolean mountedNodeFound = false;
+    // check every measurement path
+    for (String measurementId : measurementList) {
+      PartialPath fullPath = devicePath.concatNode(measurementId);
+      int index = mtree.getMountedNodeIndexOnMeasurementPath(fullPath);
+      if (index != fullPath.getNodeLength() - 1) {
+        // this measurement is in template, need to assure mounted node exists and set using
+        // template.
+        if (!mountedNodeFound) {
+          // Without allowing overlap of template and MTree, this block run only once
+          String[] mountedPathNodes = Arrays.copyOfRange(fullPath.getNodes(), 0, index + 1);
+          IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
+          setUsingSchemaTemplate(mountedNode);
+          mountedNodeFound = true;
+        }
+      }
+    }
+    // get logical device node, may be in template. will be multiple if overlap is allowed.
     IMNode deviceMNode = getDeviceNodeWithAutoCreate(devicePath);
 
     // check insert non-aligned InsertPlan for aligned timeseries
@@ -1810,7 +1828,7 @@ public class MManager {
     String measurement = measurementList[loc];
     IMeasurementMNode measurementMNode = getMeasurementMNode(deviceMNode, measurement);
     if (measurementMNode == null) {
-      measurementMNode = findTemplate(deviceMNode, measurement);
+      measurementMNode = findMeasurementInTemplate(deviceMNode, measurement);
     }
     if (measurementMNode == null) {
       if (!config.isAutoCreateSchemaEnabled()) {
@@ -1877,7 +1895,7 @@ public class MManager {
     return dataType;
   }
 
-  private IMeasurementMNode findTemplate(IMNode deviceMNode, String measurement)
+  private IMeasurementMNode findMeasurementInTemplate(IMNode deviceMNode, String measurement)
       throws MetadataException {
     Template curTemplate = deviceMNode.getUpperTemplate();
     if (curTemplate != null) {
@@ -1918,6 +1936,7 @@ public class MManager {
     }
     createAlignedTimeSeries(prefixPath, measurements, dataTypes, encodings, compressors);
   }
+
   // endregion
 
   // region Interfaces and Implementation for Template operations
@@ -2047,13 +2066,29 @@ public class MManager {
     }
   }
 
-  IEntityMNode setUsingSchemaTemplate(IMNode node) throws MetadataException {
+  IMNode setUsingSchemaTemplate(IMNode node) throws MetadataException {
     // this operation may change mtree structure and node type
     // invoke mnode.setUseTemplate is invalid
-    IEntityMNode entityMNode = mtree.setToEntity(node);
-    entityMNode.setUseTemplate(true);
-    if (node != entityMNode) {
-      mNodeCache.removeObject(entityMNode.getPartialPath());
+
+    // check alignment of template and mounted node
+    // if direct measurement exists, node will be replaced
+    IMNode mountedMNode =
+        mtree.checkTemplateAlignmentWithMountedNode(node, node.getUpperTemplate());
+
+    // if has direct measurement (be a EntityNode), to ensure alignment adapt with former node or
+    // template
+    if (mountedMNode.isEntity()) {
+      mountedMNode
+          .getAsEntityMNode()
+          .setAligned(
+              node.isEntity()
+                  ? node.getAsEntityMNode().isAligned()
+                  : node.getUpperTemplate().isDirectAligned());
+    }
+    mountedMNode.setUseTemplate(true);
+
+    if (node != mountedMNode) {
+      mNodeCache.removeObject(mountedMNode.getPartialPath());
     }
     if (!isRecovering) {
       try {
@@ -2062,7 +2097,7 @@ public class MManager {
         throw new MetadataException(e);
       }
     }
-    return entityMNode;
+    return mountedMNode;
   }
   // endregion
 
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 e6884c0..7bbb69d 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
@@ -35,8 +35,6 @@ public class EntityMNode extends InternalMNode implements IEntityMNode {
   @SuppressWarnings("squid:S3077")
   private transient volatile Map<String, IMeasurementMNode> aliasChildren = null;
 
-  private volatile boolean useTemplate = false;
-
   private volatile boolean isAligned = false;
 
   private volatile Map<String, ILastCacheContainer> lastCacheMap = null;
@@ -108,16 +106,6 @@ public class EntityMNode extends InternalMNode implements IEntityMNode {
   }
 
   @Override
-  public boolean isUseTemplate() {
-    return useTemplate;
-  }
-
-  @Override
-  public void setUseTemplate(boolean useTemplate) {
-    this.useTemplate = useTemplate;
-  }
-
-  @Override
   public boolean isAligned() {
     return isAligned;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
index 3c6638e..1e4e5d6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
@@ -61,6 +61,8 @@ public interface IMNode extends Serializable {
 
   boolean isUseTemplate();
 
+  void setUseTemplate(boolean useTemplate);
+
   Template getUpperTemplate();
 
   Template getSchemaTemplate();
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 f745fe2..84677e5 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
@@ -48,6 +48,8 @@ public class InternalMNode extends MNode {
   // schema template
   protected Template schemaTemplate = null;
 
+  private volatile boolean useTemplate = false;
+
   /** Constructor of MNode. */
   public InternalMNode(IMNode parent, String name) {
     super(parent, name);
@@ -214,6 +216,16 @@ public class InternalMNode extends MNode {
   }
 
   @Override
+  public boolean isUseTemplate() {
+    return useTemplate;
+  }
+
+  @Override
+  public void setUseTemplate(boolean useTemplate) {
+    this.useTemplate = useTemplate;
+  }
+
+  @Override
   public void serializeTo(MLogWriter logWriter) throws IOException {
     serializeChildren(logWriter);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
index 90c7635..c43c69c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
@@ -231,6 +231,9 @@ public class MeasurementMNode extends MNode implements IMeasurementMNode {
   public void setSchemaTemplate(Template schemaTemplate) {}
 
   @Override
+  public void setUseTemplate(boolean useTemplate) {}
+
+  @Override
   public boolean isMeasurement() {
     return true;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
index 8f8bf77..13e43da 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.metadata.TemplateImcompatibeException;
 import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
 import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
 import org.apache.iotdb.db.metadata.MetadataConstant;
@@ -65,7 +66,6 @@ import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
 import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 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;
@@ -352,14 +352,18 @@ public class MTree implements Serializable {
         hasSetStorageGroup = true;
       }
       String childName = nodeNames[i];
+
+      // even template not in use, measurement path shall not be conflict with MTree
+      if (upperTemplate != null && upperTemplate.getDirectNode(childName) != null) {
+        throw new TemplateImcompatibeException(
+            path.getFullPath(), upperTemplate.getName(), childName);
+      }
+
       if (!cur.hasChild(childName)) {
         if (!hasSetStorageGroup) {
           throw new StorageGroupNotSetException("Storage group should be created first");
         }
-        if (cur.isUseTemplate() && upperTemplate.isDirectNodeInTemplate(childName)) {
-          throw new PathAlreadyExistException(
-              cur.getPartialPath().concatNode(childName).getFullPath());
-        }
+
         cur.addChild(childName, new InternalMNode(cur, childName));
       }
       cur = cur.getChild(childName);
@@ -373,18 +377,6 @@ public class MTree implements Serializable {
       throw new PathAlreadyExistException(cur.getFullPath());
     }
 
-    if (upperTemplate != null
-        && upperTemplate.isDirectNodeInTemplate(nodeNames[nodeNames.length - 1])) {
-      throw new PathAlreadyExistException(
-          path.getFullPath() + " ( which is incompatible with template )");
-    }
-
-    // Quick patch for tree structure template
-    if (upperTemplate != null && upperTemplate.isDirectNodeInTemplate(cur.getName())) {
-      throw new PathAlreadyExistException(
-          path.getFullPath() + " ( which is incompatible with template )");
-    }
-
     MetaFormatUtils.checkTimeseriesProps(path.getFullPath(), props);
 
     String leafName = path.getMeasurement();
@@ -402,8 +394,7 @@ public class MTree implements Serializable {
 
       if (upperTemplate != null
           && (upperTemplate.hasSchema(leafName) || upperTemplate.hasSchema(alias))) {
-        throw new PathAlreadyExistException(
-            path.getFullPath() + " ( which is incompatible with template )");
+        throw new TemplateImcompatibeException(path.getFullPath(), upperTemplate.getName());
       }
 
       IEntityMNode entityMNode = MNodeUtils.setToEntity(cur);
@@ -456,7 +447,7 @@ public class MTree implements Serializable {
 
       if (upperTemplate != null) {
         for (String measurement : measurements) {
-          if (upperTemplate.hasSchema(measurement)) {
+          if (upperTemplate.getDirectNode(measurement) != null) {
             throw new PathAlreadyExistException(
                 devicePath.concatNode(measurement).getFullPath()
                     + " ( which is incompatible with template )");
@@ -504,7 +495,7 @@ public class MTree implements Serializable {
         if (!hasSetStorageGroup) {
           throw new StorageGroupNotSetException("Storage group should be created first");
         }
-        if (upperTemplate != null && upperTemplate.hasSchema(childName)) {
+        if (upperTemplate != null && upperTemplate.getDirectNode(childName) != null) {
           throw new PathAlreadyExistException(
               cur.getPartialPath().concatNode(childName).getFullPath()
                   + " ( which is incompatible with template )");
@@ -539,6 +530,11 @@ public class MTree implements Serializable {
       throw new IllegalPathException(path.getFullPath());
     }
 
+    if (isPathExistsWithinTemplate(path)) {
+      throw new MetadataException(
+          "Cannot delete a timeseries inside a template: " + path.toString());
+    }
+
     IMeasurementMNode deletedNode = getMeasurementMNode(path);
     IEntityMNode parent = deletedNode.getParent();
     // delete the last node of path
@@ -592,7 +588,7 @@ public class MTree implements Serializable {
     Template upperTemplate = cur.getSchemaTemplate();
     for (int i = 1; i < nodeNames.length; i++) {
       if (!cur.hasChild(nodeNames[i])) {
-        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+        if (cur.isUseTemplate() && upperTemplate.getDirectNode(nodeNames[i]) != null) {
           throw new PathAlreadyExistException(
               cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
         }
@@ -728,7 +724,10 @@ public class MTree implements Serializable {
     Template upperTemplate = cur.getSchemaTemplate();
     for (int i = 1; i < nodeNames.length; i++) {
       if (!cur.hasChild(nodeNames[i])) {
-        return cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i]);
+        if (!cur.isUseTemplate() || upperTemplate.getDirectNode(nodeNames[i]) == null) {
+          return false;
+        }
+        cur = upperTemplate.getDirectNode(nodeNames[i]);
       }
       cur = cur.getChild(nodeNames[i]);
       if (cur.isMeasurement()) {
@@ -898,8 +897,8 @@ public class MTree implements Serializable {
     EntityCollector<Set<PartialPath>> collector =
         new EntityCollector<Set<PartialPath>>(root, pathPattern) {
           @Override
-          protected void collectEntity(IEntityMNode node) {
-            result.add(node.getPartialPath());
+          protected void collectEntity(IEntityMNode node) throws MetadataException {
+            result.add(getCurrentPartialPath(node));
           }
         };
     collector.setPrefixMatch(isPrefixMatch);
@@ -914,7 +913,7 @@ public class MTree implements Serializable {
             root, plan.getPath(), plan.getLimit(), plan.getOffset()) {
           @Override
           protected void collectEntity(IEntityMNode node) throws MetadataException {
-            PartialPath device = node.getPartialPath();
+            PartialPath device = getCurrentPartialPath(node);
             if (plan.hasSgCol()) {
               res.add(
                   new ShowDevicesResult(
@@ -934,7 +933,7 @@ public class MTree implements Serializable {
         new MeasurementCollector<Set<PartialPath>>(root, timeseries) {
           @Override
           protected void collectMeasurement(IMeasurementMNode node) throws MetadataException {
-            result.add(node.getParent().getPartialPath());
+            result.add(getCurrentPartialPath(node).getDevicePath());
           }
         };
     collector.traverse();
@@ -966,8 +965,8 @@ public class MTree implements Serializable {
     MeasurementCollector<List<PartialPath>> collector =
         new MeasurementCollector<List<PartialPath>>(root, pathPattern, limit, offset) {
           @Override
-          protected void collectMeasurement(IMeasurementMNode node) {
-            MeasurementPath path = node.getMeasurementPath();
+          protected void collectMeasurement(IMeasurementMNode node) throws MetadataException {
+            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
               path.setMeasurementAlias(node.getAlias());
@@ -1028,13 +1027,13 @@ public class MTree implements Serializable {
             IMeasurementSchema measurementSchema = node.getSchema();
             String[] tsRow = new String[7];
             tsRow[0] = node.getAlias();
-            tsRow[1] = getBelongedStorageGroupPath(node).getFullPath();
+            tsRow[1] = getStorageGroupNodeInTraversePath().getFullPath();
             tsRow[2] = measurementSchema.getType().toString();
             tsRow[3] = measurementSchema.getEncodingType().toString();
             tsRow[4] = measurementSchema.getCompressor().toString();
             tsRow[5] = String.valueOf(node.getOffset());
             tsRow[6] = needLast ? String.valueOf(getLastTimeStamp(node, queryContext)) : null;
-            Pair<PartialPath, String[]> temp = new Pair<>(node.getPartialPath(), tsRow);
+            Pair<PartialPath, String[]> temp = new Pair<>(getCurrentPartialPath(node), tsRow);
             result.add(temp);
           }
         };
@@ -1066,8 +1065,8 @@ public class MTree implements Serializable {
     MeasurementCollector<List<IMeasurementSchema>> collector =
         new MeasurementCollector<List<IMeasurementSchema>>(root, prefixPath) {
           @Override
-          protected void collectMeasurement(IMeasurementMNode node) {
-            result.put(node.getPartialPath(), node.getSchema());
+          protected void collectMeasurement(IMeasurementMNode node) throws MetadataException {
+            result.put(getCurrentPartialPath(node), node.getSchema());
           }
         };
     collector.setPrefixMatch(true);
@@ -1105,11 +1104,11 @@ public class MTree implements Serializable {
     MeasurementCollector<List<IMeasurementSchema>> collector =
         new MeasurementCollector<List<IMeasurementSchema>>(root, prefixPath) {
           @Override
-          protected void collectMeasurement(IMeasurementMNode node) {
+          protected void collectMeasurement(IMeasurementMNode node) throws MetadataException {
             IMeasurementSchema nodeSchema = node.getSchema();
             timeseriesSchemas.add(
                 new TimeseriesSchema(
-                    node.getFullPath(),
+                    getCurrentPartialPath(node).getFullPath(),
                     nodeSchema.getType(),
                     nodeSchema.getEncodingType(),
                     nodeSchema.getCompressor()));
@@ -1139,7 +1138,11 @@ public class MTree implements Serializable {
           new MNodeCollector<Set<String>>(root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD)) {
             @Override
             protected void transferToResult(IMNode node) {
-              resultSet.add(node.getFullPath());
+              try {
+                resultSet.add(getCurrentPartialPath(node).getFullPath());
+              } catch (IllegalPathException e) {
+                logger.error(e.getMessage());
+              }
             }
           };
       collector.setResultSet(new TreeSet<>());
@@ -1186,7 +1189,11 @@ public class MTree implements Serializable {
         new MNodeCollector<List<PartialPath>>(root, pathPattern) {
           @Override
           protected void transferToResult(IMNode node) {
-            resultSet.add(node.getPartialPath());
+            try {
+              resultSet.add(getCurrentPartialPath(node));
+            } catch (MetadataException e) {
+              logger.error(e.getMessage());
+            }
           }
         };
     collector.setResultSet(new LinkedList<>());
@@ -1269,27 +1276,12 @@ public class MTree implements Serializable {
       }
       IMNode next = cur.getChild(nodes[i]);
       if (next == null) {
-        if (upperTemplate == null) {
+        if (upperTemplate == null
+            || !cur.isUseTemplate()
+            || upperTemplate.getDirectNode(nodes[i]) == null) {
           throw new PathNotExistException(path.getFullPath(), true);
         }
-
-        String realName = nodes[i];
-        // Quick patch for tree structured template
-        if (nodes.length - 1 != i) {
-          String[] realNameNodes = Arrays.copyOfRange(nodes, i, nodes.length);
-          StringBuilder builder = new StringBuilder(realNameNodes[0]);
-          for (int concatIndex = 1; concatIndex < realNameNodes.length; concatIndex++) {
-            builder.append(TsFileConstant.PATH_SEPARATOR);
-            builder.append(realNameNodes[concatIndex]);
-          }
-          realName = builder.toString();
-        }
-        IMeasurementSchema schema = upperTemplate.getSchemaMap().get(realName);
-        if (schema == null) {
-          throw new PathNotExistException(path.getFullPath(), true);
-        }
-        return MeasurementMNode.getMeasurementMNode(
-            cur.getAsEntityMNode(), schema.getMeasurementId(), schema, null);
+        next = upperTemplate.getDirectNode(nodes[i]);
       }
       cur = next;
     }
@@ -1308,15 +1300,28 @@ public class MTree implements Serializable {
     }
 
     IMNode cur = root;
+    Template upperTemplate = null;
 
     for (int i = 1; i < nodes.length; i++) {
-      cur = cur.getChild(nodes[i]);
-      if (cur == null) {
-        // not find
+      if (cur.getSchemaTemplate() != null) {
+        upperTemplate = cur.getSchemaTemplate();
+      }
+
+      if (cur.getChild(nodes[i]) != null) {
+        cur = cur.getChild(nodes[i]);
+      } else {
+        // seek child in template
         if (!storageGroupChecked) {
           throw new StorageGroupNotSetException(path.getFullPath());
         }
-        throw new PathNotExistException(path.getFullPath());
+
+        if (upperTemplate == null
+            || !cur.isUseTemplate()
+            || upperTemplate.getDirectNode(nodes[i]) == null) {
+          throw new PathNotExistException(path.getFullPath());
+        }
+
+        cur = upperTemplate.getDirectNode(nodes[i]);
       }
 
       if (cur.isStorageGroup()) {
@@ -1394,6 +1399,7 @@ public class MTree implements Serializable {
           path.getFullPath(), MetadataConstant.MEASUREMENT_MNODE_TYPE);
     }
   }
+
   // endregion
 
   // region Interfaces and Implementation for Template check
@@ -1426,6 +1432,52 @@ public class MTree implements Serializable {
     checkTemplateOnSubtree(cur);
   }
 
+  /**
+   * Check route 1: If template has no direct measurement, just pass the check.
+   *
+   * <p>Check route 2: If template has direct measurement and mounted node is Internal, it should be
+   * set to Entity.
+   *
+   * <p>Check route 3: If template has direct measurement and mounted node is Entity,
+   *
+   * <p>route 3.1: mounted node has no measurement child, then its alignment will be set as the
+   * template.
+   *
+   * <p>route 3.2: mounted node has measurement child, then alignment of it and template should be
+   * identical, otherwise cast a exception.
+   *
+   * @return return the node competent to be mounted.
+   */
+  public IMNode checkTemplateAlignmentWithMountedNode(IMNode mountedNode, Template template)
+      throws MetadataException {
+    boolean hasDirectMeasurement = false;
+    for (IMNode child : template.getDirectNodes()) {
+      if (child.isMeasurement()) {
+        hasDirectMeasurement = true;
+      }
+    }
+    if (hasDirectMeasurement) {
+      if (!mountedNode.isEntity()) {
+        return setToEntity(mountedNode);
+      } else {
+        for (IMNode child : mountedNode.getChildren().values()) {
+          if (child.isMeasurement()) {
+            if (template.isDirectAligned() != mountedNode.getAsEntityMNode().isAligned()) {
+              throw new MetadataException(
+                  "Template and mounted node has different alignment: "
+                      + template.getName()
+                      + mountedNode.getFullPath());
+            } else {
+              return mountedNode;
+            }
+          }
+        }
+        mountedNode.getAsEntityMNode().setAligned(template.isDirectAligned());
+      }
+    }
+    return mountedNode;
+  }
+
   // traverse  all the  descendant of the given path node
   private void checkTemplateOnSubtree(IMNode node) throws MetadataException {
     if (node.isMeasurement()) {
@@ -1456,6 +1508,93 @@ public class MTree implements Serializable {
       checkTemplateInUseOnLowerNode(child);
     }
   }
+
+  /**
+   * Note that template and MTree cannot have overlap paths.
+   *
+   * @return true iff path corresponding to a measurement inside a template, whether using or not.
+   */
+  public boolean isPathExistsWithinTemplate(PartialPath path) {
+    if (path.getNodes().length < 2) {
+      return false;
+    }
+    String[] pathNodes = path.getNodes();
+    IMNode cur = root;
+    Template upperTemplate = cur.getUpperTemplate();
+    for (int i = 1; i < pathNodes.length; i++) {
+      if (cur.hasChild(pathNodes[i])) {
+        cur = cur.getChild(pathNodes[i]);
+        if (cur.isMeasurement()) {
+          return false;
+        }
+        upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      } else if (upperTemplate != null) {
+        String suffixPath =
+            new PartialPath(Arrays.copyOfRange(pathNodes, i, pathNodes.length)).toString();
+        if (upperTemplate.hasSchema(suffixPath)) {
+          return true;
+        } else {
+          // has template, but not match
+          return false;
+        }
+      } else {
+        // no child and no template
+        return false;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check measurement path and return the mounted node index on path. The node could have not
+   * created yet. The result is used for getDeviceNodeWithAutoCreate, which return corresponding
+   * IMNode on MTree.
+   *
+   * @return index on full path of the node which matches all measurements path with its
+   *     upperTemplate.
+   */
+  public int getMountedNodeIndexOnMeasurementPath(PartialPath measurementPath)
+      throws MetadataException {
+    String[] fullPathNodes = measurementPath.getNodes();
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+
+    if (!cur.getName().equals(fullPathNodes[0])) {
+      throw new IllegalPathException(measurementPath.toString());
+    }
+
+    for (int index = 1; index < fullPathNodes.length; index++) {
+      upperTemplate = cur.getSchemaTemplate() != null ? cur.getSchemaTemplate() : upperTemplate;
+      if (!cur.hasChild(fullPathNodes[index])) {
+        if (upperTemplate != null) {
+          String suffixPath =
+              new PartialPath(Arrays.copyOfRange(fullPathNodes, index, fullPathNodes.length))
+                  .toString();
+
+          // if suffix matches template, then fullPathNodes[index-1] should be the node to use
+          // template on MTree
+          if (upperTemplate.hasSchema(suffixPath)) {
+            return index - 1;
+          }
+
+          // overlap with template, cast exception for now
+          if (upperTemplate.getDirectNode(fullPathNodes[index]) != null) {
+            throw new TemplateImcompatibeException(
+                measurementPath.getFullPath(), upperTemplate.getName(), fullPathNodes[index]);
+          }
+        } else {
+          // no matched child, no template, need to create device node as logical device path
+          return fullPathNodes.length - 1;
+        }
+      } else {
+        // has child on MTree
+        cur = cur.getChild(fullPathNodes[index]);
+      }
+    }
+    // all nodes on path exist in MTree, device node should be the penultimate one
+    return fullPathNodes.length - 1;
+  }
+
   // endregion
 
   // region TestOnly Interface
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 51cf8cb..4385a63 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
@@ -21,11 +21,13 @@ package org.apache.iotdb.db.metadata.mtree.traverser;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.metadata.template.Template;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.Iterator;
 import java.util.regex.Pattern;
 
 import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
@@ -42,6 +44,9 @@ public abstract class Traverser {
   protected IMNode startNode;
   protected String[] nodes;
 
+  // to construct full path or find mounted node on MTree when traverse into template
+  protected Deque<IMNode> traverseContext;
+
   // if isMeasurementTraverser, measurement in template should be processed
   protected boolean isMeasurementTraverser = false;
 
@@ -56,6 +61,7 @@ public abstract class Traverser {
     }
     this.startNode = startNode;
     this.nodes = nodes;
+    this.traverseContext = new ArrayDeque<>();
   }
 
   /**
@@ -136,27 +142,28 @@ public abstract class Traverser {
 
   protected void processMultiLevelWildcard(IMNode node, int idx, int level)
       throws MetadataException {
+    traverseContext.push(node);
     for (IMNode child : node.getChildren().values()) {
       traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
 
-    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+    if (!node.isUseTemplate()) {
       return;
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-          idx + 1,
-          level + 1);
+    traverseContext.push(node);
+    for (IMNode child : upperTemplate.getDirectNodes()) {
+      traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
   }
 
   protected void processOneLevelWildcard(IMNode node, int idx, int level) throws MetadataException {
     boolean multiLevelWildcard = nodes[idx].equals(MULTI_LEVEL_PATH_WILDCARD);
     String targetNameRegex = nodes[idx + 1].replace("*", ".*");
+    traverseContext.push(node);
     for (IMNode child : node.getChildren().values()) {
       if (child.isMeasurement()) {
         String alias = child.getAsMeasurementMNode().getAlias();
@@ -171,77 +178,111 @@ public abstract class Traverser {
       }
       traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
+
     if (multiLevelWildcard) {
+      traverseContext.push(node);
       for (IMNode child : node.getChildren().values()) {
         traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
 
-    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+    if (!node.isUseTemplate()) {
       return;
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-      if (!Pattern.matches(targetNameRegex, schema.getMeasurementId())) {
+
+    traverseContext.push(node);
+    for (IMNode child : upperTemplate.getDirectNodes()) {
+      if (!Pattern.matches(targetNameRegex, child.getName())) {
         continue;
       }
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-          idx + 1,
-          level + 1);
+      traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
+
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      traverseContext.push(node);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
   }
 
+  @SuppressWarnings("Duplicates")
   protected void processNameMatch(IMNode node, int idx, int level) throws MetadataException {
     boolean multiLevelWildcard = nodes[idx].equals(MULTI_LEVEL_PATH_WILDCARD);
     String targetName = nodes[idx + 1];
     IMNode next = node.getChild(targetName);
     if (next != null) {
+      traverseContext.push(node);
       traverse(next, idx + 1, level + 1);
+      traverseContext.pop();
     }
     if (multiLevelWildcard) {
+      traverseContext.push(node);
       for (IMNode child : node.getChildren().values()) {
         traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
 
-    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+    if (!node.isUseTemplate()) {
       return;
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      traverseContext.push(node);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getCurrentPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  /** @return the storage group node in the traverse path */
+  protected IMNode getStorageGroupNodeInTraversePath() {
+    Iterator<IMNode> nodes = traverseContext.iterator();
+    while (nodes.hasNext()) {
+      IMNode node = nodes.next();
+      if (node.isStorageGroup()) {
+        return node;
+      }
+    }
+    return null;
+  }
 }
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 fae6347..46f8e87 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
@@ -21,7 +21,11 @@ package org.apache.iotdb.db.metadata.mtree.traverser.collector;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
 import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+
+import java.util.Iterator;
 
 // This class defines MeasurementMNode as target node and defines the measurement process framework.
 public abstract class MeasurementCollector<T> extends CollectorTraverser<T> {
@@ -68,4 +72,29 @@ public abstract class MeasurementCollector<T> extends CollectorTraverser<T> {
    * @param node MeasurementMNode holding the measurement schema
    */
   protected abstract void collectMeasurement(IMeasurementMNode node) throws MetadataException;
+
+  /**
+   * When traverse goes into a template, IMNode.getPartialPath may not work as nodes in template has
+   * no parent on MTree. So this methods will construct a path from root to node in template using a
+   * stack traverseContext.
+   */
+  protected MeasurementPath getCurrentMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    MeasurementPath retPath =
+        new MeasurementPath(new PartialPath(builder.toString()), currentNode.getSchema());
+    retPath.setUnderAlignedEntity(par.getAsEntityMNode().isAligned());
+    return retPath;
+  }
 }
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 0c9fd97..8437297 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
@@ -47,6 +47,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Deque;
 import java.util.HashMap;
@@ -58,7 +59,7 @@ import java.util.Set;
 public class Template {
   private String name;
   private Map<String, IMNode> directNodes;
-  private Set<String> alignedPrefix;
+  private boolean isDirectAligned;
   private int measurementsCount;
   private Map<String, IMeasurementSchema> schemaMap;
 
@@ -73,7 +74,7 @@ public class Template {
     boolean isAlign;
     schemaMap = new HashMap<>();
     name = plan.getName();
-    alignedPrefix = new HashSet<>();
+    isDirectAligned = false;
     directNodes = new HashMap<>();
 
     for (int i = 0; i < plan.getMeasurements().size(); i++) {
@@ -82,7 +83,8 @@ public class Template {
       if (size > 1) {
         isAlign = true;
       } else {
-        // Patch for align designation ambiguity when creating from serialization
+        // If sublist of measurements has only one item,
+        // but it share prefix with other aligned sublist, it will be aligned too
         String[] thisMeasurement =
             MetaUtils.splitPathToDetachedPath(plan.getMeasurements().get(i).get(0));
         String thisPrefix =
@@ -135,44 +137,16 @@ public class Template {
     return schemaMap;
   }
 
-  public boolean hasSchema(String measurementId) {
-    return schemaMap.containsKey(measurementId);
+  public boolean hasSchema(String suffixPath) {
+    return schemaMap.containsKey(suffixPath);
   }
 
   public IMeasurementSchema getSchema(String measurementId) {
     return schemaMap.get(measurementId);
   }
 
-  public List<IMeasurementMNode> getMeasurementMNode() {
-    Set<IMeasurementSchema> deduplicateSchema = new HashSet<>();
-    List<IMeasurementMNode> res = new ArrayList<>();
-
-    for (IMeasurementSchema measurementSchema : schemaMap.values()) {
-      if (deduplicateSchema.add(measurementSchema)) {
-        IMeasurementMNode measurementMNode = null;
-        if (measurementSchema instanceof UnaryMeasurementSchema) {
-          measurementMNode =
-              MeasurementMNode.getMeasurementMNode(
-                  null, measurementSchema.getMeasurementId(), measurementSchema, null);
-
-        } else if (measurementSchema instanceof VectorMeasurementSchema) {
-          measurementMNode =
-              MeasurementMNode.getMeasurementMNode(
-                  null,
-                  getMeasurementNodeName(measurementSchema.getMeasurementId()),
-                  measurementSchema,
-                  null);
-        }
-
-        res.add(measurementMNode);
-      }
-    }
-
-    return res;
-  }
-
-  public String getMeasurementNodeName(String measurementName) {
-    return schemaMap.get(measurementName).getMeasurementId();
+  public boolean isDirectAligned() {
+    return isDirectAligned;
   }
 
   // region construct template tree
@@ -181,7 +155,7 @@ public class Template {
       throws IllegalPathException {
     // Only for aligned Paths, with common direct prefix
     String[] pathNodes;
-    IEntityMNode commonPar;
+    IMNode commonPar;
     String prefix = null;
     List<String> measurementNames = new ArrayList<>();
     IMeasurementMNode leafNode;
@@ -218,8 +192,9 @@ public class Template {
     }
 
     synchronized (this) {
-      // if not aligned now, it will be set to aligned
-      alignedPrefix.add(prefix);
+      if (prefix.equals("")) {
+        isDirectAligned = true;
+      }
       for (int i = 0; i <= measurementNames.size() - 1; i++) {
         // find the parent and add nodes to template
         if (prefix.equals("")) {
@@ -227,10 +202,11 @@ public class Template {
               MeasurementMNode.getMeasurementMNode(null, measurementNames.get(i), schemas[i], null);
           directNodes.put(leafNode.getName(), leafNode);
         } else {
-          commonPar = (IEntityMNode) constructEntityPath(alignedPaths[0]);
+          commonPar = constructEntityPath(alignedPaths[0]);
+          commonPar.getAsEntityMNode().setAligned(true);
           leafNode =
               MeasurementMNode.getMeasurementMNode(
-                  commonPar, measurementNames.get(i), schemas[i], null);
+                  commonPar.getAsEntityMNode(), measurementNames.get(i), schemas[i], null);
           commonPar.addChild(leafNode);
         }
         schemaMap.put(getFullPathWithoutTemplateName(leafNode), schemas[i]);
@@ -285,20 +261,37 @@ public class Template {
   // region query of template
 
   public List<String> getAllAlignedPrefix() {
-    return Arrays.asList(alignedPrefix.toArray(new String[0]));
+    List<String> alignedPrefix = new ArrayList<>();
+    if (isDirectAligned) {
+      alignedPrefix.add("");
+    }
+
+    Deque<IMNode> traverseChildren = new ArrayDeque<>();
+    directNodes.values().forEach(traverseChildren::push);
+    while (traverseChildren.size() != 0) {
+      IMNode cur = traverseChildren.pop();
+      if (cur.getChildren().size() != 0) {
+        cur.getChildren().values().forEach(traverseChildren::push);
+      }
+      if (cur.isEntity() && cur.getAsEntityMNode().isAligned()) {
+        alignedPrefix.add(cur.getFullPath());
+      }
+    }
+    return alignedPrefix;
   }
 
   public List<String> getAlignedMeasurements(String prefix) throws IllegalPathException {
-    if (!alignedPrefix.contains(prefix)) {
-      return null;
-    }
     IMNode prefixNode = getPathNodeInTemplate(prefix);
     if (prefixNode == null) {
-      throw new IllegalPathException(prefix, "there is no prefix IMNode.");
+      throw new IllegalPathException(prefix, "there is no IMNode for given prefix.");
     }
     if (prefixNode.isMeasurement()) {
       throw new IllegalPathException(prefix, "path is a measurement.");
     }
+    if (!prefixNode.isEntity() || !prefixNode.getAsEntityMNode().isAligned()) {
+      throw new IllegalPathException(prefix, "path has no child as aligned measurement.");
+    }
+
     List<String> subMeasurements = new ArrayList<>();
     for (IMNode child : prefixNode.getChildren().values()) {
       if (child.isMeasurement()) {
@@ -402,6 +395,10 @@ public class Template {
     return directNodes.getOrDefault(nodeName, null);
   }
 
+  public Collection<IMNode> getDirectNodes() {
+    return directNodes.values();
+  }
+
   // endregion
 
   // region inner utils
@@ -482,7 +479,8 @@ public class Template {
     // Prefix equality will be checked in constructTemplateTree
     pathNode = MetaUtils.splitPathToDetachedPath(measurements[0]);
     prefix = joinBySeparator(Arrays.copyOf(pathNode, pathNode.length - 1));
-    if ((getPathNodeInTemplate(prefix) != null) && (!alignedPrefix.contains(prefix))) {
+    IMNode targetNode = getPathNodeInTemplate(prefix);
+    if (targetNode != null && !targetNode.getAsEntityMNode().isAligned()) {
       throw new IllegalPathException(prefix, "path already exists but not aligned");
     }
 
@@ -514,7 +512,8 @@ public class Template {
 
       // If prefix exists and aligned, it will throw exception
       prefix = joinBySeparator(Arrays.copyOf(pathNode, pathNode.length - 1));
-      if ((getPathNodeInTemplate(prefix) != null) && (alignedPrefix.contains(prefix))) {
+      IMNode parNode = getPathNodeInTemplate(prefix);
+      if (parNode != null && parNode.getAsEntityMNode().isAligned()) {
         throw new IllegalPathException(prefix, "path already exists and aligned");
       }
 
@@ -569,7 +568,11 @@ public class Template {
       IMNode top = astack.pop();
       if (!top.isMeasurement()) {
         String thisPrefix = getFullPathWithoutTemplateName(top);
-        alignedPrefix.remove(thisPrefix);
+
+        if (thisPrefix.equals("")) {
+          isDirectAligned = false;
+        }
+
         for (IMNode child : top.getChildren().values()) {
           astack.push(child);
         }
@@ -580,8 +583,14 @@ public class Template {
     }
   }
 
-  public void deleteAlignedPrefix(String path) {
-    alignedPrefix.remove(path);
+  public void deleteAlignedPrefix(String path) throws IllegalPathException {
+    if (path.equals("")) {
+      isDirectAligned = false;
+    }
+    IMNode targetNode = getPathNodeInTemplate(path);
+    if (targetNode.isEntity()) {
+      targetNode.getAsEntityMNode().setAligned(false);
+    }
   }
   // endregion
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
index 22b3f3a..512c27d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
@@ -153,7 +153,7 @@ public class TemplateManager {
       String directNodeName = MetaUtils.splitPathToDetachedPath(measurementPath)[0];
       if (node.hasChild(directNodeName)) {
         throw new MetadataException(
-            "Schema name "
+            "Node name "
                 + directNodeName
                 + " in template has conflict with node's child "
                 + (node.getFullPath() + "." + directNodeName));
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
index 28e50b0..61bd75f 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
@@ -104,7 +104,7 @@ public class MetaUtils {
         result.add(measurementPath);
         alignedPath = null;
       } else {
-        if (alignedPath == null) {
+        if (alignedPath == null || !alignedPath.equals(measurementPath.getDevice())) {
           alignedPath = new AlignedPath(measurementPath);
           result.add(alignedPath);
         } else {
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 505c5a9..4516588 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -1883,47 +1883,17 @@ public class TSServiceImpl extends BasicServiceProvider implements TSIService.If
 
       if (AUDIT_LOGGER.isDebugEnabled()) {
         AUDIT_LOGGER.debug(
-            "Session-{} create device template {}.{}.{}.{}.{}.{}",
+            "Session-{} create schema template {}",
             sessionManager.getCurrSessionId(),
-            req.getName(),
-            req.getSchemaNames(),
-            req.getMeasurements(),
-            req.getDataTypes(),
-            req.getEncodings(),
-            req.getCompressors());
+            req.getName());
       }
 
       CreateTemplatePlan plan;
-      if (req.getMeasurements().size() == 0) {
-        // Construct plan from serialized request
-        ByteBuffer buffer = ByteBuffer.wrap(req.getSerializedTemplate());
-        plan = CreateTemplatePlan.deserializeFromReq(buffer);
-      } else {
-        int size = req.getMeasurementsSize();
-        String[][] measurements = new String[size][];
-        TSDataType[][] dataTypes = new TSDataType[size][];
-        TSEncoding[][] encodings = new TSEncoding[size][];
-        CompressionType[][] compressionTypes = new CompressionType[size][];
-
-        for (int i = 0; i < size; i++) {
-          int alignedSize = req.getMeasurements().get(i).size();
-          measurements[i] = new String[alignedSize];
-          dataTypes[i] = new TSDataType[alignedSize];
-          encodings[i] = new TSEncoding[alignedSize];
-          compressionTypes[i] = new CompressionType[alignedSize];
-          for (int j = 0; j < alignedSize; j++) {
-            measurements[i][j] = req.getMeasurements().get(i).get(j);
-            dataTypes[i][j] = TSDataType.values()[req.getDataTypes().get(i).get(j)];
-            encodings[i][j] = TSEncoding.values()[req.getEncodings().get(i).get(j)];
-            compressionTypes[i][j] = CompressionType.values()[req.getCompressors().get(i).get(j)];
-          }
-        }
-
-        plan =
-            new CreateTemplatePlan(
-                req.getName(), measurements, dataTypes, encodings, compressionTypes);
-      }
+      // Construct plan from serialized request
+      ByteBuffer buffer = ByteBuffer.wrap(req.getSerializedTemplate());
+      plan = CreateTemplatePlan.deserializeFromReq(buffer);
       TSStatus status = checkAuthority(plan, req.getSessionId());
+
       return status != null ? status : executeNonQueryPlan(plan);
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
index 434e803..92642ed 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
@@ -40,6 +40,7 @@ import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 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;
@@ -51,6 +52,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -868,11 +870,13 @@ public class MManagerBasicTest {
         new UnaryMeasurementSchema("s11", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
     assertNotNull(node.getSchemaTemplate());
 
-    Set<IMeasurementSchema> allSchema =
-        new HashSet<>(node.getSchemaTemplate().getSchemaMap().values());
+    Set<String> allSchema = new HashSet<>();
+    for (IMeasurementSchema schema : node.getSchemaTemplate().getSchemaMap().values()) {
+      allSchema.add("root.sg1.d1" + TsFileConstant.PATH_SEPARATOR + schema.getMeasurementId());
+    }
     for (MeasurementPath measurementPath :
-        manager.getMeasurementPaths(new PartialPath("root.sg1.d1.*"))) {
-      allSchema.remove(measurementPath.getMeasurementSchema());
+        manager.getMeasurementPaths(new PartialPath("root.sg1.d1.**"))) {
+      allSchema.remove(measurementPath.toString());
     }
 
     assertTrue(allSchema.isEmpty());
@@ -914,7 +918,7 @@ public class MManagerBasicTest {
       CompressionType[] compressionTypes = {CompressionType.SNAPPY, CompressionType.SNAPPY};
       template.addAlignedMeasurements(alignedMeasurements, dataTypes, encodings, compressionTypes);
 
-      assertEquals("[GPS, to.be.prefix]", template.getAllAlignedPrefix().toString());
+      assertEquals("[to.be.prefix, GPS]", template.getAllAlignedPrefix().toString());
       assertEquals("[s1, s2]", template.getAlignedMeasurements("to.be.prefix").toString());
 
       template.deleteAlignedPrefix("to.be.prefix");
@@ -923,11 +927,16 @@ public class MManagerBasicTest {
       assertEquals(null, template.getDirectNode("prefix"));
       assertEquals("to", template.getDirectNode("to").getName());
 
+      assertFalse(template.isDirectAligned());
+      template.addAlignedMeasurements(
+          new String[] {"speed", "temperature"}, dataTypes, encodings, compressionTypes);
+      assertTrue(template.isDirectAligned());
+
       try {
         template.deleteMeasurements("a.single");
         fail();
       } catch (IllegalPathException e) {
-        assertEquals("a.single is not a legal path, because Path does not exist", e.getMessage());
+        assertEquals("Path [a.single] does not exist", e.getMessage());
       }
       assertEquals(
           "[d1.s1, GPS.x, to.be.prefix.s2, GPS.y, to.be.prefix.s1, s2]",
@@ -942,13 +951,12 @@ public class MManagerBasicTest {
     }
   }
 
+  @SuppressWarnings("Duplicates")
   private CreateTemplatePlan getTreeTemplatePlan() {
     /**
      * Construct a template like: create schema template treeTemplate ( (d1.s1 INT32 GORILLA
      * SNAPPY), (s2 INT32 GORILLA SNAPPY), (GPS.x FLOAT RLE SNAPPY), (GPS.y FLOAT RLE SNAPPY), )with
      * aligned (GPS)
-     *
-     * <p>Check aligned path whether with same prefix? Construct tree
      */
     List<List<String>> measurementList = new ArrayList<>();
     measurementList.add(Collections.singletonList("d1.s1"));
@@ -1086,11 +1094,13 @@ public class MManagerBasicTest {
     CreateTemplatePlan plan = getCreateTemplatePlan();
     MManager manager = IoTDB.metaManager;
     manager.createSchemaTemplate(plan);
+    manager.createSchemaTemplate(getTreeTemplatePlan());
 
     // set device template
     SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.sg1.d1");
 
     manager.setSchemaTemplate(setTemplatePlan);
+    manager.setSchemaTemplate(new SetTemplatePlan("treeTemplate", "root.tree.sg0"));
 
     CreateTimeSeriesPlan createTimeSeriesPlan =
         new CreateTimeSeriesPlan(
@@ -1120,10 +1130,40 @@ public class MManagerBasicTest {
       manager.createTimeseries(createTimeSeriesPlan2);
       fail();
     } catch (Exception e) {
+      assertEquals("Path [root.sg1.d1.s11] already exists in [template1]", e.getMessage());
+    }
+
+    CreateTimeSeriesPlan createTimeSeriesPlan3 =
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.tree.sg0.GPS.s9"),
+            TSDataType.INT32,
+            TSEncoding.PLAIN,
+            CompressionType.GZIP,
+            null,
+            null,
+            null,
+            null);
+
+    try {
+      manager.createTimeseries(createTimeSeriesPlan3);
+      fail();
+    } catch (Exception e) {
       assertEquals(
-          "Path [root.sg1.d1.s11 ( which is incompatible with template )] already exist",
-          e.getMessage());
+          "Path [root.tree.sg0.GPS.s9] overlaps with [treeTemplate] on [GPS]", e.getMessage());
     }
+
+    CreateTimeSeriesPlan createTimeSeriesPlan4 =
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.tree.sg0.s3"),
+            TSDataType.INT32,
+            TSEncoding.PLAIN,
+            CompressionType.GZIP,
+            null,
+            null,
+            null,
+            null);
+
+    manager.createTimeseries(createTimeSeriesPlan4);
   }
 
   @Test
@@ -1131,10 +1171,13 @@ public class MManagerBasicTest {
     MManager manager = IoTDB.metaManager;
     CreateTemplatePlan plan = getCreateTemplatePlan();
     manager.createSchemaTemplate(plan);
+    manager.createSchemaTemplate(getTreeTemplatePlan());
 
     // set device template
     SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.sg1.d1");
 
+    SetTemplatePlan setSchemaTemplatePlan2 = new SetTemplatePlan("treeTemplate", "root.tree.sg0");
+
     CreateTimeSeriesPlan createTimeSeriesPlan =
         new CreateTimeSeriesPlan(
             new PartialPath("root.sg1.d1.s11"),
@@ -1148,12 +1191,45 @@ public class MManagerBasicTest {
 
     manager.createTimeseries(createTimeSeriesPlan);
 
+    manager.createTimeseries(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.tree.sg0.s1"),
+            TSDataType.INT32,
+            TSEncoding.PLAIN,
+            CompressionType.GZIP,
+            null,
+            null,
+            null,
+            null));
+
+    manager.setSchemaTemplate(setSchemaTemplatePlan2);
+    manager.unsetSchemaTemplate(new UnsetTemplatePlan("root.tree.sg0", "treeTemplate"));
     try {
       manager.setSchemaTemplate(setTemplatePlan);
       fail();
     } catch (MetadataException e) {
       assertEquals(
-          "Schema name s11 in template has conflict with node's child root.sg1.d1.s11",
+          "Node name s11 in template has conflict with node's child root.sg1.d1.s11",
+          e.getMessage());
+    }
+
+    manager.createTimeseries(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.tree.sg0.GPS.speed"),
+            TSDataType.INT32,
+            TSEncoding.PLAIN,
+            CompressionType.GZIP,
+            null,
+            null,
+            null,
+            null));
+
+    try {
+      manager.setSchemaTemplate(setSchemaTemplatePlan2);
+      fail();
+    } catch (MetadataException e) {
+      assertEquals(
+          "Node name GPS in template has conflict with node's child root.tree.sg0.GPS",
           e.getMessage());
     }
 
@@ -1302,24 +1378,39 @@ public class MManagerBasicTest {
   public void testShowTimeseriesWithTemplate() {
     List<List<String>> measurementList = new ArrayList<>();
     measurementList.add(Collections.singletonList("s0"));
-    measurementList.add(Collections.singletonList("s1"));
+    List<String> measurements = new ArrayList<>();
+    for (int i = 1; i <= 3; i++) {
+      measurements.add("vector.s" + i);
+    }
+    measurementList.add(measurements);
 
     List<List<TSDataType>> dataTypeList = new ArrayList<>();
     dataTypeList.add(Collections.singletonList(TSDataType.INT32));
-    dataTypeList.add(Collections.singletonList(TSDataType.INT32));
+    List<TSDataType> dataTypes = new ArrayList<>();
+    dataTypes.add(TSDataType.INT32);
+    dataTypes.add(TSDataType.FLOAT);
+    dataTypes.add(TSDataType.INT32);
+    dataTypeList.add(dataTypes);
 
     List<List<TSEncoding>> encodingList = new ArrayList<>();
     encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
+    List<TSEncoding> encodings = new ArrayList<>();
+    for (int i = 1; i <= 3; i++) {
+      encodings.add(TSEncoding.RLE);
+    }
+    encodingList.add(encodings);
 
     List<List<CompressionType>> compressionTypes = new ArrayList<>();
-    for (int i = 0; i < 2; i++) {
-      compressionTypes.add(Collections.singletonList(compressionType));
+    compressionTypes.add(Collections.singletonList(CompressionType.SNAPPY));
+    List<CompressionType> compressorList = new ArrayList<>();
+    for (int i = 0; i <= 2; i++) {
+      compressorList.add(compressionType);
     }
+    compressionTypes.add(compressorList);
 
     List<String> schemaNames = new ArrayList<>();
     schemaNames.add("s0");
-    schemaNames.add("s1");
+    schemaNames.add("vector");
 
     CreateTemplatePlan plan =
         new CreateTemplatePlan(
@@ -1329,14 +1420,37 @@ public class MManagerBasicTest {
             dataTypeList,
             encodingList,
             compressionTypes);
+    CreateTemplatePlan treePlan = getTreeTemplatePlan();
     MManager manager = IoTDB.metaManager;
     try {
       manager.createSchemaTemplate(plan);
+      manager.createSchemaTemplate(treePlan);
 
       // set device template
-      SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.laptop.d1");
-      manager.setSchemaTemplate(setTemplatePlan);
+
+      SetTemplatePlan setSchemaTemplatePlan = new SetTemplatePlan("template1", "root.laptop.d1");
+      SetTemplatePlan setSchemaTemplatePlan1 = new SetTemplatePlan("treeTemplate", "root.tree.d0");
+      manager.setSchemaTemplate(setSchemaTemplatePlan);
+      manager.setSchemaTemplate(setSchemaTemplatePlan1);
       manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.laptop.d1")));
+      manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.tree.d0")));
+
+      // show timeseries root.tree.d0
+      ShowTimeSeriesPlan showTreeTSPlan =
+          new ShowTimeSeriesPlan(
+              new PartialPath("root.tree.d0.**"), false, null, null, 0, 0, false);
+      List<ShowTimeSeriesResult> treeShowResult =
+          manager.showTimeseries(showTreeTSPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+      assertEquals(4, treeShowResult.size());
+      Set<String> checkSet = new HashSet<>();
+      checkSet.add("root.tree.d0.d1.s1");
+      checkSet.add("root.tree.d0.s2");
+      checkSet.add("root.tree.d0.GPS.x");
+      checkSet.add("root.tree.d0.GPS.y");
+      for (ShowTimeSeriesResult res : treeShowResult) {
+        checkSet.remove(res.getName());
+      }
+      assertTrue(checkSet.isEmpty());
 
       // show timeseries root.laptop.d1.s0
       ShowTimeSeriesPlan showTimeSeriesPlan =
@@ -1349,15 +1463,26 @@ public class MManagerBasicTest {
 
       // show timeseries root.laptop.d1.(s1,s2,s3)
       showTimeSeriesPlan =
-          new ShowTimeSeriesPlan(new PartialPath("root.**"), false, null, null, 0, 0, false);
+          new ShowTimeSeriesPlan(
+              new PartialPath("root.laptop.d1.vector.s1"), false, null, null, 0, 0, false);
       result = manager.showTimeseries(showTimeSeriesPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
-      assertEquals(2, result.size());
+
+      assertEquals(1, result.size());
+      assertEquals("root.laptop.d1.vector.s1", result.get(0).getName());
+
+      // show timeseries root.laptop.d1.(s1,s2,s3)
+      showTimeSeriesPlan =
+          new ShowTimeSeriesPlan(new PartialPath("root.laptop.**"), false, null, null, 0, 0, false);
+      result = manager.showTimeseries(showTimeSeriesPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+      assertEquals(4, result.size());
       Set<String> set = new HashSet<>();
+      for (int i = 1; i < result.size(); i++) {
+        set.add("root.laptop.d1.vector.s" + i);
+      }
       set.add("root.laptop.d1.s0");
-      set.add("root.laptop.d1.s1");
 
-      for (ShowTimeSeriesResult showTimeSeriesResult : result) {
-        set.remove(showTimeSeriesResult.getName());
+      for (int i = 0; i < result.size(); i++) {
+        set.remove(result.get(i).getName());
       }
 
       assertTrue(set.isEmpty());
@@ -1380,7 +1505,25 @@ public class MManagerBasicTest {
   }
 
   @Test
-  public void testCountTimeseriesWithTemplate() {
+  public void minimumTestForWildcardInTemplate() throws MetadataException {
+    MManager manager = IoTDB.metaManager;
+    CreateTemplatePlan treePlan = getTreeTemplatePlan();
+    manager.createSchemaTemplate(treePlan);
+
+    // set device template
+    SetTemplatePlan setSchemaTemplatePlan1 = new SetTemplatePlan("treeTemplate", "root.tree.d0");
+    manager.setSchemaTemplate(setSchemaTemplatePlan1);
+    manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.tree.d0")));
+
+    ShowTimeSeriesPlan showTimeSeriesPlan =
+        new ShowTimeSeriesPlan(new PartialPath("root.tree.**.s1"), false, null, null, 0, 0, false);
+    List<ShowTimeSeriesResult> result =
+        manager.showTimeseries(showTimeSeriesPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+    assertEquals(1, result.size());
+  }
+
+  @Test
+  public void testCountTimeseriesWithTemplate() throws IOException {
     List<List<String>> measurementList = new ArrayList<>();
     measurementList.add(Collections.singletonList("s0"));
     measurementList.add(Collections.singletonList("s1"));
@@ -1413,10 +1556,12 @@ public class MManagerBasicTest {
     MManager manager = IoTDB.metaManager;
     try {
       manager.createSchemaTemplate(plan);
+      manager.createSchemaTemplate(getTreeTemplatePlan());
 
       // set device template
-      SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.laptop.d1");
-      manager.setSchemaTemplate(setTemplatePlan);
+      SetTemplatePlan setSchemaTemplatePlan = new SetTemplatePlan("template1", "root.laptop.d1");
+      manager.setSchemaTemplate(setSchemaTemplatePlan);
+      manager.setSchemaTemplate(new SetTemplatePlan("treeTemplate", "root.tree.d0"));
       manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.laptop.d1")));
 
       manager.createTimeseries(
@@ -1426,9 +1571,14 @@ public class MManagerBasicTest {
           CompressionType.GZIP,
           null);
 
-      setTemplatePlan = new SetTemplatePlan("template1", "root.computer");
+      SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.computer");
       manager.setSchemaTemplate(setTemplatePlan);
       manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.computer.d1")));
+      manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.tree.d0")));
+      manager.getDeviceNodeWithAutoCreate(new PartialPath("root.tree.d0.v0"));
+      manager.getDeviceNodeWithAutoCreate(new PartialPath("root.tree.d0.v1"));
+      manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.tree.d0.v0")));
+      manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.tree.d0.v1")));
 
       Assert.assertEquals(2, manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1.**")));
       Assert.assertEquals(1, manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1.s1")));
@@ -1436,7 +1586,8 @@ public class MManagerBasicTest {
       Assert.assertEquals(1, manager.getAllTimeseriesCount(new PartialPath("root.computer.d1.s2")));
       Assert.assertEquals(3, manager.getAllTimeseriesCount(new PartialPath("root.computer.d1.**")));
       Assert.assertEquals(3, manager.getAllTimeseriesCount(new PartialPath("root.computer.**")));
-      Assert.assertEquals(5, manager.getAllTimeseriesCount(new PartialPath("root.**")));
+      Assert.assertEquals(12, manager.getAllTimeseriesCount(new PartialPath("root.tree.**")));
+      Assert.assertEquals(17, manager.getAllTimeseriesCount(new PartialPath("root.**")));
 
     } catch (MetadataException e) {
       e.printStackTrace();
@@ -1479,10 +1630,13 @@ public class MManagerBasicTest {
 
     try {
       manager.createSchemaTemplate(plan);
+      manager.createSchemaTemplate(getTreeTemplatePlan());
       // set device template
-      SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.laptop.d1");
-      manager.setSchemaTemplate(setTemplatePlan);
+      SetTemplatePlan setSchemaTemplatePlan = new SetTemplatePlan("template1", "root.laptop.d1");
+      manager.setSchemaTemplate(setSchemaTemplatePlan);
+      manager.setSchemaTemplate(new SetTemplatePlan("treeTemplate", "root.tree.d0"));
       manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.laptop.d1")));
+      manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.tree.d0")));
 
       manager.createTimeseries(
           new PartialPath("root.laptop.d2.s1"),
@@ -1495,6 +1649,8 @@ public class MManagerBasicTest {
       Assert.assertEquals(1, manager.getDevicesNum(new PartialPath("root.laptop.d2")));
       Assert.assertEquals(2, manager.getDevicesNum(new PartialPath("root.laptop.*")));
       Assert.assertEquals(2, manager.getDevicesNum(new PartialPath("root.laptop.**")));
+      Assert.assertEquals(3, manager.getDevicesNum(new PartialPath("root.tree.**")));
+      Assert.assertEquals(5, manager.getDevicesNum(new PartialPath("root.**")));
 
       manager.createTimeseries(
           new PartialPath("root.laptop.d1.a.s3"),
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
index 9d82894..8400f6d 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
@@ -131,7 +131,7 @@ public class TemplateTest {
       CompressionType[] compressionTypes = {CompressionType.SNAPPY, CompressionType.SNAPPY};
       template.addAlignedMeasurements(alignedMeasurements, dataTypes, encodings, compressionTypes);
 
-      assertEquals("[GPS, to.be.prefix]", template.getAllAlignedPrefix().toString());
+      assertEquals("[to.be.prefix, GPS]", template.getAllAlignedPrefix().toString());
       assertEquals("[s1, s2]", template.getAlignedMeasurements("to.be.prefix").toString());
 
       template.deleteAlignedPrefix("to.be.prefix");
@@ -184,8 +184,6 @@ public class TemplateTest {
      * Construct a template like: create schema template treeTemplate ( (d1.s1 INT32 GORILLA
      * SNAPPY), (s2 INT32 GORILLA SNAPPY), (GPS.x FLOAT RLE SNAPPY), (GPS.y FLOAT RLE SNAPPY), )with
      * aligned (GPS)
-     *
-     * <p>Check aligned path whether with same prefix? Construct tree
      */
     List<List<String>> measurementList = new ArrayList<>();
     measurementList.add(Collections.singletonList("d1.s1"));
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowPlanTest.java
index 158367b..5677ad2 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowPlanTest.java
@@ -97,7 +97,7 @@ public class InsertRowPlanTest {
     Assert.assertEquals("[s1, s2, s3]", Arrays.toString(vectorRowPlan.getMeasurementMNodes()));
 
     QueryPlan queryPlan =
-        (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1.vector");
+        (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1.GPS");
     QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
     Assert.assertEquals(1, dataSet.getPaths().size());
     while (dataSet.hasNext()) {
@@ -186,6 +186,79 @@ public class InsertRowPlanTest {
     Assert.assertEquals(plan1, plan2);
   }
 
+  @Test
+  public void testInsertRowPlanWithSchemaTemplateFormer()
+      throws QueryProcessException, MetadataException, InterruptedException,
+          QueryFilterOptimizationException, StorageEngineException, IOException {
+    List<List<String>> measurementList = new ArrayList<>();
+    List<String> v1 = new ArrayList<>();
+    v1.add("GPS.s1");
+    v1.add("GPS.s2");
+    v1.add("GPS.s3");
+    measurementList.add(v1);
+    List<String> v2 = new ArrayList<>();
+    v2.add("GPS2.s4");
+    v2.add("GPS2.s5");
+    measurementList.add(v2);
+    measurementList.add(Collections.singletonList("s6"));
+
+    List<List<TSDataType>> dataTypesList = new ArrayList<>();
+    List<TSDataType> d1 = new ArrayList<>();
+    d1.add(TSDataType.DOUBLE);
+    d1.add(TSDataType.FLOAT);
+    d1.add(TSDataType.INT64);
+    dataTypesList.add(d1);
+    List<TSDataType> d2 = new ArrayList<>();
+    d2.add(TSDataType.INT32);
+    d2.add(TSDataType.BOOLEAN);
+    dataTypesList.add(d2);
+    dataTypesList.add(Collections.singletonList(TSDataType.TEXT));
+
+    List<List<TSEncoding>> encodingList = new ArrayList<>();
+    List<TSEncoding> e1 = new ArrayList<>();
+    e1.add(TSEncoding.PLAIN);
+    e1.add(TSEncoding.PLAIN);
+    e1.add(TSEncoding.PLAIN);
+    encodingList.add(e1);
+    List<TSEncoding> e2 = new ArrayList<>();
+    e2.add(TSEncoding.PLAIN);
+    e2.add(TSEncoding.PLAIN);
+    encodingList.add(e2);
+    encodingList.add(Collections.singletonList(TSEncoding.PLAIN));
+
+    List<List<CompressionType>> compressionTypes = new ArrayList<>();
+    for (int i = 0; i < 3; i++) {
+      List<CompressionType> compressorList = new ArrayList<>();
+      for (int j = 0; j < 3; j++) {
+        compressorList.add(CompressionType.SNAPPY);
+      }
+      compressionTypes.add(compressorList);
+    }
+
+    CreateTemplatePlan plan =
+        new CreateTemplatePlan(
+            "template1", measurementList, dataTypesList, encodingList, compressionTypes);
+
+    IoTDB.metaManager.createSchemaTemplate(plan);
+    IoTDB.metaManager.setSchemaTemplate(new SetTemplatePlan("template1", "root.isp.d1"));
+
+    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(false);
+
+    InsertRowPlan rowPlan = getInsertAlignedRowPlan();
+
+    PlanExecutor executor = new PlanExecutor();
+    executor.insert(rowPlan);
+
+    QueryPlan queryPlan =
+        (QueryPlan) processor.parseSQLToPhysicalPlan("select s1 from root.isp.d1.GPS");
+    QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+    Assert.assertEquals(1, dataSet.getPaths().size());
+    while (dataSet.hasNext()) {
+      RowRecord record = dataSet.next();
+      Assert.assertEquals(3, record.getFields().size());
+    }
+  }
+
   private InsertRowPlan getInsertRowPlan() throws IllegalPathException {
     long time = 110L;
     TSDataType[] dataTypes =
@@ -225,7 +298,7 @@ public class InsertRowPlanTest {
     columns[2] = 10000 + "";
 
     return new InsertRowPlan(
-        new PartialPath("root.isp.d1.vector"),
+        new PartialPath("root.isp.d1.GPS"),
         time,
         new String[] {"s1", "s2", "s3"},
         dataTypes,
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
index df648fe..75213a2 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
@@ -29,9 +29,14 @@ import org.apache.iotdb.db.qp.executor.PlanExecutor;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan.PhysicalPlanType;
 import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+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.apache.iotdb.tsfile.read.common.Field;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
@@ -47,6 +52,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 public class InsertTabletPlanTest {
@@ -282,6 +288,137 @@ public class InsertTabletPlanTest {
     Assert.assertEquals(plan1, plan2);
   }
 
+  @Test
+  public void testInsertTabletPlanWithSchemaTemplateAndAutoCreateSchema()
+      throws QueryProcessException, MetadataException, InterruptedException,
+          QueryFilterOptimizationException, StorageEngineException, IOException {
+    CreateTemplatePlan plan = getCreateTemplatePlan();
+
+    IoTDB.metaManager.createSchemaTemplate(plan);
+    IoTDB.metaManager.setSchemaTemplate(new SetTemplatePlan("template1", "root.isp.d1"));
+    InsertTabletPlan tabletPlan = getAlignedInsertTabletPlan();
+
+    PlanExecutor executor = new PlanExecutor();
+    executor.insertTablet(tabletPlan);
+
+    QueryPlan queryPlan =
+        (QueryPlan) processor.parseSQLToPhysicalPlan("select ** from root.isp.d1");
+    QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+    Assert.assertEquals(3, dataSet.getPaths().size());
+    while (dataSet.hasNext()) {
+      RowRecord record = dataSet.next();
+      Assert.assertEquals(6, record.getFields().size());
+    }
+
+    // test recover
+    EnvironmentUtils.stopDaemon();
+    IoTDB.metaManager.clear();
+    // wait for close
+    try {
+      Thread.sleep(1000);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+      Thread.currentThread().interrupt();
+    }
+    EnvironmentUtils.activeDaemon();
+
+    queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select ** from root.isp.d1");
+    dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+    Assert.assertEquals(3, dataSet.getPaths().size());
+    while (dataSet.hasNext()) {
+      RowRecord record = dataSet.next();
+      Assert.assertEquals(6, record.getFields().size());
+    }
+  }
+
+  @Test
+  public void testInsertTabletPlanWithSchemaTemplate()
+      throws QueryProcessException, MetadataException, InterruptedException,
+          QueryFilterOptimizationException, StorageEngineException, IOException {
+    CreateTemplatePlan plan = getCreateTemplatePlan();
+
+    IoTDB.metaManager.createSchemaTemplate(plan);
+    IoTDB.metaManager.setSchemaTemplate(new SetTemplatePlan("template1", "root.isp.d1"));
+
+    InsertTabletPlan tabletPlan = getAlignedInsertTabletPlan();
+
+    PlanExecutor executor = new PlanExecutor();
+
+    // nothing can be found when we not insert data
+    QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select ** from root.isp");
+    QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+    Assert.assertEquals(0, dataSet.getPaths().size());
+
+    executor.insertTablet(tabletPlan);
+
+    queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select ** from root.isp");
+    dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+    Assert.assertEquals(3, dataSet.getPaths().size());
+    while (dataSet.hasNext()) {
+      RowRecord record = dataSet.next();
+      Assert.assertEquals(6, record.getFields().size());
+    }
+  }
+
+  private CreateTemplatePlan getCreateTemplatePlan() {
+    List<List<String>> measurementList = new ArrayList<>();
+    List<String> v1 = new ArrayList<>();
+    v1.add("vector.s1");
+    v1.add("vector.s2");
+    v1.add("vector.s3");
+    measurementList.add(v1);
+    List<String> v2 = new ArrayList<>();
+    v2.add("vector2.s4");
+    v2.add("vector2.s5");
+    measurementList.add(v2);
+    measurementList.add(Collections.singletonList("vector3.s6"));
+
+    List<List<TSDataType>> dataTypesList = new ArrayList<>();
+    List<TSDataType> d1 = new ArrayList<>();
+    d1.add(TSDataType.DOUBLE);
+    d1.add(TSDataType.FLOAT);
+    d1.add(TSDataType.INT64);
+    dataTypesList.add(d1);
+    List<TSDataType> d2 = new ArrayList<>();
+    d2.add(TSDataType.INT32);
+    d2.add(TSDataType.BOOLEAN);
+    dataTypesList.add(d2);
+    dataTypesList.add(Collections.singletonList(TSDataType.TEXT));
+
+    List<List<TSEncoding>> encodingList = new ArrayList<>();
+    List<TSEncoding> e1 = new ArrayList<>();
+    e1.add(TSEncoding.PLAIN);
+    e1.add(TSEncoding.PLAIN);
+    e1.add(TSEncoding.PLAIN);
+    encodingList.add(e1);
+    List<TSEncoding> e2 = new ArrayList<>();
+    e2.add(TSEncoding.PLAIN);
+    e2.add(TSEncoding.PLAIN);
+    encodingList.add(e2);
+    encodingList.add(Collections.singletonList(TSEncoding.PLAIN));
+
+    List<List<CompressionType>> compressionTypes = new ArrayList<>();
+    List<CompressionType> c1 = new ArrayList<>();
+    for (int i = 0; i < 3; i++) {
+      c1.add(CompressionType.SNAPPY);
+    }
+    List<CompressionType> c2 = new ArrayList<>();
+    for (int i = 0; i < 2; i++) {
+      c2.add(CompressionType.SNAPPY);
+    }
+    compressionTypes.add(c1);
+    compressionTypes.add(c2);
+    compressionTypes.add(Collections.singletonList(CompressionType.SNAPPY));
+
+    List<String> schemaNames = new ArrayList<>();
+    schemaNames.add("vector");
+    schemaNames.add("vector2");
+    schemaNames.add("s6");
+
+    return new CreateTemplatePlan(
+        "template1", schemaNames, measurementList, dataTypesList, encodingList, compressionTypes);
+  }
+
   private InsertTabletPlan getAlignedInsertTabletPlan() throws IllegalPathException {
     long[] times = new long[] {110L, 111L, 112L, 113L};
     List<Integer> dataTypes = new ArrayList<>();
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 b9b3928..ebc1354 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
@@ -57,6 +57,7 @@ public enum TSStatusCode {
   NO_TEMPLATE_ON_MNODE(324),
   DIFFERENT_TEMPLATE(325),
   TEMPLATE_IS_IN_USE(326),
+  TEMPLATE_IMCOMPATIBLE(327),
 
   EXECUTE_STATEMENT_ERROR(400),
   SQL_PARSE_ERROR(401),
diff --git a/session/src/main/java/org/apache/iotdb/session/Session.java b/session/src/main/java/org/apache/iotdb/session/Session.java
index fca13f4..afe7f0d 100644
--- a/session/src/main/java/org/apache/iotdb/session/Session.java
+++ b/session/src/main/java/org/apache/iotdb/session/Session.java
@@ -1989,57 +1989,13 @@ public class Session {
   }
 
   /**
-   * @param name template name
-   * @param schemaNames list of schema names, if this measurement is vector, name it. if this
-   *     measurement is not vector, keep this name as same as measurement's name
-   * @param measurements List of measurements, if it is a single measurement, just put it's name
-   *     into a list and add to measurements if it is a vector measurement, put all measurements of
-   *     the vector into a list and add to measurements
-   * @param dataTypes List of datatypes, if it is a single measurement, just put it's type into a
-   *     list and add to dataTypes if it is a vector measurement, put all types of the vector into a
-   *     list and add to dataTypes
-   * @param encodings List of encodings, if it is a single measurement, just put it's encoding into
-   *     a list and add to encodings if it is a vector measurement, put all encodings of the vector
-   *     into a list and add to encodings
-   * @param compressors List of compressors
-   * @throws IoTDBConnectionException
-   * @throws StatementExecutionException
-   */
-  public void createSchemaTemplate(
-      String name,
-      List<String> schemaNames,
-      List<List<String>> measurements,
-      List<List<TSDataType>> dataTypes,
-      List<List<TSEncoding>> encodings,
-      List<List<CompressionType>> compressors)
-      throws IoTDBConnectionException, StatementExecutionException {
-    TSCreateSchemaTemplateReq request =
-        getTSCreateSchemaTemplateReq(
-            name, schemaNames, measurements, dataTypes, encodings, compressors);
-    defaultSessionConnection.createSchemaTemplate(request);
-  }
-
-  /** Create schema template without schemaNames. */
-  public void createSchemaTemplate(
-      String name,
-      List<List<String>> measurements,
-      List<List<TSDataType>> dataTypes,
-      List<List<TSEncoding>> encodings,
-      List<List<CompressionType>> compressors)
-      throws IoTDBConnectionException, StatementExecutionException {
-    TSCreateSchemaTemplateReq request =
-        getTSCreateSchemaTemplateReq(name, null, measurements, dataTypes, encodings, compressors);
-    defaultSessionConnection.createSchemaTemplate(request);
-  }
-
-  /**
    * Construct Template at session and create it at server.
    *
    * @see Template
    */
   public void createSchemaTemplate(Template template)
       throws IOException, IoTDBConnectionException, StatementExecutionException {
-    TSCreateSchemaTemplateReq req = getTSCreateSchemaTemplateReq();
+    TSCreateSchemaTemplateReq req = new TSCreateSchemaTemplateReq();
     req.setName(template.getName());
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     template.serialize(baos);
@@ -2218,56 +2174,6 @@ public class Session {
     return request;
   }
 
-  private TSCreateSchemaTemplateReq getTSCreateSchemaTemplateReq(
-      String name,
-      List<String> schemaNames,
-      List<List<String>> measurements,
-      List<List<TSDataType>> dataTypes,
-      List<List<TSEncoding>> encodings,
-      List<List<CompressionType>> compressors) {
-    TSCreateSchemaTemplateReq request = new TSCreateSchemaTemplateReq();
-    request.setName(name);
-    if (schemaNames != null) {
-      request.setSchemaNames(schemaNames);
-    }
-    request.setMeasurements(measurements);
-
-    List<List<Integer>> requestType = new ArrayList<>();
-    for (List<TSDataType> typesList : dataTypes) {
-      requestType.add(typesList.stream().map(TSDataType::ordinal).collect(Collectors.toList()));
-    }
-    request.setDataTypes(requestType);
-
-    List<List<Integer>> requestEncoding = new ArrayList<>();
-    for (List<TSEncoding> encodingList : encodings) {
-      requestEncoding.add(
-          encodingList.stream().map(TSEncoding::ordinal).collect(Collectors.toList()));
-    }
-    request.setEncodings(requestEncoding);
-
-    List<List<Integer>> requestCompressor = new ArrayList<>();
-    for (List<CompressionType> compressorList : compressors) {
-      requestCompressor.add(
-          compressorList.stream().map(CompressionType::ordinal).collect(Collectors.toList()));
-    }
-    request.setCompressors(requestCompressor);
-    return request;
-  }
-
-  private TSCreateSchemaTemplateReq getTSCreateSchemaTemplateReq() {
-    // Return a empty Request to construct by serialized binary
-    TSCreateSchemaTemplateReq request = new TSCreateSchemaTemplateReq();
-    List<String> emptyList = new ArrayList<>();
-    List<List<String>> emptyStringList = new ArrayList<>();
-    List<List<Integer>> emptyIntegerList = new ArrayList<>();
-    request.setMeasurements(emptyStringList);
-    request.setSchemaNames(emptyList);
-    request.setDataTypes(emptyIntegerList);
-    request.setEncodings(emptyIntegerList);
-    request.setCompressors(emptyIntegerList);
-    return request;
-  }
-
   private TSUnsetSchemaTemplateReq getTSUnsetSchemaTemplateReq(
       String prefixPath, String templateName) {
     TSUnsetSchemaTemplateReq request = new TSUnsetSchemaTemplateReq();
diff --git a/session/src/main/java/org/apache/iotdb/session/pool/SessionPool.java b/session/src/main/java/org/apache/iotdb/session/pool/SessionPool.java
index 2469146..a8da9ec 100644
--- a/session/src/main/java/org/apache/iotdb/session/pool/SessionPool.java
+++ b/session/src/main/java/org/apache/iotdb/session/pool/SessionPool.java
@@ -1356,57 +1356,6 @@ public class SessionPool {
     return false;
   }
 
-  public void createSchemaTemplate(
-      String name,
-      List<String> schemaNames,
-      List<List<String>> measurements,
-      List<List<TSDataType>> dataTypes,
-      List<List<TSEncoding>> encodings,
-      List<List<CompressionType>> compressors)
-      throws IoTDBConnectionException, StatementExecutionException {
-    for (int i = 0; i < RETRY; i++) {
-      Session session = getSession();
-      try {
-        session.createSchemaTemplate(
-            name, schemaNames, measurements, dataTypes, encodings, compressors);
-        putBack(session);
-        return;
-      } catch (IoTDBConnectionException e) {
-        // TException means the connection is broken, remove it and get a new one.
-        logger.warn("createSchemaTemplate failed", e);
-        cleanSessionAndMayThrowConnectionException(session, i, e);
-      } catch (StatementExecutionException | RuntimeException e) {
-        putBack(session);
-        throw e;
-      }
-    }
-  }
-
-  /** New interface for no schemaNames */
-  public void createSchemaTemplate(
-      String name,
-      List<List<String>> measurements,
-      List<List<TSDataType>> dataTypes,
-      List<List<TSEncoding>> encodings,
-      List<List<CompressionType>> compressors)
-      throws IoTDBConnectionException, StatementExecutionException {
-    for (int i = 0; i < RETRY; i++) {
-      Session session = getSession();
-      try {
-        session.createSchemaTemplate(name, measurements, dataTypes, encodings, compressors);
-        putBack(session);
-        return;
-      } catch (IoTDBConnectionException e) {
-        // TException means the connection is broken, remove it and get a new one.
-        logger.warn("createSchemaTemplate failed", e);
-        cleanSessionAndMayThrowConnectionException(session, i, e);
-      } catch (StatementExecutionException | RuntimeException e) {
-        putBack(session);
-        throw e;
-      }
-    }
-  }
-
   /**
    * Construct Template at session and create it at server.
    *
diff --git a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
index 546e2b7..57f4520 100644
--- a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
+++ b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
@@ -30,6 +30,10 @@ import org.apache.iotdb.rpc.BatchExecutionException;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.session.template.InternalNode;
+import org.apache.iotdb.session.template.MeasurementNode;
+import org.apache.iotdb.session.template.Template;
+import org.apache.iotdb.session.template.TemplateNode;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -50,6 +54,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -1127,6 +1133,200 @@ public class IoTDBSessionSimpleIT {
     session.close();
   }
 
+  @Test
+  public void testInsertDeleteWithTemplate()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    session = new Session("127.0.0.1", 6667, "root", "root");
+    session.open();
+
+    initTreeTemplate("root.sg.loc1");
+    List<String> measurements = new ArrayList<>();
+    List<String> values = new ArrayList<>();
+    String deviceId = "root.sg.loc1.sector.GPS";
+    Set<String> checkSet = new HashSet<>();
+    SessionDataSet dataSet;
+
+    // insert record set using template
+
+    measurements.add("x");
+    measurements.add("y");
+    values.add("1.0");
+    values.add("2.0");
+
+    checkSet.add("root.sg.loc1.sector.GPS.x");
+    checkSet.add("root.sg.loc1.sector.GPS.y");
+    checkSet.add("root.sg.loc1.sector.y");
+    checkSet.add("root.sg.loc1.sector.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.y");
+    checkSet.add("root.sg.loc1.sector.vehicle.GPS.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.GPS.y");
+
+    session.insertRecord(deviceId, 1L, measurements, values);
+    dataSet = session.executeQueryStatement("show timeseries");
+    while (dataSet.hasNext()) {
+      checkSet.remove(dataSet.next().getFields().get(0).toString());
+    }
+    assertTrue(checkSet.isEmpty());
+
+    // insert aligned under unaligned node
+    try {
+      session.insertAlignedRecord("root.sg.loc1.sector.GPS", 3L, measurements, values);
+    } catch (StatementExecutionException e) {
+      assertEquals(
+          "303: Timeseries under path [root.sg.loc1.sector.GPS] is not aligned , please set InsertPlan.isAligned() = false",
+          e.getMessage());
+    }
+
+    // insert overlap unmatched series
+    measurements.set(1, "speed");
+    try {
+      session.insertRecord(deviceId, 5L, measurements, values);
+      fail();
+    } catch (StatementExecutionException e) {
+      assertEquals(
+          "327: Path [root.sg.loc1.sector.GPS.speed] overlaps with [treeTemplate] on [GPS]",
+          e.getMessage());
+    }
+
+    // insert tablets
+    List<IMeasurementSchema> schemaList = new ArrayList<>();
+    schemaList.add(new UnaryMeasurementSchema("x", TSDataType.FLOAT));
+    schemaList.add(new UnaryMeasurementSchema("y", TSDataType.FLOAT));
+    Tablet tablet = new Tablet("root.sg.loc1.sector", schemaList);
+    tablet.setAligned(true);
+
+    long timestamp = System.currentTimeMillis();
+
+    for (long row = 0; row < 10; row++) {
+      int rowIndex = tablet.rowSize++;
+      tablet.addTimestamp(rowIndex, timestamp);
+      tablet.addValue(
+          schemaList.get(0).getMeasurementId(), rowIndex, new SecureRandom().nextFloat());
+      tablet.addValue(
+          schemaList.get(1).getMeasurementId(), rowIndex, new SecureRandom().nextFloat());
+      timestamp++;
+    }
+
+    if (tablet.rowSize != 0) {
+      session.insertAlignedTablet(tablet);
+      tablet.reset();
+    }
+
+    dataSet = session.executeQueryStatement("select count(*) from root");
+
+    while (dataSet.hasNext()) {
+      RowRecord rowRecord = dataSet.next();
+      Assert.assertEquals(10L, rowRecord.getFields().get(0).getLongV());
+      Assert.assertEquals(10L, rowRecord.getFields().get(1).getLongV());
+    }
+
+    // delete series inside template
+    try {
+      session.deleteTimeseries("root.sg.loc1.sector.x");
+      fail();
+    } catch (StatementExecutionException e) {
+      assertEquals(
+          "Cannot delete a timeseries inside a template: root.sg.loc1.sector.x;", e.getMessage());
+    }
+
+    session.close();
+  }
+
+  @Test
+  public void testCountWithTemplate()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    session = new Session("127.0.0.1", 6667, "root", "root");
+    session.open();
+
+    initTreeTemplate("root.sg.loc");
+
+    List<String> measurements = new ArrayList<>();
+    List<String> values = new ArrayList<>();
+    Set<String> checkSet = new HashSet<>();
+    SessionDataSet dataSet;
+
+    // invoke template template
+
+    measurements.add("x");
+    measurements.add("y");
+    values.add("1.0");
+    values.add("2.0");
+
+    session.insertRecord("root.sg.loc.area", 1L, measurements, values);
+    session.insertRecord("root.sg.loc", 1L, measurements, values);
+
+    dataSet = session.executeQueryStatement("show timeseries");
+
+    checkSet.add("root.sg.loc.x");
+    checkSet.add("root.sg.loc.y");
+    checkSet.add("root.sg.loc.GPS.x");
+    checkSet.add("root.sg.loc.GPS.y");
+    checkSet.add("root.sg.loc.vehicle.GPS.x");
+    checkSet.add("root.sg.loc.vehicle.GPS.y");
+    checkSet.add("root.sg.loc.vehicle.x");
+    checkSet.add("root.sg.loc.vehicle.x");
+
+    checkSet.add("root.sg.loc.area.x");
+    checkSet.add("root.sg.loc.area.y");
+    checkSet.add("root.sg.loc.area.GPS.x");
+    checkSet.add("root.sg.loc.area.GPS.y");
+    checkSet.add("root.sg.loc.area.vehicle.GPS.x");
+    checkSet.add("root.sg.loc.area.vehicle.GPS.y");
+    checkSet.add("root.sg.loc.area.vehicle.x");
+    checkSet.add("root.sg.loc.area.vehicle.x");
+
+    while (dataSet.hasNext()) {
+      checkSet.remove(dataSet.next().getFields().get(0).toString());
+    }
+
+    assertTrue(checkSet.isEmpty());
+
+    dataSet = session.executeQueryStatement("show devices");
+
+    checkSet.add("root.sg.loc");
+    checkSet.add("root.sg.loc.GPS");
+    checkSet.add("root.sg.loc.vehicle");
+    checkSet.add("root.sg.loc.vehicle.GPS");
+    checkSet.add("root.sg.loc.area");
+    checkSet.add("root.sg.loc.area.GPS");
+    checkSet.add("root.sg.loc.area.vehicle");
+    checkSet.add("root.sg.loc.area.vehicle.GPS");
+
+    while (dataSet.hasNext()) {
+      checkSet.remove(dataSet.next().getFields().get(0).toString());
+    }
+
+    assertTrue(checkSet.isEmpty());
+  }
+
+  private void initTreeTemplate(String path)
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    Template sessionTemplate = new Template("treeTemplate", true);
+    TemplateNode iNodeGPS = new InternalNode("GPS", false);
+    TemplateNode iNodeV = new InternalNode("vehicle", true);
+    TemplateNode mNodeX =
+        new MeasurementNode("x", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY);
+    TemplateNode mNodeY =
+        new MeasurementNode("y", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY);
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+
+    iNodeGPS.addChild(mNodeX);
+    iNodeGPS.addChild(mNodeY);
+
+    iNodeV.addChild(mNodeX);
+    iNodeV.addChild(mNodeY);
+    iNodeV.addChild(iNodeGPS);
+
+    sessionTemplate.addToTemplate(iNodeGPS);
+    sessionTemplate.addToTemplate(iNodeV);
+    sessionTemplate.addToTemplate(mNodeX);
+    sessionTemplate.addToTemplate(mNodeY);
+
+    session.createSchemaTemplate(sessionTemplate);
+    session.setSchemaTemplate("treeTemplate", path);
+  }
+
   private void checkResult(Session session)
       throws StatementExecutionException, IoTDBConnectionException {
     SessionDataSet dataSet = session.executeQueryStatement("select * from root.sg.d1");
diff --git a/session/src/test/java/org/apache/iotdb/session/SessionTest.java b/session/src/test/java/org/apache/iotdb/session/SessionTest.java
index 159df03..08ab917 100644
--- a/session/src/test/java/org/apache/iotdb/session/SessionTest.java
+++ b/session/src/test/java/org/apache/iotdb/session/SessionTest.java
@@ -45,7 +45,6 @@ import java.time.ZoneId;
 import java.time.ZoneOffset;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertArrayEquals;
@@ -349,52 +348,49 @@ public class SessionTest {
   }
 
   @Test
-  public void createSchemaTemplate() throws IoTDBConnectionException, StatementExecutionException {
+  public void createSchemaTemplate()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
     session = new Session("127.0.0.1", 6667, "root", "root", ZoneId.of("+05:00"));
     session.open();
 
-    List<List<String>> measurementList = new ArrayList<>();
-    measurementList.add(Collections.singletonList("s11"));
-    List<String> measurements = new ArrayList<>();
-    for (int i = 0; i < 10; i++) {
-      measurements.add("s" + i);
-    }
-    measurementList.add(measurements);
+    InternalNode iNodeVector = new InternalNode("vector", true);
 
-    List<List<TSDataType>> dataTypeList = new ArrayList<>();
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-    List<TSDataType> dataTypes = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
-      dataTypes.add(TSDataType.INT64);
+      MeasurementNode mNodei =
+          new MeasurementNode("s" + i, TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+      iNodeVector.addChild(mNodei);
     }
-    dataTypeList.add(dataTypes);
 
-    List<List<TSEncoding>> encodingList = new ArrayList<>();
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    List<TSEncoding> encodings = new ArrayList<>();
-    for (int i = 0; i < 10; i++) {
-      encodings.add(TSEncoding.RLE);
-    }
-    encodingList.add(encodings);
+    MeasurementNode mNode11 =
+        new MeasurementNode("s11", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
 
-    List<List<CompressionType>> compressionTypes = new ArrayList<>();
-    compressionTypes.add(Collections.singletonList(CompressionType.SNAPPY));
-    List<CompressionType> compressionList = new ArrayList<>();
-    for (int i = 0; i < 10; i++) {
-      compressionList.add(CompressionType.SNAPPY);
-    }
-    compressionTypes.add(compressionList);
+    Template template = new Template("template1");
 
-    List<String> schemaNames = new ArrayList<>();
-    schemaNames.add("s11");
-    schemaNames.add("test_vector");
+    template.addToTemplate(mNode11);
+    template.addToTemplate(iNodeVector);
 
-    session.createSchemaTemplate(
-        "template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
+    session.createSchemaTemplate(template);
     session.setSchemaTemplate("template1", "root.sg.1");
   }
 
   @Test
+  public void testCreateEmptyTemplateAndAppend()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    session = new Session("127.0.0.1", 6667, "root", "root", ZoneId.of("+05:00"));
+    session.open();
+
+    List<List<String>> measurements = new ArrayList<>();
+    List<List<TSDataType>> dataTypes = new ArrayList<>();
+    List<List<TSEncoding>> encodings = new ArrayList<>();
+    List<List<CompressionType>> compressors = new ArrayList<>();
+    Template template = new Template("emptyTemplate");
+    session.createSchemaTemplate(template);
+
+    session.addAlignedMeasurementInTemplate(
+        "emptyTemplate", "speed", TSDataType.FLOAT, TSEncoding.GORILLA, CompressionType.SNAPPY);
+  }
+
+  @Test
   public void testBuilder() {
     session =
         new Session.Builder()
@@ -437,36 +433,19 @@ public class SessionTest {
 
   @Test
   public void testUnsetSchemaTemplate()
-      throws IoTDBConnectionException, StatementExecutionException {
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
     session = new Session("127.0.0.1", 6667, "root", "root", ZoneId.of("+05:00"));
     session.open();
 
-    List<List<String>> measurementList = new ArrayList<>();
-    measurementList.add(Collections.singletonList("s1"));
-    measurementList.add(Collections.singletonList("s2"));
-    measurementList.add(Collections.singletonList("s3"));
+    Template template = new Template("template1", false);
 
-    List<List<TSDataType>> dataTypeList = new ArrayList<>();
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
-
-    List<List<TSEncoding>> encodingList = new ArrayList<>();
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-    encodingList.add(Collections.singletonList(TSEncoding.RLE));
-
-    List<List<CompressionType>> compressionTypes = new ArrayList<>();
-    for (int i = 0; i < 3; i++) {
-      compressionTypes.add(Collections.singletonList(CompressionType.SNAPPY));
+    for (int i = 1; i <= 3; i++) {
+      MeasurementNode mNodei =
+          new MeasurementNode("s" + i, TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+      template.addToTemplate(mNodei);
     }
-    List<String> schemaNames = new ArrayList<>();
-    schemaNames.add("s1");
-    schemaNames.add("s2");
-    schemaNames.add("s3");
 
-    session.createSchemaTemplate(
-        "template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
+    session.createSchemaTemplate(template);
 
     // path does not exist test
     try {
diff --git a/session/src/test/java/org/apache/iotdb/session/template/TemplateUT.java b/session/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
index 2cf0289..a244b83 100644
--- a/session/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
+++ b/session/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
@@ -22,7 +22,9 @@ import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.qp.physical.sys.CreateTemplatePlan;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.session.Session;
 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;
@@ -34,11 +36,15 @@ import org.junit.Test;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.time.ZoneId;
 
+import static junit.framework.TestCase.fail;
 import static org.junit.Assert.assertEquals;
 
 public class TemplateUT {
 
+  private Session session;
+
   @Before
   public void setUp() throws Exception {
     System.setProperty(IoTDBConstant.IOTDB_CONF, "src/test/resources/");
@@ -48,11 +54,15 @@ public class TemplateUT {
 
   @After
   public void tearDown() throws Exception {
+    session.close();
     EnvironmentUtils.cleanEnv();
   }
 
   @Test
-  public void testTemplateTree() {
+  public void testTemplateTree()
+      throws IOException, MetadataException, StatementExecutionException, IoTDBConnectionException {
+    session = new Session("127.0.0.1", 6667, "root", "root", ZoneId.of("+05:00"));
+    session.open();
     Template sessionTemplate = new Template("treeTemplate", true);
     TemplateNode iNodeGPS = new InternalNode("GPS", false);
     TemplateNode iNodeV = new InternalNode("vehicle", true);
@@ -62,32 +72,32 @@ public class TemplateUT {
         new MeasurementNode("y", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY);
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
 
-    try {
-      iNodeGPS.addChild(mNodeX);
-      iNodeGPS.addChild(mNodeY);
-      iNodeV.addChild(mNodeX);
-      iNodeV.addChild(mNodeY);
-      iNodeV.addChild(iNodeGPS);
-      sessionTemplate.addToTemplate(iNodeGPS);
-      sessionTemplate.addToTemplate(iNodeV);
-      sessionTemplate.addToTemplate(mNodeX);
-      sessionTemplate.addToTemplate(mNodeY);
+    iNodeGPS.addChild(mNodeX);
+    iNodeGPS.addChild(mNodeY);
+    iNodeV.addChild(mNodeX);
+    iNodeV.addChild(mNodeY);
+    iNodeV.addChild(iNodeGPS);
+    sessionTemplate.addToTemplate(iNodeGPS);
+    sessionTemplate.addToTemplate(iNodeV);
+    sessionTemplate.addToTemplate(mNodeX);
+    sessionTemplate.addToTemplate(mNodeY);
 
-      sessionTemplate.serialize(stream);
-      ByteBuffer buffer = ByteBuffer.wrap(stream.toByteArray());
+    sessionTemplate.serialize(stream);
+    ByteBuffer buffer = ByteBuffer.wrap(stream.toByteArray());
 
-      CreateTemplatePlan plan = CreateTemplatePlan.deserializeFromReq(buffer);
-      assertEquals("treeTemplate", plan.getName());
-      assertEquals(
-          "[[vehicle.GPS.y], [vehicle.GPS.x], [GPS.y], [GPS.x], [y, x], [vehicle.y, vehicle.x]]",
-          plan.getMeasurements().toString());
+    CreateTemplatePlan plan = CreateTemplatePlan.deserializeFromReq(buffer);
+    assertEquals("treeTemplate", plan.getName());
+    assertEquals(
+        "[[vehicle.GPS.y], [vehicle.GPS.x], [GPS.y], [GPS.x], [y, x], [vehicle.y, vehicle.x]]",
+        plan.getMeasurements().toString());
 
+    session.createSchemaTemplate(sessionTemplate);
+    session.setSchemaTemplate("treeTemplate", "root.sg.d0");
+    try {
+      session.setSchemaTemplate("treeTemplate", "root.sg.d0");
+      fail();
     } catch (StatementExecutionException e) {
-      e.printStackTrace();
-    } catch (IOException e) {
-      e.printStackTrace();
-    } catch (MetadataException e) {
-      e.printStackTrace();
+      assertEquals("303: Template already exists on root.sg.d0", e.getMessage());
     }
   }
 }
diff --git a/thrift/src/main/thrift/rpc.thrift b/thrift/src/main/thrift/rpc.thrift
index 7a30ad8..a981a8c 100644
--- a/thrift/src/main/thrift/rpc.thrift
+++ b/thrift/src/main/thrift/rpc.thrift
@@ -362,12 +362,7 @@ struct TSSetSchemaTemplateReq {
 struct TSCreateSchemaTemplateReq {
   1: required i64 sessionId
   2: required string name
-  3: optional list<string> schemaNames
-  4: required list<list<string>> measurements
-  5: required list<list<i32>> dataTypes
-  6: required list<list<i32>> encodings
-  7: required list<list<i32>> compressors
-  8: optional binary serializedTemplate
+  3: required binary serializedTemplate
 }
 
 struct TSAppendSchemaTemplateReq {