You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/11/05 14:40:12 UTC

[GitHub] [iotdb] qiaojialin commented on a change in pull request #4326: [IOTDB-1883] Extension of schema template to tree-structured

qiaojialin commented on a change in pull request #4326:
URL: https://github.com/apache/iotdb/pull/4326#discussion_r743694243



##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -2101,6 +2122,89 @@ public TSStatus createSchemaTemplate(TSCreateSchemaTemplateReq req) throws TExce
     }
   }
 
+  @Override
+  public TSStatus createSchemaTemplateSerialized(TSCreateSchemaTemplateSerializedReq req)

Review comment:
       ```suggestion
     public TSStatus createSchemaTemplate(TSCreateSchemaTemplateReq req)
   ```

##########
File path: thrift/src/main/thrift/rpc.thrift
##########
@@ -366,9 +366,46 @@ struct TSCreateSchemaTemplateReq {
   4: required list<list<string>> measurements
   5: required list<list<i32>> dataTypes
   6: required list<list<i32>> encodings
+  7: required list<list<i32>> compressors
+}
+
+struct TSCreateSchemaTemplateSerializedReq {
+  1: required i64 sessionId
+  2: required string name
+  3: required binary template
+}
+
+struct TSAppendSchemaTemplateReq {
+  1: required i64 sessionId
+  2: required string name
+  3: required bool isAligned
+  4: required list<string> measurements
+  5: required list<i32> dataTypes
+  6: required list<i32> encodings
   7: required list<i32> compressors
 }
 
+struct TSPruneSchemaTemplateReq {
+  1: required i64 sessionId
+  2: required string name
+  3: required string measurement

Review comment:
       ```suggestion
     3: required string path
   ```

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;
+  private boolean shareTime;
+
+  private List<String> measurementsPath;
+  private List<Boolean> alignedList;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+
+  // sync with metadata.Template
+  public enum TemplateQueryType {
+    NULL,
+    COUNT_MEA,
+    IS_MEA,
+    IS_SERIES,
+    SHOW_MEA
+  }
+
+  public Template(String name, boolean isShareTime) {
+    this.name = name;
+    moutedNode = new InternalNode(name, isShareTime);
+    this.shareTime = isShareTime;
+
+    this.measurementsPath = new ArrayList<>();
+    this.alignedList = new ArrayList<>();
+    this.dataTypes = new ArrayList<>();
+    this.encodings = new ArrayList<>();
+    this.compressors = new ArrayList<>();
+  }
+
+  public Template(String name) {
+    this(name, false);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isShareTime() {
+    return shareTime;
+  }
+
+  public void setShareTime(boolean shareTime) {
+    this.shareTime = shareTime;
+  }
+
+  public List<Boolean> getAlignedList() {
+    return this.alignedList;
+  }
+
+  public List<String> getMeasurementsPath() {
+    return this.measurementsPath;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  // region Interface to manipulate Template
+
+  public void addToTemplate(Node child) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(child.getName()))
+      throw new StatementExecutionException("Duplicated child of node in template.");
+    moutedNode.getChildren().put(child.getName(), child);
+  }
+
+  public void deleteFromTemplate(String name) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(name)) moutedNode.getChildren().remove(name);
+    else throw new StatementExecutionException("It is not a direct child of the template: " + name);
+  }
+
+  public void traverseTemplate() throws StatementExecutionException {
+    List<String> result = new ArrayList<>();
+    List<String> nowPaths = new ArrayList<>();
+    Deque<Integer> childSize = new ArrayDeque<>();
+    Deque<Boolean> alignStack = new ArrayDeque<>();
+    Deque<Node> stack = new ArrayDeque<>();
+    this.measurementsPath.clear();
+    this.alignedList.clear();
+    this.dataTypes.clear();
+    this.encodings.clear();
+    this.compressors.clear();
+
+    Node cur;
+    MeasurementNode mNode;
+    int count;
+    boolean isAligned;
+
+    for (Node child : moutedNode.getChildren().values()) stack.push(child);
+    count = moutedNode.getChildren().size();
+    isAligned = moutedNode.isShareTime();
+
+    while (stack.size() != 0) {
+      if (stack.size() > 64)
+        throw new StatementExecutionException(
+            "Too long prefix in template, please check iterations.");
+
+      cur = stack.pop();
+      count--;
+
+      if ((count < 0) && (childSize.size() > 0)) {
+        isAligned = alignStack.pop();
+        count = childSize.pop();
+        count--;
+        nowPaths.remove(nowPaths.size() - 1);
+      }
+
+      if (cur != moutedNode) nowPaths.add(cur.getName());
+
+      if (cur.getChildren() != null) {
+        childSize.push(count);
+        alignStack.push(isAligned);
+
+        for (Node child : cur.getChildren().values()) stack.push(child);
+        count = cur.getChildren().size();
+        isAligned = cur.isShareTime();
+      }
+
+      if (cur.isMeasurement()) {
+        mNode = (MeasurementNode) cur;
+        result.add(convertList2String(nowPaths));
+        measurementsPath.add(convertList2String(nowPaths));
+        alignedList.add(isAligned);
+        dataTypes.add(mNode.getDataType());
+        encodings.add(mNode.getEncoding());
+        compressors.add(mNode.getCompressionType());
+        nowPaths.remove(nowPaths.size() - 1);
+      }
+    }
+    measurementsPath = result;
+  }
+
+  public byte[] serialize() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Deque<Node> stack = new ArrayDeque<>();
+    stack.push(moutedNode);
+
+    while (stack.size() != 0) {
+      Node cur = stack.pop();
+      cur.serialize(baos);
+      if (!cur.isMeasurement()) {
+        for (Node child : cur.getChildren().values()) stack.push(child);
+      }
+    }
+    return baos.toByteArray();
+  }
+
+  // endregion
+
+  // region Inner Utility
+  public static void deserialize(ByteBuffer buffer) {
+    while (buffer.position() != buffer.limit()) {
+      String nodeName = readStringFromBuffer(buffer);
+      boolean isInternal = readBoolFromBuffer(buffer);
+
+      if (isInternal) {
+        boolean isAligned = readBoolFromBuffer(buffer);
+        int childSize = readIntFromBuffer(buffer);
+
+        System.out.println(
+            String.format("%s, %s, %s, %d", nodeName, isInternal, isAligned, childSize));
+      } else {
+        TSDataType dataType = TSDataType.values()[readByteFromBuffer(buffer)];
+        TSEncoding encoding = TSEncoding.values()[readByteFromBuffer(buffer)];
+        CompressionType compressor = CompressionType.values()[readByteFromBuffer(buffer)];
+
+        System.out.println(
+            String.format(
+                "%s, %s, %s, %s, %s", nodeName, isInternal, dataType, encoding, compressor));
+      }
+    }
+  }
+
+  private static int readIntFromBuffer(ByteBuffer buffer) {
+    return buffer.getInt();
+  }
+
+  private static String readStringFromBuffer(ByteBuffer buffer) {
+    int size = buffer.getInt();
+    byte[] bytes = new byte[size];
+    buffer.get(bytes, 0, size);
+    return new String(bytes, 0, size);
+  }
+
+  private static Byte readByteFromBuffer(ByteBuffer buffer) {
+    return buffer.get();
+  }
+
+  private static boolean readBoolFromBuffer(ByteBuffer buffer) {
+    return (buffer.get() == 1);
+  }

