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 2020/12/14 11:30:41 UTC

[iotdb] branch master updated: [IOTDB-868] change mlog from txt to bin (#1721)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a75b722  [IOTDB-868] change mlog from txt to bin (#1721)
a75b722 is described below

commit a75b7225e915b98e8d5453ffd31a57cb1ad48917
Author: chaow <ru...@foxmail.com>
AuthorDate: Mon Dec 14 19:30:24 2020 +0800

    [IOTDB-868] change mlog from txt to bin (#1721)
---
 docs/SystemDesign/SchemaManager/SchemaManager.md   |  20 +-
 .../UserGuide/System Tools/MLogParser Tool.md      |  21 +-
 .../zh/SystemDesign/SchemaManager/SchemaManager.md |  20 +-
 .../zh/UserGuide/System Tools/MLogParser Tool.md   |  21 +-
 server/file-changelists/mlog-changelist.md         |   4 +
 .../resources/conf/iotdb-engine.properties         |  11 +-
 .../assembly/resources/tools/mlog/mLogParser.bat   |  62 +++
 .../assembly/resources/tools/mlog/mLogParser.sh    |  48 +++
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  16 +-
 .../org/apache/iotdb/db/conf/IoTDBConfigCheck.java |  35 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |   6 +
 .../{MLogWriter.java => MLogTxtWriter.java}        |  72 +++-
 .../org/apache/iotdb/db/metadata/MManager.java     | 236 +++++------
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 127 +++---
 .../apache/iotdb/db/metadata/MetadataConstant.java |  10 +-
 .../iotdb/db/metadata/logfile/MLogReader.java      |  74 ++++
 .../iotdb/db/metadata/logfile/MLogTxtReader.java   |  77 ++++
 .../iotdb/db/metadata/logfile/MLogWriter.java      | 450 +++++++++++++++++++++
 .../org/apache/iotdb/db/metadata/mnode/MNode.java  |  25 +-
 .../iotdb/db/metadata/mnode/MeasurementMNode.java  |  37 +-
 .../iotdb/db/metadata/mnode/StorageGroupMNode.java |  22 +-
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   5 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |  28 +-
 ...tStorageGroupPlan.java => ChangeAliasPlan.java} |  74 ++--
 ...rageGroupPlan.java => ChangeTagOffsetPlan.java} |  76 ++--
 .../qp/physical/sys/CreateMultiTimeSeriesPlan.java |   4 +-
 .../db/qp/physical/sys/CreateTimeSeriesPlan.java   |  66 ++-
 .../db/qp/physical/sys/DeleteTimeSeriesPlan.java   |   4 +
 .../{SetStorageGroupPlan.java => MNodePlan.java}   |  88 ++--
 .../db/qp/physical/sys/MeasurementMNodePlan.java   | 143 +++++++
 .../db/qp/physical/sys/SetStorageGroupPlan.java    |  18 +-
 .../iotdb/db/qp/physical/sys/SetTTLPlan.java       |   1 -
 ...geGroupPlan.java => StorageGroupMNodePlan.java} |  79 ++--
 .../db/sync/receiver/transfer/SyncServiceImpl.java |  37 +-
 .../org/apache/iotdb/db/tools/mlog/MLogParser.java | 194 +++++++++
 .../db/tools/upgrade/TsFileOnlineUpgradeTool.java  |   1 +
 .../org/apache/iotdb/db/writelog/io/LogWriter.java |  56 ++-
 .../db/writelog/node/ExclusiveWriteLogNode.java    |  13 +-
 .../db/integration/IoTDBCreateSnapshotIT.java      |  74 ++--
 .../iotdb/db/integration/IoTDBRestartIT.java       |   2 -
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  |   9 +-
 .../org/apache/iotdb/db/tools/MLogParserTest.java  | 150 +++++++
 .../org/apache/iotdb/db/tools/WalCheckerTest.java  |   5 +-
 .../iotdb/db/writelog/io/LogWriterReaderTest.java  |   4 +-
 .../db/writelog/io/MultiFileLogReaderTest.java     |   4 +-
 45 files changed, 2019 insertions(+), 510 deletions(-)

diff --git a/docs/SystemDesign/SchemaManager/SchemaManager.md b/docs/SystemDesign/SchemaManager/SchemaManager.md
index 4773289..53d97d1 100644
--- a/docs/SystemDesign/SchemaManager/SchemaManager.md
+++ b/docs/SystemDesign/SchemaManager/SchemaManager.md
@@ -179,10 +179,10 @@ The root node exists by default. Creating storage groups, deleting storage group
 
 ### Create condition
 
-To speed up restarting of IoTDB, we set checkpoint for MTree to avoid reading `mlog.txt` and executing the commands line by line. There are two ways to create MTree snapshot:
+To speed up restarting of IoTDB, we set checkpoint for MTree to avoid reading `mlog.bin` and executing the commands line by line. There are two ways to create MTree snapshot:
 1. Background checking and creating automatically: Every 10 minutes, background thread checks the last modified time of MTree. If:
-  * If users haven’t modified MTree for more than 1 hour (could be configured), which means `mlog.txt` hasn’t been updated for more than 1 hour
-  * `mlog.txt` has reached 100000 lines (could be configured)
+  * If users haven’t modified MTree for more than 1 hour (could be configured), which means `mlog.bin` hasn’t been updated for more than 1 hour
+  * `mlog.bin` has reached 100000 lines (could be configured)
 
 2. Creating manually: Users can use `create snapshot for schema` to create MTree snapshot
 
@@ -196,10 +196,10 @@ The method is `MManager.createMTreeSnapshot()`:
   * MeasurementMNode: 2, name, alias, TSDataType, TSEncoding, CompressionType, props, offset, children size
 
 3. After serialization, rename the temp file to a formal file (`mtree.snapshot`), to avoid crush of server and failure of serialization.
-4. Clear `mlog.txt` by `MLogWriter.clear()` method:
-  * Close BufferedWriter and delete `mlog.txt` file
+4. Clear `mlog.bin` by `MLogWriter.clear()` method:
+  * Close BufferedWriter and delete `mlog.bin` file
   * Create a new BufferedWriter
-  * Set `lineNumber` as 0. `lineNumber` records the line number of `mlog.txt`, which is used for background thread to check whether it is larger than the threshold configured by user.
+  * Set `logNumber` as 0. `logNumber` records the log number of `mlog.bin`, which is used for background thread to check whether it is larger than the threshold configured by user.
 
 5. Release the read lock.
 
@@ -209,13 +209,13 @@ The method is `MManager.initFromLog()`:
 
 1. Check whether the temp file `mtree.snapshot.tmp` exists. If so, there may exist crush of server and failure of serialization. Delete the temp file.
 2. Check whether the snapshot file `mtree.snapshot` exists. If not, use a new MTree; otherwise, start deserializing from snapshot and get MTree
-3. Read and operate all lines in `mlog.txt` and finish the recover process of MTree. Update `lineNumber` at the same time and return it for recording the line number of `mlog.txt` afterwards.
+3. Read and operate all lines in `mlog.bin` and finish the recover process of MTree. Update `lineNumber` at the same time and return it for recording the line number of `mlog.bin` afterwards.
 
 ## Log management of metadata
 
-* org.apache.iotdb.db.metadata.MLogWriter
+* org.apache.iotdb.db.metadata.logfile.MLogWriter
 
-All metadata operations are recorded in a metadata log file, which defaults to data/system/schema/mlog.txt.
+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.
 
@@ -270,7 +270,7 @@ sql examples and the corresponding mlog record:
 ## TLog
 * org.apache.iotdb.db.metadata.TagLogFile
 
-All timeseries tag/attribute information will be saved in the tag file, which defaults to data/system/schema/mlog.txt.
+All timeseries tag/attribute information will be saved in the tag file, which defaults to data/system/schema/mlog.bin.
 
 * Total number of bytes of persistence for tags and attributes of each time series is L, which can be configured in the iotdb-engine.properties
 
diff --git a/server/file-changelists/mlog-changelist.md b/docs/UserGuide/System Tools/MLogParser Tool.md
similarity index 66%
copy from server/file-changelists/mlog-changelist.md
copy to docs/UserGuide/System Tools/MLogParser Tool.md
index 4c0b45d..4446c16 100644
--- a/server/file-changelists/mlog-changelist.md
+++ b/docs/UserGuide/System Tools/MLogParser Tool.md	
@@ -7,9 +7,9 @@
     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
@@ -19,8 +19,19 @@
 
 -->
 
-# 0.10.x -> 0.11.x
+# What
+
+After version 0.12.x, IoTDB encodes metadata files into binary format.
+
+If you want to parse metadata into a human readable way, you can use this tool to parse the specified metadata file.
+
+The tool can parse snapshot files and mlog files.
+
+# How to use
+
+Linux/MacOS
+> ./mLogParser.sh -f /your path/mlog.bin -o /your path/mlog.txt
 
-Last updated on 2020-11-5 by Haonan Hou
+Windows
 
-No change.
\ No newline at end of file
+> .\mLogParser.bat -f \your path\mlog.bin -o \your path\mlog.txt
\ No newline at end of file
diff --git a/docs/zh/SystemDesign/SchemaManager/SchemaManager.md b/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
index 365ca8a..a651746 100644
--- a/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
+++ b/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
@@ -176,10 +176,10 @@ IoTDB 的元数据管理采用目录树的形式,倒数第二层为设备层
 
 ### 创建条件
 
-为了加快 IoTDB 重启速度,我们为 MTree 设置了检查点,这样避免了在重启时按行读取并复现 `mlog.txt` 中的信息。创建 MTree 的快照有两种方式:
+为了加快 IoTDB 重启速度,我们为 MTree 设置了检查点,这样避免了在重启时按行读取并复现 `mlog.bin` 中的信息。创建 MTree 的快照有两种方式:
 1. 后台线程检查自动创建:每隔10分钟,后台线程检查 MTree 的最后修改时间,需要同时满足
-  * 用户超过1小时(可配置)没修改 MTree,即`mlog.txt` 文件超过1小时没有修改
-  * `mlog.txt` 中积累了100000行日志(可配置)
+  * 用户超过1小时(可配置)没修改 MTree,即`mlog.bin` 文件超过1小时没有修改
+  * `mlog.bin` 中积累了100000行日志(可配置)
   
 2. 手动创建:使用`create snapshot for schema`命令手动触发创建 MTree 快照
 
@@ -194,10 +194,10 @@ IoTDB 的元数据管理采用目录树的形式,倒数第二层为设备层
   * 传感器节点:2,名字,别名,数据类型,编码,压缩方式,属性,偏移量,子节点个数
   
 3. 序列化结束后,将临时文件重命名为正式文件(`mtree.snapshot`),防止在序列化过程中出现服务器人为或意外关闭,导致序列化失败的情况。
-4. 调用`MLogWriter.clear()`方法,清空 `mlog.txt`:
-  * 关闭 BufferedWriter,删除`mlog.txt`文件;
+4. 调用`MLogWriter.clear()`方法,清空 `mlog.bin`:
+  * 关闭 BufferedWriter,删除`mlog.bin`文件;
   * 新建一个 BufferedWriter;
-  * 将 `lineNumber` 置为0,`lineNumber` 记录`mlog.txt`的行数,用于在后台检查时判断其是否超过用户配置的阈值而触发自动创建快照。
+  * 将 `logNumber` 置为0,`logNumber` 记录`mlog.bin`的日志行数,用于在后台检查时判断其是否超过用户配置的阈值而触发自动创建快照。
 
 5. 释放 MTree 读锁
 
@@ -207,13 +207,13 @@ IoTDB 的元数据管理采用目录树的形式,倒数第二层为设备层
 
 1. 检查临时文件`mtree.snapshot.tmp`是否存在,如果存在证明在创建快照的序列化过程中出现服务器人为或意外关闭,导致序列化失败,删除临时文件;
 2. 检查快照文件`mtree.snapshot`是否存在。如果不存在,则使用新的 MTree;否则启动反序列化过程,得到 MTree
-3. 对于`mlog.txt`中的内容,逐行读取并操作,完成 MTree 的恢复。读取过程中更新 `lineNumber`,并返回,用于后面`mlog.txt`行数的记录。
+3. 对于`mlog.bin`中的内容,逐行读取并操作,完成 MTree 的恢复。读取过程中更新 `logNumber`,并返回,用于后面`mlog.bin`行数的记录。
 
 ## 元数据日志管理
 
-* org.apache.iotdb.db.metadata.MLogWriter
+* org.apache.iotdb.db.metadata.logfile.MLogWriter
 
-所有元数据的操作均会记录到元数据日志文件中,此文件默认为 data/system/schema/mlog.txt。
+所有元数据的操作均会记录到元数据日志文件中,此文件默认为 data/system/schema/mlog.bin。
 
 系统重启时会重做 mlog 中的日志,重做之前需要标记不需要记录日志。当重启结束后,标记需要记录日志。
 
@@ -268,7 +268,7 @@ IoTDB 的元数据管理采用目录树的形式,倒数第二层为设备层
 * org.apache.iotdb.db.metadata.TagLogFile
 
 
-所有时间序列的标签/属性信息都会保存在标签文件中,此文件默认为 data/system/schema/mlog.txt。
+所有时间序列的标签/属性信息都会保存在标签文件中,此文件默认为 data/system/schema/mlog.bin。
 
 * 每条时间序列的 tags 和 attributes 持久化总字节数为 L,在 iotdb-engine.properties 中配置。
 
diff --git a/server/file-changelists/mlog-changelist.md b/docs/zh/UserGuide/System Tools/MLogParser Tool.md
similarity index 66%
copy from server/file-changelists/mlog-changelist.md
copy to docs/zh/UserGuide/System Tools/MLogParser Tool.md
index 4c0b45d..5f66eee 100644
--- a/server/file-changelists/mlog-changelist.md
+++ b/docs/zh/UserGuide/System Tools/MLogParser Tool.md	
@@ -7,9 +7,9 @@
     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
@@ -19,8 +19,19 @@
 
 -->
 
-# 0.10.x -> 0.11.x
+# 工具说明
+
+0.12.x版本之后,IoTDB将元数据文件编码成二进制。
+
+如果想要解析元数据成人可读的方式,可以使用本工具来解析指定元数据文件。
+
+该工具可以同时解析snapshot文件和mlog文件。
+
+# 使用方式
+
+Linux/MacOS
+> ./mLogParser.sh -f /your path/mlog.bin -o /your path/mlog.txt
 
-Last updated on 2020-11-5 by Haonan Hou
+Windows
 
-No change.
\ No newline at end of file
+> .\mLogParser.bat -f \your path\mlog.bin -o \your path\mlog.txt
\ No newline at end of file
diff --git a/server/file-changelists/mlog-changelist.md b/server/file-changelists/mlog-changelist.md
index 4c0b45d..d70ce90 100644
--- a/server/file-changelists/mlog-changelist.md
+++ b/server/file-changelists/mlog-changelist.md
@@ -19,6 +19,10 @@
 
 -->
 
+# 0.11.x -> 0.12.x
+
+from txt format to binary format, updated by Chao Wang
+
 # 0.10.x -> 0.11.x
 
 Last updated on 2020-11-5 by Haonan Hou
diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index 006ddbe..05d773e 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -178,7 +178,7 @@ timestamp_precision=ms
 # default_ttl=36000000
 
 # Size of log buffer in each log node(in byte).
-# If WAL is enabled and the size of a insert plan is smaller than this parameter, then the insert plan will be rejected by WAL
+# If WAL is enabled and the size of a insert plan is larger than this parameter, then the insert plan will be rejected by WAL
 # If it sets a value smaller than 0, use the default value 16777216
 wal_buffer_size=16777216
 
