You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by su...@apache.org on 2020/06/17 05:20:43 UTC

[incubator-iotdb] branch master updated: [IOTDB-759] Refactor MNode by removing InternalMNode (#1345)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 44c9e49  [IOTDB-759] Refactor MNode by removing InternalMNode (#1345)
44c9e49 is described below

commit 44c9e4925301fc51fc0c4af89593eba1217f5706
Author: Zesong Sun <sz...@mails.tsinghua.edu.cn>
AuthorDate: Wed Jun 17 13:20:36 2020 +0800

    [IOTDB-759] Refactor MNode by removing InternalMNode (#1345)
    
    * [IOTDB-759] Refactor MNode by removing InternalMNode
    
    * [IOTDB-726] CheckPoint of MTree
    
    * Merge master
    
    * Fix review comment and code smell
    
    * Fix code smell
    
    * Add props of MeasurementMNode
    
    * init children of MNode when used
    
    * fix getChildren()
    
    * optimize getChild()
    
    * use StringBuilder
    
    * fix typo
    
    * Merge master
---
 .../iotdb/db/engine/merge/task/MergeTask.java      |  11 +-
 .../engine/storagegroup/StorageGroupProcessor.java |  13 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |  19 +--
 .../java/org/apache/iotdb/db/metadata/MTree.java   |  32 ++---
 .../apache/iotdb/db/metadata/MetadataConstant.java |  11 +-
 .../iotdb/db/metadata/mnode/InternalMNode.java     | 135 --------------------
 .../org/apache/iotdb/db/metadata/mnode/MNode.java  | 137 +++++++++++++++++++--
 .../iotdb/db/metadata/mnode/MeasurementMNode.java  |  77 ++++++++++--
 .../iotdb/db/metadata/mnode/StorageGroupMNode.java |  22 +++-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  46 ++++---
 .../iotdb/db/metadata/MManagerImproveTest.java     |   5 +-
 .../org/apache/iotdb/db/metadata/MTreeTest.java    |  40 +++---
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |  12 +-
 13 files changed, 307 insertions(+), 253 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
index b2cc984..a5652f3 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
@@ -35,9 +35,8 @@ import org.apache.iotdb.db.engine.merge.recover.MergeLogger;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.metadata.mnode.InternalMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.utils.MergeUtils;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -97,7 +96,7 @@ public class MergeTask implements Callable<Void> {
 
   @Override
   public Void call() throws Exception {
-    try  {
+    try {
       doMerge();
     } catch (Exception e) {
       logger.error("Runtime exception in merge {}", taskName, e);
@@ -111,7 +110,8 @@ public class MergeTask implements Callable<Void> {
     cleanUp(false);
     // call the callback to make sure the StorageGroup exit merging status, but passing 2
     // empty file lists to avoid files being deleted.
-    callback.call(Collections.emptyList(), Collections.emptyList(), new File(storageGroupSysDir, MergeLogger.MERGE_LOG_NAME));
+    callback.call(Collections.emptyList(), Collections.emptyList(),
+        new File(storageGroupSysDir, MergeLogger.MERGE_LOG_NAME));
   }
 
   private void doMerge() throws IOException, MetadataException {
@@ -130,7 +130,7 @@ public class MergeTask implements Callable<Void> {
     Map<Path, MeasurementSchema> measurementSchemaMap = new HashMap<>();
     List<Path> unmergedSeries = new ArrayList<>();
     for (String device : devices) {
-      InternalMNode deviceNode = (InternalMNode) MManager.getInstance().getNodeByPath(device);
+      MNode deviceNode = MManager.getInstance().getNodeByPath(device);
       for (Entry<String, MNode> entry : deviceNode.getChildren().entrySet()) {
         Path path = new Path(device, entry.getKey());
         measurementSchemaMap.put(path, ((MeasurementMNode) entry.getValue()).getSchema());
@@ -151,7 +151,6 @@ public class MergeTask implements Callable<Void> {
       return;
     }
 
-
     fileTask = new MergeFileTask(taskName, mergeContext, mergeLogger, resource,
         resource.getSeqFiles());
     states = States.MERGE_FILES;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index b4b3f72..029126b 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -74,7 +74,6 @@ import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.OutOfTTLException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.metadata.mnode.InternalMNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
@@ -790,7 +789,7 @@ public class StorageGroupProcessor {
       throw new WriteProcessException(e);
     } finally {
       if (node != null) {
-        ((InternalMNode) node).readUnlock();
+        node.readUnlock();
       }
     }
   }
@@ -848,7 +847,7 @@ public class StorageGroupProcessor {
       throw new WriteProcessException(e);
     } finally {
       if (node != null) {
-        ((InternalMNode) node).readUnlock();
+        node.readUnlock();
       }
     }
   }
@@ -1263,7 +1262,7 @@ public class StorageGroupProcessor {
                   .query(deviceId, measurementId, schema.getType(), schema.getEncodingType(),
                       schema.getProps(), context);
 
-          tsfileResourcesForQuery.add(new TsFileResource(tsFileResource.getFile(), 
+          tsfileResourcesForQuery.add(new TsFileResource(tsFileResource.getFile(),
               tsFileResource.getDeviceToIndexMap(),
               tsFileResource.getStartTimes(), tsFileResource.getEndTimes(), pair.left,
               pair.right));
@@ -1525,8 +1524,8 @@ public class StorageGroupProcessor {
     List<TsFileResource> upgradedResources = tsFileResource.getUpgradedResources();
     for (TsFileResource resource : upgradedResources) {
       long partitionId = resource.getTimePartition();
-      resource.getDeviceToIndexMap().forEach((device, index) -> 
-        updateNewlyFlushedPartitionLatestFlushedTimeForEachDevice(partitionId, device, 
+      resource.getDeviceToIndexMap().forEach((device, index) ->
+        updateNewlyFlushedPartitionLatestFlushedTimeForEachDevice(partitionId, device,
             resource.getEndTime(index))
       );
     }
@@ -1541,7 +1540,7 @@ public class StorageGroupProcessor {
     }
     mergeLock.writeLock().unlock();
     insertLock.writeLock().unlock();
-    
+
     // after upgrade complete, update partitionLatestFlushedTimeForEachDevice
     if (countUpgradeFiles() == 0) {
       for (Entry<Long, Map<String, Long>> entry : newlyFlushedPartitionLatestFlushedTimeForEachDevice
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 fd5ff14..23b292b 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
@@ -53,9 +53,8 @@ import org.apache.iotdb.db.exception.metadata.MetadataException;
 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.mnode.InternalMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 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;
@@ -442,9 +441,6 @@ public class MManager {
 
   /**
    * remove the node from the tag inverted index
-   *
-   * @param node
-   * @throws IOException
    */
   private void removeFromTagInvertedIndex(MeasurementMNode node) throws IOException {
     if (node.getOffset() < 0) {
@@ -921,8 +917,7 @@ public class MManager {
       throws MetadataException {
     lock.readLock().lock();
     try {
-      InternalMNode node = (InternalMNode) mtree.getNodeByPath(device);
-      MNode leaf = node.getChild(measurement);
+      MNode leaf = mtree.getNodeByPath(device).getChild(measurement);
       if (leaf != null) {
         return ((MeasurementMNode) leaf).getSchema();
       } else {
@@ -1031,7 +1026,7 @@ public class MManager {
       }
     } finally {
       if (node != null) {
-        ((InternalMNode) node).readLock();
+        node.readLock();
       }
       lock.readLock().unlock();
     }
@@ -1057,7 +1052,7 @@ public class MManager {
       return node;
     } finally {
       if (node != null) {
-        ((InternalMNode) node).readLock();
+        node.readLock();
       }
       lock.writeLock().unlock();
     }
@@ -1661,9 +1656,6 @@ public class MManager {
   /**
    * Collect the timeseries schemas under "startingPath". Notice the measurements in the collected
    * MeasurementSchemas are the full path here.
-   *
-   * @param startingPath
-   * @param timeseriesSchemas
    */
   public void collectSeries(String startingPath, List<MeasurementSchema> timeseriesSchemas) {
     MNode mNode;
@@ -1711,9 +1703,6 @@ public class MManager {
   /**
    * if the path is in local mtree, nothing needed to do (because mtree is in the memory); Otherwise
    * cache the path to mRemoteSchemaCache
-   *
-   * @param path
-   * @param schema
    */
   public void cacheSchema(String path, MeasurementSchema schema) {
     // check schema is in local
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 612a2f8..089be37 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
@@ -52,7 +52,6 @@ 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.mnode.InternalMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
@@ -81,7 +80,7 @@ public class MTree implements Serializable {
   private transient ThreadLocal<Integer> curOffset = new ThreadLocal<>();
 
   MTree() {
-    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+    this.root = new MNode(null, IoTDBConstant.PATH_ROOT);
   }
 
   /**
@@ -119,7 +118,7 @@ public class MTree implements Serializable {
         if (!hasSetStorageGroup) {
           throw new StorageGroupNotSetException("Storage group should be created first");
         }
-        cur.addChild(nodeName, new InternalMNode(cur, nodeName));
+        cur.addChild(nodeName, new MNode(cur, nodeName));
       }
       cur = cur.getChild(nodeName);
     }
@@ -157,7 +156,7 @@ public class MTree implements Serializable {
           cur.addChild(nodeNames[i], new StorageGroupMNode(cur, nodeNames[i],
               IoTDBDescriptor.getInstance().getConfig().getDefaultTTL()));
         } else {
-          cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+          cur.addChild(nodeNames[i], new MNode(cur, nodeNames[i]));
         }
       }
       cur = cur.getChild(nodeNames[i]);
@@ -203,7 +202,7 @@ public class MTree implements Serializable {
     while (i < nodeNames.length - 1) {
       MNode temp = cur.getChild(nodeNames[i]);
       if (temp == null) {
-        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+        cur.addChild(nodeNames[i], new MNode(cur, nodeNames[i]));
       } else if (temp instanceof StorageGroupMNode) {
         // before set storage group, check whether the exists or not
         throw new StorageGroupAlreadySetException(temp.getFullPath());
@@ -373,13 +372,6 @@ public class MTree implements Serializable {
   }
 
   /**
-   * Get device node, if the give path is not a device, throw exception
-   */
-  MNode getDeviceNode(String path) throws MetadataException {
-    return getNodeByPath(path);
-  }
-
-  /**
    * Get node by the path
    *
    * @return last node in given seriesPath
@@ -475,7 +467,7 @@ public class MTree implements Serializable {
       MNode current = nodeStack.pop();
       if (current instanceof StorageGroupMNode) {
         ret.add((StorageGroupMNode) current);
-      } else if (current instanceof InternalMNode) {
+      } else {
         nodeStack.addAll(current.getChildren().values());
       }
     }
@@ -624,10 +616,8 @@ public class MTree implements Serializable {
       return 1;
     }
     int cnt = 0;
-    if (node instanceof InternalMNode) {
-      for (MNode child : node.getChildren().values()) {
-        cnt += getCountInGivenLevel(child, targetLevel - 1);
-      }
+    for (MNode child : node.getChildren().values()) {
+      cnt += getCountInGivenLevel(child, targetLevel - 1);
     }
     return cnt;
   }
@@ -801,7 +791,7 @@ public class MTree implements Serializable {
             parent + node.getName() + PATH_SEPARATOR, res, length);
       }
     } else {
-      if (node instanceof InternalMNode && node.getChildren().size() > 0) {
+      if (node.getChildren().size() > 0) {
         for (MNode child : node.getChildren().values()) {
           if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
             continue;
@@ -904,10 +894,8 @@ public class MTree implements Serializable {
       res.add(path);
       return;
     }
-    if (node instanceof InternalMNode) {
-      for (MNode child : node.getChildren().values()) {
-        findNodes(child, path + PATH_SEPARATOR + child.toString(), res, targetLevel - 1);
-      }
+    for (MNode child : node.getChildren().values()) {
+      findNodes(child, path + PATH_SEPARATOR + child.toString(), res, targetLevel - 1);
     }
   }
 
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 ee096bf..5aeab5b 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
@@ -19,10 +19,17 @@
 package org.apache.iotdb.db.metadata;
 
 public class MetadataConstant {
-  private MetadataConstant(){
-    //allowed to do nothing
+
+  private MetadataConstant() {
+    // allowed to do nothing
   }
+
   public static final String ROOT = "root";
   public static final String METADATA_LOG = "mlog.txt";
   public static final String TAG_LOG = "tlog.txt";
+  public static final String MTREE_SNAPSHOT = "mtree.snapshot";
+
+  public static final short MNODE_TYPE = 0;
+  public static final short STORAGE_GROUP_MNODE_TYPE = 1;
+  public static final short MEASUREMENT_MNODE_TYPE = 2;
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
deleted file mode 100644
index fa2861f..0000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * 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.mnode;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
-
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
-
-public class InternalMNode extends MNode {
-
-  private static final long serialVersionUID = 7999036474525817732L;
-
-  private Map<String, MNode> children;
-  private Map<String, MNode> aliasChildren;
-
-  protected ReadWriteLock lock = new ReentrantReadWriteLock();
-
-  public InternalMNode(MNode parent, String name) {
-    super(parent, name);
-    this.children = new LinkedHashMap<>();
-  }
-
-  @Override
-  public boolean hasChild(String name) {
-    return this.children.containsKey(name) ||
-        (aliasChildren != null && aliasChildren.containsKey(name));
-  }
-
-  @Override
-  public void addChild(String name, MNode child) {
-    children.put(name, child);
-  }
-
-
-  /**
-   * If delete a leafMNode, lock its parent, if delete an InternalNode, lock itself
-   */
-  @Override
-  public void deleteChild(String name) throws DeleteFailedException {
-    if (children.containsKey(name)) {
-      Lock writeLock;
-      // if its child node is leaf node, we need to acquire the write lock of the current device node
-      if (children.get(name) instanceof MeasurementMNode) {
-        writeLock = lock.writeLock();
-      } else {
-        // otherwise, we only need to acquire the write lock of its child node.
-        writeLock = ((InternalMNode) children.get(name)).lock.writeLock();
-      }
-      if (writeLock.tryLock()) {
-        children.remove(name);
-        writeLock.unlock();
-      } else {
-        throw new DeleteFailedException(getFullPath() + PATH_SEPARATOR + name);
-      }
-    }
-  }
-
-  @Override
-  public void deleteAliasChild(String alias) throws DeleteFailedException {
-    if (aliasChildren == null) {
-      return;
-    }
-    if (lock.writeLock().tryLock()) {
-      aliasChildren.remove(alias);
-      lock.writeLock().unlock();
-    } else {
-      throw new DeleteFailedException(getFullPath() + PATH_SEPARATOR + alias);
-    }
-  }
-
-  @Override
-  public MNode getChild(String name) {
-    return children.containsKey(name) ? children.get(name)
-        : (aliasChildren == null ? null : aliasChildren.get(name));
-  }
-
-  @Override
-  public int getLeafCount() {
-    int leafCount = 0;
-    for (MNode child : this.children.values()) {
-      leafCount += child.getLeafCount();
-    }
-    return leafCount;
-  }
-
-  @Override
-  public void addAlias(String alias, MNode child) {
-    if (aliasChildren == null) {
-      aliasChildren = new LinkedHashMap<>();
-    }
-    aliasChildren.put(alias, child);
-  }
-
-  @Override
-  public Map<String, MNode> getChildren() {
-    return children;
-  }
-
-  public void readLock() {
-    InternalMNode node = this;
-    while (node != null) {
-      node.lock.readLock().lock();
-      node = (InternalMNode) node.parent;
-    }
-  }
-
-  public void readUnlock() {
-    InternalMNode node = this;
-    while (node != null) {
-      node.lock.readLock().unlock();
-      node = (InternalMNode) node.parent;
-    }
-  }
-}
\ No newline at end of file
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 181c309..a34df03 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,17 +18,26 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
 
+import java.io.BufferedWriter;
+import java.io.IOException;
 import java.io.Serializable;
+import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.metadata.MetadataConstant;
 
 /**
  * This class is the implementation of Metadata Node. One MNode instance represents one node in the
  * Metadata Tree
  */
-public abstract class MNode implements Serializable {
+public class MNode implements Serializable {
 
   private static final long serialVersionUID = -770028375899514063L;
 
@@ -44,6 +53,10 @@ public abstract class MNode implements Serializable {
    */
   protected String fullPath;
 
+  transient Map<String, MNode> children;
+  transient Map<String, MNode> aliasChildren;
+
+  protected transient ReadWriteLock lock = new ReentrantReadWriteLock();
 
   /**
    * Constructor of MNode.
@@ -56,37 +69,89 @@ public abstract class MNode implements Serializable {
   /**
    * check whether the MNode has a child with the name
    */
-  public abstract boolean hasChild(String name);
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name)) ||
+        (aliasChildren != null && aliasChildren.containsKey(name));
+  }
 
   /**
    * node key, name or alias
    */
-  public abstract void addChild(String name, MNode child);
+  public void addChild(String name, MNode child) {
+    if (children == null) {
+      children = new LinkedHashMap<>();
+    }
+    children.put(name, child);
+  }
 
   /**
-   * delete a child
+   * If delete a leafMNode, lock its parent, if delete an InternalNode, lock itself
    */
-  public abstract void deleteChild(String name) throws DeleteFailedException;
+  public void deleteChild(String name) throws DeleteFailedException {
+    if (children != null && children.containsKey(name)) {
+      // acquire the write lock of its child node.
+      Lock writeLock = (children.get(name)).lock.writeLock();
+      if (writeLock.tryLock()) {
+        children.remove(name);
+        writeLock.unlock();
+      } else {
+        throw new DeleteFailedException(getFullPath() + PATH_SEPARATOR + name);
+      }
+    }
+  }
 
   /**
    * delete the alias of a child
    */
-  public abstract void deleteAliasChild(String alias) throws DeleteFailedException;
+  public void deleteAliasChild(String alias) throws DeleteFailedException {
+    if (aliasChildren == null) {
+      return;
+    }
+    if (lock.writeLock().tryLock()) {
+      aliasChildren.remove(alias);
+      lock.writeLock().unlock();
+    } else {
+      throw new DeleteFailedException(getFullPath() + PATH_SEPARATOR + alias);
+    }
+  }
 
   /**
    * get the child with the name
    */
-  public abstract MNode getChild(String name);
+  public MNode getChild(String name) {
+    MNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
 
   /**
    * get the count of all leaves whose ancestor is current node
    */
-  public abstract int getLeafCount();
+  public int getLeafCount() {
+    if (children == null) {
+      return 0;
+    }
+    int leafCount = 0;
+    for (MNode child : children.values()) {
+      leafCount += child.getLeafCount();
+    }
+    return leafCount;
+  }
 
   /**
    * add an alias
    */
-  public abstract void addAlias(String alias, MNode child);
+  public void addAlias(String alias, MNode child) {
+    if (aliasChildren == null) {
+      aliasChildren = new LinkedHashMap<>();
+    }
+    aliasChildren.put(alias, child);
+  }
 
   /**
    * get full path
@@ -118,7 +183,16 @@ public abstract class MNode implements Serializable {
     return parent;
   }
 
-  public abstract Map<String, MNode> getChildren();
+  public void setParent(MNode parent) {
+    this.parent = parent;
+  }
+
+  public Map<String, MNode> getChildren() {
+    if (children == null) {
+      return new LinkedHashMap<>();
+    }
+    return children;
+  }
 
   public String getName() {
     return name;
@@ -127,4 +201,43 @@ public abstract class MNode implements Serializable {
   public void setName(String name) {
     this.name = name;
   }
+
+  public void setChildren(Map<String, MNode> children) {
+    this.children = children;
+  }
+
+  public void serializeTo(BufferedWriter bw) throws IOException {
+    serializeChildren(bw);
+
+    StringBuilder s = new StringBuilder(String.valueOf(MetadataConstant.MNODE_TYPE));
+    s.append(",").append(name).append(",");
+    s.append(children == null ? "0" : children.size());
+    bw.write(s.toString());
+    bw.newLine();
+  }
+
+  void serializeChildren(BufferedWriter bw) throws IOException {
+    if (children == null) {
+      return;
+    }
+    for (Entry<String, MNode> entry : children.entrySet()) {
+      entry.getValue().serializeTo(bw);
+    }
+  }
+
+  public void readLock() {
+    MNode node = this;
+    while (node != null) {
+      node.lock.readLock().lock();
+      node = node.parent;
+    }
+  }
+
+  public void readUnlock() {
+    MNode node = this;
+    while (node != null) {
+      node.lock.readLock().unlock();
+      node = node.parent;
+    }
+  }
 }
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 6dd4aa1..b31edc1 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,18 +18,21 @@
  */
 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.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
-import java.util.Map;
-
 /**
- * Represents an (Internal-)MNode which has a Measurement or Sensor attached to it.
+ * Represents an MNode which has a Measurement or Sensor attached to it.
  */
-public class MeasurementMNode extends InternalMNode {
+public class MeasurementMNode extends MNode {
 
   private static final long serialVersionUID = -1199657856921206435L;
 
@@ -47,12 +50,19 @@ public class MeasurementMNode extends InternalMNode {
    * @param alias alias of measurementName
    */
   public MeasurementMNode(MNode parent, String measurementName, String alias, TSDataType dataType,
-                          TSEncoding encoding, CompressionType type, Map<String, String> props) {
+      TSEncoding encoding, CompressionType type, Map<String, String> props) {
     super(parent, measurementName);
     this.schema = new MeasurementSchema(measurementName, dataType, encoding, type, props);
     this.alias = alias;
   }
 
+  public MeasurementMNode(MNode parent, String measurementName, MeasurementSchema schema,
+      String alias) {
+    super(parent, measurementName);
+    this.schema = schema;
+    this.alias = alias;
+  }
+
   public MeasurementSchema getSchema() {
     return schema;
   }
@@ -63,7 +73,9 @@ public class MeasurementMNode extends InternalMNode {
 
   public synchronized void updateCachedLast(
       TimeValuePair timeValuePair, boolean highPriorityUpdate, Long latestFlushedTime) {
-    if (timeValuePair == null || timeValuePair.getValue() == null) return;
+    if (timeValuePair == null || timeValuePair.getValue() == null) {
+      return;
+    }
 
     if (cachedLastValuePair == null) {
       // If no cached last, (1) a last query (2) an unseq insertion or (3) a seq insertion will update cache.
@@ -73,7 +85,7 @@ public class MeasurementMNode extends InternalMNode {
       }
     } else if (timeValuePair.getTimestamp() > cachedLastValuePair.getTimestamp()
         || (timeValuePair.getTimestamp() == cachedLastValuePair.getTimestamp()
-            && highPriorityUpdate)) {
+        && highPriorityUpdate)) {
       cachedLastValuePair.setTimestamp(timeValuePair.getTimestamp());
       cachedLastValuePair.setValue(timeValuePair.getValue());
     }
@@ -103,4 +115,55 @@ public class MeasurementMNode extends InternalMNode {
   public void setAlias(String alias) {
     this.alias = alias;
   }
+
+  public void setSchema(MeasurementSchema schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  public void serializeTo(BufferedWriter bw) throws IOException {
+    serializeChildren(bw);
+
+    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(",");
+    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();
+  }
+
+  /**
+   * deserialize MeasuremetMNode from string array
+   *
+   * @param nodeInfo node information array. For example: "2,s0,speed,2,2,1,year:2020;month:jan;,-1,0"
+   * representing: [0] nodeType [1] name [2] alias [3] TSDataType.ordinal() [4] TSEncoding.ordinal()
+   * [5] CompressionType.ordinal() [6] props [7] offset [8] children size
+   */
+  public static MeasurementMNode deserializeFrom(String[] nodeInfo) {
+    String name = nodeInfo[1];
+    String alias = nodeInfo[2].equals("") ? null : nodeInfo[2];
+    Map<String, String> props = new HashMap<>();
+    if (!nodeInfo[6].equals("")) {
+      for (String propInfo : nodeInfo[6].split(";")) {
+        props.put(propInfo.split(":")[0], propInfo.split(":")[1]);
+      }
+    }
+    MeasurementSchema schema = new MeasurementSchema(name,
+        TSDataType.deserialize(Short.valueOf(nodeInfo[3])),
+        TSEncoding.deserialize(Short.valueOf(nodeInfo[4])),
+        CompressionType.deserialize(Short.valueOf(nodeInfo[5])), props);
+    MeasurementMNode node = new MeasurementMNode(null, name, schema, alias);
+    node.setOffset(Long.valueOf(nodeInfo[7]));
+
+    return node;
+  }
 }
\ No newline at end of file
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 02c668f..1072e14 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,7 +18,11 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-public class StorageGroupMNode extends InternalMNode {
+import java.io.BufferedWriter;
+import java.io.IOException;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+
+public class StorageGroupMNode extends MNode {
 
   private static final long serialVersionUID = 7999036474525817732L;
 
@@ -28,7 +32,6 @@ public class StorageGroupMNode extends InternalMNode {
    */
   private long dataTTL;
 
-
   public StorageGroupMNode(MNode parent, String name, long dataTTL) {
     super(parent, name);
     this.dataTTL = dataTTL;
@@ -43,4 +46,19 @@ public class StorageGroupMNode extends InternalMNode {
     this.dataTTL = dataTTL;
   }
 
+  @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 static StorageGroupMNode deserializeFrom(String[] nodeInfo) {
+    return new StorageGroupMNode(null, nodeInfo[1], Long.valueOf(nodeInfo[2]));
+  }
 }
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index d84d28e..45ea1fd 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -56,7 +56,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
-
 import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.auth.authorizer.BasicAuthorizer;
 import org.apache.iotdb.db.auth.authorizer.IAuthorizer;
@@ -76,7 +75,6 @@ import org.apache.iotdb.db.engine.flush.pool.FlushTaskPoolManager;
 import org.apache.iotdb.db.engine.merge.manage.MergeManager;
 import org.apache.iotdb.db.engine.merge.manage.MergeManager.TaskStatus;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.BatchInsertionException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -85,16 +83,25 @@ import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.metadata.mnode.InternalMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 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.logical.Operator.OperatorType;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator.AuthorType;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.*;
+import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
+import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByTimeFillPlan;
 import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.UpdatePlan;
 import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
 import org.apache.iotdb.db.qp.physical.sys.ClearCachePlan;
@@ -103,8 +110,8 @@ import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
 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.LoadConfigurationPlan;
 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.MergePlan;
 import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
@@ -125,7 +132,6 @@ import org.apache.iotdb.db.utils.AuthUtils;
 import org.apache.iotdb.db.utils.FileLoaderUtils;
 import org.apache.iotdb.db.utils.TypeInferenceUtils;
 import org.apache.iotdb.db.utils.UpgradeUtils;
-import org.apache.iotdb.service.rpc.thrift.TSStatus;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
@@ -824,7 +830,8 @@ public class PlanExecutor implements IPlanExecutor {
                   schema.getEncodingType(),
                   schema.getCompressor(),
                   Collections.emptyMap());
-            } else if (!(node.getChild(chunkMetadata.getMeasurementUid()) instanceof MeasurementMNode)) {
+            } else if (!(node
+                .getChild(chunkMetadata.getMeasurementUid()) instanceof MeasurementMNode)) {
               throw new QueryProcessException(
                   String.format("Current Path is not leaf node. %s", series));
             }
@@ -832,7 +839,7 @@ public class PlanExecutor implements IPlanExecutor {
         }
       } finally {
         if (node != null) {
-          ((InternalMNode) node).readUnlock();
+          node.readUnlock();
         }
       }
     }
@@ -907,7 +914,8 @@ public class PlanExecutor implements IPlanExecutor {
       insertPlan.setSchemasAndTransferType(schemas);
       StorageEngine.getInstance().insert(insertPlan);
       if (insertPlan.getFailedMeasurements() != null) {
-        throw new StorageEngineException("failed to insert points " + insertPlan.getFailedMeasurements());
+        throw new StorageEngineException(
+            "failed to insert points " + insertPlan.getFailedMeasurements());
       }
     } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
@@ -935,7 +943,8 @@ public class PlanExecutor implements IPlanExecutor {
             measurementList[i] = schemas[i].getMeasurementId();
           }
         } catch (MetadataException e) {
-          logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i], e.getMessage());
+          logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i],
+              e.getMessage());
           if (enablePartialInsert) {
             insertPlan.markMeasurementInsertionFailed(i);
           } else {
@@ -945,7 +954,7 @@ public class PlanExecutor implements IPlanExecutor {
       }
     } finally {
       if (node != null) {
-        ((InternalMNode) node).readUnlock();
+        node.readUnlock();
       }
     }
     return schemas;
@@ -954,7 +963,8 @@ public class PlanExecutor implements IPlanExecutor {
   /**
    * @param loc index of measurement in insertPlan
    */
-  private MeasurementSchema getSeriesSchema(MNode deviceNode, InsertPlan insertPlan, int loc) throws MetadataException {
+  private MeasurementSchema getSeriesSchema(MNode deviceNode, InsertPlan insertPlan, int loc)
+      throws MetadataException {
     String measurement = insertPlan.getMeasurements()[loc];
     String deviceId = insertPlan.getDeviceId();
     Object value = insertPlan.getValues()[loc];
@@ -975,15 +985,17 @@ public class PlanExecutor implements IPlanExecutor {
         Path path = new Path(deviceId, measurement);
         internalCreateTimeseries(path.toString(), dataType);
 
-        MeasurementMNode measurementNode = (MeasurementMNode) mManager.getChild(deviceNode, measurement);
+        MeasurementMNode measurementNode = (MeasurementMNode) mManager
+            .getChild(deviceNode, measurement);
         measurementSchema = measurementNode.getSchema();
-        if(!isInferType) {
+        if (!isInferType) {
           checkType(insertPlan, loc, measurementNode.getSchema().getType());
         }
       }
     } else if (deviceNode != null) {
       // device and measurement exists in MTree
-      MeasurementMNode measurementNode = (MeasurementMNode) MManager.getInstance().getChild(deviceNode, measurement);
+      MeasurementMNode measurementNode = (MeasurementMNode) MManager.getInstance()
+          .getChild(deviceNode, measurement);
       measurementSchema = measurementNode.getSchema();
     } else {
       // device in not in MTree, try the cache
@@ -1125,7 +1137,7 @@ public class PlanExecutor implements IPlanExecutor {
       throw new QueryProcessException(e);
     } finally {
       if (node != null) {
-        ((InternalMNode) node).readUnlock();
+        node.readUnlock();
       }
     }
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
index 424ad52..5949d0e 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
@@ -26,9 +26,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.mnode.InternalMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -142,7 +141,7 @@ public class MManagerImproveTest {
       }
     } finally {
       if (node != null) {
-        ((InternalMNode) node).readUnlock();
+        node.readUnlock();
       }
     }
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
index 539585a..0f49a9a 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
@@ -18,6 +18,14 @@
  */
 package org.apache.iotdb.db.metadata;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -31,15 +39,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class MTreeTest {
 
   @Before
@@ -58,14 +57,16 @@ public class MTreeTest {
     root.setStorageGroup("root.laptop");
     try {
       root.createTimeseries("root.laptop.d1.s1", TSDataType.INT32, TSEncoding.RLE,
-              TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), "status");
+          TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(),
+          "status");
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
     try {
       root.createTimeseries("root.laptop.d1.s2", TSDataType.INT32, TSEncoding.RLE,
-              TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), "status");
+          TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(),
+          "status");
       fail();
     } catch (MetadataException e) {
       assertTrue(e instanceof AliasAlreadyExistException);
@@ -148,21 +149,24 @@ public class MTreeTest {
       assertFalse(root.checkStorageGroupByPath("root.a.d0"));
       root.setStorageGroup("root.a.d0");
       root.createTimeseries("root.a.d0.s0", TSDataType.INT32, TSEncoding.RLE,
-              TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), "temperature");
+          TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(),
+          "temperature");
       root.createTimeseries("root.a.d0.s1", TSDataType.INT32, TSEncoding.RLE,
-              TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), "status");
+          TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(),
+          "status");
 
       assertFalse(root.isPathExist("root.a.d1"));
       assertFalse(root.checkStorageGroupByPath("root.a.d1"));
       root.setStorageGroup("root.a.d1");
       root.createTimeseries("root.a.d1.s0", TSDataType.INT32, TSEncoding.RLE,
-              TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), "temperature");
+          TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(),
+          "temperature");
       root.createTimeseries("root.a.d1.s1", TSDataType.INT32, TSEncoding.RLE,
-              TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), null);
+          TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), null);
 
       root.setStorageGroup("root.a.b.d0");
       root.createTimeseries("root.a.b.d0.s0", TSDataType.INT32, TSEncoding.RLE,
-              TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), null);
+          TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap(), null);
 
     } catch (MetadataException e1) {
       e1.printStackTrace();
@@ -174,13 +178,11 @@ public class MTreeTest {
       assertEquals("root.a.d0.s0", result.get(0));
       assertEquals("root.a.d1.s0", result.get(1));
 
-
       result = root.getAllTimeseriesName("root.a.*.temperature");
       assertEquals(2, result.size());
       assertEquals("root.a.d0.s0", result.get(0));
       assertEquals("root.a.d1.s0", result.get(1));
 
-
       List<Path> result2 = root.getAllTimeseriesPath("root.a.*.s0");
       assertEquals(2, result2.size());
       assertEquals("root.a.d0.s0", result2.get(0).getFullPath());
@@ -432,7 +434,7 @@ public class MTreeTest {
   }
 
   @Test
-  public void addSubDevice() throws MetadataException {
+  public void testAddSubDevice() throws MetadataException {
     MTree root = new MTree();
     root.setStorageGroup("root.laptop");
     root.createTimeseries("root.laptop.d1.s1", TSDataType.INT32, TSEncoding.RLE,
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
index 1173749..c88481c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
@@ -103,17 +103,17 @@ public class ReadWriteIOUtils {
   }
 
   /**
-   * write if the object not equals null. Eg, object eauals null, then write false.
+   * write if the object equals null. Eg, object equals null, then write true.
    */
-  public static int writeIsNotNull(Object object, OutputStream outputStream) throws IOException {
-    return write(object != null, outputStream);
+  public static int writeIsNull(Object object, OutputStream outputStream) throws IOException {
+    return write(object == null, outputStream);
   }
 
   /**
-   * write if the object not equals null. Eg, object eauals null, then write false.
+   * write if the object equals null. Eg, object equals null, then write true.
    */
-  public static int writeIsNotNull(Object object, ByteBuffer buffer) {
-    return write(object != null, buffer);
+  public static int writeIsNull(Object object, ByteBuffer buffer) {
+    return write(object == null, buffer);
   }
 
   /**