Review comment:
       these methods are already exist in ReadWriteIOUtils

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;
+  private boolean shareTime;
+
+  private List<String> measurementsPath;
+  private List<Boolean> alignedList;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+
+  // sync with metadata.Template
+  public enum TemplateQueryType {
+    NULL,
+    COUNT_MEA,
+    IS_MEA,
+    IS_SERIES,
+    SHOW_MEA
+  }
+
+  public Template(String name, boolean isShareTime) {
+    this.name = name;
+    moutedNode = new InternalNode(name, isShareTime);
+    this.shareTime = isShareTime;
+
+    this.measurementsPath = new ArrayList<>();
+    this.alignedList = new ArrayList<>();
+    this.dataTypes = new ArrayList<>();
+    this.encodings = new ArrayList<>();
+    this.compressors = new ArrayList<>();
+  }
+
+  public Template(String name) {
+    this(name, false);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isShareTime() {
+    return shareTime;
+  }
+
+  public void setShareTime(boolean shareTime) {
+    this.shareTime = shareTime;
+  }
+
+  public List<Boolean> getAlignedList() {
+    return this.alignedList;
+  }
+
+  public List<String> getMeasurementsPath() {
+    return this.measurementsPath;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  // region Interface to manipulate Template
+
+  public void addToTemplate(Node child) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(child.getName()))
+      throw new StatementExecutionException("Duplicated child of node in template.");
+    moutedNode.getChildren().put(child.getName(), child);
+  }
+
+  public void deleteFromTemplate(String name) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(name)) moutedNode.getChildren().remove(name);
+    else throw new StatementExecutionException("It is not a direct child of the template: " + name);