@@ -186,6 +186,11 @@ wal_buffer_size=16777216
 # It may cause memTable size smaller if it is a large value
 tsfile_size_threshold=1
 
+# Size of log buffer in each metadata operation plan(in byte).
+# If the size of a metadata operation plan is larger than this parameter, then it will be rejected by MManager
+# If it sets a value smaller than 0, use the default value 1024*1024
+mlog_buffer_size=1048576
+
 # When a memTable's size (in byte) exceeds this, the memtable is flushed to disk. The default threshold is 1 GB.
 memtable_size_threshold=1073741824
 
@@ -402,7 +407,7 @@ enable_external_sort=true
 # If the num of simultaneous chunk reading is greater than external_sort_threshold, external sorting is used.
 # When external_sort_threshold increases, the number of chunks sorted at the same time in memory may increase and this will occupy more memory.
 # When external_sort_threshold decreases, triggering external sorting will increase the time-consuming.
-external_sort_threshold = 1000
+external_sort_threshold=1000
 
 
 ####################
@@ -593,4 +598,4 @@ concurrent_index_build_thread=0
 default_index_window_range=10
 
 # buffer parameter for index processor.
-index_buffer_size = 134217728
\ No newline at end of file
+index_buffer_size=134217728
\ No newline at end of file
diff --git a/server/src/assembly/resources/tools/mlog/mLogParser.bat b/server/src/assembly/resources/tools/mlog/mLogParser.bat
new file mode 100644
index 0000000..72599e3
--- /dev/null
+++ b/server/src/assembly/resources/tools/mlog/mLogParser.bat
@@ -0,0 +1,62 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one
+@REM or more contributor license agreements.  See the NOTICE file
+@REM distributed with this work for additional information
+@REM regarding copyright ownership.  The ASF licenses this file
+@REM to you under the Apache License, Version 2.0 (the
+@REM "License"); you may not use this file except in compliance
+@REM with the License.  You may obtain a copy of the License at
+@REM
+@REM     http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing,
+@REM software distributed under the License is distributed on an
+@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+@REM KIND, either express or implied.  See the License for the
+@REM specific language governing permissions and limitations
+@REM under the License.
+@REM
+
+
+@echo off
+echo ````````````````````````
+echo Starting Parsing the IoTDB MLog or Snapshot
+echo ````````````````````````
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd %~dp0..\..
+if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%CD%
+popd
+
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.tools.mlog.MLogParser
+if NOT DEFINED JAVA_HOME goto :err
+
+@REM -----------------------------------------------------------------------------
+@REM ***** CLASSPATH library setting *****
+@REM Ensure that any user defined CLASSPATH variables are not used on startup
+set CLASSPATH="%IOTDB_HOME%\lib\*"
+
+goto okClasspath
+
+:append
+set CLASSPATH=%CLASSPATH%;%1
+goto :eof
+
+@REM -----------------------------------------------------------------------------
+:okClasspath
+
+"%JAVA_HOME%\bin\java" -cp "%CLASSPATH%" %MAIN_CLASS% %*
+
+goto finally
+
+
+:err
+echo JAVA_HOME environment variable must be set!
+pause
+
+
+@REM -----------------------------------------------------------------------------
+:finally
+
+ENDLOCAL
\ No newline at end of file
diff --git a/server/src/assembly/resources/tools/mlog/mLogParser.sh b/server/src/assembly/resources/tools/mlog/mLogParser.sh
new file mode 100644
index 0000000..ee94a77
--- /dev/null
+++ b/server/src/assembly/resources/tools/mlog/mLogParser.sh
@@ -0,0 +1,48 @@
+#!/bin/sh
+#
+# 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.
+#
+
+echo ---------------------
+echo Starting Parsing the IoTDB Mlog or Snapshot
+echo ---------------------
+
+if [ -z "${IOTDB_HOME}" ]; then
+  export IOTDB_HOME="$(cd "`dirname "$0"`"/../..; pwd)"
+fi
+
+if [ -n "$JAVA_HOME" ]; then
+    for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
+        if [ -x "$java" ]; then
+            JAVA="$java"
+            break
+        fi
+    done
+else
+    JAVA=java
+fi
+
+CLASSPATH=""
+for f in ${IOTDB_HOME}/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}":"$f
+done
+
+MAIN_CLASS=org.apache.iotdb.db.tools.mlog.MLogParser
+
+"$JAVA" -cp "$CLASSPATH" "$MAIN_CLASS" "$@"
+exit $?
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index b620ab9..27af6b1 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -204,13 +204,19 @@ public class IoTDBConfig {
 
   /**
    * Size of log buffer in each log node(in byte). If WAL is enabled and the size of a insert plan
-   * is smaller than this parameter, then the insert plan will be rejected by WAL.
+   * is larger than this parameter, then the insert plan will be rejected by WAL.
    */
   private int walBufferSize = 16 * 1024 * 1024;
 
   private int estimatedSeriesSize = 300;
 
   /**
+   * Size of log buffer for every MetaData operation. If the size of a MetaData operation plan
+   * is larger than this parameter, then the MetaData operation plan will be rejected by MManager.
+   */
+  private int mlogBufferSize = 1024 * 1024;
+
+  /**
    * default base dir, stores all IoTDB runtime files
    */
   private static final String DEFAULT_BASE_DIR = "data";
@@ -2108,4 +2114,12 @@ public class IoTDBConfig {
     this.rpcAdvancedCompressionEnable = rpcAdvancedCompressionEnable;
     RpcTransportFactory.setUseSnappy(this.rpcAdvancedCompressionEnable);
   }
+
+  public int getMlogBufferSize() {
+    return mlogBufferSize;
+  }
+
+  public void setMlogBufferSize(int mlogBufferSize) {
+    this.mlogBufferSize = mlogBufferSize;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
index 6b4ad13..9d7efd1 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
@@ -33,6 +33,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
@@ -184,18 +185,29 @@ public class IoTDBConfigCheck {
             inputStream, TSFileConfig.STRING_CHARSET)) {
       properties.load(inputStreamReader);
     }
-    // check whether upgrading
-    if (!properties.containsKey(IOTDB_VERSION_STRING)
-        || properties.getProperty(IOTDB_VERSION_STRING).startsWith("0.10")
-        || properties.getProperty(IOTDB_VERSION_STRING).startsWith("0.11")) {
-      logger.error("DO NOT UPGRADE IoTDB from v0.11 or lower version NOW");
+    // check whether upgrading from <=v0.10 to v0.12
+    if (!properties.containsKey(IOTDB_VERSION_STRING) ||
+      properties.getProperty(IOTDB_VERSION_STRING).startsWith("0.10")) {
+      logger.error("DO NOT UPGRADE IoTDB from v0.10 or lower version to v0.12!"
+          + " Please upgrade to v0.11 first");
       System.exit(-1);
     }
+
+    // check whether upgrading from v0.11 to v0.12
+    if (properties.getProperty(IOTDB_VERSION_STRING).startsWith("0.11")) {
+      logger.info("Upgrading IoTDB from v0.11 to v0.12, checking files...");
+      checkUnClosedTsFileV2();
+      upgradePropertiesFile();
+      logger.info("Upgrade to IoTDB v0.12 successfully!");
+    }
+
+    MLogWriter.upgradeMLog();
+
     checkProperties();
   }
 
   /**
-   * upgrade 0.10 properties to 0.11 properties
+   * upgrade 0.11 properties to 0.12 properties
    */
   @SuppressWarnings("unused")
   private void upgradePropertiesFile()
@@ -294,14 +306,14 @@ public class IoTDBConfigCheck {
   }
 
   /**
-   * ensure all TsFiles are closed in 0.10 when starting 0.11
+   * ensure all TsFiles are closed in 0.11 when starting 0.12
    */
   @SuppressWarnings("unused")
   private void checkUnClosedTsFileV2() {
     if (SystemFileFactory.INSTANCE.getFile(WAL_DIR).isDirectory()
-        && SystemFileFactory.INSTANCE.getFile(WAL_DIR).list().length != 0) {
-      logger.error("Unclosed Version-2 TsFile detected, please run 'flush' on v0.10 IoTDB"
-          + " before upgrading to v0.11");
+      && SystemFileFactory.INSTANCE.getFile(WAL_DIR).list().length != 0) {
+      logger.error("Unclosed Version-2 TsFile detected, please run 'flush' on v0.11 IoTDB"
+        + " before upgrading to v0.12");
       System.exit(-1);
     }
     checkUnClosedTsFileV2InFolders(DirectoryManager.getInstance().getAllSequenceFileFolders());
@@ -328,13 +340,12 @@ public class IoTDBConfigCheck {
             .listFilesBySuffix(partitionDir.toString(), TsFileResource.RESOURCE_SUFFIX);
           if (tsfiles.length != resources.length) {
             logger.error("Unclosed Version-2 TsFile detected, please run 'flush' on v0.10 IoTDB"
-                + " before upgrading to v0.11");
+              + " before upgrading to v0.11");
             System.exit(-1);
           }
         }
       }
     }
   }
-
 }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 2a4827e..c6b7847 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -239,6 +239,12 @@ public class IoTDBDescriptor {
         conf.setWalBufferSize(walBufferSize);
       }
 
