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/07/15 01:12:57 UTC

[iotdb] branch rel/0.12 updated: [To rel/0.12][IOTDB-1492] Fix mLogParser.sh cannot parse the device template (#3557)

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

qiaojialin pushed a commit to branch rel/0.12
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/0.12 by this push:
     new c257e4a  [To rel/0.12][IOTDB-1492] Fix mLogParser.sh cannot parse the device template (#3557)
c257e4a is described below

commit c257e4a85efbcae460efd428fd8d36d06210594f
Author: Haonan <hh...@outlook.com>
AuthorDate: Thu Jul 15 09:12:32 2021 +0800

    [To rel/0.12][IOTDB-1492] Fix mLogParser.sh cannot parse the device template (#3557)
---
 docs/SystemDesign/SchemaManager/SchemaManager.md   | 37 +++++++++--
 .../zh/SystemDesign/SchemaManager/SchemaManager.md | 53 ++++++++++++----
 .../org/apache/iotdb/db/metadata/MManager.java     |  1 +
 .../iotdb/db/metadata/MetadataOperationType.java   |  4 ++
 .../db/metadata/{ => logfile}/MLogTxtWriter.java   | 71 +++++++++++++++++++++-
 .../db/metadata/{ => logfile}/TagLogFile.java      |  2 +-
 .../org/apache/iotdb/db/tools/mlog/MLogParser.java | 19 +++++-
 .../org/apache/iotdb/db/tools/MLogParserTest.java  | 71 +++++++++++++++++-----
 8 files changed, 221 insertions(+), 37 deletions(-)

diff --git a/docs/SystemDesign/SchemaManager/SchemaManager.md b/docs/SystemDesign/SchemaManager/SchemaManager.md
index 53d97d1..aec2adf 100644
--- a/docs/SystemDesign/SchemaManager/SchemaManager.md
+++ b/docs/SystemDesign/SchemaManager/SchemaManager.md
@@ -213,15 +213,14 @@ The method is `MManager.initFromLog()`:
 
 ## Log management of metadata
 
-* org.apache.iotdb.db.metadata.logfile.MLogWriter
 
 All metadata operations are recorded in a metadata log file, which defaults to data/system/schema/mlog.bin.
 
 When the system restarted, the logs in mlog will be replayed. Until the replaying finished, you need to mark writeToLog to false. When the restart is complete, the writeToLog needs to be set to true.
 
-The type of metadata log is recorded by the MetadataOperationType class. mlog directly stores the corresponding string encoding.
+mlog stores the binary encoding. We can use [MlogParser Tool](https://iotdb.apache.org/UserGuide/Master/System-Tools/MLogParser-Tool.html) to parse the mlog.bin to a human-readable txt version.
 
-sql examples and the corresponding mlog record:
+Schema operation examples and the corresponding parsed mlog record:
 
 * set storage group to root.turbine
 
@@ -265,8 +264,36 @@ sql examples and the corresponding mlog record:
    > mlog: 13,root.turbine.d1.s1,newAlias
    
    > format: 13,path,[new alias]
-                                                                                                                
-                                                                                                              
+                                                                                                               
+* create schema template temp1(
+  s1 INT32 with encoding=Gorilla and compression SNAPPY,
+  s2 FLOAT with encoding=RLE and compression=SNAPPY
+)
+   
+   > mlog:5,temp1,0,s1,1,8,1
+   
+   > mlog:5,temp1,0,s2,3,2,1
+   
+   > format: 5,template name,is Aligned Timeseries,measurementId,TSDataType,TSEncoding,CompressionType
+
+* set schema template temp1 to root.turbine
+ 
+    > mlog: 6,temp1,root.turbine
+   
+    > format: 6,template name,path
+
+* Auto create device root.turbine.d1 (after set a template to a prefix path,  create a device path in mtree automatically when insert data to the device)
+ 
+    > mlog: 4,root.turbine.d1
+   
+    > format: 4,path
+
+* set root.turbine.d1 is using template (after set a template to a device path, this log shows the device is using template)
+ 
+    > mlog: 61,root.turbine.d1
+   
+    > format: 61,path                                                                                                              
+
 ## TLog
 * org.apache.iotdb.db.metadata.TagLogFile
 
diff --git a/docs/zh/SystemDesign/SchemaManager/SchemaManager.md b/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
index a651746..97eb4a0 100644
--- a/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
+++ b/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
@@ -207,45 +207,43 @@ IoTDB 的元数据管理采用目录树的形式,倒数第二层为设备层
 
 1. 检查临时文件`mtree.snapshot.tmp`是否存在,如果存在证明在创建快照的序列化过程中出现服务器人为或意外关闭,导致序列化失败,删除临时文件;
 2. 检查快照文件`mtree.snapshot`是否存在。如果不存在,则使用新的 MTree;否则启动反序列化过程,得到 MTree
-3. 对于`mlog.bin`中的内容,逐行读取并操作,完成 MTree 的恢复。读取过程中更新 `logNumber`,并返回,用于后面`mlog.bin`行数的记录。
+3. 对于`mlog.bin`中的内容,逐行读取并操作,完成 MTree 的恢复。
 
 ## 元数据日志管理
 
-* org.apache.iotdb.db.metadata.logfile.MLogWriter
-
 所有元数据的操作均会记录到元数据日志文件中,此文件默认为 data/system/schema/mlog.bin。
 
-系统重启时会重做 mlog 中的日志,重做之前需要标记不需要记录日志。当重启结束后,标记需要记录日志。
+系统重启时会重做 mlog.bin 中的日志,重做之前需要标记不需要记录日志。当重启结束后,标记需要记录日志。
 
-元数据日志的类型由 MetadataOperationType 类记录。mlog 直接存储字符串编码。
+mlog.bin 存储二进制编码。我们可以使用[MlogParser Tool](https://iotdb.apache.org/UserGuide/Master/System-Tools/MLogParser-Tool.html) 将mlog解析为可读的mlog.txt版本。
 
-示例 sql 及对应的 mlog 记录:
+示例元数据操作及对应的 mlog 解析后记录:
 
-* set storage group to root.turbine
+* 创建一个名为root.turbine的存储组
 
 	> mlog: 2,root.turbine
 	
 	> 格式: 2,path
 
-* delete storage group root.turbine	
+* 删除一个名为root.turbine的存储组
 	
-	> mlog: 1,root.turbine
+	> mlog: 11,root.turbine
 	
-	> 格式: 1,path
+	> 格式: 11,path
 
-* create timeseries root.turbine.d1.s1(temprature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)
+* 创建时间序列 create timeseries root.turbine.d1.s1(temprature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)
 
 	> mlog: 0,root.turbine.d1.s1,3,2,1,,温度,offset
 	
 	> 格式: 0,path,TSDataType,TSEncoding,CompressionType,[properties],[alias],[tag-attribute offset]
 
-* delete timeseries root.turbine.d1.s1
+* 删除时间序列 root.turbine.d1.s1
 
 	> mlog: 1,root.turbine.d1.s1
 	
 	> 格式: 1,path	
 
-* set ttl to root.turbine 10
+* 给 root.turbine 设置时间为 10 秒的 ttl
 	
 	> mlog: 10,root.turbine,10
 		
@@ -263,6 +261,35 @@ IoTDB 的元数据管理采用目录树的形式,倒数第二层为设备层
    > mlog: 13,root.turbine.d1.s1,newAlias
    
    > 格式: 13,path,[new alias]
+   
+* 创建元数据模版 create schema template temp1(
+  s1 INT32 with encoding=Gorilla and compression SNAPPY,
+  s2 FLOAT with encoding=RLE and compression=SNAPPY
+)
+   
+   > mlog:5,temp1,0,s1,1,8,1
+   
+   > mlog:5,temp1,0,s2,3,2,1
+   
+   > 格式: 5,template name,is Aligned Timeseries,measurementId,TSDataType,TSEncoding,CompressionType
+
+* 在某前缀路径上设置元数据模版 set schema template temp1 to root.turbine
+ 
+    > mlog: 6,temp1,root.turbine
+   
+    > 格式: 6,template name,path
+
+* 自动创建设备 (应用场景为在某个前缀路径上设置模版之后,写入时会自动创建设备)
+ 
+    > mlog: 4,root.turbine.d1
+   
+    > 格式: 4,path
+
+* 设置某设备正在使用模版 (应用场景为在某个设备路径上设置模版之后,表示该设备正在应用模版)
+ 
+    > mlog: 61,root.turbine.d1
+   
+    > 格式: 61,path
 
 ## 标签文件
 * org.apache.iotdb.db.metadata.TagLogFile
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 6eda19a..674c90e 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
@@ -37,6 +37,7 @@ import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
 import org.apache.iotdb.db.metadata.logfile.MLogReader;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.logfile.TagLogFile;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
index 466c0a2..8faf87a 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
@@ -33,4 +33,8 @@ public class MetadataOperationType {
   public static final String DROP_INDEX = "32";
   public static final String CHANGE_OFFSET = "12";
   public static final String CHANGE_ALIAS = "13";
+  public static final String AUTO_CREATE_DEVICE = "4";
+  public static final String CREATE_TEMPLATE = "5";
+  public static final String SET_TEMPLATE = "6";
+  public static final String SET_USING_TEMPLATE = "61";
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogTxtWriter.java
similarity index 77%
rename from server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogTxtWriter.java
index ebfa2c6..6991cc7 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogTxtWriter.java
@@ -16,13 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata;
+package org.apache.iotdb.db.metadata.logfile;
 
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.MetadataOperationType;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetUsingDeviceTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
@@ -256,6 +262,69 @@ public class MLogTxtWriter implements AutoCloseable {
     lineNumber.incrementAndGet();
   }
 
+  public void setTemplate(SetDeviceTemplatePlan plan) throws IOException {
+    StringBuilder buf = new StringBuilder(String.valueOf(MetadataOperationType.SET_TEMPLATE));
+    buf.append(",");
+    buf.append(plan.getTemplateName());
+    buf.append(",");
+    buf.append(plan.getPrefixPath());
+    buf.append(LINE_SEPARATOR);
+    ByteBuffer buff = ByteBuffer.wrap(buf.toString().getBytes());
+    channel.write(buff);
+    lineNumber.incrementAndGet();
+  }
+
+  public void setUsingTemplate(SetUsingDeviceTemplatePlan plan) throws IOException {
+    StringBuilder buf = new StringBuilder(String.valueOf(MetadataOperationType.SET_USING_TEMPLATE));
+    buf.append(",");
+    buf.append(plan.getPrefixPath());
+    buf.append(LINE_SEPARATOR);
+    ByteBuffer buff = ByteBuffer.wrap(buf.toString().getBytes());
+    channel.write(buff);
+    lineNumber.incrementAndGet();
+  }
+
+  public void createTemplate(CreateTemplatePlan plan) throws IOException {
+    StringBuilder buf = new StringBuilder();
+    for (int i = 0; i < plan.getSchemaNames().size(); i++) {
+      for (int j = 0; j < plan.getMeasurements().get(i).size(); j++) {
+        String measurement;
+        boolean isAligned = false;
+        if (plan.getMeasurements().get(i).size() == 1) {
+          measurement = plan.getSchemaNames().get(i);
+        } else {
+          // for aligned timeseries
+          isAligned = true;
+          measurement =
+              plan.getSchemaNames().get(i)
+                  + TsFileConstant.PATH_SEPARATOR
+                  + plan.getMeasurements().get(i).get(j);
+        }
+        buf.append(
+            String.format(
+                "%s,%s,%s,%s,%s,%s,%s",
+                MetadataOperationType.CREATE_TEMPLATE,
+                plan.getName(),
+                isAligned ? 1 : 0,
+                measurement,
+                plan.getDataTypes().get(i).get(j).serialize(),
+                plan.getEncodings().get(i).get(j).serialize(),
+                plan.getCompressors().get(i).serialize()));
+        buf.append(LINE_SEPARATOR);
+        lineNumber.incrementAndGet();
+      }
+    }
+    ByteBuffer buff = ByteBuffer.wrap(buf.toString().getBytes());
+    channel.write(buff);
+  }
+
+  public void autoCreateDeviceNode(String Device) throws IOException {
+    String outputStr = MetadataOperationType.AUTO_CREATE_DEVICE + "," + Device + LINE_SEPARATOR;
+    ByteBuffer buff = ByteBuffer.wrap(outputStr.getBytes());
+    channel.write(buff);
+    lineNumber.incrementAndGet();
+  }
+
   int getLineNumber() {
     return lineNumber.get();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/TagLogFile.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/TagLogFile.java
similarity index 99%
rename from server/src/main/java/org/apache/iotdb/db/metadata/TagLogFile.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/logfile/TagLogFile.java
index 0c5f2bd..e2731f4 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/TagLogFile.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/TagLogFile.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata;
+package org.apache.iotdb.db.metadata.logfile;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java b/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
index 85af420..cfb5fab 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
@@ -18,10 +18,13 @@
  */
 package org.apache.iotdb.db.tools.mlog;
 
-import org.apache.iotdb.db.metadata.MLogTxtWriter;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogTxtWriter;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
 import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
@@ -29,6 +32,7 @@ import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetUsingDeviceTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 
 import org.apache.commons.cli.CommandLine;
@@ -198,6 +202,19 @@ public class MLogParser {
           case MNODE:
             mLogTxtWriter.serializeMNode((MNodePlan) plan);
             break;
+          case CREATE_TEMPLATE:
+            mLogTxtWriter.createTemplate((CreateTemplatePlan) plan);
+            break;
+          case SET_DEVICE_TEMPLATE:
+            mLogTxtWriter.setTemplate((SetDeviceTemplatePlan) plan);
+            break;
+          case SET_USING_DEVICE_TEMPLATE:
+            mLogTxtWriter.setUsingTemplate((SetUsingDeviceTemplatePlan) plan);
+            break;
+          case AUTO_CREATE_DEVICE_MNODE:
+            mLogTxtWriter.autoCreateDeviceNode(
+                ((AutoCreateDeviceMNodePlan) plan).getPath().getFullPath());
+            break;
           default:
             logger.warn("unknown plan {}", plan);
         }
diff --git a/server/src/test/java/org/apache/iotdb/db/tools/MLogParserTest.java b/server/src/test/java/org/apache/iotdb/db/tools/MLogParserTest.java
index 1cd6cf6..0a033fa 100644
--- a/server/src/test/java/org/apache/iotdb/db/tools/MLogParserTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/tools/MLogParserTest.java
@@ -23,6 +23,8 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.MetadataConstant;
 import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.tools.mlog.MLogParser;
@@ -35,22 +37,19 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 public class MLogParserTest {
 
-  private static final Logger logger = LoggerFactory.getLogger(MLogParserTest.class);
-
   @Before
   public void setUp() {
     EnvironmentUtils.envSetUp();
@@ -99,6 +98,42 @@ public class MLogParserTest {
     } catch (MetadataException | IOException e) {
       e.printStackTrace();
     }
+
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath("root.sg"));
+      IoTDB.metaManager.createDeviceTemplate(genCreateTemplatePlan());
+      SetDeviceTemplatePlan setDeviceTemplatePlan =
+          new SetDeviceTemplatePlan("template1", "root.sg");
+      IoTDB.metaManager.setDeviceTemplate(setDeviceTemplatePlan);
+      IoTDB.metaManager.getDeviceNodeWithAutoCreate(new PartialPath("root.sg.d1"), true, true, 1);
+    } catch (MetadataException | IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private CreateTemplatePlan genCreateTemplatePlan() {
+    List<List<String>> measurementList = new ArrayList<>();
+    measurementList.add(Collections.singletonList("s11"));
+    measurementList.add(Collections.singletonList("s12"));
+
+    List<List<TSDataType>> dataTypeList = new ArrayList<>();
+    dataTypeList.add(Collections.singletonList(TSDataType.INT64));
+    dataTypeList.add(Collections.singletonList(TSDataType.DOUBLE));
+
+    List<List<TSEncoding>> encodingList = new ArrayList<>();
+    encodingList.add(Collections.singletonList(TSEncoding.RLE));
+    encodingList.add(Collections.singletonList(TSEncoding.GORILLA));
+
+    List<CompressionType> compressionTypes = new ArrayList<>();
+    compressionTypes.add(CompressionType.SNAPPY);
+    compressionTypes.add(CompressionType.SNAPPY);
+
+    List<String> schemaNames = new ArrayList<>();
+    schemaNames.add("s11");
+    schemaNames.add("s12");
+
+    return new CreateTemplatePlan(
+        "template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
   }
 
   @Test
@@ -126,20 +161,22 @@ public class MLogParserTest {
         lineNum++;
         lines.add(line);
       }
-      if (lineNum != 108) {
-        // We prepare 2 storage groups, each one has 5 devices, and every device has 10
+      if (lineNum != 113) {
+        // First, we prepare 2 storage groups, each one has 5 devices, and every device has 10
         // measurements.
         // So, mlog records 2 * 5 * 10 = 100 CreateTimeSeriesPlan, and 2 SetStorageGroupPlan.
         // Next, we do 6 operations which will be written into mlog, include set 2 sgs, set ttl,
         // delete timeseries, delete sg, add tags.
         // The final operation changeAlias only change the mtree in memory, so it will not write
         // record to mlog.
-        // Finally, the mlog should have 100 + 2  + 6 = 108 records
+        // Then, we set 1 more storage group, create a template with 2 measurements, set
+        // the template to this storage group and create 1 device using template.
+        // Finally, the mlog should have 100 + 2 + 6 + 1 + 2 + 1 + 1 = 113 records
         for (String content : lines) {
-          logger.info(content);
+          System.out.println(content);
         }
       }
-      Assert.assertEquals(108, lineNum);
+      Assert.assertEquals(113, lineNum);
     } catch (IOException e) {
       Assert.fail(e.getMessage());
     }
@@ -170,20 +207,22 @@ public class MLogParserTest {
         lineNum++;
         lines.add(line);
       }
-      if (lineNum != 113) {
-        // We prepare 2 storage groups, each one has 5 devices, and every device has 10
+      if (lineNum != 115) {
+        // First, we prepare 2 storage groups, each one has 5 devices, and every device has 10
         // measurements.
         // So, mtree records 2 * 5 * 10 = 100 TimeSeries, and 2 SetStorageGroup, 2 * 5 devices.
         // Next, we do 4 operations which will be record in mtree, include set 2 sgs, delete
         // timeseries, delete sg.
-        // The snapshot should have 100 + 2 + 5 * 2 + 2 - 1 - 1 = 112 records, and we have root
-        // record,
-        // so we have 112 + 1 = 113 records finally.
+        // Then, we set 1 more storage group, create a template with 2 measurements and set
+        // the template to this storage group and create 1 device using template.
+        // The snapshot should have 100 + 2 + 5 * 2 + 2 - 1 - 1 + 1 + 1 = 114 records,
+        // and we have root record,
+        // so we have 114 + 1 = 115 records finally.
         for (String content : lines) {
-          logger.info(content);
+          System.out.println(content);
         }
       }
-      Assert.assertEquals(113, lineNum);
+      Assert.assertEquals(115, lineNum);
     } catch (IOException e) {
       Assert.fail(e.getMessage());
     }