Review comment:
       add {}:
   
   if () {
   } else {
   }

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;
+  private boolean shareTime;
+
+  private List<String> measurementsPath;
+  private List<Boolean> alignedList;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+
+  // sync with metadata.Template
+  public enum TemplateQueryType {
+    NULL,
+    COUNT_MEA,
+    IS_MEA,
+    IS_SERIES,
+    SHOW_MEA
+  }
+
+  public Template(String name, boolean isShareTime) {
+    this.name = name;
+    moutedNode = new InternalNode(name, isShareTime);
+    this.shareTime = isShareTime;
+
+    this.measurementsPath = new ArrayList<>();
+    this.alignedList = new ArrayList<>();
+    this.dataTypes = new ArrayList<>();
+    this.encodings = new ArrayList<>();
+    this.compressors = new ArrayList<>();
+  }
+
+  public Template(String name) {
+    this(name, false);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isShareTime() {
+    return shareTime;
+  }
+
+  public void setShareTime(boolean shareTime) {
+    this.shareTime = shareTime;
+  }
+
+  public List<Boolean> getAlignedList() {
+    return this.alignedList;
+  }
+
+  public List<String> getMeasurementsPath() {
+    return this.measurementsPath;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  // region Interface to manipulate Template
+
+  public void addToTemplate(Node child) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(child.getName()))
+      throw new StatementExecutionException("Duplicated child of node in template.");
+    moutedNode.getChildren().put(child.getName(), child);
+  }
+
+  public void deleteFromTemplate(String name) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(name)) moutedNode.getChildren().remove(name);
+    else throw new StatementExecutionException("It is not a direct child of the template: " + name);
+  }
+
+  public void traverseTemplate() throws StatementExecutionException {
+    List<String> result = new ArrayList<>();
+    List<String> nowPaths = new ArrayList<>();
+    Deque<Integer> childSize = new ArrayDeque<>();
+    Deque<Boolean> alignStack = new ArrayDeque<>();
+    Deque<Node> stack = new ArrayDeque<>();
+    this.measurementsPath.clear();
+    this.alignedList.clear();
+    this.dataTypes.clear();
+    this.encodings.clear();
+    this.compressors.clear();
+
+    Node cur;
+    MeasurementNode mNode;
+    int count;
+    boolean isAligned;
+
+    for (Node child : moutedNode.getChildren().values()) stack.push(child);
+    count = moutedNode.getChildren().size();
+    isAligned = moutedNode.isShareTime();
+
+    while (stack.size() != 0) {
+      if (stack.size() > 64)
+        throw new StatementExecutionException(
+            "Too long prefix in template, please check iterations.");
+
+      cur = stack.pop();
+      count--;
+
+      if ((count < 0) && (childSize.size() > 0)) {
+        isAligned = alignStack.pop();
+        count = childSize.pop();
+        count--;
+        nowPaths.remove(nowPaths.size() - 1);
+      }
+
+      if (cur != moutedNode) nowPaths.add(cur.getName());
+
+      if (cur.getChildren() != null) {
+        childSize.push(count);
+        alignStack.push(isAligned);
+
+        for (Node child : cur.getChildren().values()) stack.push(child);
+        count = cur.getChildren().size();
+        isAligned = cur.isShareTime();
+      }
+
+      if (cur.isMeasurement()) {
+        mNode = (MeasurementNode) cur;
+        result.add(convertList2String(nowPaths));
+        measurementsPath.add(convertList2String(nowPaths));
+        alignedList.add(isAligned);
+        dataTypes.add(mNode.getDataType());
+        encodings.add(mNode.getEncoding());
+        compressors.add(mNode.getCompressionType());
+        nowPaths.remove(nowPaths.size() - 1);
+      }
+    }
+    measurementsPath = result;
+  }
+
+  public byte[] serialize() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Deque<Node> stack = new ArrayDeque<>();
+    stack.push(moutedNode);
+
+    while (stack.size() != 0) {
+      Node cur = stack.pop();
+      cur.serialize(baos);
+      if (!cur.isMeasurement()) {
+        for (Node child : cur.getChildren().values()) stack.push(child);
+      }
+    }
+    return baos.toByteArray();
+  }
+
+  // endregion
+
+  // region Inner Utility
+  public static void deserialize(ByteBuffer buffer) {

Review comment:
       We usually construct an object from buffer an return it:
   
   public statistic Template deserialized(ByteBuffer buffer)

##########
File path: session/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TemplateUT {
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(IoTDBConstant.IOTDB_CONF, "src/test/resources/");
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.envSetUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testTemplateTree() {
+    Template template = new Template("treeTemplate", true);
+    Node iNodeGPS = new InternalNode("GPS", false);
+    Node iNodeV = new InternalNode("vehicle", true);
+    Node mNodeX =
+        new MeasurementNode("x", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY);
+    Node mNodeY =
+        new MeasurementNode("y", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY);
+    try {
+      iNodeGPS.addChild(mNodeX);
+      iNodeGPS.addChild(mNodeY);
+      iNodeV.addChild(mNodeX);
+      iNodeV.addChild(mNodeY);
+      iNodeV.addChild(iNodeGPS);
+      template.addToTemplate(iNodeGPS);
+      template.addToTemplate(iNodeV);
+      template.addToTemplate(mNodeX);
+      template.addToTemplate(mNodeY);
+
+      template.traverseTemplate();
+      System.out.println(template.getAlignedList());
+      System.out.println(template.getMeasurementsPath());
+
+      template.serialize();

Review comment:
       store the serialized bytes, then deserialize it and assert

##########
File path: session/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TemplateUT {
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(IoTDBConstant.IOTDB_CONF, "src/test/resources/");
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.envSetUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testTemplateTree() {
+    Template template = new Template("treeTemplate", true);
+    Node iNodeGPS = new InternalNode("GPS", false);
+    Node iNodeV = new InternalNode("vehicle", true);
+    Node mNodeX =
+        new MeasurementNode("x", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY);
+    Node mNodeY =
+        new MeasurementNode("y", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY);
+    try {
+      iNodeGPS.addChild(mNodeX);
+      iNodeGPS.addChild(mNodeY);
+      iNodeV.addChild(mNodeX);
+      iNodeV.addChild(mNodeY);
+      iNodeV.addChild(iNodeGPS);
+      template.addToTemplate(iNodeGPS);
+      template.addToTemplate(iNodeV);
+      template.addToTemplate(mNodeX);
+      template.addToTemplate(mNodeY);
+
+      template.traverseTemplate();
+      System.out.println(template.getAlignedList());
+      System.out.println(template.getMeasurementsPath());

Review comment:
       print is of no use in test, we use assert 

##########
File path: thrift/src/main/thrift/rpc.thrift
##########
@@ -446,5 +483,13 @@ service TSIService {
 
   TSStatus createSchemaTemplate(1:TSCreateSchemaTemplateReq req);
 
+  TSStatus createSchemaTemplateSerialized(1:TSCreateSchemaTemplateSerializedReq req);

Review comment:
       ```suggestion
     TSStatus createSchemaTemplate(1:TSCreateSchemaTemplateReq req);
   ```

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;
+  private boolean shareTime;
+
+  private List<String> measurementsPath;
+  private List<Boolean> alignedList;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+
+  // sync with metadata.Template
+  public enum TemplateQueryType {
+    NULL,
+    COUNT_MEA,
+    IS_MEA,
+    IS_SERIES,
+    SHOW_MEA
+  }
+
+  public Template(String name, boolean isShareTime) {
+    this.name = name;
+    moutedNode = new InternalNode(name, isShareTime);
+    this.shareTime = isShareTime;
+
+    this.measurementsPath = new ArrayList<>();
+    this.alignedList = new ArrayList<>();
+    this.dataTypes = new ArrayList<>();
+    this.encodings = new ArrayList<>();
+    this.compressors = new ArrayList<>();
+  }
+
+  public Template(String name) {
+    this(name, false);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isShareTime() {
+    return shareTime;
+  }
+
+  public void setShareTime(boolean shareTime) {
+    this.shareTime = shareTime;
+  }
+
+  public List<Boolean> getAlignedList() {
+    return this.alignedList;
+  }
+
+  public List<String> getMeasurementsPath() {
+    return this.measurementsPath;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  // region Interface to manipulate Template
+
+  public void addToTemplate(Node child) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(child.getName()))
+      throw new StatementExecutionException("Duplicated child of node in template.");
+    moutedNode.getChildren().put(child.getName(), child);

Review comment:
       how could you construct a tree using a non-recursion method? This should be a bug

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;
+  private boolean shareTime;
+
+  private List<String> measurementsPath;
+  private List<Boolean> alignedList;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+
+  // sync with metadata.Template
+  public enum TemplateQueryType {
+    NULL,
+    COUNT_MEA,
+    IS_MEA,
+    IS_SERIES,
+    SHOW_MEA
+  }
+
+  public Template(String name, boolean isShareTime) {
+    this.name = name;
+    moutedNode = new InternalNode(name, isShareTime);
+    this.shareTime = isShareTime;
+
+    this.measurementsPath = new ArrayList<>();
+    this.alignedList = new ArrayList<>();
+    this.dataTypes = new ArrayList<>();
+    this.encodings = new ArrayList<>();
+    this.compressors = new ArrayList<>();
+  }
+
+  public Template(String name) {
+    this(name, false);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isShareTime() {
+    return shareTime;
+  }
+
+  public void setShareTime(boolean shareTime) {
+    this.shareTime = shareTime;
+  }
+
+  public List<Boolean> getAlignedList() {
+    return this.alignedList;
+  }
+
+  public List<String> getMeasurementsPath() {
+    return this.measurementsPath;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  // region Interface to manipulate Template
+
+  public void addToTemplate(Node child) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(child.getName()))
+      throw new StatementExecutionException("Duplicated child of node in template.");
+    moutedNode.getChildren().put(child.getName(), child);
+  }
+
+  public void deleteFromTemplate(String name) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(name)) moutedNode.getChildren().remove(name);
+    else throw new StatementExecutionException("It is not a direct child of the template: " + name);
+  }
+
+  public void traverseTemplate() throws StatementExecutionException {

Review comment:
       what is this method for?

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;
+  private boolean shareTime;
+
+  private List<String> measurementsPath;
+  private List<Boolean> alignedList;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+
+  // sync with metadata.Template
+  public enum TemplateQueryType {
+    NULL,
+    COUNT_MEA,
+    IS_MEA,
+    IS_SERIES,
+    SHOW_MEA
+  }
+
+  public Template(String name, boolean isShareTime) {
+    this.name = name;
+    moutedNode = new InternalNode(name, isShareTime);
+    this.shareTime = isShareTime;
+
+    this.measurementsPath = new ArrayList<>();
+    this.alignedList = new ArrayList<>();
+    this.dataTypes = new ArrayList<>();
+    this.encodings = new ArrayList<>();
+    this.compressors = new ArrayList<>();
+  }
+
+  public Template(String name) {
+    this(name, false);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isShareTime() {
+    return shareTime;
+  }
+
+  public void setShareTime(boolean shareTime) {
+    this.shareTime = shareTime;
+  }
+
+  public List<Boolean> getAlignedList() {
+    return this.alignedList;
+  }
+
+  public List<String> getMeasurementsPath() {
+    return this.measurementsPath;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  // region Interface to manipulate Template
+
+  public void addToTemplate(Node child) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(child.getName()))
+      throw new StatementExecutionException("Duplicated child of node in template.");
+    moutedNode.getChildren().put(child.getName(), child);
+  }
+
+  public void deleteFromTemplate(String name) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(name)) moutedNode.getChildren().remove(name);
+    else throw new StatementExecutionException("It is not a direct child of the template: " + name);
+  }
+
+  public void traverseTemplate() throws StatementExecutionException {
+    List<String> result = new ArrayList<>();
+    List<String> nowPaths = new ArrayList<>();
+    Deque<Integer> childSize = new ArrayDeque<>();
+    Deque<Boolean> alignStack = new ArrayDeque<>();
+    Deque<Node> stack = new ArrayDeque<>();
+    this.measurementsPath.clear();
+    this.alignedList.clear();
+    this.dataTypes.clear();
+    this.encodings.clear();
+    this.compressors.clear();
+
+    Node cur;
+    MeasurementNode mNode;
+    int count;
+    boolean isAligned;
+
+    for (Node child : moutedNode.getChildren().values()) stack.push(child);
+    count = moutedNode.getChildren().size();
+    isAligned = moutedNode.isShareTime();
+
+    while (stack.size() != 0) {
+      if (stack.size() > 64)
+        throw new StatementExecutionException(
+            "Too long prefix in template, please check iterations.");
+
+      cur = stack.pop();
+      count--;
+
+      if ((count < 0) && (childSize.size() > 0)) {
+        isAligned = alignStack.pop();
+        count = childSize.pop();
+        count--;
+        nowPaths.remove(nowPaths.size() - 1);
+      }
+
+      if (cur != moutedNode) nowPaths.add(cur.getName());
+
+      if (cur.getChildren() != null) {
+        childSize.push(count);
+        alignStack.push(isAligned);
+
+        for (Node child : cur.getChildren().values()) stack.push(child);
+        count = cur.getChildren().size();
+        isAligned = cur.isShareTime();
+      }
+
+      if (cur.isMeasurement()) {
+        mNode = (MeasurementNode) cur;
+        result.add(convertList2String(nowPaths));
+        measurementsPath.add(convertList2String(nowPaths));
+        alignedList.add(isAligned);
+        dataTypes.add(mNode.getDataType());
+        encodings.add(mNode.getEncoding());
+        compressors.add(mNode.getCompressionType());
+        nowPaths.remove(nowPaths.size() - 1);
+      }
+    }
+    measurementsPath = result;
+  }
+
+  public byte[] serialize() throws IOException {

Review comment:
       ```suggestion
     public byte[] serialize(ByteBuffer or OutputStream) throws IOException {
   ```
   
   refer to  serializeTo() in ChunkMetadata

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -800,6 +805,85 @@ protected void createSchemaTemplate(TSCreateSchemaTemplateReq request)
     }
   }
 
+  protected void createSchemaTemplate(TSCreateSchemaTemplateSerializedReq request)

Review comment:
       also add these methods to SessionPool

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
##########
@@ -177,6 +208,406 @@ public String getMeasurementNodeName(String measurementName) {
     return res;
   }
 
+  // region construct template tree
+  /** Construct aligned measurements, checks prefix equality, path duplication and conflict */
+  private void constructTemplateTree(String[] alignedPaths, IMeasurementSchema[] schemas)
+      throws IllegalPathException {
+    // Only for aligned Paths, with common direct prefix
+    String[] pathNodes;
+    IMNode cur;
+    IEntityMNode commonPar;
+    String prefix = null;
+    String thisPrefix = null;
+    List<String> measurementNames = new ArrayList<>();
+    IMeasurementMNode leafNode;
+
+    // deduplicate
+    Set<String> pathSet = new HashSet<>(Arrays.asList(alignedPaths));
+    if (pathSet.size() != alignedPaths.length)
+      throw new IllegalPathException("Duplication in paths.");
+
+    for (String path : alignedPaths) {
+      // check aligned whether legal, and records measurements name
+      pathNodes = MetaUtils.splitPathToDetachedPath(path);
+
+      if (pathNodes.length == 1) thisPrefix = "";
+      else thisPrefix = joinBySeparator(Arrays.copyOf(pathNodes, pathNodes.length - 1));
+      if (prefix == null) prefix = thisPrefix;
+      if (!prefix.equals(thisPrefix))
+        throw new IllegalPathException(
+            "Aligned measurements get different paths, " + alignedPaths[0]);
+      if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + prefix);
+
+      measurementNames.add(pathNodes[pathNodes.length - 1]);
+    }
+    if (prefix.equals("")) {
+      commonPar = convertInternalToEntity(mountedNode);
+      mountedNode = commonPar;
+    } else {
+      cur = isPathExist(prefix);
+      if (cur == null) cur = constructEntityPath(alignedPaths[0]);
+      if (cur.isMeasurement())
+        throw new IllegalPathException(prefix, "Measurement node amid path.");
+      commonPar = convertInternalToEntity(cur);
+    }
+
+    synchronized (this) {
+      if (!alignedPrefix.contains(prefix)) alignedPrefix.add(prefix);
+      for (int i = 0; i <= measurementNames.size() - 1; i++) {
+        leafNode =
+            MeasurementMNode.getMeasurementMNode(
+                commonPar, measurementNames.get(i), schemas[i], "");
+        commonPar.addChild(leafNode);
+      }
+    }
+  }
+
+  /** Construct single measurement, only check path conflict and duplication */
+  private IMeasurementMNode constructTemplateTree(String path, IMeasurementSchema schema)
+      throws IllegalPathException {
+    IEntityMNode entityMNode;
+    if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + path);
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = constructEntityPath(path);
+    if (cur.isMeasurement()) throw new IllegalPathException("Path duplicated: " + path);
+
+    entityMNode = convertInternalToEntity(cur);
+
+    if (entityMNode.getParent() == null) mountedNode = entityMNode;
+
+    synchronized (this) {
+      IMeasurementMNode leafNode =
+          MeasurementMNode.getMeasurementMNode(
+              entityMNode, pathNode[pathNode.length - 1], schema, "");
+      entityMNode.addChild(leafNode.getName(), leafNode);
+      return leafNode;
+    }
+  }
+
+  private IMeasurementSchema constructSchema(
+      String nodeName, TSDataType dataType, TSEncoding encoding, CompressionType compressor) {
+    return new UnaryMeasurementSchema(nodeName, dataType, encoding, compressor);
+  }
+
+  private IMeasurementSchema[] constructSchemas(
+      String[] nodeNames,
+      TSDataType[] dataTypes,
+      TSEncoding[] encodings,
+      CompressionType[] compressors) {
+    UnaryMeasurementSchema[] schemas = new UnaryMeasurementSchema[nodeNames.length];
+    schemas[0] =
+        new UnaryMeasurementSchema(nodeNames[0], dataTypes[0], encodings[0], compressors[0]);
+    return schemas;
+  }
+  // endregion
+
+  // region test methods
+  @TestOnly
+  public IMeasurementSchema getVirtualSchema(String nodeName) {
+    return constructSchema(nodeName, TSDataType.INT32, TSEncoding.GORILLA, CompressionType.SNAPPY);
+  }
+
+  @TestOnly
+  public void constructVirtualSchemaMeasurement(String path) throws IllegalPathException {
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    constructTemplateTree(path, getVirtualSchema(pathNode[pathNode.length - 1]));
+  }
+  // endregion
+
+  // region query of template
+
+  public List<String> getAllAlignedPrefix() {
+    return Arrays.asList(alignedPrefix.toArray(new String[0]));
+  }
+
+  public List<String> getAlignedMeasurements(String prefix) throws IllegalPathException {
+    if (!alignedPrefix.contains(prefix)) return null;
+    IMNode prefixNode = isPathExist(prefix);
+    if (prefixNode == null) throw new IllegalPathException(prefix, "there is no prefix IMNode.");
+    if (prefixNode.isMeasurement())
+      throw new IllegalPathException(prefix, "path is a measurement.");
+    List<String> subMeasurements = new ArrayList<>();
+    for (IMNode child : prefixNode.getChildren().values()) {
+      if (child.isMeasurement()) subMeasurements.add(child.getName());
+    }
+    return subMeasurements;
+  }
+
+  public List<String> getAllMeasurementsPaths() {
+    traverse();
+    return allSeriesPaths;
+  }
+
+  public List<String> getMeasurementsUnderPath(String path) {
+    if (path.equals("")) return getAllMeasurementsPaths();
+    List<String> res = new ArrayList<>();
+    try {
+      IMNode cur = isPathExist(path);
+      if (cur == null) throw new IllegalPathException(path, "Path not exists.");
+      if (cur.isMeasurement())
+        return Collections.singletonList(getFullPathWithoutTemplateName(cur));
+      Deque<IMNode> stack = new ArrayDeque<>();
+      stack.push(cur);
+      while (stack.size() != 0) {
+        cur = stack.pop();
+        if (cur.isMeasurement()) {
+          res.add(getFullPathWithoutTemplateName(cur));
+        } else {
+          for (IMNode child : cur.getChildren().values()) stack.push(child);
+        }
+      }
+    } catch (IllegalPathException e) {
+      e.printStackTrace();
+    }
+    return res;
+  }
+
+  public int getMeasurementsCount() {
+    traverse();
+    return measurementsCount;
+  }
+
+  public IMNode isPathExist(String path) throws IllegalPathException {
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);
+      else return null;
+    }
+    return cur;
+  }
+
+  public IMNode isPathExist(PartialPath partialPath) throws IllegalPathException {
+    String path = partialPath.toString();
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);
+      else return null;
+    }
+    return cur;
+  }
+
+  public boolean isPathMeasurement(String path) throws IllegalPathException {
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);
+      else return false;
+    }
+    if (cur.isMeasurement()) return true;
+    else return false;
+  }
+
+  public boolean isPathSeries(String path) throws IllegalPathException {
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);