+      int mlogBufferSize = Integer.parseInt(properties.getProperty("mlog_buffer_size",
+        Integer.toString(conf.getMlogBufferSize())));
+      if (mlogBufferSize > 0) {
+        conf.setMlogBufferSize(mlogBufferSize);
+      }
+
       conf.setMultiDirStrategyClassName(properties.getProperty("multi_dir_strategy",
           conf.getMultiDirStrategyClassName()));
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MLogWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
similarity index 70%
rename from server/src/main/java/org/apache/iotdb/db/metadata/MLogWriter.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
index 538e6ea..e814277 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.metadata;
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.FileWriter;
@@ -28,17 +29,22 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 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.StorageGroupMNodePlan;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class MLogWriter {
+public class MLogTxtWriter implements AutoCloseable {
 
-  private static final Logger logger = LoggerFactory.getLogger(MLogWriter.class);
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtWriter.class);
   private static final String STRING_TYPE = "%s,%s,%s" + System.lineSeparator();
   private static final String LINE_SEPARATOR = System.lineSeparator();
   private final File logFile;
@@ -46,7 +52,7 @@ public class MLogWriter {
   private FileChannel channel;
   private final AtomicInteger lineNumber;
 
-  public MLogWriter(String schemaDir, String logFileName) throws IOException {
+  public MLogTxtWriter(String schemaDir, String logFileName) throws IOException {
     File metadataDir = SystemFileFactory.INSTANCE.getFile(schemaDir);
     if (!metadataDir.exists()) {
       if (metadataDir.mkdirs()) {
@@ -62,6 +68,14 @@ public class MLogWriter {
     lineNumber = new AtomicInteger(0);
   }
 
+  public MLogTxtWriter(String logFileName) throws FileNotFoundException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFileName);
+    fileOutputStream = new FileOutputStream(logFile, true);
+    channel = fileOutputStream.getChannel();
+    lineNumber = new AtomicInteger(0);
+  }
+
+  @Override
   public void close() throws IOException {
     fileOutputStream.close();
   }
@@ -154,8 +168,12 @@ public class MLogWriter {
     }
 
     // if both old mlog and mlog.tmp exist, delete mlog tmp, then do upgrading
-    if (tmpLogFile.exists() && !tmpLogFile.delete()) {
-      throw new IOException("Deleting " + tmpLogFile + "failed.");
+    if (tmpLogFile.exists()) {
+      try {
+        Files.delete(Paths.get(tmpLogFile.toURI()));
+      } catch (IOException e) {
+        throw new IOException("Deleting " + tmpLogFile + "failed with exception " + e.getMessage());
+      }
     }
     // upgrading
     try (BufferedReader reader = new BufferedReader(new FileReader(logFile));
@@ -184,6 +202,50 @@ public class MLogWriter {
     lineNumber.set(0);
   }
 
+  public void serializeMNode(MNodePlan plan) throws IOException {
+    StringBuilder s = new StringBuilder(String.valueOf(MetadataConstant.STORAGE_GROUP_MNODE_TYPE));
+    s.append(",").append(plan.getName()).append(",");
+    s.append(plan.getChildSize());
+    s.append(LINE_SEPARATOR);
+    ByteBuffer buff = ByteBuffer.wrap(s.toString().getBytes());
+    channel.write(buff);
+    lineNumber.incrementAndGet();
+  }
+
+  public void serializeMeasurementMNode(MeasurementMNodePlan plan) throws IOException {
+    StringBuilder s = new StringBuilder(String.valueOf(MetadataConstant.MEASUREMENT_MNODE_TYPE));
+    s.append(",").append(plan.getName()).append(",");
+    if (plan.getAlias() != null) {
+      s.append(plan.getAlias());
+    }
+    MeasurementSchema schema = plan.getSchema();
+    s.append(",").append(schema.getType().ordinal()).append(",");
+    s.append(schema.getEncodingType().ordinal()).append(",");
+    s.append(schema.getCompressor().ordinal()).append(",");
+    if (schema.getProps() != null) {
+      for (Map.Entry<String, String> entry : schema.getProps().entrySet()) {
+        s.append(entry.getKey()).append(":").append(entry.getValue()).append(";");
+      }
+    }
+    s.append(",").append(plan.getOffset()).append(",");
+    s.append(plan.getChildSize());
+    s.append(LINE_SEPARATOR);
+    ByteBuffer buff = ByteBuffer.wrap(s.toString().getBytes());
+    channel.write(buff);
+    lineNumber.incrementAndGet();
+  }
+
+  public void serializeStorageGroupMNode(StorageGroupMNodePlan plan) throws IOException {
+    StringBuilder s = new StringBuilder(String.valueOf(MetadataConstant.STORAGE_GROUP_MNODE_TYPE));
+    s.append(",").append(plan.getName()).append(",");
+    s.append(plan.getDataTTL()).append(",");
+    s.append(plan.getChildSize());
+    s.append(LINE_SEPARATOR);
+    ByteBuffer buff = ByteBuffer.wrap(s.toString().getBytes());
+    channel.write(buff);
+    lineNumber.incrementAndGet();
+  }
+
   int getLineNumber() {
     return lineNumber.get();
   }
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 397805c..f7bc057 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
@@ -21,9 +21,7 @@ package org.apache.iotdb.db.metadata;
 import static java.util.stream.Collectors.toList;
 import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
 
-import java.io.BufferedReader;
 import java.io.File;
-import java.io.FileReader;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.util.ArrayDeque;
@@ -45,6 +43,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
@@ -60,21 +59,31 @@ import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
 import org.apache.iotdb.db.monitor.MonitorConstants;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+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;
+import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+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.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.utils.RandomDeleteCache;
 import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.db.utils.TypeInferenceUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.exception.cache.CacheException;
@@ -89,6 +98,7 @@ import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 /**
  * This class takes the responsibility of serialization of all the metadata info and persistent it
  * into files. This class contains all the interfaces to modify the metadata for delta system. All
@@ -136,6 +146,7 @@ public class MManager {
 
   private File logFile;
   private ScheduledExecutorService timedCreateMTreeSnapshotThread;
+  private ScheduledExecutorService timedForceMLogThread;
 
   /**
    * threshold total size of MTree
@@ -187,12 +198,13 @@ public class MManager {
       }
     };
 
+
     if (config.isEnableMTreeSnapshot()) {
       timedCreateMTreeSnapshotThread = Executors.newSingleThreadScheduledExecutor(r -> new Thread(r,
-          "timedCreateMTreeSnapshotThread"));
+        "timedCreateMTreeSnapshotThread"));
       timedCreateMTreeSnapshotThread
-          .scheduleAtFixedRate(this::checkMTreeModified, MTREE_SNAPSHOT_THREAD_CHECK_TIME,
-              MTREE_SNAPSHOT_THREAD_CHECK_TIME, TimeUnit.SECONDS);
+        .scheduleAtFixedRate(this::checkMTreeModified, MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME, TimeUnit.SECONDS);
     }
   }
 
@@ -223,11 +235,10 @@ public class MManager {
       }
 
       logWriter = new MLogWriter(config.getSchemaDir(), MetadataConstant.METADATA_LOG);
-      logWriter.setLineNumber(lineNumber);
+      logWriter.setLogNum(lineNumber);
       isRecovering = false;
     } catch (IOException | MetadataException e) {
-      mtree = new MTree();
-      logger.error("Cannot read MTree from file, using an empty new one", e);
+      logger.error("Cannot recover all MTree from file, we try to recover as possible as we can", e);
     }
     reportedDataTypeTotalNum = 0L;
     initialized = true;
@@ -258,28 +269,37 @@ public class MManager {
     // init the metadata from the operation log
     if (logFile.exists()) {
       int idx = 0;
-      try (FileReader fr = new FileReader(logFile);
-          BufferedReader br = new BufferedReader(fr)) {
-        String cmd;
-        while ((cmd = br.readLine()) != null) {
-          try {
-            operation(cmd);
-            idx++;
-          } catch (Exception e) {
-            logger.error("Can not operate cmd {}", cmd, e);
-          }
-        }
+      try (MLogReader mLogReader = new MLogReader(config.getSchemaDir(), MetadataConstant.METADATA_LOG);) {
+        idx = applyMlog(mLogReader);
+        logger.debug("spend {} ms to deserialize mtree from mlog.bin",
+            System.currentTimeMillis() - time);
+        return idx;
+      } catch (Exception e) {
+        throw new IOException("Failed to parser mlog.bin for err:" +  e.toString());
       }
-      logger.debug("spend {} ms to deserialize mtree from mlog.txt",
-          System.currentTimeMillis() - time);
-      return idx;
-    } else if (mtreeSnapshot.exists()) {
-      throw new IOException("mtree snapshot file exists but mlog.txt does not exist.");
     } else {
       return 0;
     }
   }
 
+  private int applyMlog(MLogReader mLogReader) {
+    int idx = 0;
+    while (mLogReader.hasNext()) {
+      PhysicalPlan plan = null;
+      try {
+        plan = mLogReader.next();
+        if (plan == null) {
+          continue;
+        }
+        operation(plan);
+        idx++;
+      } catch (Exception e) {
+        logger.error("Can not operate cmd {} for err:", plan == null ? "" : plan.getOperatorType(), e);
+      }
+    }
+    return idx;
+  }
+
   /**
    * function for clearing MTree
    */
@@ -304,106 +324,48 @@ public class MManager {
         timedCreateMTreeSnapshotThread.shutdownNow();
         timedCreateMTreeSnapshotThread = null;
       }
+      if (timedForceMLogThread != null) {
+        timedForceMLogThread.shutdownNow();
+        timedForceMLogThread = null;
+      }
     } catch (IOException e) {
       logger.error("Cannot close metadata log writer, because:", e);
     }
   }
 
-  private void operateCreateTimeseries(String[] args) throws IOException, MetadataException {
-    if (args.length > 8) {
-      String[] tmpArgs = new String[8];
-      tmpArgs[0] = args[0];
-      int i = 1;
-      tmpArgs[1] = "";
-      for (; i < args.length - 7; i++) {
-        tmpArgs[1] += args[i] + ",";
-      }
-      tmpArgs[1] += args[i++];
-      for (int j = 2; j < 8; j++) {
-        tmpArgs[j] = args[i++];
-      }
-      args = tmpArgs;
-    }
-    Map<String, String> props = null;
-    if (!args[5].isEmpty()) {
-      String[] keyValues = args[5].split("&");
-      String[] kv;
-      props = new HashMap<>();
-      for (String keyValue : keyValues) {
-        kv = keyValue.split("=");
-        props.put(kv[0], kv[1]);
-      }
-    }
-
-    String alias = null;
-    if (!args[6].isEmpty()) {
-      alias = args[6];
-    }
-    long offset = -1L;
-    Map<String, String> tagMap = null;
-    if (!args[7].isEmpty()) {
-      offset = Long.parseLong(args[7]);
-      tagMap = tagLogFile.readTag(config.getTagAttributeTotalSize(), offset);
-    }
-
-    CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(new PartialPath(args[1]),
-        TSDataType.deserialize(Short.parseShort(args[2])),
-        TSEncoding.deserialize(Short.parseShort(args[3])),
-        CompressionType.deserialize(Short.parseShort(args[4])), props, tagMap, null, alias);
-
-    createTimeseries(plan, offset);
-  }
-
-  private void operateDeleteTimeseries(String[] args)
-      throws MetadataException {
-    if (args.length > 2) {
-      StringBuilder tmp = new StringBuilder();
-      for (int i = 1; i < args.length - 1; i++) {
-        tmp.append(args[i]).append(",");
-      }
-      tmp.append(args[args.length - 1]);
-      args[1] = tmp.toString();
-    }
-    String failedTimeseries = deleteTimeseries(new PartialPath(args[1]));
-    if (!failedTimeseries.isEmpty()) {
-      throw new DeleteFailedException(failedTimeseries);
-    }
-  }
-
-  public void operation(String cmd) throws IOException, MetadataException {
-    // see createTimeseries() to get the detailed format of the cmd
-    String[] args = cmd.trim().split(",", -1);
-    switch (args[0]) {
-      case MetadataOperationType.CREATE_TIMESERIES:
-        operateCreateTimeseries(args);
+  public void operation(PhysicalPlan plan) throws IOException, MetadataException {
+    switch (plan.getOperatorType()) {
+      case CREATE_TIMESERIES:
+        CreateTimeSeriesPlan createTimeSeriesPlan = (CreateTimeSeriesPlan) plan;
+        createTimeseries(createTimeSeriesPlan, createTimeSeriesPlan.getTagOffset());
         break;
-      case MetadataOperationType.DELETE_TIMESERIES:
-        operateDeleteTimeseries(args);
+      case DELETE_TIMESERIES:
+        DeleteTimeSeriesPlan deleteTimeSeriesPlan = (DeleteTimeSeriesPlan) plan;
+        // cause we only has one path for one DeleteTimeSeriesPlan
+        deleteTimeseries(deleteTimeSeriesPlan.getPaths().get(0));
         break;
-      case MetadataOperationType.SET_STORAGE_GROUP:
-        try {
-          setStorageGroup(new PartialPath(args[1]));
-        }
-        // two time series may set one storage group concurrently,
-        // that's normal in our concurrency control protocol
-        catch (MetadataException e){
-          logger.info("concurrently operate set storage group cmd {} twice", cmd);
-        }
+      case SET_STORAGE_GROUP:
+        SetStorageGroupPlan setStorageGroupPlan = (SetStorageGroupPlan) plan;
+        setStorageGroup(setStorageGroupPlan.getPath());
         break;
-      case MetadataOperationType.DELETE_STORAGE_GROUP:
-        deleteStorageGroups(Collections.singletonList(new PartialPath(args[1])));
+      case DELETE_STORAGE_GROUP:
+        DeleteStorageGroupPlan deleteStorageGroupPlan = (DeleteStorageGroupPlan) plan;
+        deleteStorageGroups(deleteStorageGroupPlan.getPaths());
         break;
-      case MetadataOperationType.SET_TTL:
-        setTTL(new PartialPath(args[1]), Long.parseLong(args[2]));
+      case TTL:
+        SetTTLPlan setTTLPlan = (SetTTLPlan) plan;
+        setTTL(setTTLPlan.getStorageGroup(), setTTLPlan.getDataTTL());
         break;
-      case MetadataOperationType.CHANGE_OFFSET:
-        changeOffset(new PartialPath(args[1]), Long.parseLong(args[2]));
+      case CHANGE_ALIAS:
+        ChangeAliasPlan changeAliasPlan = (ChangeAliasPlan) plan;
+        changeAlias(changeAliasPlan.getPath(), changeAliasPlan.getAlias());
         break;
-      case MetadataOperationType.CHANGE_ALIAS:
-        changeAlias(new PartialPath(args[1]), args[2]);
+      case CHANGE_TAG_OFFSET:
+        ChangeTagOffsetPlan changeTagOffsetPlan = (ChangeTagOffsetPlan) plan;
+        changeOffset(changeTagOffsetPlan.getPath(), changeTagOffsetPlan.getOffset());
         break;
       default:
-        logger.error("Unrecognizable command {}", cmd);
+        logger.error("Unrecognizable command {}", plan.getOperatorType());
     }
   }
 
@@ -469,7 +431,8 @@ public class MManager {
             || (plan.getAttributes() != null && !plan.getAttributes().isEmpty())) {
           offset = tagLogFile.write(plan.getTags(), plan.getAttributes());
         }
-        logWriter.createTimeseries(plan, offset);
+        plan.setTagOffset(offset);
+        logWriter.createTimeseries(plan);
       }
       leafMNode.setOffset(offset);
 
@@ -527,13 +490,15 @@ public class MManager {
 
   private void deleteSingleTimeseriesInternal(PartialPath p, Set<String> failedNames)
       throws MetadataException, IOException {
+    DeleteTimeSeriesPlan deleteTimeSeriesPlan = new DeleteTimeSeriesPlan();
     try {
       PartialPath emptyStorageGroup = deleteOneTimeseriesAndUpdateStatistics(p);
       if (!isRecovering) {
         if (emptyStorageGroup != null) {
           StorageEngine.getInstance().deleteAllDataFilesInOneStorageGroup(emptyStorageGroup);
         }
-        logWriter.deleteTimeseries(p.getFullPath());
+        deleteTimeSeriesPlan.setDeletePathList(Collections.singletonList(p));
+        logWriter.deleteTimeseries(deleteTimeSeriesPlan);
       }
     } catch (DeleteFailedException e) {
       failedNames.add(e.getName());
@@ -610,7 +575,7 @@ public class MManager {
     try {
       mtree.setStorageGroup(storageGroup);
       if (!isRecovering) {
-        logWriter.setStorageGroup(storageGroup.getFullPath());
+        logWriter.setStorageGroup(storageGroup);
       }
     } catch (IOException e) {
       throw new MetadataException(e.getMessage());
@@ -644,7 +609,7 @@ public class MManager {
 
         // if success
         if (!isRecovering) {
-          logWriter.deleteStorageGroup(storageGroup.getFullPath());
+          logWriter.deleteStorageGroup(storageGroup);
         }
       }
     } catch (IOException e) {
@@ -836,7 +801,7 @@ public class MManager {
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   private List<ShowTimeSeriesResult> showTimeseriesWithIndex(ShowTimeSeriesPlan plan,
-      QueryContext context) throws MetadataException {
+                                                             QueryContext context) throws MetadataException {
     if (!tagIndex.containsKey(plan.getKey())) {
       throw new MetadataException("The key " + plan.getKey() + " is not a tag.");
     }
@@ -1128,7 +1093,7 @@ public class MManager {
   public void setTTL(PartialPath storageGroup, long dataTTL) throws MetadataException, IOException {
     getStorageGroupNodeByStorageGroupPath(storageGroup).setDataTTL(dataTTL);
     if (!isRecovering) {
-      logWriter.setTTL(storageGroup.getFullPath(), dataTTL);
+      logWriter.setTTL(storageGroup, dataTTL);
     }
   }
 
@@ -1201,7 +1166,7 @@ public class MManager {
 
       leafMNode.setAlias(alias);
       // persist to WAL
-      logWriter.changeAlias(fullPath.getFullPath(), alias);
+      logWriter.changeAlias(fullPath, alias);
     }
 
     if (tagsMap == null && attributesMap == null) {
@@ -1210,20 +1175,20 @@ public class MManager {
     // no tag or attribute, we need to add a new record in log
     if (leafMNode.getOffset() < 0) {
       long offset = tagLogFile.write(tagsMap, attributesMap);
-      logWriter.changeOffset(fullPath.getFullPath(), offset);
+      logWriter.changeOffset(fullPath, offset);
       leafMNode.setOffset(offset);
       // update inverted Index map
       if (tagsMap != null) {
         for (Entry<String, String> entry : tagsMap.entrySet()) {
           tagIndex.computeIfAbsent(entry.getKey(), k -> new ConcurrentHashMap<>())
-              .computeIfAbsent(entry.getValue(), v -> new CopyOnWriteArraySet<>()).add(leafMNode);
+            .computeIfAbsent(entry.getValue(), v -> new CopyOnWriteArraySet<>()).add(leafMNode);
         }
       }
       return;
     }
 
     Pair<Map<String, String>, Map<String, String>> pair =
-        tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+      tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
 
     if (tagsMap != null) {
       for (Entry<String, String> entry : tagsMap.entrySet()) {
@@ -1259,7 +1224,7 @@ public class MManager {
         // we should add a new key-value to inverted index map
         if (beforeValue == null || !beforeValue.equals(value)) {
           tagIndex.computeIfAbsent(key, k -> new ConcurrentHashMap<>())
-              .computeIfAbsent(value, v -> new CopyOnWriteArraySet<>()).add(leafMNode);
+            .computeIfAbsent(value, v -> new CopyOnWriteArraySet<>()).add(leafMNode);
         }
       }
     }
@@ -1289,20 +1254,20 @@ public class MManager {
     // no tag or attribute, we need to add a new record in log
     if (leafMNode.getOffset() < 0) {
       long offset = tagLogFile.write(Collections.emptyMap(), attributesMap);
-      logWriter.changeOffset(fullPath.getFullPath(), offset);
+      logWriter.changeOffset(fullPath, offset);
       leafMNode.setOffset(offset);
       return;
     }
 
     Pair<Map<String, String>, Map<String, String>> pair =
-        tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+      tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
 
     for (Entry<String, String> entry : attributesMap.entrySet()) {
       String key = entry.getKey();
       String value = entry.getValue();
       if (pair.right.containsKey(key)) {
         throw new MetadataException(
-            String.format("TimeSeries [%s] already has the attribute [%s].", fullPath, key));
+          String.format("TimeSeries [%s] already has the attribute [%s].", fullPath, key));
       }
       pair.right.put(key, value);
     }
@@ -1327,25 +1292,25 @@ public class MManager {
     // no tag or attribute, we need to add a new record in log
     if (leafMNode.getOffset() < 0) {
       long offset = tagLogFile.write(tagsMap, Collections.emptyMap());
-      logWriter.changeOffset(fullPath.getFullPath(), offset);
+      logWriter.changeOffset(fullPath, offset);
       leafMNode.setOffset(offset);
       // update inverted Index map
       for (Entry<String, String> entry : tagsMap.entrySet()) {
         tagIndex.computeIfAbsent(entry.getKey(), k -> new ConcurrentHashMap<>())
-            .computeIfAbsent(entry.getValue(), v -> new CopyOnWriteArraySet<>()).add(leafMNode);
+          .computeIfAbsent(entry.getValue(), v -> new CopyOnWriteArraySet<>()).add(leafMNode);
       }
       return;
     }
 
     Pair<Map<String, String>, Map<String, String>> pair =
-        tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+      tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
 
     for (Entry<String, String> entry : tagsMap.entrySet()) {
       String key = entry.getKey();
       String value = entry.getValue();
       if (pair.left.containsKey(key)) {
         throw new MetadataException(
-            String.format("TimeSeries [%s] already has the tag [%s].", fullPath, key));
+          String.format("TimeSeries [%s] already has the tag [%s].", fullPath, key));
       }
       pair.left.put(key, value);
     }
@@ -1355,7 +1320,7 @@ public class MManager {
 
     // update tag inverted map
     tagsMap.forEach((key, value) -> tagIndex.computeIfAbsent(key, k -> new ConcurrentHashMap<>())
-        .computeIfAbsent(value, v -> new CopyOnWriteArraySet<>()).add(leafMNode));
+      .computeIfAbsent(value, v -> new CopyOnWriteArraySet<>()).add(leafMNode));
   }
 
   /**
@@ -1696,6 +1661,11 @@ public class MManager {
     return null;
   }
 
+  @TestOnly
+  public void flushAllMlogForTest() throws IOException {
+    logWriter.close();
+  }
+
   private void checkMTreeModified() {
     if (logWriter == null || logFile == null) {
       // the logWriter is not initialized now, we skip the check once.
@@ -1706,14 +1676,14 @@ public class MManager {
         logger.debug("MTree snapshot need not be created. Time from last modification: {} ms.",
             System.currentTimeMillis() - logFile.lastModified());
       }
-    } else if (logWriter.getLineNumber() < mtreeSnapshotInterval) {
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
       if (logger.isDebugEnabled()) {
         logger.debug("MTree snapshot need not be created. New mlog line number: {}.",
-            logWriter.getLineNumber());
+            logWriter.getLogNum());
       }
     } else {
       logger.info("New mlog line number: {}, time from last modification: {} ms",
-          logWriter.getLineNumber(), System.currentTimeMillis() - logFile.lastModified());
+          logWriter.getLogNum(), System.currentTimeMillis() - logFile.lastModified());
       createMTreeSnapshot();
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index c1f9316..a0e62f2 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -26,11 +26,7 @@ import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.JsonElement;
 import com.google.gson.JsonObject;
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
 import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayDeque;
@@ -52,7 +48,6 @@ import java.util.stream.Stream;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -62,10 +57,16 @@ import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.executor.fill.LastPointReader;
@@ -125,56 +126,6 @@ public class MTree implements Serializable {
     }
   }
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public static MTree deserializeFrom(File mtreeSnapshot) {
-    try (BufferedReader br = new BufferedReader(new FileReader(mtreeSnapshot))) {
-      String s;
-      Deque<MNode> nodeStack = new ArrayDeque<>();
-      MNode node = null;
-
-      while ((s = br.readLine()) != null) {
-        String[] nodeInfo = s.split(",");
-        short nodeType = Short.parseShort(nodeInfo[0]);
-        if (nodeType == MetadataConstant.STORAGE_GROUP_MNODE_TYPE) {
-          node = StorageGroupMNode.deserializeFrom(nodeInfo);
-        } else if (nodeType == MetadataConstant.MEASUREMENT_MNODE_TYPE) {
-          node = MeasurementMNode.deserializeFrom(nodeInfo);
-        } else {
-          node = new MNode(null, nodeInfo[1]);
-        }
-
-        int childrenSize = Integer.parseInt(nodeInfo[nodeInfo.length - 1]);
-        if (childrenSize == 0) {
-          nodeStack.push(node);
-        } else {
-          ConcurrentHashMap<String, MNode> childrenMap = new ConcurrentHashMap<>();
-          for (int i = 0; i < childrenSize; i++) {
-            MNode child = nodeStack.removeFirst();
-            child.setParent(node);
-            childrenMap.put(child.getName(), child);
-            if (child instanceof MeasurementMNode) {
-              String alias = ((MeasurementMNode) child).getAlias();
-              if (alias != null) {
-                node.addAlias(alias, child);
-              }
-            }
-          }
-          node.setChildren(childrenMap);
-          nodeStack.push(node);
-        }
-      }
-      return new MTree(node);
-    } catch (IOException e) {
-      logger.warn("Failed to deserialize from {}. Use a new MTree.", mtreeSnapshot.getPath());
-      return new MTree();
-    } finally {
-      limit = new ThreadLocal<>();
-      offset = new ThreadLocal<>();
-      count = new ThreadLocal<>();
-      curOffset = new ThreadLocal<>();
-    }
-  }
-
   private static String jsonToString(JsonObject jsonObject) {
     return GSON.toJson(jsonObject);
   }
@@ -1266,12 +1217,72 @@ public class MTree implements Serializable {
   }
 
   public void serializeTo(String snapshotPath) throws IOException {
-    try (BufferedWriter bw = new BufferedWriter(
-        new FileWriter(SystemFileFactory.INSTANCE.getFile(snapshotPath)))) {
-      root.serializeTo(bw);
+    try (MLogWriter mLogWriter = new MLogWriter(snapshotPath)) {
+      root.serializeTo(mLogWriter);
     }
   }
 
+  public static MTree deserializeFrom(File mtreeSnapshot) {
+    try (MLogReader mLogReader = new MLogReader(mtreeSnapshot)) {
+      return deserializeFromReader(mLogReader);
+    } catch (IOException e) {
+      logger.warn("Failed to deserialize from {}. Use a new MTree.", mtreeSnapshot.getPath());
+      return new MTree();
+    } finally {
+      limit = new ThreadLocal<>();
+      offset = new ThreadLocal<>();
+      count = new ThreadLocal<>();
+      curOffset = new ThreadLocal<>();
+    }
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private static MTree deserializeFromReader(MLogReader mLogReader) {
+    Deque<MNode> nodeStack = new ArrayDeque<>();
+    MNode node = null;
+    while (mLogReader.hasNext()) {
+      PhysicalPlan plan = null;
+      try {
+        plan = mLogReader.next();
+        if (plan == null) {
+          continue;
+        }
+        int childrenSize = 0;
+        if (plan instanceof StorageGroupMNodePlan) {
+          node = StorageGroupMNode.deserializeFrom((StorageGroupMNodePlan) plan);
+          childrenSize = ((StorageGroupMNodePlan) plan).getChildSize();
+        } else if (plan instanceof MeasurementMNodePlan) {
+          node = MeasurementMNode.deserializeFrom((MeasurementMNodePlan) plan);
+          childrenSize = ((MeasurementMNodePlan) plan).getChildSize();
+        } else if (plan instanceof MNodePlan) {
+          node = new MNode(null, ((MNodePlan) plan).getName());
+          childrenSize = ((MNodePlan) plan).getChildSize();
+        }
+
+        if (childrenSize != 0) {
+          ConcurrentHashMap<String, MNode> childrenMap = new ConcurrentHashMap<>();
+          for (int i = 0; i < childrenSize; i++) {
+            MNode child = nodeStack.removeFirst();
+            child.setParent(node);
+            childrenMap.put(child.getName(), child);
+            if (child instanceof MeasurementMNode) {
+              String alias = ((MeasurementMNode) child).getAlias();
+              if (alias != null) {
+                node.addAlias(alias, child);
+              }
+            }
+          }
+          node.setChildren(childrenMap);
+        }
+        nodeStack.push(node);
+      } catch (Exception e) {
+        logger.error("Can not operate cmd {} for err:", plan == null ? "" : plan.getOperatorType(), e);
+      }
+    }
+
+    return new MTree(node);
+  }
+
   @Override
   public String toString() {
     JsonObject jsonObject = new JsonObject();
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataConstant.java b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataConstant.java
index 372e9fb..92c6359 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataConstant.java
@@ -29,12 +29,16 @@ public class MetadataConstant {
   private static final String MTREE_VERSION = "1";
 
   public static final String ROOT = "root";
-  public static final String METADATA_LOG = "mlog.txt";
+  public static final String METADATA_TXT_LOG = "mlog.txt";
+  public static final String METADATA_LOG = "mlog.bin";
   public static final String TAG_LOG = "tlog.txt";
+  public static final String MTREE_PREFIX = "mtree";
+  public static final String MTREE_TXT_SNAPSHOT =
+    MTREE_PREFIX + IoTDBConstant.FILE_NAME_SEPARATOR + MTREE_VERSION + ".snapshot";
   public static final String MTREE_SNAPSHOT =
-      "mtree" + IoTDBConstant.FILE_NAME_SEPARATOR + MTREE_VERSION + ".snapshot";
+    MTREE_PREFIX+ IoTDBConstant.FILE_NAME_SEPARATOR + MTREE_VERSION + ".snapshot.bin";
   public static final String MTREE_SNAPSHOT_TMP =
-      "mtree" + IoTDBConstant.FILE_NAME_SEPARATOR + MTREE_VERSION + ".snapshot.tmp";
+    MTREE_PREFIX + IoTDBConstant.FILE_NAME_SEPARATOR + MTREE_VERSION + ".snapshot.bin.tmp";
 
 
   public static final short MNODE_TYPE = 0;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogReader.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogReader.java
new file mode 100644
index 0000000..b87186c
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogReader.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.logfile;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+public class MLogReader implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogReader.class);
+  private File logFile;
+
+  SingleFileLogReader singleFileLogReader;
+
+  public MLogReader(String schemaDir, String logFileName) throws IOException {
+    File metadataDir = SystemFileFactory.INSTANCE.getFile(schemaDir);
+    if (!metadataDir.exists()) {
+      logger.error("no mlog.bin to init MManager.");
+      throw new IOException("mlog.bin does not exist.");
+    }
+
+    logFile = SystemFileFactory.INSTANCE.getFile(schemaDir + File.separator + logFileName);
+    singleFileLogReader = new SingleFileLogReader(logFile);
+  }
+
+  public MLogReader(String logFilePath) throws IOException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+    singleFileLogReader = new SingleFileLogReader(logFile);
+  }
+
+  public MLogReader(File logFile) throws IOException {
+    this.logFile = logFile;
+    singleFileLogReader = new SingleFileLogReader(this.logFile);
+  }
+
+  public boolean hasNext() {
+    return !singleFileLogReader.isFileCorrupted() && singleFileLogReader.hasNext();
+  }
+
+  public PhysicalPlan next() {
+    return singleFileLogReader.next();
+  }
+
+  @Override
+  public void close() {
+    singleFileLogReader.close();
+  }
+
+  public boolean isFileCorrupted() {
+    return singleFileLogReader.isFileCorrupted();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogTxtReader.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogTxtReader.java
new file mode 100644
index 0000000..022d3be
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogTxtReader.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.logfile;
+
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+
+/**
+ * reader for reading mlog.txt
+ */
+public class MLogTxtReader implements AutoCloseable {
+  private static final Logger logger = LoggerFactory.getLogger(MLogTxtReader.class);
+
+  private BufferedReader bufferedReader;
+  private File logFile;
+  private String cmd;
+
+  public MLogTxtReader(String schemaDir, String logFileName) throws IOException {
+    File metadataDir = SystemFileFactory.INSTANCE.getFile(schemaDir);
+    if (!metadataDir.exists()) {
+      logger.error("no mlog.txt to init MManager.");
+      throw new IOException("mlog.txt does not exist.");
+    }
+
+    logFile = SystemFileFactory.INSTANCE.getFile(schemaDir + File.separator + logFileName);
+    bufferedReader = new BufferedReader(new FileReader(logFile));
+  }
+
+  public String next() {
+    return cmd;
+  }
+
+  public boolean hasNext() {
+    if (cmd != null) {
+      return true;
+    }
+    try {
+      return (cmd = bufferedReader.readLine()) != null;
+    } catch (IOException e) {
+      logger.warn("Read mlog error.");
+      cmd = null;
+      return false;
+    }
+  }
+
+  @Override
+  public void close() {
+    try {
+      bufferedReader.close();
+    } catch (IOException e) {
+      logger.error("Failed to close mlog.txt");
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
new file mode 100644
index 0000000..9fea132
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
@@ -0,0 +1,450 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.logfile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.MetadataOperationType;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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;
+import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+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.StorageGroupMNodePlan;
+import org.apache.iotdb.db.writelog.io.LogWriter;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class MLogWriter implements AutoCloseable {
+
+  private static final Logger logger = LoggerFactory.getLogger(MLogWriter.class);
+  private File logFile;
+  private LogWriter logWriter;
+  private int logNum;
+  private static final String DELETE_FAILED_FORMAT = "Deleting %s failed with exception %s";
+  private final ByteBuffer mlogBuffer = ByteBuffer.allocate(
+    IoTDBDescriptor.getInstance().getConfig().getMlogBufferSize());
+
+  private static final String LOG_TOO_LARGE_INFO = "Log cannot fit into buffer, please increase mlog_buffer_size";
+
+  public MLogWriter(String schemaDir, String logFileName) throws IOException {
+    File metadataDir = SystemFileFactory.INSTANCE.getFile(schemaDir);
+    if (!metadataDir.exists()) {
+      if (metadataDir.mkdirs()) {
+        logger.info("create schema folder {}.", metadataDir);
+      } else {
+        logger.warn("create schema folder {} failed.", metadataDir);
+      }
+    }
+
+    logFile = SystemFileFactory.INSTANCE.getFile(schemaDir + File.separator + logFileName);
+    logWriter = new LogWriter(logFile, false);
+  }
+
+  public MLogWriter(String logFilePath) throws IOException {
+    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+    logWriter = new LogWriter(logFile, false);
+  }
+
+  @Override
+  public void close() throws IOException {
+    logWriter.close();
+  }
+
+  private void sync() {
+    try {
+      logWriter.write(mlogBuffer);
+    } catch (IOException e) {
+      logger.error("MLog {} sync failed, change system mode to read-only", logFile.getAbsoluteFile(), e);
+      IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
+    }
+    mlogBuffer.clear();
+  }
+
+  private void putLog(PhysicalPlan plan) {
+    try {
+      plan.serialize(mlogBuffer);
+      sync();
+      logNum ++;
+    } catch (BufferOverflowException e) {
+      logger.warn("MLog {} BufferOverflow !", plan.getOperatorType(), e);
+    }
+  }
+
+  public void createTimeseries(CreateTimeSeriesPlan createTimeSeriesPlan) throws IOException {
+    try {
+      putLog(createTimeSeriesPlan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void deleteTimeseries(DeleteTimeSeriesPlan deleteTimeSeriesPlan) throws IOException {
+    try {
+      putLog(deleteTimeSeriesPlan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void setStorageGroup(PartialPath storageGroup) throws IOException {
+    try {
+      SetStorageGroupPlan plan = new SetStorageGroupPlan(storageGroup);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void deleteStorageGroup(PartialPath storageGroup) throws IOException {
+    try {
+      DeleteStorageGroupPlan plan = new DeleteStorageGroupPlan(Collections.singletonList(storageGroup));
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void setTTL(PartialPath storageGroup, long ttl) throws IOException {
+    try {
+      SetTTLPlan plan = new SetTTLPlan(storageGroup, ttl);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void changeOffset(PartialPath path, long offset) throws IOException {
+    try {
+      ChangeTagOffsetPlan plan = new ChangeTagOffsetPlan(path, offset);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void changeAlias(PartialPath path, String alias) throws IOException {
+    try {
+      ChangeAliasPlan plan = new ChangeAliasPlan(path, alias);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void serializeMNode(MNode node) throws IOException {
+    try {
+      int childSize = 0;
+      if (node.getChildren() != null) {
+        childSize = node.getChildren().size();
+      }
+      MNodePlan plan = new MNodePlan(node.getName(), childSize);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void serializeMeasurementMNode(MeasurementMNode node) throws IOException {
+    try {
+      int childSize = 0;
+      if (node.getChildren() != null) {
+        childSize = node.getChildren().size();
+      }
+      MeasurementMNodePlan plan = new MeasurementMNodePlan(node.getName(), node.getAlias(),
+        node.getOffset(), childSize, node.getSchema());
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  public void serializeStorageGroupMNode(StorageGroupMNode node) throws IOException {
+    try {
+      int childSize = 0;
+      if (node.getChildren() != null) {
+        childSize = node.getChildren().size();
+      }
+      StorageGroupMNodePlan plan = new StorageGroupMNodePlan(node.getName(), node.getDataTTL(), childSize);
+      putLog(plan);
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+        LOG_TOO_LARGE_INFO, e);
+    }
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  public static void upgradeTxtToBin(String schemaDir, String oldFileName,
+                                     String newFileName, boolean isSnapshot) throws IOException {
+    File logFile = SystemFileFactory.INSTANCE.getFile(schemaDir + File.separator + newFileName);
+    File tmpLogFile = SystemFileFactory.INSTANCE.getFile(logFile.getAbsolutePath() + ".tmp");
+    File oldLogFile = SystemFileFactory.INSTANCE.getFile(
+      schemaDir + File.separator + oldFileName);
+    File tmpOldLogFile = SystemFileFactory.INSTANCE.getFile(oldLogFile.getAbsolutePath()
+      + ".tmp");
+
+    if (oldLogFile.exists() || tmpOldLogFile.exists()) {
+
+      if (tmpOldLogFile.exists() && !oldLogFile.exists()) {
+        FileUtils.moveFile(tmpOldLogFile, oldLogFile);
+      }
+
+      try (MLogWriter mLogWriter = new MLogWriter(schemaDir, newFileName + ".tmp");
+           MLogTxtReader mLogTxtReader = new MLogTxtReader(schemaDir, oldFileName)) {
+        // upgrade from old character log file to new binary mlog
+        while (mLogTxtReader.hasNext()) {
+          String cmd = mLogTxtReader.next();
+          try {
+            mLogWriter.operation(cmd, isSnapshot);
+          } catch (MetadataException e) {
+            logger.error("failed to upgrade cmd {}.", cmd, e);
+          }
+        }
+      }
+    } else if (!logFile.exists() && !tmpLogFile.exists()) {
+      // if both .bin and .bin.tmp do not exist, nothing to do
+    } else if (!logFile.exists() && tmpLogFile.exists()) {
+      // if old .bin doesn't exist but .bin.tmp exists, rename tmp file to .bin
+      FSFactoryProducer.getFSFactory().moveFile(tmpLogFile, logFile);
+    } else if (tmpLogFile.exists()) {
+      // if both .bin and .bin.tmp exist, delete .bin.tmp
+      try {
+        Files.delete(Paths.get(tmpLogFile.toURI()));
+      } catch (IOException e) {
+        throw new IOException(String.format(DELETE_FAILED_FORMAT, tmpLogFile, e.getMessage()));
+      }
+    }
+
+    // do some clean job
+    // remove old .txt and .txt.tmp
+    if (oldLogFile.exists()) {
+      try {
+        Files.delete(Paths.get(oldLogFile.toURI()));
+      } catch (IOException e) {
+        throw new IOException(String.format(DELETE_FAILED_FORMAT, oldLogFile, e.getMessage()));
+      }
+    }
+
+    if (tmpOldLogFile.exists()) {
+      try {
+        Files.delete(Paths.get(tmpOldLogFile.toURI()));
+      } catch (IOException e) {
+        throw new IOException(String.format(DELETE_FAILED_FORMAT, tmpOldLogFile, e.getMessage()));
+      }
+    }
+
+    // rename .bin.tmp to .bin
+    FSFactoryProducer.getFSFactory().moveFile(tmpLogFile, logFile);
+  }
+
+  public static void upgradeMLog() throws IOException {
+    String schemaDir = IoTDBDescriptor.getInstance().getConfig().getSchemaDir();
+    upgradeTxtToBin(schemaDir, MetadataConstant.METADATA_TXT_LOG, MetadataConstant.METADATA_LOG, false);
+    upgradeTxtToBin(schemaDir, MetadataConstant.MTREE_TXT_SNAPSHOT, MetadataConstant.MTREE_SNAPSHOT, true);
+  }
+
+  public void clear() throws IOException {
+    sync();
+    logWriter.close();
+    mlogBuffer.clear();
+    if (logFile != null && logFile.exists()) {
+      Files.delete(logFile.toPath());
+    }
+    logNum = 0;
+    logWriter = new LogWriter(logFile, false);
+  }
+
+  public int getLogNum() {
+    return logNum;
+  }
+
+  /**
+   * only used for initialize a mlog file writer.
+   */
+  public void setLogNum(int number) {
+    logNum = number;
+  }
+
+  public void operation(String cmd, boolean isSnapshot) throws IOException, MetadataException {
+    if (!isSnapshot) {
+      operation(cmd);
+    } else {
+      PhysicalPlan plan = convertFromString(cmd);
+      try {
+        if (plan != null) {
+          putLog(plan);
+        }
+      } catch (BufferOverflowException e) {
+        throw new IOException(
+          LOG_TOO_LARGE_INFO, e);
+      }
+    }
+  }
+
+  /**
+   * upgrade from mlog.txt to mlog.bin
+   * @param cmd, the old meta operation
+   * @throws IOException
+   * @throws MetadataException
+   */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  public void operation(String cmd) throws IOException, MetadataException {
+    // see createTimeseries() to get the detailed format of the cmd
+    String[] args = cmd.trim().split(",", -1);
+    switch (args[0]) {
+      case MetadataOperationType.CREATE_TIMESERIES:
+        if (args.length > 8) {
+          String[] tmpArgs = new String[8];
+          tmpArgs[0] = args[0];
+          int i = 1;
+          tmpArgs[1] = "";
+          for (; i < args.length - 7; i++) {
+            tmpArgs[1] += args[i] + ",";
+          }
+          tmpArgs[1] += args[i++];
+          for (int j = 2; j < 8; j++) {
+            tmpArgs[j] = args[i++];
+          }
+          args = tmpArgs;
+        }
+        Map<String, String> props = null;
+        if (!args[5].isEmpty()) {
+          String[] keyValues = args[5].split("&");
+          String[] kv;
+          props = new HashMap<>();
+          for (String keyValue : keyValues) {
+            kv = keyValue.split("=");
+            props.put(kv[0], kv[1]);
+          }
+        }
+
+        String alias = null;
+        if (!args[6].isEmpty()) {
+          alias = args[6];
+        }
+        long offset = -1L;
+        if (!args[7].isEmpty()) {
+          offset = Long.parseLong(args[7]);
+        }
+
+        CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(new PartialPath(args[1]),
+          TSDataType.deserialize(Short.parseShort(args[2])),
+          TSEncoding.deserialize(Short.parseShort(args[3])),
+          CompressionType.deserialize(Short.parseShort(args[4])), props, null, null, alias);
+
+        plan.setTagOffset(offset);
+        createTimeseries(plan);
+        break;
+      case MetadataOperationType.DELETE_TIMESERIES:
+        if (args.length > 2) {
+          StringBuilder tmp = new StringBuilder();
+          for (int i = 1; i < args.length - 1; i++) {
+            tmp.append(args[i]).append(",");
+          }
+          tmp.append(args[args.length - 1]);
+          args[1] = tmp.toString();
+        }
+        deleteTimeseries(new DeleteTimeSeriesPlan(Collections.singletonList(new PartialPath(args[1]))));
+        break;
+      case MetadataOperationType.SET_STORAGE_GROUP:
+        try {
+          setStorageGroup(new PartialPath(args[1]));
+        }
+        // two time series may set one storage group concurrently,
+        // that's normal in our concurrency control protocol
+        catch (MetadataException e){
+          logger.info("concurrently operate set storage group cmd {} twice", cmd);
+        }
+        break;
+      case MetadataOperationType.DELETE_STORAGE_GROUP:
+        deleteStorageGroup(new PartialPath(args[1]));
+        break;
+      case MetadataOperationType.SET_TTL:
+        setTTL(new PartialPath(args[1]), Long.parseLong(args[2]));
+        break;
+      case MetadataOperationType.CHANGE_OFFSET:
+        changeOffset(new PartialPath(args[1]), Long.parseLong(args[2]));
+        break;
+      case MetadataOperationType.CHANGE_ALIAS:
+        changeAlias(new PartialPath(args[1]), args[2]);
+        break;
+      default:
+        logger.error("Unrecognizable command {}", cmd);
+    }
+  }
+
+  public void force() throws IOException {
+    logWriter.force();
+  }
+
+  public static PhysicalPlan convertFromString(String str) {
+    String[] words = str.split(",");
+    switch (words[0]) {
+      case "2":
+        return new MeasurementMNodePlan(words[1], words[2].equals("") ? null :  words[2], Long.parseLong(words[words.length - 2]),
+          Integer.parseInt(words[words.length - 1]),
+          new MeasurementSchema(words[1], TSDataType.values()[Integer.parseInt(words[3])],
+            TSEncoding.values()[Integer.parseInt(words[4])], CompressionType.values()[Integer.parseInt(words[5])]
+          ));
+      case "1":
+        return new StorageGroupMNodePlan(words[1], Long.parseLong(words[2]), Integer.parseInt(words[3]));
+      case "0":
+        return new MNodePlan(words[1], Integer.parseInt(words[2]));
+      default:
+        logger.error("unknown cmd {}", str);
+    }
+    return null;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java
index f2939b6..d9c11d1 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java
@@ -18,20 +18,20 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-import java.io.BufferedWriter;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.rescon.CachedStringPool;
+
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.Collections;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.metadata.MetadataConstant;
-import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.rescon.CachedStringPool;
 
 /**
  * This class is the implementation of Metadata Node. One MNode instance represents one node in the
@@ -236,21 +236,18 @@ public class MNode implements Serializable {
     this.name = name;
   }
 
-  public void serializeTo(BufferedWriter bw) throws IOException {
-    serializeChildren(bw);
+  public void serializeTo(MLogWriter logWriter) throws IOException {
+    serializeChildren(logWriter);
 
-    String s = String.valueOf(MetadataConstant.MNODE_TYPE) + "," + name + ","
-        + (children == null ? "0" : children.size());
-    bw.write(s);
-    bw.newLine();
+    logWriter.serializeMNode(this);
   }
 
-  void serializeChildren(BufferedWriter bw) throws IOException {
+  void serializeChildren(MLogWriter logWriter) throws IOException {
     if (children == null) {
       return;
     }
     for (Entry<String, MNode> entry : children.entrySet()) {
-      entry.getValue().serializeTo(bw);
+      entry.getValue().serializeTo(logWriter);
     }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
index 5da9d1c..6ee34ae 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
@@ -18,11 +18,12 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-import java.io.BufferedWriter;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.iotdb.db.metadata.MetadataConstant;
+
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
 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;
@@ -121,26 +122,10 @@ public class MeasurementMNode extends MNode {
   }
 
   @Override
-  public void serializeTo(BufferedWriter bw) throws IOException {
-    serializeChildren(bw);
+  public void serializeTo(MLogWriter logWriter) throws IOException {
+    serializeChildren(logWriter);
 
-    StringBuilder s = new StringBuilder(String.valueOf(MetadataConstant.MEASUREMENT_MNODE_TYPE));
-    s.append(",").append(name).append(",");
-    if (alias != null) {
-      s.append(alias);
-    }
-    s.append(",").append(schema.getType().ordinal()).append(",");
-    s.append(schema.getEncodingType().ordinal()).append(",");
-    s.append(schema.getCompressor().ordinal()).append(",");
-    if (schema.getProps() != null) {
-      for (Map.Entry<String, String> entry : schema.getProps().entrySet()) {
-        s.append(entry.getKey()).append(":").append(entry.getValue()).append(";");
-      }
-    }
-    s.append(",").append(offset).append(",");
-    s.append(children == null ? "0" : children.size());
-    bw.write(s.toString());
-    bw.newLine();
+    logWriter.serializeMeasurementMNode(this);
   }
 
   /**
@@ -164,6 +149,16 @@ public class MeasurementMNode extends MNode {
         Byte.parseByte(nodeInfo[4]), Byte.parseByte(nodeInfo[5]), props);
     MeasurementMNode node = new MeasurementMNode(null, name, schema, alias);
     node.setOffset(Long.parseLong(nodeInfo[7]));
+    return node;
+  }
+
+  /**
+   * deserialize MeasuremetMNode from MeasurementNodePlan
+   */
+  public static MeasurementMNode deserializeFrom(MeasurementMNodePlan plan) {
+    MeasurementMNode node = new MeasurementMNode(null, plan.getName(),
+      plan.getSchema(), plan.getAlias());
+    node.setOffset(plan.getOffset());
 
     return node;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java
index ce47e81..9041c9d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java
@@ -18,9 +18,10 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-import java.io.BufferedWriter;
 import java.io.IOException;
-import org.apache.iotdb.db.metadata.MetadataConstant;
+
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 
 public class StorageGroupMNode extends MNode {
 
@@ -46,15 +47,14 @@ public class StorageGroupMNode extends MNode {
   }
 
   @Override
-  public void serializeTo(BufferedWriter bw) throws IOException {
-    serializeChildren(bw);
-
-    StringBuilder s = new StringBuilder(String.valueOf(MetadataConstant.STORAGE_GROUP_MNODE_TYPE));
-    s.append(",").append(name).append(",");
-    s.append(dataTTL).append(",");
-    s.append(children == null ? "0" : children.size());
-    bw.write(s.toString());
-    bw.newLine();
+  public void serializeTo(MLogWriter logWriter) throws IOException {
+    serializeChildren(logWriter);
+
+    logWriter.serializeStorageGroupMNode(this);
+  }
+
+  public static StorageGroupMNode deserializeFrom(StorageGroupMNodePlan plan) {
+    return new StorageGroupMNode(null, plan.getName(), plan.getDataTTL());
   }
 
   public static StorageGroupMNode deserializeFrom(String[] nodeInfo) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index 7347d6a..13e3749 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -77,7 +77,8 @@ public abstract class Operator {
     TTL, DELETE_STORAGE_GROUP, LOAD_CONFIGURATION, SHOW, LOAD_FILES, REMOVE_FILE, MOVE_FILE, LAST, GROUP_BY_FILL,
     ALTER_TIMESERIES, FLUSH, MERGE, FULL_MERGE, CLEAR_CACHE,
     SHOW_MERGE_STATUS, CREATE_SCHEMA_SNAPSHOT, TRACING, DELETE_PARTITION,
-    CREATE_MULTI_TIMESERIES
-    , CREATE_INDEX, DROP_INDEX, QUERY_INDEX,
+    CREATE_MULTI_TIMESERIES, CREATE_INDEX, DROP_INDEX, QUERY_INDEX,
+    CHANGE_TAG_OFFSET, CHANGE_ALIAS, MNODE,
+    MEASUREMENT_MNODE, STORAGE_GROUP_MNODE;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
index b3dc487..d40f1eb 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
@@ -35,6 +35,8 @@ import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
+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.CreateIndexPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
@@ -43,9 +45,12 @@ import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DropIndexPlan;
 import org.apache.iotdb.db.qp.physical.sys.FlushPlan;
 import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan;
+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.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 /**
@@ -323,6 +328,26 @@ public abstract class PhysicalPlan {
           plan = new CreateMultiTimeSeriesPlan();
           plan.deserialize(buffer);
           break;
+        case CHANGE_ALIAS:
+          plan = new ChangeAliasPlan();
+          plan.deserialize(buffer);
+          break;
+        case CHANGE_TAG_OFFSET:
+          plan = new ChangeTagOffsetPlan();
+          plan.deserialize(buffer);
+          break;
+        case MNODE:
+          plan = new MNodePlan();
+          plan.deserialize(buffer);
+          break;
+        case MEASUREMENT_MNODE:
+          plan = new MeasurementMNodePlan();
+          plan.deserialize(buffer);
+          break;
+        case STORAGE_GROUP_MNODE:
+          plan = new StorageGroupMNodePlan();
+          plan.deserialize(buffer);
+          break;
         default:
           throw new IOException("unrecognized log type " + type);
       }
@@ -335,7 +360,8 @@ public abstract class PhysicalPlan {
     REVOKE_WATERMARK_EMBEDDING, CREATE_ROLE, DELETE_ROLE, CREATE_USER, REVOKE_USER_ROLE, REVOKE_ROLE_PRIVILEGE,
     REVOKE_USER_PRIVILEGE, GRANT_ROLE_PRIVILEGE, GRANT_USER_PRIVILEGE, GRANT_USER_ROLE, MODIFY_PASSWORD, DELETE_USER,
     DELETE_STORAGE_GROUP, SHOW_TIMESERIES, DELETE_TIMESERIES, LOAD_CONFIGURATION, CREATE_MULTI_TIMESERIES,
-    ALTER_TIMESERIES, FLUSH, CREATE_INDEX, DROP_INDEX
+    ALTER_TIMESERIES, FLUSH, CREATE_INDEX, DROP_INDEX,
+    CHANGE_TAG_OFFSET, CHANGE_ALIAS, MNODE, MEASUREMENT_MNODE, STORAGE_GROUP_MNODE
   }
 
   public long getIndex() {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ChangeAliasPlan.java
similarity index 60%
copy from server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ChangeAliasPlan.java
index bf81f02..a6bf1aa 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ChangeAliasPlan.java
@@ -16,32 +16,35 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.qp.physical.sys;
 
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
-public class SetStorageGroupPlan extends PhysicalPlan {
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
 
+public class ChangeAliasPlan extends PhysicalPlan {
   private PartialPath path;
+  private String alias;
 
-  public SetStorageGroupPlan() {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
+  public ChangeAliasPlan() {
+    super(false, Operator.OperatorType.CHANGE_ALIAS);
   }
 
-  public SetStorageGroupPlan(PartialPath path) {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
+  public ChangeAliasPlan(PartialPath path, String alias) {
+    super(false, Operator.OperatorType.CHANGE_ALIAS);
     this.path = path;
+    this.alias = alias;
   }
-  
+
   public PartialPath getPath() {
     return path;
   }
@@ -50,34 +53,47 @@ public class SetStorageGroupPlan extends PhysicalPlan {
     this.path = path;
   }
 
+  public String getAlias() {
+    return alias;
+  }
+
+  public void setAlias(String alias) {
+    this.alias = alias;
+  }
+
   @Override
   public List<PartialPath> getPaths() {
-    return path != null ? Collections.singletonList(path) : Collections.emptyList();
+    List<PartialPath> ret = new ArrayList<>();
+    if (path != null) {
+      ret.add(path);
+    }
+    return ret;
   }
 
   @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    stream.write((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal());
-    byte[] fullPathBytes = path.getFullPath().getBytes();
-    stream.writeInt(fullPathBytes.length);
-    stream.write(fullPathBytes);
+  public void serialize(ByteBuffer buffer) {
+    int type = PhysicalPlanType.CHANGE_ALIAS.ordinal();
+    buffer.put((byte) type);
+    putString(buffer, path.getFullPath());
+    putString(buffer, alias);
+  }
 
-    stream.writeLong(index);
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    stream.write((byte) PhysicalPlanType.CHANGE_ALIAS.ordinal());
+    putString(stream, path.getFullPath());
+    putString(stream, alias);
   }
 
   @Override
   public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    int length = buffer.getInt();
-    byte[] fullPathBytes = new byte[length];
-    buffer.get(fullPathBytes);
-    path = new PartialPath(new String(fullPathBytes));
-
-    this.index = buffer.getLong();
+    path = new PartialPath(readString(buffer));
+    alias = readString(buffer);
   }
 
   @Override
   public String toString() {
-    return "SetStorageGroup{" + path + '}';
+    return "ChangeAlias{" + path + "," + alias + "}";
   }
 
   @Override
@@ -88,12 +104,12 @@ public class SetStorageGroupPlan extends PhysicalPlan {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    SetStorageGroupPlan that = (SetStorageGroupPlan) o;
-    return Objects.equals(path, that.path);
+    ChangeAliasPlan that = (ChangeAliasPlan) o;
+    return Objects.equals(path, that.path) && Objects.equals(alias, that.alias);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(path);
+    return Objects.hash(path, alias);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ChangeTagOffsetPlan.java
similarity index 59%
copy from server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ChangeTagOffsetPlan.java
index bf81f02..ba80502 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ChangeTagOffsetPlan.java
@@ -16,32 +16,35 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.qp.physical.sys;
 
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
-public class SetStorageGroupPlan extends PhysicalPlan {
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
 
+public class ChangeTagOffsetPlan extends PhysicalPlan {
   private PartialPath path;
+  private long offset;
 
-  public SetStorageGroupPlan() {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
+  public ChangeTagOffsetPlan() {
+    super(false, Operator.OperatorType.CHANGE_TAG_OFFSET);
   }
 
-  public SetStorageGroupPlan(PartialPath path) {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
-    this.path = path;
+  public ChangeTagOffsetPlan(PartialPath partialPath, long offset) {
+    super(false, Operator.OperatorType.CHANGE_TAG_OFFSET);
+    path = partialPath;
+    this.offset = offset;
   }
-  
+
   public PartialPath getPath() {
     return path;
   }
@@ -50,34 +53,47 @@ public class SetStorageGroupPlan extends PhysicalPlan {
     this.path = path;
   }
 
+  public long getOffset() {
+    return offset;
+  }
+
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
   @Override
   public List<PartialPath> getPaths() {
-    return path != null ? Collections.singletonList(path) : Collections.emptyList();
+    List<PartialPath> ret = new ArrayList<>();
+    if (path != null) {
+      ret.add(path);
+    }
+    return ret;
   }
 
   @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    stream.write((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal());
-    byte[] fullPathBytes = path.getFullPath().getBytes();
-    stream.writeInt(fullPathBytes.length);
-    stream.write(fullPathBytes);
+  public void serialize(ByteBuffer buffer) {
+    int type = PhysicalPlanType.CHANGE_TAG_OFFSET.ordinal();
+    buffer.put((byte) type);
+    putString(buffer, path.getFullPath());
+    buffer.putLong(offset);
+  }
 
-    stream.writeLong(index);
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    stream.write((byte) PhysicalPlanType.CHANGE_TAG_OFFSET.ordinal());
+    putString(stream, path.getFullPath());
+    stream.writeLong(offset);
   }
 
   @Override
   public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    int length = buffer.getInt();
-    byte[] fullPathBytes = new byte[length];
-    buffer.get(fullPathBytes);
-    path = new PartialPath(new String(fullPathBytes));
-
-    this.index = buffer.getLong();
+    path = new PartialPath(readString(buffer));
+    offset = buffer.getLong();
   }
 
   @Override
   public String toString() {
-    return "SetStorageGroup{" + path + '}';
+    return "ChangeTagOffset{" + path + "," + offset + "}";
   }
 
   @Override
@@ -88,12 +104,12 @@ public class SetStorageGroupPlan extends PhysicalPlan {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    SetStorageGroupPlan that = (SetStorageGroupPlan) o;
-    return Objects.equals(path, that.path);
+    ChangeTagOffsetPlan that = (ChangeTagOffsetPlan) o;
+    return Objects.equals(path, that.path) && Objects.equals(offset, that.offset);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(path);
+    return Objects.hash(path, offset);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
index d30d776..e381998 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
@@ -51,8 +51,8 @@ public class CreateMultiTimeSeriesPlan extends PhysicalPlan {
   private List<Map<String, String>> tags = null;
   private List<Map<String, String>> attributes = null;
 
-  /*
-   ** record the result of creation of time series
+  /**
+   * record the result of creation of time series
    */
   private Map<Integer, TSStatus> results = new TreeMap<>();
   private List<Integer> indexes;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
index 9ca7427..7716af0 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
@@ -45,6 +45,7 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
   private Map<String, String> props = null;
   private Map<String, String> tags = null;
   private Map<String, String> attributes = null;
+  private long tagOffset = -1;
 
   public CreateTimeSeriesPlan() {
     super(false, Operator.OperatorType.CREATE_TIMESERIES);
@@ -133,10 +134,18 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
     this.props = props;
   }
 
+  public long getTagOffset() {
+    return tagOffset;
+  }
+
+  public void setTagOffset(long tagOffset) {
+    this.tagOffset = tagOffset;
+  }
+
   @Override
   public String toString() {
-    return String.format("seriesPath: %s, resultDataType: %s, encoding: %s, compression: %s", path,
-        dataType, encoding, compressor);
+    return String.format("seriesPath: %s, resultDataType: %s, encoding: %s, compression: %s, tagOffset: %s", path,
+        dataType, encoding, compressor, tagOffset);
   }
 
   @Override
@@ -153,6 +162,7 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
     stream.write(dataType.ordinal());
     stream.write(encoding.ordinal());
     stream.write(compressor.ordinal());
+    stream.writeLong(tagOffset);
 
     // alias
     if (alias != null) {
@@ -190,6 +200,52 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
   }
 
   @Override
+  public void serialize(ByteBuffer buffer) {
+    buffer.put((byte) PhysicalPlanType.CREATE_TIMESERIES.ordinal());
+    byte[] bytes = path.getFullPath().getBytes();
+    buffer.putInt(bytes.length);
+    buffer.put(bytes);
+    buffer.put((byte) dataType.ordinal());
+    buffer.put((byte) encoding.ordinal());
+    buffer.put((byte) compressor.ordinal());
+    buffer.putLong(tagOffset);
+
+    // alias
+    if (alias != null) {
+      buffer.put((byte) 1);
+      ReadWriteIOUtils.write(alias, buffer);
+    } else {
+      buffer.put((byte) 0);
+    }
+
+    // props
+    if (props != null && !props.isEmpty()) {
+      buffer.put((byte) 1);
+      ReadWriteIOUtils.write(props, buffer);
+    } else {
+      buffer.put((byte) 0);
+    }
+
+    // tags
+    if (tags != null && !tags.isEmpty()) {
+      buffer.put((byte) 1);
+      ReadWriteIOUtils.write(tags, buffer);
+    } else {
+      buffer.put((byte) 0);
+    }
+
+    // attributes
+    if (attributes != null && !attributes.isEmpty()) {
+      buffer.put((byte) 1);
+      ReadWriteIOUtils.write(attributes, buffer);
+    } else {
+      buffer.put((byte) 0);
+    }
+
+    buffer.putLong(index);
+  }
+
+  @Override
   public void deserialize(ByteBuffer buffer) throws IllegalPathException {
     int length = buffer.getInt();
     byte[] bytes = new byte[length];
@@ -198,6 +254,7 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
     dataType = TSDataType.values()[buffer.get()];
     encoding = TSEncoding.values()[buffer.get()];
     compressor = CompressionType.values()[buffer.get()];
+    tagOffset = buffer.getLong();
 
     // alias
     if (buffer.get() == 1) {
@@ -234,11 +291,12 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
     return Objects.equals(path, that.path) &&
         dataType == that.dataType &&
         encoding == that.encoding &&
-        compressor == that.compressor;
+        compressor == that.compressor &&
+        tagOffset == that.tagOffset;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(path, dataType, encoding, compressor);
+    return Objects.hash(path, dataType, encoding, compressor, tagOffset);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/DeleteTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/DeleteTimeSeriesPlan.java
index f70e6ec..24373be 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/DeleteTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/DeleteTimeSeriesPlan.java
@@ -46,6 +46,10 @@ public class DeleteTimeSeriesPlan extends PhysicalPlan {
     return deletePathList;
   }
 
+  public void setDeletePathList(List<PartialPath> deletePathList) {
+    this.deletePathList = deletePathList;
+  }
+
   @Override
   public void serialize(DataOutputStream stream) throws IOException {
     int type = PhysicalPlanType.DELETE_TIMESERIES.ordinal();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MNodePlan.java
similarity index 53%
copy from server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MNodePlan.java
index bf81f02..e5c5140 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MNodePlan.java
@@ -16,68 +16,85 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.qp.physical.sys;
 
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Collections;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
-import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
-public class SetStorageGroupPlan extends PhysicalPlan {
+public class MNodePlan extends PhysicalPlan {
+  protected String name;
+  protected int childSize;
 
-  private PartialPath path;
+  public MNodePlan() {
+    super(false, Operator.OperatorType.MNODE);
+  }
 
-  public SetStorageGroupPlan() {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
+  public MNodePlan(String name, int childSize) {
+    super(false, Operator.OperatorType.MNODE);
+    this.name = name;
+    this.childSize = childSize;
   }
 
-  public SetStorageGroupPlan(PartialPath path) {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
-    this.path = path;
+  public MNodePlan(boolean isQuery, Operator.OperatorType operatorType) {
+    super(isQuery, operatorType);
   }
-  
-  public PartialPath getPath() {
-    return path;
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return new ArrayList<>();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public int getChildSize() {
+    return childSize;
+  }
+
+  public void setChildSize(int childSize) {
+    this.childSize = childSize;
   }
 
   @Override
-  public List<PartialPath> getPaths() {
-    return path != null ? Collections.singletonList(path) : Collections.emptyList();
+  public void serialize(ByteBuffer buffer) {
+    buffer.put((byte) PhysicalPlanType.MNODE.ordinal());
+    putString(buffer, name);
+    buffer.putInt(childSize);
+    buffer.putLong(index);
   }
 
   @Override
   public void serialize(DataOutputStream stream) throws IOException {
-    stream.write((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal());
-    byte[] fullPathBytes = path.getFullPath().getBytes();
-    stream.writeInt(fullPathBytes.length);
-    stream.write(fullPathBytes);
-
+    stream.write((byte) PhysicalPlanType.MNODE.ordinal());
+    putString(stream, name);
+    stream.writeInt(childSize);
     stream.writeLong(index);
   }
 
   @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    int length = buffer.getInt();
-    byte[] fullPathBytes = new byte[length];
-    buffer.get(fullPathBytes);
-    path = new PartialPath(new String(fullPathBytes));
-
-    this.index = buffer.getLong();
+  public void deserialize(ByteBuffer buffer) {
+    name = readString(buffer);
+    childSize = buffer.getInt();
+    index = buffer.getLong();
   }
 
   @Override
   public String toString() {
-    return "SetStorageGroup{" + path + '}';
+    return "MNode{" + name + "," + childSize + "}";
   }
 
   @Override
@@ -88,12 +105,13 @@ public class SetStorageGroupPlan extends PhysicalPlan {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    SetStorageGroupPlan that = (SetStorageGroupPlan) o;
-    return Objects.equals(path, that.path);
+    MNodePlan that = (MNodePlan) o;
+    return Objects.equals(name, that.name)
+      && Objects.equals(childSize, that.childSize);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(path);
+    return Objects.hash(name, childSize);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MeasurementMNodePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MeasurementMNodePlan.java
new file mode 100644
index 0000000..c69f53f
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MeasurementMNodePlan.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.qp.physical.sys;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+public class MeasurementMNodePlan extends MNodePlan {
+  private MeasurementSchema schema;
+  private String alias;
+  private long offset;
+
+  public MeasurementMNodePlan() {
+    super(false, Operator.OperatorType.MEASUREMENT_MNODE);
+  }
+
+  public MeasurementMNodePlan(String name, String alias,
+                              long offset, int childSize, MeasurementSchema schema) {
+    super(false, Operator.OperatorType.MEASUREMENT_MNODE);
+    this.name = name;
+    this.alias = alias;
+    this.offset = offset;
+    this.childSize = childSize;
+    this.schema = schema;
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return new ArrayList<>();
+  }
+
+  @Override
+  public void serialize(ByteBuffer buffer) {
+    buffer.put((byte) PhysicalPlanType.MEASUREMENT_MNODE.ordinal());
+
+    putString(buffer, name);
+    putString(buffer, alias);
+    buffer.putLong(offset);
+    buffer.putInt(childSize);
+    schema.serializeTo(buffer);
+
+    buffer.putLong(index);
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    stream.write((byte) PhysicalPlanType.MEASUREMENT_MNODE.ordinal());
+
+    putString(stream, name);
+    putString(stream, alias);
+    stream.writeLong(offset);
+    stream.writeInt(childSize);
+    schema.serializeTo(stream);
+
+    stream.writeLong(index);
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+    name = readString(buffer);
+    alias = readString(buffer);
+    offset = buffer.getLong();
+    childSize = buffer.getInt();
+    schema = MeasurementSchema.deserializeFrom(buffer);
+
+    index = buffer.getLong();
+  }
+
+  public MeasurementSchema getSchema() {
+    return schema;
+  }
+
+  public void setSchema(MeasurementSchema schema) {
+    this.schema = schema;
+  }
+
+  public String getAlias() {
+    return alias;
+  }
+
+  public void setAlias(String alias) {
+    this.alias = alias;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  @Override
+  public String toString() {
+    return "MeasurementMNode{" + name + "," + alias+ "," +
+      schema + "," + offset + "," + childSize + "}";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    MeasurementMNodePlan that = (MeasurementMNodePlan) o;
+    return Objects.equals(name, that.name)
+      && Objects.equals(alias, that.alias)
+      && Objects.equals(schema, that.schema)
+      && Objects.equals(offset, that.offset)
+      && Objects.equals(childSize, that.childSize);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name, alias, schema, offset, childSize);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
index bf81f02..a7a6e18 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
@@ -58,20 +58,20 @@ public class SetStorageGroupPlan extends PhysicalPlan {
   @Override
   public void serialize(DataOutputStream stream) throws IOException {
     stream.write((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal());
-    byte[] fullPathBytes = path.getFullPath().getBytes();
-    stream.writeInt(fullPathBytes.length);
-    stream.write(fullPathBytes);
-
+    putString(stream, path.getFullPath());
     stream.writeLong(index);
   }
 
   @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    int length = buffer.getInt();
-    byte[] fullPathBytes = new byte[length];
-    buffer.get(fullPathBytes);
-    path = new PartialPath(new String(fullPathBytes));
+  public void serialize(ByteBuffer buffer) {
+    buffer.put((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal());
+    putString(buffer, path.getFullPath());
+    buffer.putLong(index);
+  }
 
+  @Override
+  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
+    path = new PartialPath(readString(buffer));
     this.index = buffer.getLong();
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTTLPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTTLPlan.java
index aef7363..deea2db 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTTLPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTTLPlan.java
@@ -15,7 +15,6 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
  */
 
 package org.apache.iotdb.db.qp.physical.sys;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java
similarity index 51%
copy from server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java
index bf81f02..451475b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java
@@ -16,68 +16,77 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.qp.physical.sys;
 
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Collections;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
-import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
-public class SetStorageGroupPlan extends PhysicalPlan {
+public class StorageGroupMNodePlan extends MNodePlan {
+  private long dataTTL;
 
-  private PartialPath path;
+  public StorageGroupMNodePlan() {
+    super(false, Operator.OperatorType.STORAGE_GROUP_MNODE);
+  }
 
-  public SetStorageGroupPlan() {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
+  public StorageGroupMNodePlan(String name, long dataTTL, int childSize) {
+    super(false, Operator.OperatorType.STORAGE_GROUP_MNODE);
+    this.name = name;
+    this.dataTTL = dataTTL;
+    this.childSize = childSize;
   }
 
-  public SetStorageGroupPlan(PartialPath path) {
-    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
-    this.path = path;
+  @Override
+  public List<PartialPath> getPaths() {
+    return new ArrayList<>();
   }
-  
-  public PartialPath getPath() {
-    return path;
+
+  public long getDataTTL() {
+    return dataTTL;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setDataTTL(long dataTTL) {
+    this.dataTTL = dataTTL;
   }
 
   @Override
-  public List<PartialPath> getPaths() {
-    return path != null ? Collections.singletonList(path) : Collections.emptyList();
+  public void serialize(ByteBuffer buffer) {
+    buffer.put((byte) PhysicalPlanType.STORAGE_GROUP_MNODE.ordinal());
+    putString(buffer, name);
+    buffer.putLong(dataTTL);
+    buffer.putInt(childSize);
+
+    buffer.putLong(index);
   }
 
   @Override
   public void serialize(DataOutputStream stream) throws IOException {
-    stream.write((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal());
-    byte[] fullPathBytes = path.getFullPath().getBytes();
-    stream.writeInt(fullPathBytes.length);
-    stream.write(fullPathBytes);
+    stream.write((byte) PhysicalPlanType.STORAGE_GROUP_MNODE.ordinal());
+    putString(stream, name);
+    stream.writeLong(dataTTL);
+    stream.writeInt(childSize);
 
     stream.writeLong(index);
   }
 
   @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    int length = buffer.getInt();
-    byte[] fullPathBytes = new byte[length];
-    buffer.get(fullPathBytes);
-    path = new PartialPath(new String(fullPathBytes));
-
-    this.index = buffer.getLong();
+  public void deserialize(ByteBuffer buffer) {
+    name = readString(buffer);
+    dataTTL = buffer.getLong();
+    childSize = buffer.getInt();
+    index = buffer.getLong();
   }
 
   @Override
   public String toString() {
-    return "SetStorageGroup{" + path + '}';
+    return "StorageGroupMNode{" + name + "," + dataTTL + "," + childSize + "}";
   }
 
   @Override
@@ -88,12 +97,14 @@ public class SetStorageGroupPlan extends PhysicalPlan {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    SetStorageGroupPlan that = (SetStorageGroupPlan) o;
-    return Objects.equals(path, that.path);
+    StorageGroupMNodePlan that = (StorageGroupMNodePlan) o;
+    return Objects.equals(name, that.name)
+      && Objects.equals(dataTTL, that.dataTTL)
+      && Objects.equals(childSize, that.childSize);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(path);
+    return Objects.hash(name, dataTTL, childSize);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
index 2639ce6..7d8df48 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
@@ -18,15 +18,6 @@
  */
 package org.apache.iotdb.db.sync.receiver.transfer;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 import org.apache.iotdb.db.concurrent.ThreadName;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
@@ -35,8 +26,9 @@ import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
 import org.apache.iotdb.db.exception.SyncDeviceOwnerConflictException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.sync.conf.SyncConstant;
 import org.apache.iotdb.db.sync.receiver.load.FileLoader;
@@ -53,6 +45,15 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
 public class SyncServiceImpl implements SyncService.Iface {
 
   private static final Logger logger = LoggerFactory.getLogger(SyncServiceImpl.class);
@@ -269,14 +270,16 @@ public class SyncServiceImpl implements SyncService.Iface {
   private void loadMetadata() {
     logger.info("Start to load metadata in sync process.");
     if (currentFile.get().exists()) {
-      try (BufferedReader br = new BufferedReader(
-          new java.io.FileReader(currentFile.get()))) {
-        String metadataOperation;
-        while ((metadataOperation = br.readLine()) != null) {
+      try (MLogReader mLogReader = new MLogReader(config.getSchemaDir(), MetadataConstant.METADATA_LOG)) {
+        while (mLogReader.hasNext()) {
+          PhysicalPlan plan = mLogReader.next();
           try {
-            IoTDB.metaManager.operation(metadataOperation);
-          } catch (IOException | MetadataException e) {
-            logger.error("Can not operate metadata operation {} ", metadataOperation, e);
+            if (plan == null) {
+              continue;
+            }
+            IoTDB.metaManager.operation(plan);
+          } catch (Exception e) {
+            logger.error("Can not operate metadata operation {} for err:{}", plan.getOperatorType(), e);
           }
         }
       } catch (IOException e) {
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
new file mode 100644
index 0000000..f8bbbc6
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.tools.mlog;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+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.qp.physical.PhysicalPlan;
+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;
+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.StorageGroupMNodePlan;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * parse the binary mlog or snapshot to text
+ */
+public class MLogParser {
+
+  private static final Logger logger = LoggerFactory.getLogger(MLogParser.class);
+  private static final String MLOG_CLI_PREFIX = "MlogParser";
+
+  private static final String FILE_ARGS = "f";
+  private static final String FILE_NAME = "mlog file";
+
+  private static final String OUT_ARGS = "o";
+  private static final String OUT_NAME = "output txt file";
+
+  private static final String HELP_ARGS = "help";
+
+  private static String inputFile;
+  private static String outputFile;
+
+  /**
+   * create the commandline options.
+   *
+   * @return object Options
+   */
+  public static Options createOptions() {
+    Options options = new Options();
+
+    Option opFile = Option.builder(FILE_ARGS).required().argName(FILE_NAME).hasArg().desc(
+      "Need to specify a binary mlog file to parse (required)")
+      .build();
+    options.addOption(opFile);
+
+    Option opOut = Option.builder(OUT_ARGS).required(false).argName(OUT_NAME).hasArg().desc(
+      "Could specify the output file after parse (optional)")
+      .build();
+    options.addOption(opOut);
+
+    Option opHelp = Option.builder(HELP_ARGS).longOpt(HELP_ARGS)
+      .hasArg(false).desc("Display help information")
+      .build();
+    options.addOption(opHelp);
+
+    return options;
+  }
+
+  public static void main(String[] args) {
+    Options options = createOptions();
+    HelpFormatter hf = new HelpFormatter();
+    hf.setOptionComparator(null);
+    CommandLine commandLine;
+    CommandLineParser parser = new DefaultParser();
+
+    if (args == null || args.length == 0) {
+      logger.warn("Too few params input, please check the following hint.");
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    try {
+      commandLine = parser.parse(options, args);
+    } catch (ParseException e) {
+      logger.error("Parse error: {}", e.getMessage());
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+    if (commandLine.hasOption(HELP_ARGS)) {
+      hf.printHelp(MLOG_CLI_PREFIX, options, true);
+      return;
+    }
+
+    try {
+      parseBasicParams(commandLine);
+      parseFromFile(inputFile, outputFile);
+    } catch (Exception e) {
+      logger.error("Encounter an error, because: {} ", e.getMessage());
+    }
+  }
+
+  public static void parseBasicParams(CommandLine commandLine) throws ParseException {
+    inputFile = checkRequiredArg(FILE_ARGS, FILE_NAME, commandLine);
+    outputFile = commandLine.getOptionValue(OUT_ARGS);
+
+    if (outputFile == null) {
+      outputFile = "tmp.txt";
+    }
+  }
+
+  public static String checkRequiredArg(String arg, String name, CommandLine commandLine)
+    throws ParseException {
+    String str = commandLine.getOptionValue(arg);
+    if (str == null) {
+      String msg = String.format("Required values for option '%s' not provided", name);
+      logger.info(msg);
+      logger.info("Use -help for more information");
+      throw new ParseException(msg);
+    }
+    return str;
+  }
+
+  public static void parseFromFile(String inputFile, String outputFile) throws IOException {
+    try (MLogReader mLogReader = new MLogReader(inputFile);
+         MLogTxtWriter mLogTxtWriter = new MLogTxtWriter(outputFile)) {
+
+      while (mLogReader.hasNext()) {
+        PhysicalPlan plan = mLogReader.next();
+        switch (plan.getOperatorType()) {
+          case CREATE_TIMESERIES:
+            mLogTxtWriter.createTimeseries((CreateTimeSeriesPlan)plan,
+              ((CreateTimeSeriesPlan) plan).getTagOffset());
+            break;
+          case DELETE_TIMESERIES:
+            for (PartialPath partialPath : plan.getPaths()) {
+              mLogTxtWriter.deleteTimeseries(partialPath.getFullPath());
+            }
+            break;
+          case SET_STORAGE_GROUP:
+            mLogTxtWriter.setStorageGroup(((SetStorageGroupPlan) plan).getPath().getFullPath());
+            break;
+          case DELETE_STORAGE_GROUP:
+            for (PartialPath partialPath : plan.getPaths()) {
+              mLogTxtWriter.deleteStorageGroup(partialPath.getFullPath());
+            }
+            break;
+          case TTL:
+            mLogTxtWriter.setTTL(((SetTTLPlan) plan).getStorageGroup().getFullPath(),
+              ((SetTTLPlan) plan).getDataTTL());
+            break;
+          case CHANGE_ALIAS:
+            mLogTxtWriter.changeAlias(((ChangeAliasPlan) plan).getPath().getFullPath(),
+              ((ChangeAliasPlan) plan).getAlias());
+            break;
+          case CHANGE_TAG_OFFSET:
+            mLogTxtWriter.changeOffset(((ChangeTagOffsetPlan) plan).getPath().getFullPath(),
+              ((ChangeTagOffsetPlan) plan).getOffset());
+            break;
+          case MEASUREMENT_MNODE:
+            mLogTxtWriter.serializeMeasurementMNode((MeasurementMNodePlan) plan);
+            break;
+          case STORAGE_GROUP_MNODE:
+            mLogTxtWriter.serializeStorageGroupMNode((StorageGroupMNodePlan) plan);
+            break;
+          case MNODE:
+            mLogTxtWriter.serializeMNode((MNodePlan) plan);
+            break;
+          default:
+            logger.warn("unknown plan {}", plan);
+        }
+      }
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
index 405b4c2..59c2b96 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
@@ -279,6 +279,7 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
     return readData(position, Byte.BYTES).get();
   }
 
+  @Override
   public void close() throws IOException {
     this.tsFileInput.close();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/io/LogWriter.java b/server/src/main/java/org/apache/iotdb/db/writelog/io/LogWriter.java
index 0ba9f74..16ae427 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/io/LogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/io/LogWriter.java
@@ -18,36 +18,55 @@
  */
 package org.apache.iotdb.db.writelog.io;
 
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
 import java.util.zip.CRC32;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 
 /**
- * LogWriter writes the binarized logs into a file using FileChannel together with check sums of
+ * LogWriter writes the binary logs into a file using FileChannel together with check sums of
  * each log calculated using CRC32.
  */
 public class LogWriter implements ILogWriter {
+  private static final Logger logger = LoggerFactory.getLogger(LogWriter.class);
 
   private File logFile;
   private FileOutputStream fileOutputStream;
   private FileChannel channel;
-  private CRC32 checkSummer = new CRC32();
-  private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  private ByteBuffer lengthBuffer = ByteBuffer.allocate(4);
-  private ByteBuffer checkSumBuffer = ByteBuffer.allocate(8);
+  private final CRC32 checkSummer = new CRC32();
+  private final ByteBuffer lengthBuffer = ByteBuffer.allocate(4);
+  private final ByteBuffer checkSumBuffer = ByteBuffer.allocate(8);
+  private final boolean forceEachWrite;
 
-  public LogWriter(String logFilePath) {
+  /**
+   * @param logFilePath
+   * @param forceEachWrite
+   * @throws FileNotFoundException
+   */
+  @TestOnly
+  public LogWriter(String logFilePath, boolean forceEachWrite) throws FileNotFoundException {
     logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+    this.forceEachWrite = forceEachWrite;
+
+    fileOutputStream = new FileOutputStream(logFile, true);
+    channel = fileOutputStream.getChannel();
   }
 
-  public LogWriter(File logFile) {
+  public LogWriter(File logFile, boolean forceEachWrite) throws FileNotFoundException {
     this.logFile = logFile;
+    this.forceEachWrite = forceEachWrite;
+
+    fileOutputStream = new FileOutputStream(logFile, true);
+    channel = fileOutputStream.getChannel();
   }
 
   @Override
@@ -73,12 +92,16 @@ public class LogWriter implements ILogWriter {
     lengthBuffer.flip();
     checkSumBuffer.flip();
 
-    channel.write(lengthBuffer);
-    channel.write(logBuffer);
-    channel.write(checkSumBuffer);
+    try {
+      channel.write(lengthBuffer);
+      channel.write(logBuffer);
+      channel.write(checkSumBuffer);
 
-    if (config.getForceWalPeriodInMs() == 0) {
-      channel.force(true);
+      if (this.forceEachWrite) {
+        channel.force(true);
+      }
+    } catch (ClosedChannelException ignored) {
+      logger.warn("someone interrupt current thread, so no need to do write for io safety");
     }
   }
 
@@ -92,6 +115,9 @@ public class LogWriter implements ILogWriter {
   @Override
   public void close() throws IOException {
     if (channel != null) {
+      if (channel.isOpen()) {
+        channel.force(true);
+      }
       fileOutputStream.close();
       fileOutputStream = null;
       channel.close();
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java b/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
index 28264f3..f241db5 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.writelog.node;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
@@ -87,7 +88,7 @@ public class ExclusiveWriteLogNode implements WriteLogNode, Comparable<Exclusive
     this.logDirectory =
         DirectoryManager.getInstance().getWALFolder() + File.separator + this.identifier;
     if (SystemFileFactory.INSTANCE.getFile(logDirectory).mkdirs()) {
-      logger.info("create the WAL folder {}", logDirectory);
+      logger.info("create the WAL folder {}.", logDirectory);
     }
   }
 
@@ -163,7 +164,7 @@ public class ExclusiveWriteLogNode implements WriteLogNode, Comparable<Exclusive
 
 
   @Override
-  public void notifyStartFlush() {
+  public void notifyStartFlush() throws FileNotFoundException {
     lock.lock();
     try {
       close();
@@ -259,6 +260,8 @@ public class ExclusiveWriteLogNode implements WriteLogNode, Comparable<Exclusive
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       logger.warn("Waiting for available buffer interrupted");
+    } catch (FileNotFoundException e) {
+      logger.warn("can not found file {}", identifier, e);
     } finally {
       lock.unlock();
     }
@@ -317,21 +320,21 @@ public class ExclusiveWriteLogNode implements WriteLogNode, Comparable<Exclusive
     }
   }
 
-  private ILogWriter getCurrentFileWriter() {
+  private ILogWriter getCurrentFileWriter() throws FileNotFoundException {
     if (currentFileWriter == null) {
       nextFileWriter();
     }
     return currentFileWriter;
   }
 
-  private void nextFileWriter() {
+  private void nextFileWriter() throws FileNotFoundException {
     fileId++;
     File newFile = SystemFileFactory.INSTANCE.getFile(logDirectory, WAL_FILE_NAME + fileId);
     if (newFile.getParentFile().mkdirs()) {
       logger.info("create WAL parent folder {}.", newFile.getParent());
     }
     logger.debug("WAL file {} is opened", newFile);
-    currentFileWriter = new LogWriter(newFile);
+    currentFileWriter = new LogWriter(newFile, config.getForceWalPeriodInMs() == 0);
   }
 
   @Override
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateSnapshotIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateSnapshotIT.java
index f7f1558..e52ea00 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateSnapshotIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateSnapshotIT.java
@@ -19,12 +19,7 @@
 
 package org.apache.iotdb.db.integration;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.BufferedReader;
 import java.io.File;
-import java.io.FileReader;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -35,13 +30,25 @@ import java.util.HashSet;
 import java.util.Set;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.StorageGroupMNodePlan;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import static org.junit.Assert.*;
+
 public class IoTDBCreateSnapshotIT {
 
   private static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
@@ -71,35 +78,52 @@ public class IoTDBCreateSnapshotIT {
 
       // create snapshot
       statement.execute("CREATE SNAPSHOT FOR SCHEMA");
-      File snapshotFile = new File(config.getSchemaDir() + File.separator + "mtree-1.snapshot");
+      File snapshotFile = new File(config.getSchemaDir() + File.separator + "mtree-1.snapshot.bin");
 
       // test snapshot file exists
       Assert.assertTrue(snapshotFile.exists());
 
       // test snapshot content correct
-      Set<String> e1 = new HashSet<>(Arrays.asList("2,s0,,1,2,1,,-1,0", "2,s1,,2,2,1,,-1,0",
-          "2,s2,,3,2,1,,-1,0", "2,s3,,5,0,1,,-1,0", "2,s4,,0,0,1,,-1,0"));
-      Set<String> e2 = new HashSet<>(Arrays.asList("2,s0,,1,2,1,,-1,0", "2,s1,,5,0,1,,-1,0",
-          "2,s2,,0,0,1,,-1,0"));
-
-      try (BufferedReader br = new BufferedReader(new FileReader(snapshotFile))) {
-        for (int i = 0; i < 5; ++i) {
-          String actual = br.readLine();
-          Assert.assertTrue(e1.removeIf(candidate -> candidate.equals(actual)));
-        }
-        Assert.assertTrue(e1.isEmpty());
+      String[] exp = new String[]{
+        "2,s0,,1,2,1,,-1,0",
+        "2,s1,,2,2,1,,-1,0",
+        "2,s2,,3,2,1,,-1,0",
+        "2,s3,,5,0,1,,-1,0",
+        "2,s4,,0,0,1,,-1,0",
+        "1,d0,9223372036854775807,5",
+        "2,s0,,1,2,1,,-1,0",
+        "2,s1,,5,0,1,,-1,0",
+        "2,s2,,0,0,1,,-1,0",
+        "1,d1,9223372036854775807,3",
+        "0,vehicle,2",
+        "0,root,1"
+      };
 
-        Assert.assertEquals("1,d0,9223372036854775807,5", br.readLine());
+      Set<PhysicalPlan> d0Plans = new HashSet<>(6);
+      for (int i = 0; i < 6; i++) {
+        d0Plans.add(MLogWriter.convertFromString(exp[i]));
+      }
+
+      Set<PhysicalPlan> d1Plans = new HashSet<>(6);
+      for (int i = 0; i < 6; i++) {
+        d1Plans.add(MLogWriter.convertFromString(exp[i+6]));
+      }
 
-        for (int i = 0; i < 3; ++i) {
-          String actual = br.readLine();
-          Assert.assertTrue(e2.removeIf(candidate -> candidate.equals(actual)));
+      try (MLogReader mLogReader = new MLogReader(snapshotFile)){
+        int i = 0;
+        while (i < 6 && mLogReader.hasNext()) {
+          PhysicalPlan plan = mLogReader.next();
+          assertTrue(d0Plans.removeIf(candidate -> candidate.equals(plan)));
+          i++;
         }
-        Assert.assertTrue(e2.isEmpty());
+        assertTrue(d0Plans.isEmpty());
 
-        Assert.assertEquals("1,d1,9223372036854775807,3", br.readLine());
-        Assert.assertEquals("0,vehicle,2", br.readLine());
-        Assert.assertEquals("0,root,1", br.readLine());
+        while (i < 12 && mLogReader.hasNext()) {
+          PhysicalPlan plan = mLogReader.next();
+          assertTrue(d1Plans.removeIf(candidate -> candidate.equals(plan)));
+          i++;
+        }
+        assertTrue(d1Plans.isEmpty());
       }
     } catch (Exception e) {
       e.printStackTrace();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
index c40f9b2..9f643c6 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
@@ -219,8 +219,6 @@ public class IoTDBRestartIT {
     EnvironmentUtils.cleanEnv();
   }
 
-
-
   @Test
   public void testRestartEndTime()
       throws SQLException, ClassNotFoundException, IOException, StorageEngineException {
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index e8a5a0e..24ff3a9 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
@@ -91,7 +91,7 @@ public class PhysicalPlanTest {
     Planner processor = new Planner();
     CreateTimeSeriesPlan plan = (CreateTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata);
     assertEquals(
-        "seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY",
+        "seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY, tagOffset: -1",
         plan.toString());
   }
 
@@ -101,18 +101,19 @@ public class PhysicalPlanTest {
     Planner processor = new Planner();
     CreateTimeSeriesPlan plan = (CreateTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata);
     assertEquals(
-        "seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY",
+        "seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY, tagOffset: -1",
         plan.toString());
   }
 
   @Test
   public void testMetadata3() throws QueryProcessException {
-    String metadata = "create timeseries root.vehicle.d1.s2(温度) with datatype=int32,encoding=rle, compression=SNAPPY tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)";
+    String metadata = "create timeseries root.vehicle.d1.s2(温度) with datatype=int32,encoding=rle, compression=SNAPPY " +
+      "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)";
     System.out.println(metadata.length());
     Planner processor = new Planner();
     CreateTimeSeriesPlan plan = (CreateTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata);
     assertEquals(
-        "seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY",
+        "seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY, tagOffset: -1",
         plan.toString());
   }
 
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
new file mode 100644
index 0000000..3fd8422
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/tools/MLogParserTest.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.tools;
+
+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.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.tools.mlog.MLogParser;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+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.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class MLogParserTest {
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.envSetUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+    File file = new File("target" + File.separator
+      + "tmp"  + File.separator + "text.mlog");
+    file.deleteOnExit();
+    file = new File("target" + File.separator
+      + "tmp"  + File.separator + "text.snapshot");
+    file.deleteOnExit();
+  }
+
+  public void prepareData() {
+    // prepare data
+    for (int i = 0; i < 2; i++) {
+      for (int j = 0; j < 5; j++) {
+        for (int k = 0; k < 10; k++) {
+          CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan();
+          try {
+            plan.setPath(new PartialPath("root.sg" + i + "." + "device" + j + "." + "s" + k));
+            plan.setDataType(TSDataType.INT32);
+            plan.setEncoding(TSEncoding.PLAIN);
+            plan.setCompressor(CompressionType.GZIP);
+            IoTDB.metaManager.createTimeseries(plan);
+          } catch (MetadataException e) {
+            e.printStackTrace();
+          }
+        }
+      }
+    }
+
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath("root.ln.cc"));
+      IoTDB.metaManager.setStorageGroup(new PartialPath("root.sgcc"));
+      IoTDB.metaManager.setTTL(new PartialPath("root.sgcc"), 1234L);
+      IoTDB.metaManager.deleteTimeseries(new PartialPath("root.sg1.device1.s1"));
+      List<PartialPath> paths = new ArrayList<>();
+      paths.add(new PartialPath("root.ln.cc"));
+      IoTDB.metaManager.deleteStorageGroups(paths);
+      Map<String, String> tags = new HashMap<String, String>();
+      tags.put("tag1", "value1");
+      IoTDB.metaManager.addTags(tags, new PartialPath("root.sg1.device1.s2"));
+      IoTDB.metaManager.changeAlias(new PartialPath("root.sg1.device1.s3"), "hello");
+    } catch (MetadataException | IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  @Test
+  public void testParseMLog() throws IOException {
+    prepareData();
+    IoTDB.metaManager.flushAllMlogForTest();
+
+    try {
+      MLogParser.parseFromFile(IoTDBDescriptor.getInstance().getConfig().getSchemaDir()
+        + File.separator + MetadataConstant.METADATA_LOG,
+        "target" + File.separator + "tmp"  + File.separator + "text.mlog");
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    try (BufferedReader reader = new BufferedReader(new FileReader("target" + File.separator
+      + "tmp"  + File.separator + "text.mlog"))) {
+      int lineNum = 0;
+      while (reader.readLine() != null) {
+        lineNum++;
+      }
+      Assert.assertEquals(108, lineNum);
+    } catch (IOException e) {
+      Assert.fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testParseSnapshot() {
+    prepareData();
+    IoTDB.metaManager.createMTreeSnapshot();
+
+    try {
+      MLogParser.parseFromFile(IoTDBDescriptor.getInstance().getConfig().getSchemaDir()
+          + File.separator + MetadataConstant.MTREE_SNAPSHOT,
+        "target" + File.separator + "tmp"  + File.separator + "text.snapshot");
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    try (BufferedReader reader = new BufferedReader(new FileReader("target" + File.separator
+      + "tmp"  + File.separator + "text.snapshot"))) {
+      int lineNum = 0;
+      while (reader.readLine() != null) {
+        lineNum++;
+      }
+      Assert.assertEquals(113, lineNum);
+    } catch (IOException e) {
+      Assert.fail(e.getMessage());
+    }
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/tools/WalCheckerTest.java b/server/src/test/java/org/apache/iotdb/db/tools/WalCheckerTest.java
index 46af8c1..1e8f4be 100644
--- a/server/src/test/java/org/apache/iotdb/db/tools/WalCheckerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/tools/WalCheckerTest.java
@@ -28,6 +28,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.exception.SystemCheckException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -74,7 +75,7 @@ public class WalCheckerTest {
         File subDir = new File(tempRoot, "storage_group" + i);
         subDir.mkdir();
         LogWriter logWriter = new LogWriter(subDir.getPath() + File.separator
-            + WAL_FILE_NAME);
+            + WAL_FILE_NAME, IoTDBDescriptor.getInstance().getConfig().getForceWalPeriodInMs() == 0);
 
         ByteBuffer binaryPlans = ByteBuffer.allocate(64 * 1024);
         String deviceId = "device1";
@@ -108,7 +109,7 @@ public class WalCheckerTest {
         File subDir = new File(tempRoot, "storage_group" + i);
         subDir.mkdir();
         LogWriter logWriter = new LogWriter(subDir.getPath() + File.separator
-            + WAL_FILE_NAME);
+            + WAL_FILE_NAME, IoTDBDescriptor.getInstance().getConfig().getForceWalPeriodInMs() == 0);
 
         ByteBuffer binaryPlans = ByteBuffer.allocate(64 * 1024);
         String deviceId = "device1";
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/io/LogWriterReaderTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/io/LogWriterReaderTest.java
index e8e5f8b..f23fb4a 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/io/LogWriterReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/io/LogWriterReaderTest.java
@@ -26,6 +26,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
@@ -65,7 +67,7 @@ public class LogWriterReaderTest {
 
   @Test
   public void testWriteAndRead() throws IOException {
-    LogWriter writer = new LogWriter(filePath);
+    LogWriter writer = new LogWriter(filePath, IoTDBDescriptor.getInstance().getConfig().getForceWalPeriodInMs() == 0);
     writer.write(logsBuffer);
     try {
       writer.force();
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/io/MultiFileLogReaderTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/io/MultiFileLogReaderTest.java
index f7b5178..2e010bc 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/io/MultiFileLogReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/io/MultiFileLogReaderTest.java
@@ -25,6 +25,7 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
@@ -55,7 +56,8 @@ public class MultiFileLogReaderTest {
       for (PhysicalPlan plan : fileLogs[i]) {
         plan.serialize(buffer);
       }
-      ILogWriter writer = new LogWriter(logFiles[i]);
+      ILogWriter writer = new LogWriter(logFiles[i],
+        IoTDBDescriptor.getInstance().getConfig().getForceWalPeriodInMs() == 0);
       writer.write(buffer);
       writer.force();
       writer.close();