Review comment:
       if {}
   else {}

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;

Review comment:
       This is better a Map<String, Node>

##########
File path: session/src/main/java/org/apache/iotdb/session/template/Template.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.session.template;
+
+import org.apache.iotdb.rpc.StatementExecutionException;
+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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+
+public class Template {
+  private String name;
+  private Node moutedNode;
+  private boolean shareTime;
+
+  private List<String> measurementsPath;
+  private List<Boolean> alignedList;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+
+  // sync with metadata.Template
+  public enum TemplateQueryType {
+    NULL,
+    COUNT_MEA,
+    IS_MEA,
+    IS_SERIES,
+    SHOW_MEA
+  }
+
+  public Template(String name, boolean isShareTime) {
+    this.name = name;
+    moutedNode = new InternalNode(name, isShareTime);
+    this.shareTime = isShareTime;
+
+    this.measurementsPath = new ArrayList<>();
+    this.alignedList = new ArrayList<>();
+    this.dataTypes = new ArrayList<>();
+    this.encodings = new ArrayList<>();
+    this.compressors = new ArrayList<>();
+  }
+
+  public Template(String name) {
+    this(name, false);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isShareTime() {
+    return shareTime;
+  }
+
+  public void setShareTime(boolean shareTime) {
+    this.shareTime = shareTime;
+  }
+
+  public List<Boolean> getAlignedList() {
+    return this.alignedList;
+  }
+
+  public List<String> getMeasurementsPath() {
+    return this.measurementsPath;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  // region Interface to manipulate Template
+
+  public void addToTemplate(Node child) throws StatementExecutionException {
+    if (moutedNode.getChildren().containsKey(child.getName()))

Review comment:
       add {}

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
##########
@@ -177,6 +208,406 @@ public String getMeasurementNodeName(String measurementName) {
     return res;
   }
 
+  // region construct template tree
+  /** Construct aligned measurements, checks prefix equality, path duplication and conflict */
+  private void constructTemplateTree(String[] alignedPaths, IMeasurementSchema[] schemas)
+      throws IllegalPathException {
+    // Only for aligned Paths, with common direct prefix
+    String[] pathNodes;
+    IMNode cur;
+    IEntityMNode commonPar;
+    String prefix = null;
+    String thisPrefix = null;
+    List<String> measurementNames = new ArrayList<>();
+    IMeasurementMNode leafNode;
+
+    // deduplicate
+    Set<String> pathSet = new HashSet<>(Arrays.asList(alignedPaths));
+    if (pathSet.size() != alignedPaths.length)
+      throw new IllegalPathException("Duplication in paths.");
+
+    for (String path : alignedPaths) {
+      // check aligned whether legal, and records measurements name
+      pathNodes = MetaUtils.splitPathToDetachedPath(path);
+
+      if (pathNodes.length == 1) thisPrefix = "";
+      else thisPrefix = joinBySeparator(Arrays.copyOf(pathNodes, pathNodes.length - 1));
+      if (prefix == null) prefix = thisPrefix;
+      if (!prefix.equals(thisPrefix))
+        throw new IllegalPathException(
+            "Aligned measurements get different paths, " + alignedPaths[0]);
+      if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + prefix);
+
+      measurementNames.add(pathNodes[pathNodes.length - 1]);
+    }
+    if (prefix.equals("")) {
+      commonPar = convertInternalToEntity(mountedNode);
+      mountedNode = commonPar;
+    } else {
+      cur = isPathExist(prefix);
+      if (cur == null) cur = constructEntityPath(alignedPaths[0]);
+      if (cur.isMeasurement())
+        throw new IllegalPathException(prefix, "Measurement node amid path.");
+      commonPar = convertInternalToEntity(cur);
+    }
+
+    synchronized (this) {
+      if (!alignedPrefix.contains(prefix)) alignedPrefix.add(prefix);
+      for (int i = 0; i <= measurementNames.size() - 1; i++) {
+        leafNode =
+            MeasurementMNode.getMeasurementMNode(
+                commonPar, measurementNames.get(i), schemas[i], "");
+        commonPar.addChild(leafNode);
+      }
+    }
+  }
+
+  /** Construct single measurement, only check path conflict and duplication */
+  private IMeasurementMNode constructTemplateTree(String path, IMeasurementSchema schema)
+      throws IllegalPathException {
+    IEntityMNode entityMNode;
+    if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + path);
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = constructEntityPath(path);
+    if (cur.isMeasurement()) throw new IllegalPathException("Path duplicated: " + path);
+
+    entityMNode = convertInternalToEntity(cur);
+
+    if (entityMNode.getParent() == null) mountedNode = entityMNode;
+
+    synchronized (this) {
+      IMeasurementMNode leafNode =
+          MeasurementMNode.getMeasurementMNode(
+              entityMNode, pathNode[pathNode.length - 1], schema, "");
+      entityMNode.addChild(leafNode.getName(), leafNode);
+      return leafNode;
+    }
+  }
+
+  private IMeasurementSchema constructSchema(
+      String nodeName, TSDataType dataType, TSEncoding encoding, CompressionType compressor) {
+    return new UnaryMeasurementSchema(nodeName, dataType, encoding, compressor);
+  }
+
+  private IMeasurementSchema[] constructSchemas(
+      String[] nodeNames,
+      TSDataType[] dataTypes,
+      TSEncoding[] encodings,
+      CompressionType[] compressors) {
+    UnaryMeasurementSchema[] schemas = new UnaryMeasurementSchema[nodeNames.length];
+    schemas[0] =
+        new UnaryMeasurementSchema(nodeNames[0], dataTypes[0], encodings[0], compressors[0]);
+    return schemas;
+  }
+  // endregion
+
+  // region test methods
+  @TestOnly
+  public IMeasurementSchema getVirtualSchema(String nodeName) {
+    return constructSchema(nodeName, TSDataType.INT32, TSEncoding.GORILLA, CompressionType.SNAPPY);
+  }
+
+  @TestOnly
+  public void constructVirtualSchemaMeasurement(String path) throws IllegalPathException {
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    constructTemplateTree(path, getVirtualSchema(pathNode[pathNode.length - 1]));
+  }
+  // endregion
+
+  // region query of template
+
+  public List<String> getAllAlignedPrefix() {
+    return Arrays.asList(alignedPrefix.toArray(new String[0]));
+  }
+
+  public List<String> getAlignedMeasurements(String prefix) throws IllegalPathException {
+    if (!alignedPrefix.contains(prefix)) return null;
+    IMNode prefixNode = isPathExist(prefix);
+    if (prefixNode == null) throw new IllegalPathException(prefix, "there is no prefix IMNode.");
+    if (prefixNode.isMeasurement())
+      throw new IllegalPathException(prefix, "path is a measurement.");
+    List<String> subMeasurements = new ArrayList<>();
+    for (IMNode child : prefixNode.getChildren().values()) {
+      if (child.isMeasurement()) subMeasurements.add(child.getName());
+    }
+    return subMeasurements;
+  }
+
+  public List<String> getAllMeasurementsPaths() {
+    traverse();
+    return allSeriesPaths;
+  }
+
+  public List<String> getMeasurementsUnderPath(String path) {
+    if (path.equals("")) return getAllMeasurementsPaths();
+    List<String> res = new ArrayList<>();
+    try {
+      IMNode cur = isPathExist(path);
+      if (cur == null) throw new IllegalPathException(path, "Path not exists.");
+      if (cur.isMeasurement())
+        return Collections.singletonList(getFullPathWithoutTemplateName(cur));
+      Deque<IMNode> stack = new ArrayDeque<>();
+      stack.push(cur);
+      while (stack.size() != 0) {
+        cur = stack.pop();
+        if (cur.isMeasurement()) {
+          res.add(getFullPathWithoutTemplateName(cur));
+        } else {
+          for (IMNode child : cur.getChildren().values()) stack.push(child);
+        }
+      }
+    } catch (IllegalPathException e) {
+      e.printStackTrace();
+    }
+    return res;
+  }
+
+  public int getMeasurementsCount() {
+    traverse();
+    return measurementsCount;
+  }
+
+  public IMNode isPathExist(String path) throws IllegalPathException {
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);
+      else return null;
+    }
+    return cur;
+  }
+
+  public IMNode isPathExist(PartialPath partialPath) throws IllegalPathException {

Review comment:
       isPathExist should return boolean

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
##########
@@ -177,6 +208,406 @@ public String getMeasurementNodeName(String measurementName) {
     return res;
   }
 
+  // region construct template tree
+  /** Construct aligned measurements, checks prefix equality, path duplication and conflict */
+  private void constructTemplateTree(String[] alignedPaths, IMeasurementSchema[] schemas)
+      throws IllegalPathException {
+    // Only for aligned Paths, with common direct prefix
+    String[] pathNodes;
+    IMNode cur;
+    IEntityMNode commonPar;
+    String prefix = null;
+    String thisPrefix = null;
+    List<String> measurementNames = new ArrayList<>();
+    IMeasurementMNode leafNode;
+
+    // deduplicate
+    Set<String> pathSet = new HashSet<>(Arrays.asList(alignedPaths));
+    if (pathSet.size() != alignedPaths.length)
+      throw new IllegalPathException("Duplication in paths.");
+
+    for (String path : alignedPaths) {
+      // check aligned whether legal, and records measurements name
+      pathNodes = MetaUtils.splitPathToDetachedPath(path);
+
+      if (pathNodes.length == 1) thisPrefix = "";
+      else thisPrefix = joinBySeparator(Arrays.copyOf(pathNodes, pathNodes.length - 1));
+      if (prefix == null) prefix = thisPrefix;
+      if (!prefix.equals(thisPrefix))
+        throw new IllegalPathException(
+            "Aligned measurements get different paths, " + alignedPaths[0]);
+      if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + prefix);
+
+      measurementNames.add(pathNodes[pathNodes.length - 1]);
+    }
+    if (prefix.equals("")) {
+      commonPar = convertInternalToEntity(mountedNode);
+      mountedNode = commonPar;
+    } else {
+      cur = isPathExist(prefix);
+      if (cur == null) cur = constructEntityPath(alignedPaths[0]);
+      if (cur.isMeasurement())
+        throw new IllegalPathException(prefix, "Measurement node amid path.");
+      commonPar = convertInternalToEntity(cur);
+    }
+
+    synchronized (this) {
+      if (!alignedPrefix.contains(prefix)) alignedPrefix.add(prefix);
+      for (int i = 0; i <= measurementNames.size() - 1; i++) {
+        leafNode =
+            MeasurementMNode.getMeasurementMNode(
+                commonPar, measurementNames.get(i), schemas[i], "");
+        commonPar.addChild(leafNode);
+      }
+    }
+  }
+
+  /** Construct single measurement, only check path conflict and duplication */
+  private IMeasurementMNode constructTemplateTree(String path, IMeasurementSchema schema)
+      throws IllegalPathException {
+    IEntityMNode entityMNode;
+    if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + path);
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = constructEntityPath(path);
+    if (cur.isMeasurement()) throw new IllegalPathException("Path duplicated: " + path);
+
+    entityMNode = convertInternalToEntity(cur);
+
+    if (entityMNode.getParent() == null) mountedNode = entityMNode;
+
+    synchronized (this) {
+      IMeasurementMNode leafNode =
+          MeasurementMNode.getMeasurementMNode(
+              entityMNode, pathNode[pathNode.length - 1], schema, "");
+      entityMNode.addChild(leafNode.getName(), leafNode);
+      return leafNode;
+    }
+  }
+
+  private IMeasurementSchema constructSchema(
+      String nodeName, TSDataType dataType, TSEncoding encoding, CompressionType compressor) {
+    return new UnaryMeasurementSchema(nodeName, dataType, encoding, compressor);
+  }
+
+  private IMeasurementSchema[] constructSchemas(
+      String[] nodeNames,
+      TSDataType[] dataTypes,
+      TSEncoding[] encodings,
+      CompressionType[] compressors) {
+    UnaryMeasurementSchema[] schemas = new UnaryMeasurementSchema[nodeNames.length];
+    schemas[0] =
+        new UnaryMeasurementSchema(nodeNames[0], dataTypes[0], encodings[0], compressors[0]);
+    return schemas;
+  }
+  // endregion
+
+  // region test methods
+  @TestOnly
+  public IMeasurementSchema getVirtualSchema(String nodeName) {
+    return constructSchema(nodeName, TSDataType.INT32, TSEncoding.GORILLA, CompressionType.SNAPPY);
+  }
+
+  @TestOnly
+  public void constructVirtualSchemaMeasurement(String path) throws IllegalPathException {
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    constructTemplateTree(path, getVirtualSchema(pathNode[pathNode.length - 1]));
+  }
+  // endregion
+
+  // region query of template
+
+  public List<String> getAllAlignedPrefix() {
+    return Arrays.asList(alignedPrefix.toArray(new String[0]));
+  }
+
+  public List<String> getAlignedMeasurements(String prefix) throws IllegalPathException {
+    if (!alignedPrefix.contains(prefix)) return null;
+    IMNode prefixNode = isPathExist(prefix);
+    if (prefixNode == null) throw new IllegalPathException(prefix, "there is no prefix IMNode.");
+    if (prefixNode.isMeasurement())
+      throw new IllegalPathException(prefix, "path is a measurement.");
+    List<String> subMeasurements = new ArrayList<>();
+    for (IMNode child : prefixNode.getChildren().values()) {
+      if (child.isMeasurement()) subMeasurements.add(child.getName());
+    }
+    return subMeasurements;
+  }
+
+  public List<String> getAllMeasurementsPaths() {
+    traverse();
+    return allSeriesPaths;
+  }
+
+  public List<String> getMeasurementsUnderPath(String path) {
+    if (path.equals("")) return getAllMeasurementsPaths();
+    List<String> res = new ArrayList<>();
+    try {
+      IMNode cur = isPathExist(path);
+      if (cur == null) throw new IllegalPathException(path, "Path not exists.");
+      if (cur.isMeasurement())
+        return Collections.singletonList(getFullPathWithoutTemplateName(cur));
+      Deque<IMNode> stack = new ArrayDeque<>();
+      stack.push(cur);
+      while (stack.size() != 0) {
+        cur = stack.pop();
+        if (cur.isMeasurement()) {
+          res.add(getFullPathWithoutTemplateName(cur));
+        } else {
+          for (IMNode child : cur.getChildren().values()) stack.push(child);
+        }
+      }
+    } catch (IllegalPathException e) {
+      e.printStackTrace();
+    }
+    return res;
+  }
+
+  public int getMeasurementsCount() {
+    traverse();
+    return measurementsCount;
+  }
+
+  public IMNode isPathExist(String path) throws IllegalPathException {

Review comment:
       isPathExist should return boolean

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
##########
@@ -177,6 +208,406 @@ public String getMeasurementNodeName(String measurementName) {
     return res;
   }
 
+  // region construct template tree
+  /** Construct aligned measurements, checks prefix equality, path duplication and conflict */
+  private void constructTemplateTree(String[] alignedPaths, IMeasurementSchema[] schemas)
+      throws IllegalPathException {
+    // Only for aligned Paths, with common direct prefix
+    String[] pathNodes;
+    IMNode cur;
+    IEntityMNode commonPar;
+    String prefix = null;
+    String thisPrefix = null;
+    List<String> measurementNames = new ArrayList<>();
+    IMeasurementMNode leafNode;
+
+    // deduplicate
+    Set<String> pathSet = new HashSet<>(Arrays.asList(alignedPaths));
+    if (pathSet.size() != alignedPaths.length)
+      throw new IllegalPathException("Duplication in paths.");
+
+    for (String path : alignedPaths) {
+      // check aligned whether legal, and records measurements name
+      pathNodes = MetaUtils.splitPathToDetachedPath(path);
+
+      if (pathNodes.length == 1) thisPrefix = "";
+      else thisPrefix = joinBySeparator(Arrays.copyOf(pathNodes, pathNodes.length - 1));
+      if (prefix == null) prefix = thisPrefix;
+      if (!prefix.equals(thisPrefix))
+        throw new IllegalPathException(
+            "Aligned measurements get different paths, " + alignedPaths[0]);
+      if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + prefix);
+
+      measurementNames.add(pathNodes[pathNodes.length - 1]);
+    }
+    if (prefix.equals("")) {
+      commonPar = convertInternalToEntity(mountedNode);
+      mountedNode = commonPar;
+    } else {
+      cur = isPathExist(prefix);
+      if (cur == null) cur = constructEntityPath(alignedPaths[0]);
+      if (cur.isMeasurement())
+        throw new IllegalPathException(prefix, "Measurement node amid path.");
+      commonPar = convertInternalToEntity(cur);
+    }
+
+    synchronized (this) {
+      if (!alignedPrefix.contains(prefix)) alignedPrefix.add(prefix);
+      for (int i = 0; i <= measurementNames.size() - 1; i++) {
+        leafNode =
+            MeasurementMNode.getMeasurementMNode(
+                commonPar, measurementNames.get(i), schemas[i], "");
+        commonPar.addChild(leafNode);
+      }
+    }
+  }
+
+  /** Construct single measurement, only check path conflict and duplication */
+  private IMeasurementMNode constructTemplateTree(String path, IMeasurementSchema schema)
+      throws IllegalPathException {
+    IEntityMNode entityMNode;
+    if (isPathExist(path) != null) throw new IllegalPathException("Path duplicated: " + path);
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = constructEntityPath(path);
+    if (cur.isMeasurement()) throw new IllegalPathException("Path duplicated: " + path);
+
+    entityMNode = convertInternalToEntity(cur);
+
+    if (entityMNode.getParent() == null) mountedNode = entityMNode;
+
+    synchronized (this) {
+      IMeasurementMNode leafNode =
+          MeasurementMNode.getMeasurementMNode(
+              entityMNode, pathNode[pathNode.length - 1], schema, "");
+      entityMNode.addChild(leafNode.getName(), leafNode);
+      return leafNode;
+    }
+  }
+
+  private IMeasurementSchema constructSchema(
+      String nodeName, TSDataType dataType, TSEncoding encoding, CompressionType compressor) {
+    return new UnaryMeasurementSchema(nodeName, dataType, encoding, compressor);
+  }
+
+  private IMeasurementSchema[] constructSchemas(
+      String[] nodeNames,
+      TSDataType[] dataTypes,
+      TSEncoding[] encodings,
+      CompressionType[] compressors) {
+    UnaryMeasurementSchema[] schemas = new UnaryMeasurementSchema[nodeNames.length];
+    schemas[0] =
+        new UnaryMeasurementSchema(nodeNames[0], dataTypes[0], encodings[0], compressors[0]);
+    return schemas;
+  }
+  // endregion
+
+  // region test methods
+  @TestOnly
+  public IMeasurementSchema getVirtualSchema(String nodeName) {
+    return constructSchema(nodeName, TSDataType.INT32, TSEncoding.GORILLA, CompressionType.SNAPPY);
+  }
+
+  @TestOnly
+  public void constructVirtualSchemaMeasurement(String path) throws IllegalPathException {
+    String[] pathNode = MetaUtils.splitPathToDetachedPath(path);
+    constructTemplateTree(path, getVirtualSchema(pathNode[pathNode.length - 1]));
+  }
+  // endregion
+
+  // region query of template
+
+  public List<String> getAllAlignedPrefix() {
+    return Arrays.asList(alignedPrefix.toArray(new String[0]));
+  }
+
+  public List<String> getAlignedMeasurements(String prefix) throws IllegalPathException {
+    if (!alignedPrefix.contains(prefix)) return null;
+    IMNode prefixNode = isPathExist(prefix);
+    if (prefixNode == null) throw new IllegalPathException(prefix, "there is no prefix IMNode.");
+    if (prefixNode.isMeasurement())
+      throw new IllegalPathException(prefix, "path is a measurement.");
+    List<String> subMeasurements = new ArrayList<>();
+    for (IMNode child : prefixNode.getChildren().values()) {
+      if (child.isMeasurement()) subMeasurements.add(child.getName());
+    }
+    return subMeasurements;
+  }
+
+  public List<String> getAllMeasurementsPaths() {
+    traverse();
+    return allSeriesPaths;
+  }
+
+  public List<String> getMeasurementsUnderPath(String path) {
+    if (path.equals("")) return getAllMeasurementsPaths();
+    List<String> res = new ArrayList<>();
+    try {
+      IMNode cur = isPathExist(path);
+      if (cur == null) throw new IllegalPathException(path, "Path not exists.");
+      if (cur.isMeasurement())
+        return Collections.singletonList(getFullPathWithoutTemplateName(cur));
+      Deque<IMNode> stack = new ArrayDeque<>();
+      stack.push(cur);
+      while (stack.size() != 0) {
+        cur = stack.pop();
+        if (cur.isMeasurement()) {
+          res.add(getFullPathWithoutTemplateName(cur));
+        } else {
+          for (IMNode child : cur.getChildren().values()) stack.push(child);
+        }
+      }
+    } catch (IllegalPathException e) {
+      e.printStackTrace();
+    }
+    return res;
+  }
+
+  public int getMeasurementsCount() {
+    traverse();
+    return measurementsCount;
+  }
+
+  public IMNode isPathExist(String path) throws IllegalPathException {
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);
+      else return null;
+    }
+    return cur;
+  }
+
+  public IMNode isPathExist(PartialPath partialPath) throws IllegalPathException {
+    String path = partialPath.toString();
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);
+      else return null;
+    }
+    return cur;
+  }
+
+  public boolean isPathMeasurement(String path) throws IllegalPathException {
+    String[] pathNodes = MetaUtils.splitPathToDetachedPath(path);
+    IMNode cur = mountedNode;
+    for (String node : pathNodes) {
+      if (cur.hasChild(node)) cur = cur.getChild(node);
+      else return false;
+    }
+    if (cur.isMeasurement()) return true;

Review comment:
       if {}
   else {}
   
   format code




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org