You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by zy...@apache.org on 2023/03/15 13:15:56 UTC

[iotdb] branch master updated: [IOTDB-5674] Multi-Mode IMNode Management (#9243)

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

zyk 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 78b2b279f5 [IOTDB-5674] Multi-Mode IMNode Management (#9243)
78b2b279f5 is described below

commit 78b2b279f5da5990234b194969892755a5086b0b
Author: Chen YZ <43...@users.noreply.github.com>
AuthorDate: Wed Mar 15 21:15:48 2023 +0800

    [IOTDB-5674] Multi-Mode IMNode Management (#9243)
---
 .../request/write/database/DatabaseSchemaPlan.java |   4 +-
 .../persistence/executor/ConfigPlanExecutor.java   |   2 +-
 .../persistence/schema/ClusterSchemaInfo.java      |  42 ++-
 .../impl/schema/DeleteDatabaseProcedure.java       |   4 +-
 .../iotdb/db/it/schema/AbstractSchemaIT.java       |   2 +-
 .../db/it/schema/IoTDBSortedShowTimeseriesIT.java  |   2 +-
 .../org/apache/iotdb/db/it/schema/IoTDBTagIT.java  |   2 +-
 .../apache/iotdb/commons/schema/node}/IMNode.java  |  68 ++--
 .../iotdb/commons/schema/node}/MNodeType.java      |   2 +-
 .../node/common/AbstractAboveDatabaseMNode.java    | 191 ++++++++++
 .../node/common/AbstractDatabaseDeviceMNode.java   | 290 ++++++++++++++
 .../schema/node/common/AbstractDatabaseMNode.java  | 215 +++++++++++
 .../schema/node/common/AbstractDeviceMNode.java    | 280 ++++++++++++++
 .../node/common/AbstractMeasurementMNode.java      | 256 +++++++++++++
 .../schema/node/info/IDatabaseDeviceInfo.java      |  11 +-
 .../commons/schema/node/info/IDatabaseInfo.java    |  14 +-
 .../commons/schema/node/info/IDeviceInfo.java      |  31 +-
 .../commons/schema/node/info/IMeasurementInfo.java |  20 +-
 .../commons/schema/node/role/IDatabaseMNode.java   |  14 +-
 .../commons/schema/node/role/IDeviceMNode.java     |  22 +-
 .../schema/node/role}/IMeasurementMNode.java       |  16 +-
 .../schema/node/utils}/IMNodeContainer.java        |   6 +-
 .../commons/schema/node/utils/IMNodeFactory.java   |  34 +-
 .../commons/schema/node/utils}/IMNodeIterator.java |   6 +-
 .../commons/schema/node/visitor/MNodeVisitor.java  |  42 +++
 .../commons/utils/ThriftConfigNodeSerDeUtils.java  |   8 +-
 .../utils/ThriftConfigNodeSerDeUtilsTest.java      |   4 +-
 .../rocksdb/RSchemaReadWriteHandler.java           |   4 +-
 .../schemaregion/rocksdb/RSchemaRegion.java        |  10 +-
 .../schemaregion/rocksdb/RSchemaUtils.java         |  10 +-
 ...RStorageGroupMNode.java => RDatabaseMNode.java} |  45 ++-
 .../mnode/{REntityMNode.java => RDeviceMNode.java} |  55 ++-
 .../schemaregion/rocksdb/mnode/RInternalMNode.java |  30 +-
 .../schemaregion/rocksdb/mnode/RMNode.java         | 101 ++---
 .../rocksdb/mnode/RMeasurementMNode.java           |  47 ++-
 .../metadata/tagSchemaRegion/TagSchemaRegion.java  |   6 +-
 .../idtable/entry/InsertMeasurementMNode.java      | 287 --------------
 .../org/apache/iotdb/db/metadata/mnode/MNode.java  | 200 ----------
 .../apache/iotdb/db/metadata/mnode/MNodeUtils.java |  77 ----
 .../iotdb/db/metadata/mnode/MeasurementMNode.java  | 233 ------------
 .../db/metadata/mnode/StorageGroupEntityMNode.java |  90 -----
 .../iotdb/db/metadata/mnode/StorageGroupMNode.java | 102 -----
 .../IConfigMNode.java}                             |  26 +-
 .../mnode/config/basic/ConfigBasicMNode.java       | 279 ++++++++++++++
 .../container/ConfigMNodeContainer.java}           |  45 +--
 .../mnode/config/factory/ConfigMNodeFactory.java   |  81 ++++
 .../impl/ConfigBasicInternalMNode.java}            | 134 ++-----
 .../mnode/config/impl/ConfigDatabaseMNode.java     |  80 ++++
 .../mnode/config/info/ConfigDatabaseInfo.java      |  70 ++++
 .../mnode/config/info/ConfigMNodeInfo.java         |  82 ++++
 .../mnode/estimator/BasicMNodSizeEstimator.java    | 114 ------
 .../mnode/estimator/IMNodeSizeEstimator.java       |  38 --
 .../mem/IMemMNode.java}                            |  10 +-
 .../db/metadata/mnode/mem/basic/BasicMNode.java    | 252 +++++++++++++
 .../container/MemMNodeContainer.java}              |  44 +--
 .../estimator/MNodeSizeEstimator.java}             |  12 +-
 .../mnode/mem/factory/MemMNodeFactory.java         |  84 +++++
 .../mnode/mem/impl/AboveDatabaseMNode.java}        |  22 +-
 .../impl/BasicInternalMNode.java}                  | 134 ++-----
 .../mnode/mem/impl/DatabaseDeviceMNode.java}       |  29 +-
 .../impl/DatabaseMNode.java}                       |  35 +-
 .../impl/DeviceMNode.java}                         |  32 +-
 .../metadata/mnode/mem/impl/MeasurementMNode.java  |  49 +++
 .../info/BasicMNodeInfo.java}                      |  32 +-
 .../mnode/mem/info/DatabaseDeviceInfo.java         |  59 +++
 .../info/DatabaseInfo.java}                        |  38 +-
 .../{EntityMNode.java => mem/info/DeviceInfo.java} | 131 ++++---
 .../metadata/mnode/mem/info/MeasurementInfo.java   | 113 ++++++
 .../iterator/AbstractTraverserIterator.java        |  35 +-
 .../mnode/{ => mem}/iterator/MNodeIterator.java    |  13 +-
 .../iterator/MemoryTraverserIterator.java          |  16 +-
 .../metadata/mnode/schemafile/ICachedMNode.java}   |  14 +-
 .../mnode/schemafile/basic/CachedBasicMNode.java   | 264 +++++++++++++
 .../container}/CachedMNodeContainer.java           | 166 +++++---
 .../container}/ICachedMNodeContainer.java          |  35 +-
 .../schemafile/factory/CacheMNodeFactory.java      |  84 +++++
 .../impl/CachedAboveDatabaseMNode.java}            |  31 +-
 .../impl/CachedBasicInternalMNode.java}            | 140 ++-----
 .../schemafile/impl/CachedDatabaseDeviceMNode.java |  49 +++
 .../mnode/schemafile/impl/CachedDatabaseMNode.java |  54 +++
 .../mnode/schemafile/impl/CachedDeviceMNode.java   |  48 +++
 .../schemafile/impl/CachedMeasurementMNode.java    |  60 +++
 .../info/CacheMNodeInfo.java}                      |  31 +-
 .../iterator/CachedTraverserIterator.java          |  18 +-
 .../iotdb/db/metadata/mnode/utils/MNodeUtils.java  | 123 ++++++
 .../db/metadata/mnode/visitor/MNodeVisitor.java    |  39 --
 .../iotdb/db/metadata/mtree/ConfigMTree.java       | 249 ++++++------
 .../iotdb/db/metadata/mtree/IMTreeBelowSG.java     | 195 ----------
 .../db/metadata/mtree/MTreeBelowSGCachedImpl.java  | 419 ++++++++++-----------
 .../db/metadata/mtree/MTreeBelowSGMemoryImpl.java  | 378 +++++++++----------
 .../mtree/snapshot/MemMTreeSnapshotUtil.java       | 142 +++----
 .../db/metadata/mtree/store/CachedMTreeStore.java  | 167 ++++----
 .../db/metadata/mtree/store/ConfigMTreeStore.java  | 143 +++++++
 .../iotdb/db/metadata/mtree/store/IMTreeStore.java |  44 +--
 .../db/metadata/mtree/store/MemMTreeStore.java     | 147 ++++----
 .../store/ReentrantReadOnlyCachedMTreeStore.java   |  44 +--
 .../mtree/store/disk/cache/CacheManager.java       | 139 +++----
 .../mtree/store/disk/cache/ICacheManager.java      |  28 +-
 .../mtree/store/disk/cache/LRUCacheManager.java    |  24 +-
 .../mtree/store/disk/cache/PlainCacheManager.java  |  10 +-
 .../disk/memcontrol/CachedMNodeSizeEstimator.java  |  72 ----
 .../mtree/store/disk/memcontrol/MemManager.java    |  26 +-
 .../mtree/store/disk/schemafile/ISchemaFile.java   |  17 +-
 .../store/disk/schemafile/ISegmentedPage.java      |   8 +-
 .../store/disk/schemafile/MockSchemaFile.java      | 128 +++----
 .../mtree/store/disk/schemafile/RecordUtils.java   |  84 +++--
 .../mtree/store/disk/schemafile/SchemaFile.java    |  57 +--
 .../mtree/store/disk/schemafile/SegmentedPage.java |  41 +-
 .../store/disk/schemafile/WrappedSegment.java      |  20 +-
 .../disk/schemafile/pagemgr/BTreePageManager.java  |  30 +-
 .../disk/schemafile/pagemgr/IPageManager.java      |  13 +-
 .../store/disk/schemafile/pagemgr/PageManager.java |  34 +-
 .../db/metadata/mtree/traverser/Traverser.java     |  52 +--
 .../traverser/TraverserWithLimitOffsetWrapper.java |  18 +-
 .../mtree/traverser/basic/DatabaseTraverser.java   |  22 +-
 .../mtree/traverser/basic/EntityTraverser.java     |  19 +-
 .../mtree/traverser/basic/MNodeTraverser.java      |  21 +-
 .../traverser/basic/MeasurementTraverser.java      |  14 +-
 .../traverser/collector/DatabaseCollector.java     |  14 +-
 .../mtree/traverser/collector/EntityCollector.java |  14 +-
 .../traverser/collector/MNodeAboveDBCollector.java |  14 +-
 .../mtree/traverser/collector/MNodeCollector.java  |  10 +-
 .../traverser/collector/MeasurementCollector.java  |  22 +-
 .../mtree/traverser/counter/DatabaseCounter.java   |  10 +-
 .../mtree/traverser/counter/EntityCounter.java     |   9 +-
 .../traverser/counter/MeasurementCounter.java      |   9 +-
 .../mtree/traverser/updater/EntityUpdater.java     |  15 +-
 .../traverser/updater/MeasurementUpdater.java      |  13 +-
 .../plan/schemaregion/result/ShowNodesResult.java  |   2 +-
 .../db/metadata/query/info/INodeSchemaInfo.java    |   2 +-
 .../schemaregion/SchemaRegionMemoryImpl.java       |  52 +--
 .../schemaregion/SchemaRegionSchemaFileImpl.java   | 102 ++---
 .../apache/iotdb/db/metadata/tag/TagManager.java   |  55 +--
 .../metadata/template/TemplateMNodeGenerator.java  |  51 ---
 .../apache/iotdb/db/metadata/utils/MetaUtils.java  |   2 +-
 .../schema/NodeManageMemoryMergeOperator.java      |   2 +-
 .../memory/StatementMemorySourceVisitor.java       |   2 +-
 .../apache/iotdb/db/metadata/MetaUtilsTest.java    |  18 +-
 .../iotdb/db/metadata/idtable/IDTableTest.java     |   2 +-
 .../apache/iotdb/db/metadata/mnode/MNodeTest.java  |  64 ++--
 .../iotdb/db/metadata/mtree/ConfigMTreeTest.java   |  17 +-
 .../iotdb/db/metadata/mtree/MTreeBelowSGTest.java  | 257 -------------
 .../mtree/disk/CachedMNodeContainerTest.java       |  25 +-
 .../mtree/multimode/MTreePartialMemoryTest.java    |  28 --
 .../metadata/mtree/schemafile/RecordUtilTests.java |  32 +-
 .../mtree/schemafile/SchemaFileLogTest.java        |  21 +-
 .../metadata/mtree/schemafile/SchemaFileTest.java  | 295 ++++++++-------
 .../metadata/mtree/schemafile/SchemaPageTest.java  |  26 +-
 .../mtree/schemafile/WrappedSegmentTest.java       |  48 +--
 .../schemaRegion/SchemaRegionBasicTest.java        |   2 +-
 .../schemaRegion/SchemaStatisticsTest.java         |  97 +++--
 .../NodeManagementMemoryMergeNodeSerdeTest.java    |   2 +-
 .../iotdb/db/tools/SchemaFileSketchTest.java       |  43 +--
 153 files changed, 5813 insertions(+), 4499 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/DatabaseSchemaPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/DatabaseSchemaPlan.java
index 2430b77369..96910a8e92 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/DatabaseSchemaPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/DatabaseSchemaPlan.java
@@ -50,12 +50,12 @@ public class DatabaseSchemaPlan extends ConfigPhysicalPlan {
   @Override
   protected void serializeImpl(DataOutputStream stream) throws IOException {
     stream.writeShort(getType().getPlanType());
-    ThriftConfigNodeSerDeUtils.serializeTStorageGroupSchema(schema, stream);
+    ThriftConfigNodeSerDeUtils.serializeTDatabaseSchema(schema, stream);
   }
 
   @Override
   protected void deserializeImpl(ByteBuffer buffer) throws IOException {
-    schema = ThriftConfigNodeSerDeUtils.deserializeTStorageGroupSchema(buffer);
+    schema = ThriftConfigNodeSerDeUtils.deserializeTDatabaseSchema(buffer);
   }
 
   @Override
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
index d88cdd2dfb..86910b222a 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.common.rpc.thrift.TSchemaNode;
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.MNodeType;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
 import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
 import org.apache.iotdb.confignode.consensus.request.auth.AuthorPlan;
@@ -115,7 +116,6 @@ import org.apache.iotdb.confignode.persistence.sync.ClusterSyncInfo;
 import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq;
 import org.apache.iotdb.consensus.common.DataSet;
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.tsfile.utils.Pair;
 
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
index 8d9200256e..6f05cff326 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
@@ -133,7 +133,10 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       mTree.setStorageGroup(partialPathName);
 
       // Set DatabaseSchema
-      mTree.getDatabaseNodeByDatabasePath(partialPathName).setStorageGroupSchema(databaseSchema);
+      mTree
+          .getDatabaseNodeByDatabasePath(partialPathName)
+          .getAsMNode()
+          .setDatabaseSchema(databaseSchema);
 
       result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
     } catch (MetadataException e) {
@@ -159,7 +162,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       PartialPath partialPathName = new PartialPath(alterSchema.getName());
 
       TDatabaseSchema currentSchema =
-          mTree.getDatabaseNodeByDatabasePath(partialPathName).getStorageGroupSchema();
+          mTree.getDatabaseNodeByDatabasePath(partialPathName).getAsMNode().getDatabaseSchema();
       // TODO: Support alter other fields
       if (alterSchema.isSetMinSchemaRegionGroupNum()) {
         currentSchema.setMinSchemaRegionGroupNum(alterSchema.getMinSchemaRegionGroupNum());
@@ -192,7 +195,10 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
             currentSchema.getMaxDataRegionGroupNum());
       }
 
-      mTree.getDatabaseNodeByDatabasePath(partialPathName).setStorageGroupSchema(currentSchema);
+      mTree
+          .getDatabaseNodeByDatabasePath(partialPathName)
+          .getAsMNode()
+          .setDatabaseSchema(currentSchema);
       result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
     } catch (MetadataException e) {
       LOGGER.error(ERROR_NAME, e);
@@ -259,7 +265,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       List<PartialPath> matchedPaths = mTree.getMatchedDatabases(patternPath, false);
       for (PartialPath path : matchedPaths) {
         schemaMap.put(
-            path.getFullPath(), mTree.getDatabaseNodeByDatabasePath(path).getStorageGroupSchema());
+            path.getFullPath(),
+            mTree.getDatabaseNodeByDatabasePath(path).getAsMNode().getDatabaseSchema());
       }
       result.setSchemaMap(schemaMap);
       result.setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()));
@@ -282,7 +289,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       List<PartialPath> matchedPaths = mTree.getBelongedDatabases(patternPath);
       if (!matchedPaths.isEmpty()) {
         for (PartialPath path : matchedPaths) {
-          mTree.getDatabaseNodeByDatabasePath(path).getStorageGroupSchema().setTTL(plan.getTTL());
+          mTree.getDatabaseNodeByDatabasePath(path).setDataTTL(plan.getTTL());
         }
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
       } else {
@@ -306,7 +313,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       if (mTree.isDatabaseAlreadySet(path)) {
         mTree
             .getDatabaseNodeByDatabasePath(path)
-            .getStorageGroupSchema()
+            .getAsMNode()
+            .getDatabaseSchema()
             .setSchemaReplicationFactor(plan.getSchemaReplicationFactor());
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
       } else {
@@ -329,7 +337,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       if (mTree.isDatabaseAlreadySet(path)) {
         mTree
             .getDatabaseNodeByDatabasePath(path)
-            .getStorageGroupSchema()
+            .getAsMNode()
+            .getDatabaseSchema()
             .setDataReplicationFactor(plan.getDataReplicationFactor());
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
       } else {
@@ -352,7 +361,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       if (mTree.isDatabaseAlreadySet(path)) {
         mTree
             .getDatabaseNodeByDatabasePath(path)
-            .getStorageGroupSchema()
+            .getAsMNode()
+            .getDatabaseSchema()
             .setTimePartitionInterval(plan.getTimePartitionInterval());
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
       } else {
@@ -381,7 +391,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
           plan.getMaxRegionGroupNumMap().entrySet()) {
         PartialPath path = new PartialPath(entry.getKey());
         TDatabaseSchema databaseSchema =
-            mTree.getDatabaseNodeByDatabasePath(path).getStorageGroupSchema();
+            mTree.getDatabaseNodeByDatabasePath(path).getAsMNode().getDatabaseSchema();
         databaseSchema.setMaxSchemaRegionGroupNum(entry.getValue().getLeft());
         databaseSchema.setMaxDataRegionGroupNum(entry.getValue().getRight());
       }
@@ -445,7 +455,10 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       throws DatabaseNotExistsException {
     databaseReadWriteLock.readLock().lock();
     try {
-      return mTree.getDatabaseNodeByDatabasePath(new PartialPath(database)).getStorageGroupSchema();
+      return mTree
+          .getDatabaseNodeByDatabasePath(new PartialPath(database))
+          .getAsMNode()
+          .getDatabaseSchema();
     } catch (MetadataException e) {
       throw new DatabaseNotExistsException(database);
     } finally {
@@ -468,7 +481,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
         List<PartialPath> matchedPaths = mTree.getMatchedDatabases(patternPath, false);
         for (PartialPath path : matchedPaths) {
           schemaMap.put(
-              path.getFullPath(), mTree.getDatabaseNodeByPath(path).getStorageGroupSchema());
+              path.getFullPath(),
+              mTree.getDatabaseNodeByPath(path).getAsMNode().getDatabaseSchema());
         }
       }
     } catch (MetadataException e) {
@@ -491,7 +505,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
     try {
       PartialPath path = new PartialPath(database);
       TDatabaseSchema storageGroupSchema =
-          mTree.getDatabaseNodeByDatabasePath(path).getStorageGroupSchema();
+          mTree.getDatabaseNodeByDatabasePath(path).getAsMNode().getDatabaseSchema();
       switch (consensusGroupType) {
         case SchemaRegion:
           return storageGroupSchema.getMinSchemaRegionGroupNum();
@@ -519,7 +533,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
     try {
       PartialPath path = new PartialPath(database);
       TDatabaseSchema storageGroupSchema =
-          mTree.getDatabaseNodeByDatabasePath(path).getStorageGroupSchema();
+          mTree.getDatabaseNodeByDatabasePath(path).getAsMNode().getDatabaseSchema();
       switch (consensusGroupType) {
         case SchemaRegion:
           return storageGroupSchema.getMaxSchemaRegionGroupNum();
@@ -853,7 +867,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       List<PartialPath> matchedPaths = mTree.getBelongedDatabases(patternPath);
       for (PartialPath path : matchedPaths) {
         schemaMap.put(
-            path.getFullPath(), mTree.getDatabaseNodeByPath(path).getStorageGroupSchema());
+            path.getFullPath(), mTree.getDatabaseNodeByPath(path).getAsMNode().getDatabaseSchema());
       }
     } catch (MetadataException e) {
       LOGGER.warn(ERROR_NAME, e);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
index b2beaf2f35..f1271c25b1 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
@@ -274,14 +274,14 @@ public class DeleteDatabaseProcedure
   public void serialize(DataOutputStream stream) throws IOException {
     stream.writeShort(ProcedureType.DELETE_STORAGE_GROUP_PROCEDURE.getTypeCode());
     super.serialize(stream);
-    ThriftConfigNodeSerDeUtils.serializeTStorageGroupSchema(deleteDatabaseSchema, stream);
+    ThriftConfigNodeSerDeUtils.serializeTDatabaseSchema(deleteDatabaseSchema, stream);
   }
 
   @Override
   public void deserialize(ByteBuffer byteBuffer) {
     super.deserialize(byteBuffer);
     try {
-      deleteDatabaseSchema = ThriftConfigNodeSerDeUtils.deserializeTStorageGroupSchema(byteBuffer);
+      deleteDatabaseSchema = ThriftConfigNodeSerDeUtils.deserializeTDatabaseSchema(byteBuffer);
     } catch (ThriftSerDeException e) {
       LOG.error("Error in deserialize DeleteStorageGroupProcedure", e);
     }
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/AbstractSchemaIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/AbstractSchemaIT.java
index 380be26487..0c74d46225 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/AbstractSchemaIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/AbstractSchemaIT.java
@@ -60,7 +60,7 @@ public abstract class AbstractSchemaIT {
         break;
       case SchemaFile:
         EnvFactory.getEnv().getConfig().getCommonConfig().setSchemaEngineMode("Schema_File");
-        allocateMemoryForSchemaRegion(3600);
+        allocateMemoryForSchemaRegion(4000);
         break;
     }
   }
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBSortedShowTimeseriesIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBSortedShowTimeseriesIT.java
index 3c70fc0511..c3fcbfe250 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBSortedShowTimeseriesIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBSortedShowTimeseriesIT.java
@@ -100,7 +100,7 @@ public class IoTDBSortedShowTimeseriesIT extends AbstractSchemaIT {
   public void setUp() throws Exception {
     super.setUp();
     if (schemaTestMode.equals(SchemaTestMode.SchemaFile)) {
-      allocateMemoryForSchemaRegion(5500);
+      allocateMemoryForSchemaRegion(10000);
     }
     EnvFactory.getEnv().initClusterEnvironment();
     createSchema();
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java
index 98e0e40863..cc94dec785 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java
@@ -53,7 +53,7 @@ public class IoTDBTagIT extends AbstractSchemaIT {
   public void setUp() throws Exception {
     super.setUp();
     if (schemaTestMode.equals(SchemaTestMode.SchemaFile)) {
-      allocateMemoryForSchemaRegion(5500);
+      allocateMemoryForSchemaRegion(10000);
     }
     EnvFactory.getEnv().initClusterEnvironment();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/IMNode.java
similarity index 51%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/IMNode.java
index b7b2e452e8..f3f8f48b80 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/IMNode.java
@@ -16,24 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.commons.schema.node;
 
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
 import org.apache.iotdb.commons.schema.tree.ITreeNode;
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
 
 /** This interface defines a MNode's operation interfaces. */
-public interface IMNode extends ITreeNode {
+public interface IMNode<N extends IMNode<N>> extends ITreeNode {
 
   String getName();
 
   void setName(String name);
 
-  IMNode getParent();
+  N getParent();
 
-  void setParent(IMNode parent);
+  void setParent(N parent);
 
   String getFullPath();
 
@@ -43,64 +45,44 @@ public interface IMNode extends ITreeNode {
 
   boolean hasChild(String name);
 
-  IMNode getChild(String name);
+  N getChild(String name);
 
-  IMNode addChild(String name, IMNode child);
+  N addChild(String name, N child);
 
-  IMNode addChild(IMNode child);
+  N addChild(N child);
 
-  IMNode deleteChild(String name);
+  N deleteChild(String name);
 
   // this method will replace the oldChild with the newChild, the data of oldChild will be moved to
   // newChild
-  void replaceChild(String oldChildName, IMNode newChildNode);
+  void replaceChild(String oldChildName, N newChildNode);
 
   // this method will move all the reference or value of current node's attributes to newMNode
-  void moveDataToNewMNode(IMNode newMNode);
+  void moveDataToNewMNode(N newMNode);
 
-  IMNodeContainer getChildren();
+  IMNodeContainer<N> getChildren();
 
-  void setChildren(IMNodeContainer children);
-
-  boolean isUseTemplate();
-
-  void setUseTemplate(boolean useTemplate);
-
-  /** @return the logic id of template set or activated on this node, id>=-1 */
-  int getSchemaTemplateId();
-
-  /** @return the template id with current state, may be negative since unset or deactivation */
-  int getSchemaTemplateIdWithState();
-
-  void setSchemaTemplateId(int schemaTemplateId);
-
-  void preUnsetSchemaTemplate();
-
-  void rollbackUnsetSchemaTemplate();
-
-  boolean isSchemaTemplatePreUnset();
-
-  void unsetSchemaTemplate();
+  void setChildren(IMNodeContainer<N> children);
 
   boolean isAboveDatabase();
 
-  boolean isStorageGroup();
+  boolean isDatabase();
 
-  boolean isEntity();
+  boolean isDevice();
 
   boolean isMeasurement();
 
   MNodeType getMNodeType(Boolean isConfig);
 
-  IStorageGroupMNode getAsStorageGroupMNode();
+  IDatabaseMNode<N> getAsDatabaseMNode();
 
-  IEntityMNode getAsEntityMNode();
+  IDeviceMNode<N> getAsDeviceMNode();
 
-  IMeasurementMNode getAsMeasurementMNode();
+  IMeasurementMNode<N> getAsMeasurementMNode();
 
-  CacheEntry getCacheEntry();
+  <R, C> R accept(MNodeVisitor<R, C> visitor, C context);
 
-  void setCacheEntry(CacheEntry cacheEntry);
+  int estimateSize();
 
-  <R, C> R accept(MNodeVisitor<R, C> visitor, C context);
+  N getAsMNode();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNodeType.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/MNodeType.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNodeType.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/MNodeType.java
index 6b498eb2b7..133bc37a7c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNodeType.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/MNodeType.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.commons.schema.node;
 
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractAboveDatabaseMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractAboveDatabaseMNode.java
new file mode 100644
index 0000000000..454a0e2038
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractAboveDatabaseMNode.java
@@ -0,0 +1,191 @@
+/*
+ * 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.commons.schema.node.common;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+
+public abstract class AbstractAboveDatabaseMNode<N extends IMNode<N>, BasicNode extends IMNode<N>>
+    implements IMNode<N> {
+
+  protected final BasicNode basicMNode;
+
+  public AbstractAboveDatabaseMNode(BasicNode basicMNode) {
+    this.basicMNode = basicMNode;
+  }
+
+  @Override
+  public String getName() {
+    return basicMNode.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    basicMNode.setName(name);
+  }
+
+  @Override
+  public N getParent() {
+    if (basicMNode.getParent() == null) {
+      return null;
+    }
+    return basicMNode.getParent();
+  }
+
+  @Override
+  public void setParent(N parent) {
+    basicMNode.setParent(parent);
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public String getFullPath() {
+    return basicMNode.getFullPath();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    basicMNode.setFullPath(fullPath);
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    return basicMNode.getPartialPath();
+  }
+
+  @Override
+  public boolean hasChild(String name) {
+    return basicMNode.hasChild(name);
+  }
+
+  @Override
+  public N getChild(String name) {
+    return basicMNode.getChild(name);
+  }
+
+  @Override
+  public N addChild(String name, N child) {
+    N res = basicMNode.addChild(name, child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N addChild(N child) {
+    N res = basicMNode.addChild(child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N deleteChild(String name) {
+    return basicMNode.deleteChild(name);
+  }
+
+  @Override
+  public void replaceChild(String oldChildName, N newChildNode) {
+    basicMNode.replaceChild(oldChildName, newChildNode);
+  }
+
+  @Override
+  public void moveDataToNewMNode(N newMNode) {
+    basicMNode.moveDataToNewMNode(newMNode);
+  }
+
+  @Override
+  public IMNodeContainer<N> getChildren() {
+    return basicMNode.getChildren();
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<N> children) {
+    basicMNode.setChildren(children);
+  }
+
+  @Override
+  public boolean isAboveDatabase() {
+    return true;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return false;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return false;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return MNodeType.INTERNAL;
+  }
+
+  @Override
+  public IDatabaseMNode<N> getAsDatabaseMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IDeviceMNode<N> getAsDeviceMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IMeasurementMNode<N> getAsMeasurementMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  /**
+   * The basic memory occupied by any AbstractDatabaseDeviceMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>basicMNode reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + basicMNode.estimateSize();
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDatabaseDeviceMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDatabaseDeviceMNode.java
new file mode 100644
index 0000000000..2577a426a4
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDatabaseDeviceMNode.java
@@ -0,0 +1,290 @@
+/*
+ * 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.commons.schema.node.common;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.info.IDatabaseDeviceInfo;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+
+import java.util.Map;
+
+public abstract class AbstractDatabaseDeviceMNode<N extends IMNode<N>, BasicNode extends IMNode<N>>
+    implements IDatabaseMNode<N>, IDeviceMNode<N> {
+
+  private final IDatabaseDeviceInfo<N> databaseDeviceInfo;
+  protected final BasicNode basicMNode;
+
+  public AbstractDatabaseDeviceMNode(
+      BasicNode basicMNode, IDatabaseDeviceInfo<N> databaseDeviceInfo) {
+    this.basicMNode = basicMNode;
+    this.databaseDeviceInfo = databaseDeviceInfo;
+  }
+
+  public BasicNode getBasicMNode() {
+    return basicMNode;
+  }
+
+  @Override
+  public String getName() {
+    return basicMNode.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    basicMNode.setName(name);
+  }
+
+  @Override
+  public N getParent() {
+    return basicMNode.getParent();
+  }
+
+  @Override
+  public void setParent(N parent) {
+    basicMNode.setParent(parent);
+  }
+
+  @Override
+  public String getFullPath() {
+    return basicMNode.getFullPath();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    basicMNode.setFullPath(fullPath);
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    return basicMNode.getPartialPath();
+  }
+
+  @Override
+  public boolean hasChild(String name) {
+    return basicMNode.hasChild(name);
+  }
+
+  @Override
+  public N getChild(String name) {
+    return basicMNode.getChild(name);
+  }
+
+  @Override
+  public N addChild(String name, N child) {
+    N res = basicMNode.addChild(name, child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N addChild(N child) {
+    N res = basicMNode.addChild(child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N deleteChild(String name) {
+    return basicMNode.deleteChild(name);
+  }
+
+  @Override
+  public void replaceChild(String oldChildName, N newChildNode) {
+    basicMNode.replaceChild(oldChildName, newChildNode);
+  }
+
+  @Override
+  public void moveDataToNewMNode(N newMNode) {
+    basicMNode.moveDataToNewMNode(newMNode);
+    if (newMNode.isDevice()) {
+      databaseDeviceInfo.moveDataToNewMNode(newMNode.getAsDeviceMNode());
+    }
+    if (newMNode.isDatabase()) {
+      databaseDeviceInfo.moveDataToNewMNode(newMNode.getAsDatabaseMNode());
+    }
+  }
+
+  @Override
+  public IMNodeContainer<N> getChildren() {
+    return basicMNode.getChildren();
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<N> children) {
+    basicMNode.setChildren(children);
+  }
+
+  @Override
+  public boolean isAboveDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return true;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return true;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return false;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return MNodeType.STORAGE_GROUP;
+  }
+
+  @Override
+  public IDatabaseMNode<N> getAsDatabaseMNode() {
+    return this;
+  }
+
+  @Override
+  public IDeviceMNode<N> getAsDeviceMNode() {
+    return this;
+  }
+
+  @Override
+  public IMeasurementMNode<N> getAsMeasurementMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    return visitor.visitDatabaseDeviceMNode(this, context);
+  }
+
+  @Override
+  public long getDataTTL() {
+    return databaseDeviceInfo.getDataTTL();
+  }
+
+  @Override
+  public void setDataTTL(long dataTTL) {
+    databaseDeviceInfo.setDataTTL(dataTTL);
+  }
+
+  @Override
+  public boolean addAlias(String alias, IMeasurementMNode<N> child) {
+    return databaseDeviceInfo.addAlias(alias, child);
+  }
+
+  @Override
+  public void deleteAliasChild(String alias) {
+    databaseDeviceInfo.deleteAliasChild(alias);
+  }
+
+  @Override
+  public Map<String, IMeasurementMNode<N>> getAliasChildren() {
+    return databaseDeviceInfo.getAliasChildren();
+  }
+
+  @Override
+  public void setAliasChildren(Map<String, IMeasurementMNode<N>> aliasChildren) {
+    databaseDeviceInfo.setAliasChildren(aliasChildren);
+  }
+
+  @Override
+  public boolean isUseTemplate() {
+    return databaseDeviceInfo.isUseTemplate();
+  }
+
+  @Override
+  public void setUseTemplate(boolean useTemplate) {
+    databaseDeviceInfo.setUseTemplate(useTemplate);
+  }
+
+  @Override
+  public void setSchemaTemplateId(int schemaTemplateId) {
+    databaseDeviceInfo.setSchemaTemplateId(schemaTemplateId);
+  }
+
+  @Override
+  public int getSchemaTemplateId() {
+    return databaseDeviceInfo.getSchemaTemplateId();
+  }
+
+  @Override
+  public int getSchemaTemplateIdWithState() {
+    return databaseDeviceInfo.getSchemaTemplateIdWithState();
+  }
+
+  @Override
+  public boolean isPreDeactivateTemplate() {
+    return databaseDeviceInfo.isPreDeactivateTemplate();
+  }
+
+  @Override
+  public void preDeactivateTemplate() {
+    databaseDeviceInfo.preDeactivateTemplate();
+  }
+
+  @Override
+  public void rollbackPreDeactivateTemplate() {
+    databaseDeviceInfo.rollbackPreDeactivateTemplate();
+  }
+
+  @Override
+  public void deactivateTemplate() {
+    databaseDeviceInfo.deactivateTemplate();
+  }
+
+  @Override
+  public boolean isAligned() {
+    return databaseDeviceInfo.isAligned();
+  }
+
+  @Override
+  public void setAligned(boolean isAligned) {
+    databaseDeviceInfo.setAligned(isAligned);
+  }
+
+  /**
+   * The basic memory occupied by any AbstractDatabaseDeviceMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>databaseDeviceInfo reference, 8B
+   *         <li>basicMNode reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + databaseDeviceInfo.estimateSize() + basicMNode.estimateSize();
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDatabaseMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDatabaseMNode.java
new file mode 100644
index 0000000000..6781bad26a
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDatabaseMNode.java
@@ -0,0 +1,215 @@
+/*
+ * 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.commons.schema.node.common;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.info.IDatabaseInfo;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+
+public abstract class AbstractDatabaseMNode<N extends IMNode<N>, BasicNode extends IMNode<N>>
+    implements IDatabaseMNode<N> {
+
+  private static final long serialVersionUID = 7999036474525817732L;
+
+  private final IDatabaseInfo<N> databaseInfo;
+  protected final BasicNode basicMNode;
+
+  public AbstractDatabaseMNode(BasicNode basicMNode, IDatabaseInfo<N> databaseInfo) {
+    this.basicMNode = basicMNode;
+    this.databaseInfo = databaseInfo;
+  }
+
+  public BasicNode getBasicMNode() {
+    return basicMNode;
+  }
+
+  @Override
+  public String getName() {
+    return basicMNode.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    basicMNode.setName(name);
+  }
+
+  @Override
+  public N getParent() {
+    return basicMNode.getParent();
+  }
+
+  @Override
+  public void setParent(N parent) {
+    basicMNode.setParent(parent);
+  }
+
+  @Override
+  public String getFullPath() {
+    return basicMNode.getFullPath();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    basicMNode.setFullPath(fullPath);
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    return basicMNode.getPartialPath();
+  }
+
+  @Override
+  public boolean hasChild(String name) {
+    return basicMNode.hasChild(name);
+  }
+
+  @Override
+  public N getChild(String name) {
+    return basicMNode.getChild(name);
+  }
+
+  @Override
+  public N addChild(String name, N child) {
+    N res = basicMNode.addChild(name, child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N addChild(N child) {
+    N res = basicMNode.addChild(child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N deleteChild(String name) {
+    return basicMNode.deleteChild(name);
+  }
+
+  @Override
+  public void replaceChild(String oldChildName, N newChildNode) {
+    basicMNode.replaceChild(oldChildName, newChildNode);
+  }
+
+  @Override
+  public void moveDataToNewMNode(N newMNode) {
+    basicMNode.moveDataToNewMNode(newMNode);
+    if (newMNode.isDatabase()) {
+      databaseInfo.moveDataToNewMNode(newMNode.getAsDatabaseMNode());
+    }
+  }
+
+  @Override
+  public IMNodeContainer<N> getChildren() {
+    return basicMNode.getChildren();
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<N> children) {
+    basicMNode.setChildren(children);
+  }
+
+  @Override
+  public boolean isAboveDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return true;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return false;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return false;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return MNodeType.STORAGE_GROUP;
+  }
+
+  @Override
+  public IDatabaseMNode<N> getAsDatabaseMNode() {
+    return this;
+  }
+
+  @Override
+  public IDeviceMNode<N> getAsDeviceMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IMeasurementMNode<N> getAsMeasurementMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    return visitor.visitDatabaseMNode(this, context);
+  }
+
+  @Override
+  public long getDataTTL() {
+    return databaseInfo.getDataTTL();
+  }
+
+  @Override
+  public void setDataTTL(long dataTTL) {
+    databaseInfo.setDataTTL(dataTTL);
+  }
+
+  /**
+   * The basic memory occupied by any AbstractDatabaseMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>databaseInfo reference, 8B
+   *         <li>basicMNode reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + databaseInfo.estimateSize() + basicMNode.estimateSize();
+  }
+
+  protected IDatabaseInfo<N> getDatabaseInfo() {
+    return databaseInfo;
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDeviceMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDeviceMNode.java
new file mode 100644
index 0000000000..093eaaf4e6
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractDeviceMNode.java
@@ -0,0 +1,280 @@
+/*
+ * 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.commons.schema.node.common;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.info.IDeviceInfo;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+
+import java.util.Map;
+
+public abstract class AbstractDeviceMNode<N extends IMNode<N>, BasicNode extends IMNode<N>>
+    implements IDeviceMNode<N> {
+
+  private final IDeviceInfo<N> deviceInfo;
+  protected final BasicNode basicMNode;
+
+  public AbstractDeviceMNode(BasicNode basicMNode, IDeviceInfo<N> deviceInfo) {
+    this.basicMNode = basicMNode;
+    this.deviceInfo = deviceInfo;
+  }
+
+  public BasicNode getBasicMNode() {
+    return basicMNode;
+  }
+
+  @Override
+  public String getName() {
+    return basicMNode.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    basicMNode.setName(name);
+  }
+
+  @Override
+  public N getParent() {
+    return basicMNode.getParent();
+  }
+
+  @Override
+  public void setParent(N parent) {
+    basicMNode.setParent(parent);
+  }
+
+  @Override
+  public String getFullPath() {
+    return basicMNode.getFullPath();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    basicMNode.setFullPath(fullPath);
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    return basicMNode.getPartialPath();
+  }
+
+  @Override
+  public boolean hasChild(String name) {
+    return basicMNode.hasChild(name) || deviceInfo.hasAliasChild(name);
+  }
+
+  @Override
+  public N getChild(String name) {
+    N res = basicMNode.getChild(name);
+    if (res == null) {
+      res = deviceInfo.getAliasChild(name);
+    }
+    return res;
+  }
+
+  @Override
+  public N addChild(String name, N child) {
+    N res = basicMNode.addChild(name, child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N addChild(N child) {
+    N res = basicMNode.addChild(child);
+    if (res == child) {
+      child.setParent(this.getAsMNode());
+    }
+    return res;
+  }
+
+  @Override
+  public N deleteChild(String name) {
+    return basicMNode.deleteChild(name);
+  }
+
+  @Override
+  public void replaceChild(String oldChildName, N newChildNode) {
+    basicMNode.replaceChild(oldChildName, newChildNode);
+  }
+
+  @Override
+  public void moveDataToNewMNode(N newMNode) {
+    basicMNode.moveDataToNewMNode(newMNode);
+    if (newMNode.isDevice()) {
+      deviceInfo.moveDataToNewMNode(newMNode.getAsDeviceMNode());
+    }
+  }
+
+  @Override
+  public IMNodeContainer<N> getChildren() {
+    return basicMNode.getChildren();
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<N> children) {
+    basicMNode.setChildren(children);
+  }
+
+  @Override
+  public boolean isAboveDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return true;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return false;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return MNodeType.DEVICE;
+  }
+
+  @Override
+  public IDatabaseMNode<N> getAsDatabaseMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IDeviceMNode<N> getAsDeviceMNode() {
+    return this;
+  }
+
+  @Override
+  public IMeasurementMNode<N> getAsMeasurementMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    return visitor.visitDeviceMNode(this, context);
+  }
+
+  @Override
+  public boolean addAlias(String alias, IMeasurementMNode<N> child) {
+    return deviceInfo.addAlias(alias, child);
+  }
+
+  @Override
+  public void deleteAliasChild(String alias) {
+    deviceInfo.deleteAliasChild(alias);
+  }
+
+  @Override
+  public Map<String, IMeasurementMNode<N>> getAliasChildren() {
+    return deviceInfo.getAliasChildren();
+  }
+
+  @Override
+  public void setAliasChildren(Map<String, IMeasurementMNode<N>> aliasChildren) {
+    deviceInfo.setAliasChildren(aliasChildren);
+  }
+
+  @Override
+  public boolean isUseTemplate() {
+    return deviceInfo.isUseTemplate();
+  }
+
+  @Override
+  public void setUseTemplate(boolean useTemplate) {
+    deviceInfo.setUseTemplate(useTemplate);
+  }
+
+  @Override
+  public void setSchemaTemplateId(int schemaTemplateId) {
+    deviceInfo.setSchemaTemplateId(schemaTemplateId);
+  }
+
+  @Override
+  public int getSchemaTemplateId() {
+    return deviceInfo.getSchemaTemplateId();
+  }
+
+  @Override
+  public int getSchemaTemplateIdWithState() {
+    return deviceInfo.getSchemaTemplateIdWithState();
+  }
+
+  @Override
+  public boolean isPreDeactivateTemplate() {
+    return deviceInfo.isPreDeactivateTemplate();
+  }
+
+  @Override
+  public void preDeactivateTemplate() {
+    deviceInfo.preDeactivateTemplate();
+  }
+
+  @Override
+  public void rollbackPreDeactivateTemplate() {
+    deviceInfo.rollbackPreDeactivateTemplate();
+  }
+
+  @Override
+  public void deactivateTemplate() {
+    deviceInfo.deactivateTemplate();
+  }
+
+  @Override
+  public boolean isAligned() {
+    return deviceInfo.isAligned();
+  }
+
+  @Override
+  public void setAligned(boolean isAligned) {
+    deviceInfo.setAligned(isAligned);
+  }
+
+  /**
+   * The basic memory occupied by any AbstractDeviceMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>deviceInfo reference, 8B
+   *         <li>basicMNode reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + deviceInfo.estimateSize() + basicMNode.estimateSize();
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractMeasurementMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractMeasurementMNode.java
new file mode 100644
index 0000000000..602c5d8d3c
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractMeasurementMNode.java
@@ -0,0 +1,256 @@
+/*
+ * 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.commons.schema.node.common;
+
+import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.info.IMeasurementInfo;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractMeasurementMNode<N extends IMNode<N>, BasicNode extends IMNode<N>>
+    implements IMeasurementMNode<N> {
+
+  private static final Logger logger = LoggerFactory.getLogger(AbstractMeasurementMNode.class);
+
+  private final IMeasurementInfo measurementInfo;
+  protected final BasicNode basicMNode;
+
+  public AbstractMeasurementMNode(BasicNode basicMNode, IMeasurementInfo measurementInfo) {
+    this.basicMNode = basicMNode;
+    this.measurementInfo = measurementInfo;
+  }
+
+  public BasicNode getBasicMNode() {
+    return basicMNode;
+  }
+
+  @Override
+  public String getName() {
+    return basicMNode.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    basicMNode.setName(name);
+  }
+
+  @Override
+  public N getParent() {
+    if (basicMNode.getParent() == null) {
+      return null;
+    }
+    return basicMNode.getParent();
+  }
+
+  @Override
+  public void setParent(N parent) {
+    basicMNode.setParent(parent);
+  }
+
+  /**
+   * get MeasurementPath of this node
+   *
+   * @return MeasurementPath
+   */
+  @Override
+  public MeasurementPath getMeasurementPath() {
+    MeasurementPath result = new MeasurementPath(getPartialPath(), getSchema());
+    result.setUnderAlignedEntity(getParent().getAsDeviceMNode().isAligned());
+    return result;
+  }
+
+  @Override
+  public IMeasurementSchema getSchema() {
+    return measurementInfo.getSchema();
+  }
+
+  @Override
+  public void setSchema(IMeasurementSchema schema) {
+    measurementInfo.setSchema(schema);
+  }
+
+  @Override
+  public TSDataType getDataType() {
+    return measurementInfo.getDataType();
+  }
+
+  @Override
+  public long getOffset() {
+    return measurementInfo.getOffset();
+  }
+
+  @Override
+  public void setOffset(long offset) {
+    measurementInfo.setOffset(offset);
+  }
+
+  @Override
+  public String getAlias() {
+    return measurementInfo.getAlias();
+  }
+
+  @Override
+  public void setAlias(String alias) {
+    measurementInfo.setAlias(alias);
+  }
+
+  @Override
+  public boolean isPreDeleted() {
+    return measurementInfo.isPreDeleted();
+  }
+
+  @Override
+  public void setPreDeleted(boolean preDeleted) {
+    measurementInfo.setPreDeleted(preDeleted);
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    return visitor.visitMeasurementMNode(this, context);
+  }
+
+  @Override
+  public String getFullPath() {
+    return basicMNode.getFullPath();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    basicMNode.setFullPath(fullPath);
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    return basicMNode.getPartialPath();
+  }
+
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  @Override
+  public N getChild(String name) {
+    logger.warn("current node {} is a MeasurementMNode, can not get child {}", getName(), name);
+    throw new RuntimeException(
+        String.format(
+            "current node %s is a MeasurementMNode, can not get child %s", getName(), name));
+  }
+
+  @Override
+  public N addChild(String name, N child) {
+    // Do nothing
+    return null;
+  }
+
+  @Override
+  public N addChild(N child) {
+    return null;
+  }
+
+  @Override
+  public N deleteChild(String name) {
+    return null;
+  }
+
+  @Override
+  public void replaceChild(String oldChildName, N newChildNode) {}
+
+  @Override
+  public void moveDataToNewMNode(N newMNode) {
+    basicMNode.moveDataToNewMNode(newMNode);
+    if (newMNode.isMeasurement()) {
+      measurementInfo.moveDataToNewMNode(newMNode.getAsMeasurementMNode());
+    }
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<N> children) {
+    // Do nothing
+  }
+
+  @Override
+  public boolean isAboveDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return false;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return true;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return MNodeType.MEASUREMENT;
+  }
+
+  @Override
+  public IDatabaseMNode<N> getAsDatabaseMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IDeviceMNode<N> getAsDeviceMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IMeasurementMNode<N> getAsMeasurementMNode() {
+    return this;
+  }
+
+  /**
+   * The basic memory occupied by any AbstractMeasurementMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>measurementInfo reference, 8B
+   *         <li>basicMNode reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + measurementInfo.estimateSize() + basicMNode.estimateSize();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDatabaseDeviceInfo.java
similarity index 79%
copy from server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDatabaseDeviceInfo.java
index f7aa604009..35034a7464 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDatabaseDeviceInfo.java
@@ -16,12 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.commons.schema.node.info;
 
-package org.apache.iotdb.db.metadata.query.info;
+import org.apache.iotdb.commons.schema.node.IMNode;
 
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
-
-public interface INodeSchemaInfo extends ISchemaInfo {
-
-  MNodeType getNodeType();
-}
+public interface IDatabaseDeviceInfo<N extends IMNode<N>>
+    extends IDatabaseInfo<N>, IDeviceInfo<N> {}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDatabaseInfo.java
similarity index 69%
copy from server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDatabaseInfo.java
index f7aa604009..8fd6c3d2a8 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDatabaseInfo.java
@@ -16,12 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.commons.schema.node.info;
 
-package org.apache.iotdb.db.metadata.query.info;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
 
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
+public interface IDatabaseInfo<N extends IMNode<N>> {
 
-public interface INodeSchemaInfo extends ISchemaInfo {
+  void moveDataToNewMNode(IDatabaseMNode<N> newMNode);
 
-  MNodeType getNodeType();
+  long getDataTTL();
+
+  void setDataTTL(long dataTTL);
+
+  int estimateSize();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDeviceInfo.java
similarity index 55%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDeviceInfo.java
index 6bbf5717c1..bc625833d5 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IDeviceInfo.java
@@ -16,25 +16,42 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.commons.schema.node.info;
+
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
 
 import java.util.Map;
 
-public interface IEntityMNode extends IMNode {
+public interface IDeviceInfo<N extends IMNode<N>> {
+
+  void moveDataToNewMNode(IDeviceMNode<N> newMNode);
 
-  boolean addAlias(String alias, IMeasurementMNode child);
+  boolean addAlias(String alias, IMeasurementMNode<N> child);
 
   void deleteAliasChild(String alias);
 
-  Map<String, IMeasurementMNode> getAliasChildren();
+  Map<String, IMeasurementMNode<N>> getAliasChildren();
+
+  void setAliasChildren(Map<String, IMeasurementMNode<N>> aliasChildren);
+
+  boolean hasAliasChild(String name);
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+  N getAliasChild(String name);
 
-  @Override
   boolean isUseTemplate();
 
   void setUseTemplate(boolean useTemplate);
 
+  void setSchemaTemplateId(int schemaTemplateId);
+
+  /** @return the logic id of template set or activated on this node, id>=-1 */
+  int getSchemaTemplateId();
+
+  /** @return the template id with current state, may be negative since unset or deactivation */
+  int getSchemaTemplateIdWithState();
+
   boolean isPreDeactivateTemplate();
 
   void preDeactivateTemplate();
@@ -46,4 +63,6 @@ public interface IEntityMNode extends IMNode {
   boolean isAligned();
 
   void setAligned(boolean isAligned);
+
+  int estimateSize();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IMeasurementInfo.java
similarity index 76%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IMeasurementInfo.java
index 5f4cccdf93..ccb9254bd5 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/info/IMeasurementInfo.java
@@ -16,23 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.commons.schema.node.info;
 
-import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
-/** This interface defines a MeasurementMNode's operation interfaces. */
-public interface IMeasurementMNode extends IMNode {
-
-  @Override
-  IEntityMNode getParent();
-
-  MeasurementPath getMeasurementPath();
+public interface IMeasurementInfo {
 
   IMeasurementSchema getSchema();
 
-  TSDataType getDataType(String measurementId);
+  void setSchema(IMeasurementSchema schema);
+
+  TSDataType getDataType();
 
   String getAlias();
 
@@ -45,4 +41,8 @@ public interface IMeasurementMNode extends IMNode {
   boolean isPreDeleted();
 
   void setPreDeleted(boolean preDeleted);
+
+  int estimateSize();
+
+  void moveDataToNewMNode(IMeasurementMNode<?> newMNode);
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeFullMemoryTest.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IDatabaseMNode.java
similarity index 73%
rename from server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeFullMemoryTest.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IDatabaseMNode.java
index e7f58836bc..570b7a31e0 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeFullMemoryTest.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IDatabaseMNode.java
@@ -16,13 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mtree.multimode;
+package org.apache.iotdb.commons.schema.node.role;
 
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.schema.node.IMNode;
 
-public class MTreeFullMemoryTest extends MTreeDiskModeTest {
-  @Override
-  protected void setMemSize() {
-    IoTDBDescriptor.getInstance().getConfig().setCachedMNodeSizeInSchemaFileMode(10000);
-  }
+/** This interface defines a DatabaseMNode's operation interfaces. */
+public interface IDatabaseMNode<N extends IMNode<N>> extends IMNode<N> {
+  long getDataTTL();
+
+  void setDataTTL(long dataTTL);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IDeviceMNode.java
similarity index 63%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IDeviceMNode.java
index 6bbf5717c1..8713933d5e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IEntityMNode.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IDeviceMNode.java
@@ -16,25 +16,33 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.commons.schema.node.role;
 
-import java.util.Map;
+import org.apache.iotdb.commons.schema.node.IMNode;
 
-public interface IEntityMNode extends IMNode {
+import java.util.Map;
 
-  boolean addAlias(String alias, IMeasurementMNode child);
+public interface IDeviceMNode<N extends IMNode<N>> extends IMNode<N> {
+  boolean addAlias(String alias, IMeasurementMNode<N> child);
 
   void deleteAliasChild(String alias);
 
-  Map<String, IMeasurementMNode> getAliasChildren();
+  Map<String, IMeasurementMNode<N>> getAliasChildren();
 
-  void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren);
+  void setAliasChildren(Map<String, IMeasurementMNode<N>> aliasChildren);
 
-  @Override
   boolean isUseTemplate();
 
   void setUseTemplate(boolean useTemplate);
 
+  void setSchemaTemplateId(int schemaTemplateId);
+
+  /** @return the logic id of template set or activated on this node, id>=-1 */
+  int getSchemaTemplateId();
+
+  /** @return the template id with current state, may be negative since unset or deactivation */
+  int getSchemaTemplateIdWithState();
+
   boolean isPreDeactivateTemplate();
 
   void preDeactivateTemplate();
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IMeasurementMNode.java
similarity index 84%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IMeasurementMNode.java
index 5f4cccdf93..57ad4dd948 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/role/IMeasurementMNode.java
@@ -16,23 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.commons.schema.node.role;
 
 import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.schema.node.IMNode;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
 /** This interface defines a MeasurementMNode's operation interfaces. */
-public interface IMeasurementMNode extends IMNode {
-
-  @Override
-  IEntityMNode getParent();
-
-  MeasurementPath getMeasurementPath();
+public interface IMeasurementMNode<N extends IMNode<N>> extends IMNode<N> {
 
   IMeasurementSchema getSchema();
 
-  TSDataType getDataType(String measurementId);
+  void setSchema(IMeasurementSchema schema);
+
+  TSDataType getDataType();
 
   String getAlias();
 
@@ -45,4 +43,6 @@ public interface IMeasurementMNode extends IMNode {
   boolean isPreDeleted();
 
   void setPreDeleted(boolean preDeleted);
+
+  MeasurementPath getMeasurementPath();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/IMNodeContainer.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeContainer.java
similarity index 81%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/IMNodeContainer.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeContainer.java
index e8e0cb6820..f3522686f7 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/IMNodeContainer.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeContainer.java
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.container;
+package org.apache.iotdb.commons.schema.node.utils;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.commons.schema.node.IMNode;
 
 import java.util.Map;
 
-public interface IMNodeContainer extends Map<String, IMNode> {}
+public interface IMNodeContainer<N extends IMNode<?>> extends Map<String, N> {}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeFactory.java
similarity index 50%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeFactory.java
index 5f4cccdf93..f8aaa669bc 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeFactory.java
@@ -16,33 +16,27 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.commons.schema.node.utils;
 
-import org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
-/** This interface defines a MeasurementMNode's operation interfaces. */
-public interface IMeasurementMNode extends IMNode {
+public interface IMNodeFactory<N extends IMNode<N>> {
+  IMeasurementMNode<N> createMeasurementMNode(
+      IDeviceMNode<N> parent, String name, IMeasurementSchema schema, String alias);
 
-  @Override
-  IEntityMNode getParent();
+  IDeviceMNode<N> createDeviceMNode(N parent, String name);
 
-  MeasurementPath getMeasurementPath();
+  IDatabaseMNode<N> createDatabaseMNode(N parent, String name);
 
-  IMeasurementSchema getSchema();
+  IDatabaseMNode<N> createDatabaseMNode(N parent, String name, long dataTTL);
 
-  TSDataType getDataType(String measurementId);
+  N createDatabaseDeviceMNode(N parent, String name, long dataTTL);
 
-  String getAlias();
+  N createAboveDatabaseMNode(N parent, String name);
 
-  void setAlias(String alias);
-
-  long getOffset();
-
-  void setOffset(long offset);
-
-  boolean isPreDeleted();
-
-  void setPreDeleted(boolean preDeleted);
+  N createInternalMNode(N parent, String name);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/IMNodeIterator.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeIterator.java
similarity index 82%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/IMNodeIterator.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeIterator.java
index 254219a280..de1c710b22 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/IMNodeIterator.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeIterator.java
@@ -16,13 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.iterator;
+package org.apache.iotdb.commons.schema.node.utils;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.commons.schema.node.IMNode;
 
 import java.util.Iterator;
 
-public interface IMNodeIterator extends Iterator<IMNode> {
+public interface IMNodeIterator<N extends IMNode<?>> extends Iterator<N> {
 
   void close();
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/visitor/MNodeVisitor.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/visitor/MNodeVisitor.java
new file mode 100644
index 0000000000..d156295eb0
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/visitor/MNodeVisitor.java
@@ -0,0 +1,42 @@
+/*
+ * 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.commons.schema.node.visitor;
+
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.common.AbstractDatabaseDeviceMNode;
+import org.apache.iotdb.commons.schema.node.common.AbstractDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.common.AbstractDeviceMNode;
+import org.apache.iotdb.commons.schema.node.common.AbstractMeasurementMNode;
+
+public abstract class MNodeVisitor<R, C> {
+
+  public abstract R visitBasicMNode(IMNode<?> node, C context);
+
+  public abstract R visitDatabaseMNode(
+      AbstractDatabaseMNode<?, ? extends IMNode<?>> node, C context);
+
+  public abstract R visitDatabaseDeviceMNode(
+      AbstractDatabaseDeviceMNode<?, ? extends IMNode<?>> node, C context);
+
+  public abstract R visitDeviceMNode(AbstractDeviceMNode<?, ? extends IMNode<?>> node, C context);
+
+  public abstract R visitMeasurementMNode(
+      AbstractMeasurementMNode<?, ? extends IMNode<?>> node, C context);
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java
index 3a2be10483..8b2f3a3c25 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java
@@ -66,7 +66,7 @@ public class ThriftConfigNodeSerDeUtils {
     return new TBinaryProtocol(tioStreamTransport);
   }
 
-  public static void serializeTStorageGroupSchema(
+  public static void serializeTDatabaseSchema(
       TDatabaseSchema storageGroupSchema, ByteBuffer buffer) {
     try {
       storageGroupSchema.write(generateWriteProtocol(buffer));
@@ -75,7 +75,7 @@ public class ThriftConfigNodeSerDeUtils {
     }
   }
 
-  public static TDatabaseSchema deserializeTStorageGroupSchema(ByteBuffer buffer) {
+  public static TDatabaseSchema deserializeTDatabaseSchema(ByteBuffer buffer) {
     TDatabaseSchema storageGroupSchema = new TDatabaseSchema();
     try {
       storageGroupSchema.read(generateReadProtocol(buffer));
@@ -85,7 +85,7 @@ public class ThriftConfigNodeSerDeUtils {
     return storageGroupSchema;
   }
 
-  public static void serializeTStorageGroupSchema(
+  public static void serializeTDatabaseSchema(
       TDatabaseSchema storageGroupSchema, OutputStream outputStream) {
     try {
       storageGroupSchema.write(generateWriteProtocol(outputStream));
@@ -94,7 +94,7 @@ public class ThriftConfigNodeSerDeUtils {
     }
   }
 
-  public static TDatabaseSchema deserializeTStorageGroupSchema(InputStream inputStream) {
+  public static TDatabaseSchema deserializeTDatabaseSchema(InputStream inputStream) {
     TDatabaseSchema storageGroupSchema = new TDatabaseSchema();
     try {
       storageGroupSchema.read(generateReadProtocol(inputStream));
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java
index 5d949079ac..ac5b742722 100644
--- a/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java
@@ -46,10 +46,10 @@ public class ThriftConfigNodeSerDeUtilsTest {
     storageGroupSchema0.setDataReplicationFactor(3);
     storageGroupSchema0.setTimePartitionInterval(604800);
 
-    ThriftConfigNodeSerDeUtils.serializeTStorageGroupSchema(storageGroupSchema0, buffer);
+    ThriftConfigNodeSerDeUtils.serializeTDatabaseSchema(storageGroupSchema0, buffer);
     buffer.flip();
     TDatabaseSchema storageGroupSchema1 =
-        ThriftConfigNodeSerDeUtils.deserializeTStorageGroupSchema(buffer);
+        ThriftConfigNodeSerDeUtils.deserializeTDatabaseSchema(buffer);
     Assert.assertEquals(storageGroupSchema0, storageGroupSchema1);
   }
 
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaReadWriteHandler.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaReadWriteHandler.java
index ea127b46df..621897dd2a 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaReadWriteHandler.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaReadWriteHandler.java
@@ -21,11 +21,11 @@ package org.apache.iotdb.db.metadata.schemaregion.rocksdb;
 
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.impl.MeasurementMNode;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RMNodeType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
index 7a30ea527e..69aec0c956 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
@@ -27,6 +27,8 @@ import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
 import org.apache.iotdb.commons.utils.PathUtils;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -42,8 +44,6 @@ import org.apache.iotdb.db.metadata.MetadataConstant;
 import org.apache.iotdb.db.metadata.idtable.IDTable;
 import org.apache.iotdb.db.metadata.idtable.IDTableManager;
 import org.apache.iotdb.db.metadata.metric.ISchemaRegionMetric;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.plan.schemaregion.read.IShowDevicesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.read.IShowNodesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.read.IShowTimeSeriesPlan;
@@ -622,7 +622,7 @@ public class RSchemaRegion implements ISchemaRegion {
                     // TODO: tags invert index update
                   }
                   readWriteHandler.executeBatch(batch);
-                  if (!deletedNode.getParent().isStorageGroup()) {
+                  if (!deletedNode.getParent().isDatabase()) {
                     parentNeedsToCheck.add(deletedNode.getParent());
                   }
                 }
@@ -645,7 +645,7 @@ public class RSchemaRegion implements ISchemaRegion {
           Stream<IMNode> parentStream = parentNeedsToCheck.parallelStream();
           parentStream.forEach(
               currentNode -> {
-                if (!currentNode.isStorageGroup()) {
+                if (!currentNode.isDatabase()) {
                   PartialPath parentPath = currentNode.getPartialPath();
                   int level = parentPath.getNodeLength();
                   int end = parentPath.getNodeLength() - 1;
@@ -655,7 +655,7 @@ public class RSchemaRegion implements ISchemaRegion {
                       readWriteHandler.deleteNode(
                           parentPath.getNodes(), RSchemaUtils.typeOfMNode(currentNode));
                       IMNode parentNode = currentNode.getParent();
-                      if (!parentNode.isStorageGroup()) {
+                      if (!parentNode.isDatabase()) {
                         tempSet.add(currentNode.getParent());
                       }
                     } catch (Exception e) {
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java
index f38d329077..f5f31136e1 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java
@@ -22,13 +22,13 @@ package org.apache.iotdb.db.metadata.schemaregion.rocksdb;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.IMNode;
 import org.apache.iotdb.commons.utils.PathUtils;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.REntityMNode;
+import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RDatabaseMNode;
+import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RDeviceMNode;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RMNodeType;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RMNodeValueType;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RMeasurementMNode;
-import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RStorageGroupMNode;
 import org.apache.iotdb.tsfile.utils.BytesUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
@@ -170,11 +170,11 @@ public class RSchemaUtils {
 
   public static RMNodeType typeOfMNode(IMNode mNode) {
     // order sensitive
-    if (mNode instanceof REntityMNode) {
+    if (mNode instanceof RDeviceMNode) {
       return RMNodeType.ENTITY;
     }
 
-    if (mNode instanceof RStorageGroupMNode) {
+    if (mNode instanceof RDatabaseMNode) {
       return RMNodeType.STORAGE_GROUP;
     }
 
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RStorageGroupMNode.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RDatabaseMNode.java
similarity index 74%
rename from schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RStorageGroupMNode.java
rename to schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RDatabaseMNode.java
index 8efabdfb29..3fb4dbaf77 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RStorageGroupMNode.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RDatabaseMNode.java
@@ -21,16 +21,16 @@ package org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode;
 
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaConstants;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaReadWriteHandler;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaUtils;
 
 import org.rocksdb.RocksDBException;
 
-public class RStorageGroupMNode extends RInternalMNode implements IStorageGroupMNode {
+public class RDatabaseMNode extends RInternalMNode implements IDatabaseMNode<IMemMNode> {
 
   private long dataTTL;
 
@@ -39,14 +39,12 @@ public class RStorageGroupMNode extends RInternalMNode implements IStorageGroupM
    *
    * @param fullPath
    */
-  public RStorageGroupMNode(
-      String fullPath, long dataTTL, RSchemaReadWriteHandler readWriteHandler) {
+  public RDatabaseMNode(String fullPath, long dataTTL, RSchemaReadWriteHandler readWriteHandler) {
     super(fullPath, readWriteHandler);
     this.dataTTL = dataTTL;
   }
 
-  public RStorageGroupMNode(
-      String fullPath, byte[] value, RSchemaReadWriteHandler readWriteHandler) {
+  public RDatabaseMNode(String fullPath, byte[] value, RSchemaReadWriteHandler readWriteHandler) {
     super(fullPath, readWriteHandler);
     Object ttl = RSchemaUtils.parseNodeValue(value, RMNodeValueType.TTL);
     if (ttl == null) {
@@ -70,12 +68,25 @@ public class RStorageGroupMNode extends RInternalMNode implements IStorageGroupM
   }
 
   @Override
-  public boolean isStorageGroup() {
+  public IMemMNode addChild(String name, IMemMNode child) {
+    return null;
+  }
+
+  @Override
+  public IMemMNode addChild(IMemMNode child) {
+    return null;
+  }
+
+  @Override
+  public void replaceChild(String oldChildName, IMemMNode newChildNode) {}
+
+  @Override
+  public boolean isDatabase() {
     return true;
   }
 
   @Override
-  public boolean isEntity() {
+  public boolean isDevice() {
     return false;
   }
 
@@ -100,19 +111,7 @@ public class RStorageGroupMNode extends RInternalMNode implements IStorageGroupM
   }
 
   @Override
-  public void setSchemaReplicationFactor(int schemaReplicationFactor) {}
-
-  @Override
-  public void setDataReplicationFactor(int dataReplicationFactor) {}
-
-  @Override
-  public void setTimePartitionInterval(long timePartitionInterval) {}
-
-  @Override
-  public void setStorageGroupSchema(TDatabaseSchema schema) {}
-
-  @Override
-  public TDatabaseSchema getStorageGroupSchema() {
+  public IMemMNode getAsMNode() {
     return null;
   }
 }
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/REntityMNode.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RDeviceMNode.java
similarity index 70%
rename from schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/REntityMNode.java
rename to schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RDeviceMNode.java
index 4fedc54239..0ce285a719 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/REntityMNode.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RDeviceMNode.java
@@ -20,9 +20,11 @@
 package org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode;
 
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaConstants;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaReadWriteHandler;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaUtils;
@@ -33,7 +35,7 @@ import org.rocksdb.RocksDBException;
 import java.nio.ByteBuffer;
 import java.util.Map;
 
-public class REntityMNode extends RInternalMNode implements IEntityMNode {
+public class RDeviceMNode extends RInternalMNode implements IDeviceMNode<IMemMNode> {
 
   private volatile boolean isAligned = false;
 
@@ -42,7 +44,7 @@ public class REntityMNode extends RInternalMNode implements IEntityMNode {
    *
    * @param fullPath
    */
-  public REntityMNode(String fullPath, RSchemaReadWriteHandler readWriteHandler) {
+  public RDeviceMNode(String fullPath, RSchemaReadWriteHandler readWriteHandler) {
     super(fullPath, readWriteHandler);
   }
 
@@ -58,7 +60,7 @@ public class REntityMNode extends RInternalMNode implements IEntityMNode {
     }
   }
 
-  public REntityMNode(String fullPath, byte[] value, RSchemaReadWriteHandler readWriteHandler) {
+  public RDeviceMNode(String fullPath, byte[] value, RSchemaReadWriteHandler readWriteHandler) {
     super(fullPath, readWriteHandler);
     deserialize(value);
   }
@@ -76,12 +78,37 @@ public class REntityMNode extends RInternalMNode implements IEntityMNode {
   }
 
   @Override
-  public Map<String, IMeasurementMNode> getAliasChildren() {
+  public Map<String, IMeasurementMNode<IMemMNode>> getAliasChildren() {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren) {
+  public void setAliasChildren(Map<String, IMeasurementMNode<IMemMNode>> aliasChildren) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isUseTemplate() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setUseTemplate(boolean useTemplate) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setSchemaTemplateId(int schemaTemplateId) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getSchemaTemplateId() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getSchemaTemplateIdWithState() {
     throw new UnsupportedOperationException();
   }
 
@@ -123,7 +150,7 @@ public class REntityMNode extends RInternalMNode implements IEntityMNode {
   }
 
   @Override
-  public boolean isEntity() {
+  public boolean isDevice() {
     return true;
   }
 
@@ -131,4 +158,14 @@ public class REntityMNode extends RInternalMNode implements IEntityMNode {
   public MNodeType getMNodeType(Boolean isConfig) {
     return MNodeType.DEVICE;
   }
+
+  @Override
+  public Object accept(MNodeVisitor visitor, Object context) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public IMemMNode getAsMNode() {
+    return this;
+  }
 }
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RInternalMNode.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RInternalMNode.java
index cff6733d21..338dfbc790 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RInternalMNode.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RInternalMNode.java
@@ -20,16 +20,16 @@
 package org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode;
 
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaConstants;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaReadWriteHandler;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaUtils;
 
 import org.rocksdb.RocksDBException;
 
-public class RInternalMNode extends RMNode {
+public class RInternalMNode extends RMNode implements IMemMNode {
 
   private volatile boolean useTemplate = false;
 
@@ -58,13 +58,13 @@ public class RInternalMNode extends RMNode {
   @Override
   public boolean hasChild(String name) {
     String childPathName = fullPath.concat(RSchemaConstants.PATH_SEPARATOR).concat(name);
-    IMNode node = getNodeBySpecifiedPath(childPathName);
+    IMemMNode node = getNodeBySpecifiedPath(childPathName);
     return node != null;
   }
 
   /** get the child with the name */
   @Override
-  public IMNode getChild(String name) {
+  public IMemMNode getChild(String name) {
     String childPathName = fullPath.concat(RSchemaConstants.PATH_SEPARATOR).concat(name);
     return getNodeBySpecifiedPath(childPathName);
   }
@@ -77,7 +77,7 @@ public class RInternalMNode extends RMNode {
    * @return
    */
   @Override
-  public IMNode addChild(String name, IMNode child) {
+  public IMemMNode addChild(String name, IMemMNode child) {
     child.setParent(this);
     String childName = fullPath.concat(RSchemaConstants.PATH_SEPARATOR).concat(name);
     int childNameMaxLevel = RSchemaUtils.getLevelByPartialPath(childName);
@@ -105,14 +105,14 @@ public class RInternalMNode extends RMNode {
    * @return return the MNode already added
    */
   @Override
-  public IMNode addChild(IMNode child) {
+  public IMemMNode addChild(IMemMNode child) {
     addChild(child.getName(), child);
     return child;
   }
 
   /** delete a child */
   @Override
-  public IMNode deleteChild(String name) {
+  public IMemMNode deleteChild(String name) {
     String childPathName = fullPath.concat(RSchemaConstants.PATH_SEPARATOR).concat(name);
     int nodeNameMaxLevel = RSchemaUtils.getLevelByPartialPath(childPathName);
     for (RMNodeType type : RMNodeType.values()) {
@@ -141,7 +141,7 @@ public class RInternalMNode extends RMNode {
    * @param newChildNode new child node
    */
   @Override
-  public void replaceChild(String oldChildName, IMNode newChildNode) {
+  public void replaceChild(String oldChildName, IMemMNode newChildNode) {
     if (!oldChildName.equals(newChildNode.getName())) {
       throw new RuntimeException("New child's name must be the same as old child's name!");
     }
@@ -150,7 +150,7 @@ public class RInternalMNode extends RMNode {
   }
 
   @Override
-  public IMNodeContainer getChildren() {
+  public IMNodeContainer<IMemMNode> getChildren() {
     throw new UnsupportedOperationException();
   }
 
@@ -160,12 +160,12 @@ public class RInternalMNode extends RMNode {
   }
 
   @Override
-  public boolean isUseTemplate() {
-    return useTemplate;
+  public int estimateSize() {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public void setUseTemplate(boolean useTemplate) {
-    this.useTemplate = useTemplate;
+  public IMemMNode getAsMNode() {
+    throw new UnsupportedOperationException();
   }
 }
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMNode.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMNode.java
index 2dbad227c9..05cd849158 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMNode.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMNode.java
@@ -22,13 +22,13 @@ package org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaConstants;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaReadWriteHandler;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaUtils;
@@ -39,13 +39,13 @@ import org.slf4j.LoggerFactory;
 
 import java.util.Objects;
 
-public abstract class RMNode implements IMNode {
+public abstract class RMNode implements IMNode<IMemMNode> {
   /** from root to this node, only be set when used once for InternalMNode */
   protected String fullPath;
 
   protected RSchemaReadWriteHandler readWriteHandler;
 
-  protected IMNode parent;
+  protected IMemMNode parent;
 
   protected String name;
 
@@ -71,7 +71,7 @@ public abstract class RMNode implements IMNode {
   }
 
   @Override
-  public IMNode getParent() {
+  public IMemMNode getParent() {
     if (parent != null) {
       return parent;
     }
@@ -81,9 +81,9 @@ public abstract class RMNode implements IMNode {
     return parent;
   }
 
-  protected IMNode getNodeBySpecifiedPath(String keyName) {
+  protected IMemMNode getNodeBySpecifiedPath(String keyName) {
     byte[] value = null;
-    IMNode node;
+    IMemMNode node;
     int nodeNameMaxLevel = RSchemaUtils.getLevelByPartialPath(keyName);
     for (RMNodeType type : RMNodeType.values()) {
       String parentInnerName =
@@ -96,13 +96,13 @@ public abstract class RMNode implements IMNode {
       if (value != null) {
         switch (type.getValue()) {
           case RSchemaConstants.NODE_TYPE_SG:
-            node = new RStorageGroupMNode(keyName, value, readWriteHandler);
+            node = new RDatabaseMNode(keyName, value, readWriteHandler);
             return node;
           case RSchemaConstants.NODE_TYPE_INTERNAL:
             node = new RInternalMNode(keyName, readWriteHandler);
             return node;
           case RSchemaConstants.NODE_TYPE_ENTITY:
-            node = new REntityMNode(keyName, value, readWriteHandler);
+            node = new RDeviceMNode(keyName, value, readWriteHandler);
             return node;
           case RSchemaConstants.NODE_TYPE_MEASUREMENT:
             node = new RMeasurementMNode(keyName, value, readWriteHandler);
@@ -114,7 +114,7 @@ public abstract class RMNode implements IMNode {
   }
 
   @Override
-  public void setParent(IMNode parent) {
+  public void setParent(IMemMNode parent) {
     this.parent = parent;
   }
 
@@ -143,58 +143,18 @@ public abstract class RMNode implements IMNode {
     this.fullPath = fullPath;
   }
 
-  @Override
-  public boolean isUseTemplate() {
-    return false;
-  }
-
-  @Override
-  public int getSchemaTemplateId() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int getSchemaTemplateIdWithState() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setSchemaTemplateId(int schemaTemplateId) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void preUnsetSchemaTemplate() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void rollbackUnsetSchemaTemplate() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean isSchemaTemplatePreUnset() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void unsetSchemaTemplate() {
-    throw new UnsupportedOperationException();
-  }
-
   @Override
   public boolean isAboveDatabase() {
     return false;
   }
 
   @Override
-  public boolean isStorageGroup() {
+  public boolean isDatabase() {
     return false;
   }
 
   @Override
-  public boolean isEntity() {
+  public boolean isDevice() {
     return false;
   }
 
@@ -204,18 +164,18 @@ public abstract class RMNode implements IMNode {
   }
 
   @Override
-  public IStorageGroupMNode getAsStorageGroupMNode() {
-    if (isStorageGroup()) {
-      return (IStorageGroupMNode) this;
+  public IDatabaseMNode getAsDatabaseMNode() {
+    if (isDatabase()) {
+      return (IDatabaseMNode) this;
     } else {
       throw new UnsupportedOperationException("Wrong MNode Type");
     }
   }
 
   @Override
-  public IEntityMNode getAsEntityMNode() {
-    if (isEntity()) {
-      return (IEntityMNode) this;
+  public IDeviceMNode getAsDeviceMNode() {
+    if (isDevice()) {
+      return (IDeviceMNode) this;
     } else {
       throw new UnsupportedOperationException("Wrong MNode Type");
     }
@@ -253,17 +213,7 @@ public abstract class RMNode implements IMNode {
   }
 
   @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
-    throw new UnsupportedOperationException("Temporarily unsupported");
-  }
-
-  @Override
-  public CacheEntry getCacheEntry() {
-    throw new UnsupportedOperationException("Temporarily unsupported");
-  }
-
-  @Override
-  public void setCacheEntry(CacheEntry cacheEntry) {
+  public void moveDataToNewMNode(IMemMNode newMNode) {
     throw new UnsupportedOperationException("Temporarily unsupported");
   }
 
@@ -274,8 +224,7 @@ public abstract class RMNode implements IMNode {
 
   @Override
   public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    throw new UnsupportedOperationException("RMNode doesn't support this method");
+    throw new UnsupportedOperationException("Wrong MNode Type");
   }
-
   // end
 }
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java
index c13879c552..58a9708d44 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java
@@ -21,11 +21,10 @@ package org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode;
 
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaConstants;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaReadWriteHandler;
 import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaUtils;
@@ -40,7 +39,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Map;
 
-public class RMeasurementMNode extends RMNode implements IMeasurementMNode {
+public class RMeasurementMNode extends RMNode implements IMeasurementMNode<IMemMNode>, IMemMNode {
 
   protected String alias;
 
@@ -79,17 +78,22 @@ public class RMeasurementMNode extends RMNode implements IMeasurementMNode {
   }
 
   @Override
-  public IEntityMNode getParent() {
+  public IMemMNode getParent() {
     if (super.getParent() == null) {
       return null;
     }
-    return parent.getAsEntityMNode();
+    return parent;
+  }
+
+  @Override
+  public IMemMNode getAsMNode() {
+    return null;
   }
 
   @Override
   public MeasurementPath getMeasurementPath() {
     MeasurementPath result = new MeasurementPath(super.getPartialPath(), schema);
-    result.setUnderAlignedEntity(getParent().isAligned());
+    result.setUnderAlignedEntity(getParent().getAsDeviceMNode().isAligned());
     if (alias != null && !alias.isEmpty()) {
       result.setMeasurementAlias(alias);
     }
@@ -102,7 +106,12 @@ public class RMeasurementMNode extends RMNode implements IMeasurementMNode {
   }
 
   @Override
-  public TSDataType getDataType(String measurementId) {
+  public void setSchema(IMeasurementSchema schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  public TSDataType getDataType() {
     return schema.getType();
   }
 
@@ -167,40 +176,37 @@ public class RMeasurementMNode extends RMNode implements IMeasurementMNode {
   }
 
   @Override
-  public IMNode getChild(String name) {
+  public IMemMNode getChild(String name) {
     throw new RuntimeException(
         String.format(
             "current node %s is a MeasurementMNode, can not get child %s", super.name, name));
   }
 
   @Override
-  public IMNode addChild(String name, IMNode child) {
+  public IMemMNode addChild(String name, IMemMNode child) {
     // Do nothing
     return child;
   }
 
   @Override
-  public IMNode addChild(IMNode child) {
+  public IMemMNode addChild(IMemMNode child) {
     return null;
   }
 
   @Override
-  public IMNode deleteChild(String name) {
+  public IMemMNode deleteChild(String name) {
     // Do nothing
     return null;
   }
 
   @Override
-  public void replaceChild(String oldChildName, IMNode newChildNode) {}
+  public void replaceChild(String oldChildName, IMemMNode newChildNode) {}
 
   @Override
   public IMNodeContainer getChildren() {
     throw new UnsupportedOperationException();
   }
 
-  @Override
-  public void setUseTemplate(boolean useTemplate) {}
-
   @Override
   public boolean isMeasurement() {
     return true;
@@ -211,6 +217,11 @@ public class RMeasurementMNode extends RMNode implements IMeasurementMNode {
     return MNodeType.MEASUREMENT;
   }
 
+  @Override
+  public int estimateSize() {
+    throw new UnsupportedOperationException();
+  }
+
   public Map<String, String> getTags() {
     return tags;
   }
diff --git a/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java b/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
index 7aacc56740..75b73a4d9e 100644
--- a/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
+++ b/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -37,7 +38,6 @@ import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
 import org.apache.iotdb.db.metadata.idtable.entry.SHA256DeviceID;
 import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
 import org.apache.iotdb.db.metadata.metric.ISchemaRegionMetric;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
 import org.apache.iotdb.db.metadata.plan.schemaregion.impl.write.SchemaRegionWritePlanFactory;
 import org.apache.iotdb.db.metadata.plan.schemaregion.read.IShowDevicesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.read.IShowNodesPlan;
@@ -87,7 +87,7 @@ public class TagSchemaRegion implements ISchemaRegion {
   // when a path ends with ".**", it represents batch processing
   private final String TAIL = ".**";
 
-  private final IStorageGroupMNode storageGroupMNode;
+  private final IDatabaseMNode storageGroupMNode;
   private final String storageGroupFullPath;
   private final SchemaRegionId schemaRegionId;
   private final String schemaRegionDirPath;
@@ -103,7 +103,7 @@ public class TagSchemaRegion implements ISchemaRegion {
   public TagSchemaRegion(
       PartialPath storageGroup,
       SchemaRegionId schemaRegionId,
-      IStorageGroupMNode storageGroupMNode,
+      IDatabaseMNode storageGroupMNode,
       ISeriesNumerMonitor seriesNumerMonitor)
       throws MetadataException {
     storageGroupFullPath = storageGroup.getFullPath();
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/InsertMeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/InsertMeasurementMNode.java
deleted file mode 100644
index 1b376d4c51..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/InsertMeasurementMNode.java
+++ /dev/null
@@ -1,287 +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.idtable.entry;
-
-import org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
-/**
- * Generated entity implements IMeasurementMNode interface to unify insert logic through id table
- * and SchemaProcessor
- */
-public class InsertMeasurementMNode implements IMeasurementMNode {
-  SchemaEntry schemaEntry;
-
-  IMeasurementSchema schema;
-
-  public InsertMeasurementMNode(String measurementId, SchemaEntry schemaEntry) {
-    this.schemaEntry = schemaEntry;
-    schema =
-        new MeasurementSchema(
-            measurementId,
-            schemaEntry.getTSDataType(),
-            schemaEntry.getTSEncoding(),
-            schemaEntry.getCompressionType());
-  }
-
-  // region support methods
-
-  @Override
-  public boolean isPreDeleted() {
-    return false;
-  }
-
-  @Override
-  public void setPreDeleted(boolean preDeleted) {}
-
-  @Override
-  public IMeasurementSchema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public TSDataType getDataType(String measurementId) {
-    return schemaEntry.getTSDataType();
-  }
-
-  @Override
-  public IEntityMNode getParent() {
-    return null;
-  }
-
-  @Override
-  public String toString() {
-    return schema.getMeasurementId();
-  }
-
-  // endregion
-
-  // region unsupported methods
-  @Override
-  public String getName() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setName(String name) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setParent(IMNode parent) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public String getFullPath() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setFullPath(String fullPath) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public PartialPath getPartialPath() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public boolean hasChild(String name) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public IMNode getChild(String name) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public IMNode addChild(String name, IMNode child) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public IMNode addChild(IMNode child) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public IMNode deleteChild(String name) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void replaceChild(String oldChildName, IMNode newChildNode) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public IMNodeContainer getChildren() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setChildren(IMNodeContainer children) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public boolean isUseTemplate() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setUseTemplate(boolean useTemplate) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public int getSchemaTemplateId() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public int getSchemaTemplateIdWithState() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setSchemaTemplateId(int schemaTemplateId) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void preUnsetSchemaTemplate() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void rollbackUnsetSchemaTemplate() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public boolean isSchemaTemplatePreUnset() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void unsetSchemaTemplate() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public boolean isAboveDatabase() {
-    return false;
-  }
-
-  @Override
-  public boolean isStorageGroup() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public boolean isEntity() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public boolean isMeasurement() {
-    return true;
-  }
-
-  @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return MNodeType.MEASUREMENT;
-  }
-
-  @Override
-  public IStorageGroupMNode getAsStorageGroupMNode() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public IEntityMNode getAsEntityMNode() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public IMeasurementMNode getAsMeasurementMNode() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public CacheEntry getCacheEntry() {
-    return null;
-  }
-
-  @Override
-  public void setCacheEntry(CacheEntry cacheEntry) {}
-
-  @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public MeasurementPath getMeasurementPath() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public String getAlias() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setAlias(String alias) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public long getOffset() {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  @Override
-  public void setOffset(long offset) {
-    throw new UnsupportedOperationException("insert measurement mnode doesn't support this method");
-  }
-
-  // endregion
-}
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
deleted file mode 100644
index cfd7f6ee4d..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java
+++ /dev/null
@@ -1,200 +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 org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-public abstract class MNode implements IMNode {
-
-  /** Name of the MNode */
-  protected String name;
-
-  protected IMNode parent;
-
-  /** from root to this node, only be set when used once for InternalMNode */
-  protected String fullPath;
-
-  protected CacheEntry cacheEntry;
-
-  /** Constructor of MNode. */
-  protected MNode(IMNode parent, String name) {
-    this.parent = parent;
-    this.name = name;
-  }
-
-  @Override
-  public String getName() {
-    return name;
-  }
-
-  @Override
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  @Override
-  public IMNode getParent() {
-    return parent;
-  }
-
-  @Override
-  public void setParent(IMNode parent) {
-    this.parent = parent;
-  }
-
-  /**
-   * get partial path of this node
-   *
-   * @return partial path
-   */
-  @Override
-  public PartialPath getPartialPath() {
-    List<String> detachedPath = new ArrayList<>();
-    IMNode temp = this;
-    detachedPath.add(temp.getName());
-    while (temp.getParent() != null) {
-      temp = temp.getParent();
-      detachedPath.add(0, temp.getName());
-    }
-    return new PartialPath(detachedPath.toArray(new String[0]));
-  }
-
-  /** get full path */
-  @Override
-  public String getFullPath() {
-    if (fullPath == null) {
-      fullPath = concatFullPath();
-    }
-    return fullPath;
-  }
-
-  String concatFullPath() {
-    StringBuilder builder = new StringBuilder(name);
-    IMNode curr = this;
-    while (curr.getParent() != null) {
-      curr = curr.getParent();
-      builder.insert(0, IoTDBConstant.PATH_SEPARATOR).insert(0, curr.getName());
-    }
-    return builder.toString();
-  }
-
-  @Override
-  public void setFullPath(String fullPath) {
-    this.fullPath = fullPath;
-  }
-
-  @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
-    newMNode.setParent(parent);
-    newMNode.setCacheEntry(cacheEntry);
-  }
-
-  @Override
-  public boolean isUseTemplate() {
-    return false;
-  }
-
-  @Override
-  public boolean isStorageGroup() {
-    return false;
-  }
-
-  @Override
-  public boolean isEntity() {
-    return false;
-  }
-
-  @Override
-  public boolean isMeasurement() {
-    return false;
-  }
-
-  @Override
-  public IStorageGroupMNode getAsStorageGroupMNode() {
-    if (isStorageGroup()) {
-      return (IStorageGroupMNode) this;
-    } else {
-      throw new UnsupportedOperationException("Wrong MNode Type");
-    }
-  }
-
-  @Override
-  public IEntityMNode getAsEntityMNode() {
-    if (isEntity()) {
-      return (IEntityMNode) this;
-    } else {
-      throw new UnsupportedOperationException("Wrong MNode Type");
-    }
-  }
-
-  @Override
-  public IMeasurementMNode getAsMeasurementMNode() {
-    if (isMeasurement()) {
-      return (IMeasurementMNode) this;
-    } else {
-      throw new UnsupportedOperationException("Wrong MNode Type");
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    MNode mNode = (MNode) o;
-    if (fullPath == null) {
-      return Objects.equals(getFullPath(), mNode.getFullPath());
-    } else {
-      return Objects.equals(fullPath, mNode.getFullPath());
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    if (fullPath == null) {
-      return Objects.hash(getFullPath());
-    } else {
-      return Objects.hash(fullPath);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return this.getName();
-  }
-
-  @Override
-  public CacheEntry getCacheEntry() {
-    return cacheEntry;
-  }
-
-  @Override
-  public void setCacheEntry(CacheEntry cacheEntry) {
-    this.cacheEntry = cacheEntry;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNodeUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNodeUtils.java
deleted file mode 100644
index c5fed50dfd..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNodeUtils.java
+++ /dev/null
@@ -1,77 +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;
-
-public class MNodeUtils {
-
-  /**
-   * When a measurement, represented by template or MeasurementMNode, is going to be added to a
-   * node, the node should be set to entity. Before invoking this method, lock the related MTree
-   * part first.
-   *
-   * @param node node to be transformed
-   * @return generated entityMNode
-   */
-  public static IEntityMNode setToEntity(IMNode node) {
-    IEntityMNode entityMNode;
-    if (node.isEntity()) {
-      entityMNode = node.getAsEntityMNode();
-    } else {
-      if (node.isStorageGroup()) {
-        entityMNode =
-            new StorageGroupEntityMNode(
-                node.getParent(), node.getName(), node.getAsStorageGroupMNode().getDataTTL());
-        node.moveDataToNewMNode(entityMNode);
-      } else {
-        entityMNode = new EntityMNode(node.getParent(), node.getName());
-        if (node.getParent() != null) {
-          node.getParent().replaceChild(node.getName(), entityMNode);
-        } else {
-          node.moveDataToNewMNode(entityMNode);
-        }
-      }
-    }
-    return entityMNode;
-  }
-
-  /**
-   * When there's no measurement, represented by template or MeasurementMNode, is under this
-   * entityMNode, it should not act as entity anymore. Before invoking this method, lock related
-   * MTree structure first.
-   *
-   * @param entityMNode node to be transformed
-   * @return generated NoEntity node
-   */
-  public static IMNode setToInternal(IEntityMNode entityMNode) {
-    IMNode node;
-    IMNode parent = entityMNode.getParent();
-    if (entityMNode.isStorageGroup()) {
-      node =
-          new StorageGroupMNode(
-              parent, entityMNode.getName(), entityMNode.getAsStorageGroupMNode().getDataTTL());
-    } else {
-      node = new InternalMNode(parent, entityMNode.getName());
-    }
-
-    if (parent != null) {
-      parent.replaceChild(entityMNode.getName(), node);
-    }
-    return node;
-  }
-}
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
deleted file mode 100644
index 2151a170c0..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
+++ /dev/null
@@ -1,233 +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 org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.container.MNodeContainers;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MeasurementMNode extends MNode implements IMeasurementMNode {
-
-  private static final Logger logger = LoggerFactory.getLogger(MeasurementMNode.class);
-
-  /** alias name of this measurement */
-  protected String alias;
-  /** tag/attribute's start offset in tag file */
-  private long offset = -1;
-  /** measurement's Schema for one timeseries represented by current leaf node */
-  private IMeasurementSchema schema;
-  /** whether this measurement is pre deleted and considered in black list */
-  private boolean preDeleted = false;
-
-  /**
-   * MeasurementMNode factory method. The type of returned MeasurementMNode is according to the
-   * schema type. The default type is UnaryMeasurementMNode, which means if schema == null, an
-   * UnaryMeasurementMNode will return.
-   */
-  public static IMeasurementMNode getMeasurementMNode(
-      IEntityMNode parent, String measurementName, IMeasurementSchema schema, String alias) {
-    return new MeasurementMNode(parent, measurementName, schema, alias);
-  }
-
-  /** @param alias alias of measurementName */
-  public MeasurementMNode(IMNode parent, String name, IMeasurementSchema schema, String alias) {
-    super(parent, name);
-    this.schema = schema;
-    this.alias = alias;
-  }
-
-  @Override
-  public IEntityMNode getParent() {
-    if (parent == null) {
-      return null;
-    }
-    return parent.getAsEntityMNode();
-  }
-
-  /**
-   * get MeasurementPath of this node
-   *
-   * @return MeasurementPath
-   */
-  @Override
-  public MeasurementPath getMeasurementPath() {
-    MeasurementPath result = new MeasurementPath(super.getPartialPath(), schema);
-    result.setUnderAlignedEntity(getParent().isAligned());
-    return result;
-  }
-
-  @Override
-  public IMeasurementSchema getSchema() {
-    return schema;
-  }
-
-  /**
-   * get data type
-   *
-   * @param measurementId if it's a vector schema, we need sensor name of it
-   * @return measurement data type
-   */
-  @Override
-  public TSDataType getDataType(String measurementId) {
-    return schema.getType();
-  }
-
-  @Override
-  public long getOffset() {
-    return offset;
-  }
-
-  @Override
-  public void setOffset(long offset) {
-    this.offset = offset;
-  }
-
-  @Override
-  public String getAlias() {
-    return alias;
-  }
-
-  @Override
-  public void setAlias(String alias) {
-    this.alias = alias;
-  }
-
-  @Override
-  public boolean isPreDeleted() {
-    return preDeleted;
-  }
-
-  @Override
-  public void setPreDeleted(boolean preDeleted) {
-    this.preDeleted = preDeleted;
-  }
-
-  @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    return visitor.visitMeasurementMNode(this, context);
-  }
-
-  @Override
-  public String getFullPath() {
-    if (fullPath != null) {
-      return fullPath;
-    }
-    return concatFullPath();
-  }
-
-  @Override
-  public boolean hasChild(String name) {
-    return false;
-  }
-
-  @Override
-  public IMNode getChild(String name) {
-    MeasurementMNode.logger.warn(
-        "current node {} is a MeasurementMNode, can not get child {}", this.name, name);
-    throw new RuntimeException(
-        String.format(
-            "current node %s is a MeasurementMNode, can not get child %s", super.name, name));
-  }
-
-  @Override
-  public IMNode addChild(String name, IMNode child) {
-    // Do nothing
-    return null;
-  }
-
-  @Override
-  public IMNode addChild(IMNode child) {
-    return null;
-  }
-
-  @Override
-  public IMNode deleteChild(String name) {
-    return null;
-  }
-
-  @Override
-  public void replaceChild(String oldChildName, IMNode newChildNode) {}
-
-  @Override
-  public IMNodeContainer getChildren() {
-    return MNodeContainers.emptyMNodeContainer();
-  }
-
-  @Override
-  public void setChildren(IMNodeContainer children) {
-    // Do nothing
-  }
-
-  @Override
-  public int getSchemaTemplateId() {
-    MeasurementMNode.logger.warn(
-        "current node {} is a MeasurementMNode, can not get Schema Template", name);
-    throw new UnsupportedOperationException(
-        String.format("current node %s is a MeasurementMNode, can not get Schema Template", name));
-  }
-
-  @Override
-  public int getSchemaTemplateIdWithState() {
-    MeasurementMNode.logger.warn(
-        "current node {} is a MeasurementMNode, can not get Schema Template", name);
-    throw new UnsupportedOperationException(
-        String.format("current node %s is a MeasurementMNode, can not get Schema Template", name));
-  }
-
-  @Override
-  public void setSchemaTemplateId(int schemaTemplateId) {}
-
-  @Override
-  public void preUnsetSchemaTemplate() {}
-
-  @Override
-  public void rollbackUnsetSchemaTemplate() {}
-
-  @Override
-  public boolean isSchemaTemplatePreUnset() {
-    return false;
-  }
-
-  @Override
-  public void unsetSchemaTemplate() {}
-
-  @Override
-  public boolean isAboveDatabase() {
-    return false;
-  }
-
-  @Override
-  public void setUseTemplate(boolean useTemplate) {}
-
-  @Override
-  public boolean isMeasurement() {
-    return true;
-  }
-
-  @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return MNodeType.MEASUREMENT;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupEntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupEntityMNode.java
deleted file mode 100644
index f68a5ddad3..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupEntityMNode.java
+++ /dev/null
@@ -1,90 +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 org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-
-public class StorageGroupEntityMNode extends EntityMNode implements IStorageGroupMNode {
-  /**
-   * when the data file in a database is older than dataTTL, it is considered invalid and will be
-   * eventually deleted.
-   */
-  private long dataTTL;
-
-  public StorageGroupEntityMNode(IMNode parent, String name, long dataTTL) {
-    super(parent, name);
-    this.dataTTL = dataTTL;
-  }
-
-  @Override
-  public String getFullPath() {
-    if (fullPath == null) {
-      fullPath = concatFullPath().intern();
-    }
-    return fullPath;
-  }
-
-  @Override
-  public long getDataTTL() {
-    return dataTTL;
-  }
-
-  @Override
-  public void setDataTTL(long dataTTL) {
-    this.dataTTL = dataTTL;
-  }
-
-  @Override
-  public void setSchemaReplicationFactor(int schemaReplicationFactor) {}
-
-  @Override
-  public void setDataReplicationFactor(int dataReplicationFactor) {}
-
-  @Override
-  public void setTimePartitionInterval(long timePartitionInterval) {}
-
-  @Override
-  public void setStorageGroupSchema(TDatabaseSchema schema) {}
-
-  @Override
-  public TDatabaseSchema getStorageGroupSchema() {
-    return null;
-  }
-
-  @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
-    super.moveDataToNewMNode(newMNode);
-  }
-
-  @Override
-  public boolean isStorageGroup() {
-    return true;
-  }
-
-  @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return MNodeType.STORAGE_GROUP;
-  }
-
-  @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    return visitor.visitStorageGroupEntityMNode(this, context);
-  }
-}
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
deleted file mode 100644
index 2cf4d75c75..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java
+++ /dev/null
@@ -1,102 +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 org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-
-public class StorageGroupMNode extends InternalMNode implements IStorageGroupMNode {
-
-  private static final long serialVersionUID = 7999036474525817732L;
-
-  private TDatabaseSchema schema;
-
-  public StorageGroupMNode(IMNode parent, String name) {
-    super(parent, name);
-  }
-
-  // TODO: @yukun, remove this constructor
-  public StorageGroupMNode(IMNode parent, String name, long dataTTL) {
-    super(parent, name);
-    this.schema = new TDatabaseSchema(name).setTTL(dataTTL);
-  }
-
-  @Override
-  public String getFullPath() {
-    if (fullPath == null) {
-      fullPath = concatFullPath().intern();
-    }
-    return fullPath;
-  }
-
-  @Override
-  public long getDataTTL() {
-    return schema.getTTL();
-  }
-
-  @Override
-  public void setDataTTL(long dataTTL) {
-    schema.setTTL(dataTTL);
-  }
-
-  @Override
-  public void setSchemaReplicationFactor(int schemaReplicationFactor) {
-    schema.setSchemaReplicationFactor(schemaReplicationFactor);
-  }
-
-  @Override
-  public void setDataReplicationFactor(int dataReplicationFactor) {
-    schema.setDataReplicationFactor(dataReplicationFactor);
-  }
-
-  @Override
-  public void setTimePartitionInterval(long timePartitionInterval) {
-    schema.setTimePartitionInterval(timePartitionInterval);
-  }
-
-  @Override
-  public void setStorageGroupSchema(TDatabaseSchema schema) {
-    this.schema = schema;
-  }
-
-  @Override
-  public TDatabaseSchema getStorageGroupSchema() {
-    return schema;
-  }
-
-  @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
-    super.moveDataToNewMNode(newMNode);
-  }
-
-  @Override
-  public boolean isStorageGroup() {
-    return true;
-  }
-
-  @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return MNodeType.STORAGE_GROUP;
-  }
-
-  @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    return visitor.visitStorageGroupMNode(this, context);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IStorageGroupMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/IConfigMNode.java
similarity index 60%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/IStorageGroupMNode.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/IConfigMNode.java
index 25d9e2f7b0..43fcd5bf31 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IStorageGroupMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/IConfigMNode.java
@@ -16,24 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.db.metadata.mnode.config;
 
+import org.apache.iotdb.commons.schema.node.IMNode;
 import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
 
-/** This interface defines a StorageGroupMNode's operation interfaces. */
-public interface IStorageGroupMNode extends IMNode {
+public interface IConfigMNode extends IMNode<IConfigMNode> {
 
-  long getDataTTL();
+  void setSchemaTemplateId(int id);
 
-  void setDataTTL(long dataTTL);
+  int getSchemaTemplateId();
 
-  void setSchemaReplicationFactor(int schemaReplicationFactor);
+  void preUnsetSchemaTemplate();
 
-  void setDataReplicationFactor(int dataReplicationFactor);
+  void rollbackUnsetSchemaTemplate();
 
-  void setTimePartitionInterval(long timePartitionInterval);
+  boolean isSchemaTemplatePreUnset();
 
-  void setStorageGroupSchema(TDatabaseSchema schema);
+  void unsetSchemaTemplate();
 
-  TDatabaseSchema getStorageGroupSchema();
+  default void setDatabaseSchema(TDatabaseSchema schema) {
+    throw new UnsupportedOperationException();
+  }
+
+  default TDatabaseSchema getDatabaseSchema() {
+    throw new UnsupportedOperationException();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/basic/ConfigBasicMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/basic/ConfigBasicMNode.java
new file mode 100644
index 0000000000..b1866e1c21
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/basic/ConfigBasicMNode.java
@@ -0,0 +1,279 @@
+/*
+ * 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.config.basic;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+import org.apache.iotdb.db.metadata.mnode.config.IConfigMNode;
+import org.apache.iotdb.db.metadata.mnode.config.container.ConfigMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.config.info.ConfigMNodeInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ConfigBasicMNode implements IConfigMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  private IConfigMNode parent;
+  private final ConfigMNodeInfo configMNodeInfo;
+
+  /** from root to this node, only be set when used once for InternalMNode */
+  private String fullPath;
+
+  /** Constructor of MNode. */
+  public ConfigBasicMNode(IConfigMNode parent, String name) {
+    this.parent = parent;
+    this.configMNodeInfo = new ConfigMNodeInfo(name);
+  }
+
+  @Override
+  public String getName() {
+    return configMNodeInfo.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    configMNodeInfo.setName(name);
+  }
+
+  @Override
+  public IConfigMNode getParent() {
+    return parent;
+  }
+
+  @Override
+  public void setParent(IConfigMNode parent) {
+    this.parent = parent;
+  }
+
+  @Override
+  public String getFullPath() {
+    if (fullPath == null) {
+      fullPath = concatFullPath();
+    }
+    return fullPath;
+  }
+
+  String concatFullPath() {
+    StringBuilder builder = new StringBuilder(getName());
+    IConfigMNode curr = this;
+    while (curr.getParent() != null) {
+      curr = curr.getParent();
+      builder.insert(0, IoTDBConstant.PATH_SEPARATOR).insert(0, curr.getName());
+    }
+    return builder.toString();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    this.fullPath = fullPath;
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    List<String> detachedPath = new ArrayList<>();
+    IConfigMNode temp = this;
+    detachedPath.add(temp.getName());
+    while (temp.getParent() != null) {
+      temp = temp.getParent();
+      detachedPath.add(0, temp.getName());
+    }
+    return new PartialPath(detachedPath.toArray(new String[0]));
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IConfigMNode getChild(String name) {
+    return null;
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   * @return the child of this node after addChild
+   */
+  @Override
+  public IConfigMNode addChild(String name, IConfigMNode child) {
+    return null;
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  @Override
+  public IConfigMNode addChild(IConfigMNode child) {
+    return null;
+  }
+
+  /** delete a child */
+  @Override
+  public IConfigMNode deleteChild(String name) {
+    return null;
+  }
+
+  /**
+   * Replace a child of this mnode. New child's name must be the same as old child's name.
+   *
+   * @param oldChildName measurement name
+   * @param newChildNode new child node
+   */
+  @Override
+  public synchronized void replaceChild(String oldChildName, IConfigMNode newChildNode) {}
+
+  @Override
+  public void moveDataToNewMNode(IConfigMNode newMNode) {
+    newMNode.setParent(parent);
+    newMNode.setSchemaTemplateId(configMNodeInfo.getSchemaTemplateIdWithState());
+  }
+
+  @Override
+  public IMNodeContainer<IConfigMNode> getChildren() {
+    return ConfigMNodeContainer.emptyMNodeContainer();
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<IConfigMNode> children) {}
+
+  @Override
+  public boolean isAboveDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return false;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return false;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return isConfig ? MNodeType.SG_INTERNAL : MNodeType.INTERNAL;
+  }
+
+  @Override
+  public IDatabaseMNode<IConfigMNode> getAsDatabaseMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IDeviceMNode<IConfigMNode> getAsDeviceMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IMeasurementMNode<IConfigMNode> getAsMeasurementMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    return visitor.visitBasicMNode(this, context);
+  }
+
+  @Override
+  public void setSchemaTemplateId(int id) {
+    configMNodeInfo.setSchemaTemplateId(id);
+  }
+
+  @Override
+  public int getSchemaTemplateId() {
+    return configMNodeInfo.getSchemaTemplateId();
+  }
+
+  @Override
+  public void preUnsetSchemaTemplate() {
+    configMNodeInfo.preUnsetSchemaTemplate();
+  }
+
+  @Override
+  public void rollbackUnsetSchemaTemplate() {
+    configMNodeInfo.rollbackUnsetSchemaTemplate();
+  }
+
+  @Override
+  public boolean isSchemaTemplatePreUnset() {
+    return configMNodeInfo.isSchemaTemplatePreUnset();
+  }
+
+  @Override
+  public void unsetSchemaTemplate() {
+    configMNodeInfo.unsetSchemaTemplate();
+  }
+
+  /**
+   * The basic memory occupied by any ConfigBasicMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>basicMNodeInfo reference, 8B
+   *         <li>parent reference, 8B
+   *         <li>fullPath reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   *       <ol>
+   *         <li>key reference, 8B
+   *         <li>value reference, 8B
+   *         <li>entry size, see ConcurrentHashMap.Node, 28
+   *       </ol>
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + 8 + 8 + 8 + 8 + 28 + configMNodeInfo.estimateSize();
+  }
+
+  @Override
+  public IConfigMNode getAsMNode() {
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainers.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/container/ConfigMNodeContainer.java
similarity index 59%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainers.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/container/ConfigMNodeContainer.java
index ad473edac0..52e190f907 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainers.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/container/ConfigMNodeContainer.java
@@ -16,49 +16,32 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.container;
+package org.apache.iotdb.db.metadata.mnode.config.container;
 
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mtree.store.disk.CachedMNodeContainer;
-import org.apache.iotdb.db.metadata.schemaregion.SchemaEngineMode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.config.IConfigMNode;
 
 import org.jetbrains.annotations.NotNull;
 
 import java.util.AbstractMap;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 import static java.util.Collections.emptySet;
 
-public class MNodeContainers {
+public class ConfigMNodeContainer extends ConcurrentHashMap<String, IConfigMNode>
+    implements IMNodeContainer<IConfigMNode> {
 
-  @SuppressWarnings("rawtypes")
-  private static final IMNodeContainer EMPTY_CONTAINER = new EmptyContainer();
+  private static final IMNodeContainer<IConfigMNode> EMPTY_CONTAINER =
+      new ConfigMNodeContainer.EmptyContainer();
 
-  @SuppressWarnings("unchecked")
-  public static IMNodeContainer emptyMNodeContainer() {
+  public static IMNodeContainer<IConfigMNode> emptyMNodeContainer() {
     return EMPTY_CONTAINER;
   }
 
-  public static boolean isEmptyContainer(IMNodeContainer container) {
-    return EMPTY_CONTAINER.equals(container);
-  }
-
-  public static IMNodeContainer getNewMNodeContainer() {
-    if (IoTDBDescriptor.getInstance()
-        .getConfig()
-        .getSchemaEngineMode()
-        .equals(SchemaEngineMode.Schema_File.toString())) {
-      return new CachedMNodeContainer();
-    } else {
-      return new MNodeContainerMapImpl();
-    }
-  }
-
-  private static class EmptyContainer extends AbstractMap<String, IMNode>
-      implements IMNodeContainer {
+  private static class EmptyContainer extends AbstractMap<String, IConfigMNode>
+      implements IMNodeContainer<IConfigMNode> {
 
     @Override
     public int size() {
@@ -81,7 +64,7 @@ public class MNodeContainers {
     }
 
     @Override
-    public IMNode get(Object key) {
+    public IConfigMNode get(Object key) {
       return null;
     }
 
@@ -93,12 +76,12 @@ public class MNodeContainers {
 
     @Override
     @NotNull
-    public Collection<IMNode> values() {
+    public Collection<IConfigMNode> values() {
       return emptySet();
     }
 
     @NotNull
-    public Set<Map.Entry<String, IMNode>> entrySet() {
+    public Set<Entry<String, IConfigMNode>> entrySet() {
       return emptySet();
     }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/factory/ConfigMNodeFactory.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/factory/ConfigMNodeFactory.java
new file mode 100644
index 0000000000..dc1a03e36b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/factory/ConfigMNodeFactory.java
@@ -0,0 +1,81 @@
+/*
+ * 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.config.factory;
+
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.db.metadata.mnode.config.IConfigMNode;
+import org.apache.iotdb.db.metadata.mnode.config.impl.ConfigBasicInternalMNode;
+import org.apache.iotdb.db.metadata.mnode.config.impl.ConfigDatabaseMNode;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+public class ConfigMNodeFactory implements IMNodeFactory<IConfigMNode> {
+  private ConfigMNodeFactory() {}
+
+  private static class ConfigMNodeFactoryHolder {
+    private static final ConfigMNodeFactory INSTANCE = new ConfigMNodeFactory();
+
+    private ConfigMNodeFactoryHolder() {}
+  }
+
+  public static ConfigMNodeFactory getInstance() {
+    return ConfigMNodeFactory.ConfigMNodeFactoryHolder.INSTANCE;
+  }
+
+  @Override
+  public IMeasurementMNode<IConfigMNode> createMeasurementMNode(
+      IDeviceMNode<IConfigMNode> parent, String name, IMeasurementSchema schema, String alias) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public IDeviceMNode<IConfigMNode> createDeviceMNode(IConfigMNode parent, String name) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public IDatabaseMNode<IConfigMNode> createDatabaseMNode(IConfigMNode parent, String name) {
+    return new ConfigDatabaseMNode(parent, name);
+  }
+
+  @Override
+  public IDatabaseMNode<IConfigMNode> createDatabaseMNode(
+      IConfigMNode parent, String name, long dataTTL) {
+    IDatabaseMNode<IConfigMNode> res = new ConfigDatabaseMNode(parent, name);
+    res.setDataTTL(dataTTL);
+    return res;
+  }
+
+  @Override
+  public IConfigMNode createDatabaseDeviceMNode(IConfigMNode parent, String name, long dataTTL) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public IConfigMNode createAboveDatabaseMNode(IConfigMNode parent, String name) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public IConfigMNode createInternalMNode(IConfigMNode parent, String name) {
+    return new ConfigBasicInternalMNode(parent, name);
+  }
+}
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/config/impl/ConfigBasicInternalMNode.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/impl/ConfigBasicInternalMNode.java
index 47e95f0b9c..daef341a6e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/impl/ConfigBasicInternalMNode.java
@@ -16,44 +16,29 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.db.metadata.mnode.config.impl;
 
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.container.MNodeContainers;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-
-import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.config.IConfigMNode;
+import org.apache.iotdb.db.metadata.mnode.config.basic.ConfigBasicMNode;
+import org.apache.iotdb.db.metadata.mnode.config.container.ConfigMNodeContainer;
 
 /**
  * This class is the implementation of Metadata Node. One MNode instance represents one node in the
  * Metadata Tree
  */
-public class InternalMNode extends MNode {
-
-  private static final long serialVersionUID = -770028375899514063L;
+public class ConfigBasicInternalMNode extends ConfigBasicMNode {
 
   /**
-   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
-   * synchronized check
+   * suppress warnings reason: volatile for double synchronized check
    *
    * <p>This will be a ConcurrentHashMap instance
    */
   @SuppressWarnings("squid:S3077")
-  protected transient volatile IMNodeContainer children = null;
-
-  /**
-   * This field is mainly used in cluster schema template features. In InternalMNode of ConfigMTree,
-   * this field represents the template set on this node. In EntityMNode of MTree in SchemaRegion,
-   * this field represents the template activated on this node. The normal usage value range is [0,
-   * Int.MaxValue], since this is implemented as auto inc id. The default value -1 means
-   * NON_TEMPLATE. This value will be set negative to implement some pre-delete features.
-   */
-  protected int schemaTemplateId = NON_TEMPLATE;
-
-  private volatile boolean useTemplate = false;
+  private transient volatile IMNodeContainer<IConfigMNode> children = null;
 
   /** Constructor of MNode. */
-  public InternalMNode(IMNode parent, String name) {
+  public ConfigBasicInternalMNode(IConfigMNode parent, String name) {
     super(parent, name);
   }
 
@@ -65,8 +50,8 @@ public class InternalMNode extends MNode {
 
   /** get the child with the name */
   @Override
-  public IMNode getChild(String name) {
-    IMNode child = null;
+  public IConfigMNode getChild(String name) {
+    IConfigMNode child = null;
     if (children != null) {
       child = children.get(name);
     }
@@ -81,7 +66,7 @@ public class InternalMNode extends MNode {
    * @return the child of this node after addChild
    */
   @Override
-  public IMNode addChild(String name, IMNode child) {
+  public IConfigMNode addChild(String name, IConfigMNode child) {
     /* use cpu time to exchange memory
      * measurementNode's children should be null to save memory
      * add child method will only be called when writing MTree, which is not a frequent operation
@@ -90,12 +75,12 @@ public class InternalMNode extends MNode {
       // double check, children is volatile
       synchronized (this) {
         if (children == null) {
-          children = MNodeContainers.getNewMNodeContainer();
+          children = new ConfigMNodeContainer();
         }
       }
     }
     child.setParent(this);
-    IMNode existingChild = children.putIfAbsent(name, child);
+    IConfigMNode existingChild = children.putIfAbsent(name, child);
     return existingChild == null ? child : existingChild;
   }
 
@@ -111,7 +96,7 @@ public class InternalMNode extends MNode {
    * @return return the MNode already added
    */
   @Override
-  public IMNode addChild(IMNode child) {
+  public IConfigMNode addChild(IConfigMNode child) {
     /* use cpu time to exchange memory
      * measurementNode's children should be null to save memory
      * add child method will only be called when writing MTree, which is not a frequent operation
@@ -120,7 +105,7 @@ public class InternalMNode extends MNode {
       // double check, children is volatile
       synchronized (this) {
         if (children == null) {
-          children = MNodeContainers.getNewMNodeContainer();
+          children = new ConfigMNodeContainer();
         }
       }
     }
@@ -132,7 +117,7 @@ public class InternalMNode extends MNode {
 
   /** delete a child */
   @Override
-  public IMNode deleteChild(String name) {
+  public IConfigMNode deleteChild(String name) {
     if (children != null) {
       return children.remove(name);
     }
@@ -146,11 +131,11 @@ public class InternalMNode extends MNode {
    * @param newChildNode new child node
    */
   @Override
-  public synchronized void replaceChild(String oldChildName, IMNode newChildNode) {
+  public synchronized void replaceChild(String oldChildName, IConfigMNode newChildNode) {
     if (!oldChildName.equals(newChildNode.getName())) {
       throw new RuntimeException("New child's name must be the same as old child's name!");
     }
-    IMNode oldChildNode = this.getChild(oldChildName);
+    IConfigMNode oldChildNode = this.getChild(oldChildName);
     if (oldChildNode == null) {
       return;
     }
@@ -161,12 +146,9 @@ public class InternalMNode extends MNode {
   }
 
   @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
+  public void moveDataToNewMNode(IConfigMNode newMNode) {
     super.moveDataToNewMNode(newMNode);
 
-    newMNode.setUseTemplate(useTemplate);
-    newMNode.setSchemaTemplateId(schemaTemplateId);
-
     if (children != null) {
       newMNode.setChildren(children);
       children.forEach((childName, childNode) -> childNode.setParent(newMNode));
@@ -174,86 +156,26 @@ public class InternalMNode extends MNode {
   }
 
   @Override
-  public IMNodeContainer getChildren() {
+  public IMNodeContainer<IConfigMNode> getChildren() {
     if (children == null) {
-      return MNodeContainers.emptyMNodeContainer();
+      return ConfigMNodeContainer.emptyMNodeContainer();
     }
     return children;
   }
 
   @Override
-  public void setChildren(IMNodeContainer children) {
+  public void setChildren(IMNodeContainer<IConfigMNode> children) {
     this.children = children;
   }
 
+  /** MNodeContainer reference and basic occupation, 8 + 80B */
   @Override
-  public int getSchemaTemplateId() {
-    return schemaTemplateId >= -1 ? schemaTemplateId : -schemaTemplateId - 2;
-  }
-
-  @Override
-  public int getSchemaTemplateIdWithState() {
-    return schemaTemplateId;
-  }
-
-  @Override
-  public void setSchemaTemplateId(int schemaTemplateId) {
-    this.schemaTemplateId = schemaTemplateId;
-  }
-
-  /**
-   * In InternalMNode, schemaTemplateId represents the template set on this node. The pre unset
-   * mechanism is implemented by making this value negative. Since value 0 and -1 are all occupied,
-   * the available negative value range is [Int.MIN_VALUE, -2]. The value of a pre unset case equals
-   * the negative normal value minus 2. For example, if the id of set template is 0, then - 0 - 2 =
-   * -2 represents the pre unset operation of this template on this node.
-   */
-  @Override
-  public void preUnsetSchemaTemplate() {
-    if (this.schemaTemplateId > -1) {
-      this.schemaTemplateId = -schemaTemplateId - 2;
-    }
-  }
-
-  @Override
-  public void rollbackUnsetSchemaTemplate() {
-    if (schemaTemplateId < -1) {
-      schemaTemplateId = -schemaTemplateId - 2;
-    }
-  }
-
-  @Override
-  public boolean isSchemaTemplatePreUnset() {
-    return schemaTemplateId < -1;
-  }
-
-  @Override
-  public void unsetSchemaTemplate() {
-    this.schemaTemplateId = -1;
-  }
-
-  @Override
-  public boolean isAboveDatabase() {
-    return false;
-  }
-
-  @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return isConfig ? MNodeType.SG_INTERNAL : MNodeType.INTERNAL;
-  }
-
-  @Override
-  public boolean isUseTemplate() {
-    return useTemplate;
-  }
-
-  @Override
-  public void setUseTemplate(boolean useTemplate) {
-    this.useTemplate = useTemplate;
+  public int estimateSize() {
+    return 8 + 80 + super.estimateSize();
   }
 
   @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    return visitor.visitInternalMNode(this, context);
+  public IConfigMNode getAsMNode() {
+    return this;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/impl/ConfigDatabaseMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/impl/ConfigDatabaseMNode.java
new file mode 100644
index 0000000000..e2f425f2d3
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/impl/ConfigDatabaseMNode.java
@@ -0,0 +1,80 @@
+/*
+ * 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.config.impl;
+
+import org.apache.iotdb.commons.schema.node.common.AbstractDatabaseMNode;
+import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
+import org.apache.iotdb.db.metadata.mnode.config.IConfigMNode;
+import org.apache.iotdb.db.metadata.mnode.config.basic.ConfigBasicMNode;
+import org.apache.iotdb.db.metadata.mnode.config.info.ConfigDatabaseInfo;
+
+public class ConfigDatabaseMNode extends AbstractDatabaseMNode<IConfigMNode, ConfigBasicMNode>
+    implements IConfigMNode {
+  private final ConfigDatabaseInfo configDatabaseInfo;
+
+  public ConfigDatabaseMNode(IConfigMNode parent, String name) {
+    super(new ConfigBasicInternalMNode(parent, name), new ConfigDatabaseInfo(name));
+    this.configDatabaseInfo = (ConfigDatabaseInfo) getDatabaseInfo();
+  }
+
+  @Override
+  public void setSchemaTemplateId(int id) {
+    basicMNode.setSchemaTemplateId(id);
+  }
+
+  @Override
+  public int getSchemaTemplateId() {
+    return basicMNode.getSchemaTemplateId();
+  }
+
+  @Override
+  public void preUnsetSchemaTemplate() {
+    basicMNode.preUnsetSchemaTemplate();
+  }
+
+  @Override
+  public void rollbackUnsetSchemaTemplate() {
+    basicMNode.rollbackUnsetSchemaTemplate();
+  }
+
+  @Override
+  public boolean isSchemaTemplatePreUnset() {
+    return basicMNode.isSchemaTemplatePreUnset();
+  }
+
+  @Override
+  public void unsetSchemaTemplate() {
+    basicMNode.unsetSchemaTemplate();
+  }
+
+  @Override
+  public void setDatabaseSchema(TDatabaseSchema schema) {
+    configDatabaseInfo.setSchema(schema);
+  }
+
+  @Override
+  public TDatabaseSchema getDatabaseSchema() {
+    return configDatabaseInfo.getSchema();
+  }
+
+  @Override
+  public IConfigMNode getAsMNode() {
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/info/ConfigDatabaseInfo.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/info/ConfigDatabaseInfo.java
new file mode 100644
index 0000000000..c07e40c69d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/info/ConfigDatabaseInfo.java
@@ -0,0 +1,70 @@
+/*
+ * 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.config.info;
+
+import org.apache.iotdb.commons.schema.node.info.IDatabaseInfo;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
+import org.apache.iotdb.db.metadata.mnode.config.IConfigMNode;
+
+public class ConfigDatabaseInfo implements IDatabaseInfo<IConfigMNode> {
+
+  private TDatabaseSchema schema;
+
+  public ConfigDatabaseInfo(String name) {
+    this.schema = new TDatabaseSchema(name);
+  }
+
+  @Override
+  public void moveDataToNewMNode(IDatabaseMNode<IConfigMNode> newMNode) {
+    newMNode.getAsMNode().setDatabaseSchema(schema);
+  }
+
+  @Override
+  public long getDataTTL() {
+    return schema.getTTL();
+  }
+
+  @Override
+  public void setDataTTL(long dataTTL) {
+    schema.setTTL(dataTTL);
+  }
+
+  public TDatabaseSchema getSchema() {
+    return schema;
+  }
+
+  public void setSchema(TDatabaseSchema schema) {
+    this.schema = schema;
+  }
+
+  /**
+   * The memory occupied by an DatabaseDeviceInfo based occupation
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>reference schema, 8B
+   *   <li>object TDatabaseSchema, 112B (calculated by RamUsageEstimator)
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + 112;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/info/ConfigMNodeInfo.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/info/ConfigMNodeInfo.java
new file mode 100644
index 0000000000..26ac24eaf7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/config/info/ConfigMNodeInfo.java
@@ -0,0 +1,82 @@
+/*
+ * 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.config.info;
+
+import org.apache.iotdb.db.metadata.mnode.mem.info.BasicMNodeInfo;
+
+import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
+
+public class ConfigMNodeInfo extends BasicMNodeInfo {
+  /**
+   * This field is mainly used in cluster schema template features. In InternalMNode of ConfigMTree,
+   * this field represents the template set on this node. The normal usage value range is [0,
+   * Int.MaxValue], since this is implemented as auto inc id. The default value -1 means
+   * NON_TEMPLATE. This value will be set negative to implement some pre-delete features.
+   */
+  protected int schemaTemplateId = NON_TEMPLATE;
+
+  public ConfigMNodeInfo(String name) {
+    super(name);
+  }
+
+  public void setSchemaTemplateId(int schemaTemplateId) {
+    this.schemaTemplateId = schemaTemplateId;
+  }
+
+  /**
+   * In InternalMNode, schemaTemplateId represents the template set on this node. The pre unset
+   * mechanism is implemented by making this value negative. Since value 0 and -1 are all occupied,
+   * the available negative value range is [Int.MIN_VALUE, -2]. The value of a pre unset case equals
+   * the negative normal value minus 2. For example, if the id of set template is 0, then - 0 - 2 =
+   * -2 represents the pre unset operation of this template on this node.
+   */
+  public int getSchemaTemplateId() {
+    return schemaTemplateId >= -1 ? schemaTemplateId : -schemaTemplateId - 2;
+  }
+
+  public int getSchemaTemplateIdWithState() {
+    return schemaTemplateId;
+  }
+
+  public void preUnsetSchemaTemplate() {
+    if (this.schemaTemplateId > -1) {
+      this.schemaTemplateId = -schemaTemplateId - 2;
+    }
+  }
+
+  public void rollbackUnsetSchemaTemplate() {
+    if (schemaTemplateId < -1) {
+      schemaTemplateId = -schemaTemplateId - 2;
+    }
+  }
+
+  public boolean isSchemaTemplatePreUnset() {
+    return schemaTemplateId < -1;
+  }
+
+  public void unsetSchemaTemplate() {
+    this.schemaTemplateId = -1;
+  }
+
+  @Override
+  public int estimateSize() {
+    // int schemaTemplateId, 4B
+    return super.estimateSize() + 4;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/estimator/BasicMNodSizeEstimator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/estimator/BasicMNodSizeEstimator.java
deleted file mode 100644
index ff695e753b..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/estimator/BasicMNodSizeEstimator.java
+++ /dev/null
@@ -1,114 +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.estimator;
-
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-
-public class BasicMNodSizeEstimator implements IMNodeSizeEstimator {
-  /**
-   * The basic memory occupied by any MNode object
-   *
-   * <ol>
-   *   <li>object header, 8B
-   *   <li>node attributes
-   *       <ol>
-   *         <li>name reference, name length and name hash code, 8 + 4 + 4 = 16B
-   *         <li>parent reference, 8B
-   *         <li>fullPath reference, 8B
-   *         <li>cacheEntry reference, 8B
-   *       </ol>
-   *   <li>MapEntry in parent
-   *       <ol>
-   *         <li>key reference, 8B
-   *         <li>value reference, 8B
-   *         <li>entry size, see ConcurrentHashMap.Node, 28
-   *       </ol>
-   * </ol>
-   */
-  protected static final int NODE_BASE_SIZE = 92;
-
-  /**
-   * The basic extra memory occupied by an InternalMNode based on MNode occupation
-   *
-   * <ol>
-   *   <li>template reference, 8B
-   *   <li>boolean useTemplate, 1B
-   *   <li>MNodeContainer reference and basic occupation, 8 + 80B
-   * </ol>
-   */
-  protected static final int INTERNAL_NODE_BASE_SIZE = 97;
-
-  /**
-   * The basic extra memory occupied by an StorageGroupMNode based on InternalMNode occupation
-   *
-   * <ol>
-   *   <li>dataTTL, 8B
-   *   <li>fullPath length and hashCode, 4 + 4 = 8B
-   * </ol>
-   */
-  protected static final int STORAGE_GROUP_NODE_BASE_SIZE = 16;
-
-  /**
-   * The basic extra memory occupied by an EntityMNode based on InternalMNode occupation
-   *
-   * <ol>
-   *   <li>isAligned, 1B
-   *   <li>aliasChildren reference, 8B
-   * </ol>
-   */
-  protected static final int ENTITY_NODE_BASE_SIZE = 9;
-
-  /**
-   * The basic extra memory occupied by an MeasurementMNode based on MNode occupation
-   *
-   * <ol>
-   *   <li>alias reference, 8B
-   *   <li>tagOffset, 8B
-   *   <li>estimated schema size, 32B
-   * </ol>
-   */
-  protected static final int MEASUREMENT_NODE_BASE_SIZE = 48;
-  // alias length, hashCode and occupation in aliasMap, 4 + 4 + 44 = 52B
-  protected static final int ALIAS_BASE_SIZE = 52;
-
-  public int estimateSize(IMNode node) {
-    int size = NODE_BASE_SIZE + node.getName().length();
-    if (node.isMeasurement()) {
-      size += MEASUREMENT_NODE_BASE_SIZE;
-      IMeasurementMNode measurementMNode = node.getAsMeasurementMNode();
-      if (measurementMNode.getAlias() != null) {
-        size += ALIAS_BASE_SIZE + measurementMNode.getAlias().length();
-      }
-    } else {
-      size += INTERNAL_NODE_BASE_SIZE;
-      if (node.isStorageGroup()) {
-        size += STORAGE_GROUP_NODE_BASE_SIZE;
-        size += node.getAsStorageGroupMNode().getFullPath().length();
-      }
-
-      if (node.isEntity()) {
-        size += ENTITY_NODE_BASE_SIZE;
-      }
-    }
-
-    return size;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/estimator/IMNodeSizeEstimator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/estimator/IMNodeSizeEstimator.java
deleted file mode 100644
index e9ecfd4cd8..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/estimator/IMNodeSizeEstimator.java
+++ /dev/null
@@ -1,38 +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.estimator;
-
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-
-import static org.apache.iotdb.db.metadata.mnode.estimator.BasicMNodSizeEstimator.ALIAS_BASE_SIZE;
-import static org.apache.iotdb.db.metadata.mnode.estimator.BasicMNodSizeEstimator.ENTITY_NODE_BASE_SIZE;
-
-public interface IMNodeSizeEstimator {
-
-  int estimateSize(IMNode node);
-
-  static int getEntityNodeBaseSize() {
-    return ENTITY_NODE_BASE_SIZE;
-  }
-
-  static int getAliasBaseSize() {
-    return ALIAS_BASE_SIZE;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/IMemMNode.java
similarity index 81%
copy from server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/IMemMNode.java
index f7aa604009..68ef9b8605 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/query/info/INodeSchemaInfo.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/IMemMNode.java
@@ -16,12 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.metadata.mnode.mem;
 
-package org.apache.iotdb.db.metadata.query.info;
+import org.apache.iotdb.commons.schema.node.IMNode;
 
-import org.apache.iotdb.db.metadata.mnode.MNodeType;
-
-public interface INodeSchemaInfo extends ISchemaInfo {
-
-  MNodeType getNodeType();
-}
+public interface IMemMNode extends IMNode<IMemMNode> {}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/basic/BasicMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/basic/BasicMNode.java
new file mode 100644
index 0000000000..dfd23aa441
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/basic/BasicMNode.java
@@ -0,0 +1,252 @@
+/*
+ * 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.mem.basic;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.container.MemMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.info.BasicMNodeInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class BasicMNode implements IMemMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  private IMemMNode parent;
+  private final BasicMNodeInfo basicMNodeInfo;
+
+  /** from root to this node, only be set when used once for InternalMNode */
+  private String fullPath;
+
+  /** Constructor of MNode. */
+  public BasicMNode(IMemMNode parent, String name) {
+    this.parent = parent;
+    this.basicMNodeInfo = new BasicMNodeInfo(name);
+  }
+
+  @Override
+  public String getName() {
+    return basicMNodeInfo.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    basicMNodeInfo.setName(name);
+  }
+
+  @Override
+  public IMemMNode getParent() {
+    return parent;
+  }
+
+  @Override
+  public void setParent(IMemMNode parent) {
+    this.parent = parent;
+  }
+
+  @Override
+  public String getFullPath() {
+    if (fullPath == null) {
+      fullPath = concatFullPath();
+    }
+    return fullPath;
+  }
+
+  String concatFullPath() {
+    StringBuilder builder = new StringBuilder(getName());
+    IMemMNode curr = this;
+    while (curr.getParent() != null) {
+      curr = curr.getParent();
+      builder.insert(0, IoTDBConstant.PATH_SEPARATOR).insert(0, curr.getName());
+    }
+    return builder.toString();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    this.fullPath = fullPath;
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    List<String> detachedPath = new ArrayList<>();
+    IMemMNode temp = this;
+    detachedPath.add(temp.getName());
+    while (temp.getParent() != null) {
+      temp = temp.getParent();
+      detachedPath.add(0, temp.getName());
+    }
+    return new PartialPath(detachedPath.toArray(new String[0]));
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMemMNode getChild(String name) {
+    return null;
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   * @return the child of this node after addChild
+   */
+  @Override
+  public IMemMNode addChild(String name, IMemMNode child) {
+    return null;
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  @Override
+  public IMemMNode addChild(IMemMNode child) {
+    return null;
+  }
+
+  /** delete a child */
+  @Override
+  public IMemMNode deleteChild(String name) {
+    return null;
+  }
+
+  /**
+   * Replace a child of this mnode. New child's name must be the same as old child's name.
+   *
+   * @param oldChildName measurement name
+   * @param newChildNode new child node
+   */
+  @Override
+  public synchronized void replaceChild(String oldChildName, IMemMNode newChildNode) {}
+
+  @Override
+  public void moveDataToNewMNode(IMemMNode newMNode) {
+    newMNode.setParent(parent);
+  }
+
+  @Override
+  public IMNodeContainer<IMemMNode> getChildren() {
+    return MemMNodeContainer.emptyMNodeContainer();
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<IMemMNode> children) {}
+
+  @Override
+  public boolean isAboveDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return false;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return false;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return isConfig ? MNodeType.SG_INTERNAL : MNodeType.INTERNAL;
+  }
+
+  @Override
+  public IDatabaseMNode<IMemMNode> getAsDatabaseMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IDeviceMNode<IMemMNode> getAsDeviceMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IMeasurementMNode<IMemMNode> getAsMeasurementMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    return visitor.visitBasicMNode(this, context);
+  }
+
+  /**
+   * The basic memory occupied by any BasicMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>basicMNodeInfo reference, 8B
+   *         <li>parent reference, 8B
+   *         <li>fullPath reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   *       <ol>
+   *         <li>key reference, 8B
+   *         <li>value reference, 8B
+   *         <li>entry size, see ConcurrentHashMap.Node, 28
+   *       </ol>
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + 8 + 8 + 8 + 8 + 28 + basicMNodeInfo.estimateSize();
+  }
+
+  @Override
+  public IMemMNode getAsMNode() {
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainers.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/container/MemMNodeContainer.java
similarity index 60%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainers.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/container/MemMNodeContainer.java
index ad473edac0..813793383f 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainers.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/container/MemMNodeContainer.java
@@ -16,12 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.container;
+package org.apache.iotdb.db.metadata.mnode.mem.container;
 
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mtree.store.disk.CachedMNodeContainer;
-import org.apache.iotdb.db.metadata.schemaregion.SchemaEngineMode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
 
 import org.jetbrains.annotations.NotNull;
 
@@ -29,36 +27,22 @@ import java.util.AbstractMap;
 import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 import static java.util.Collections.emptySet;
 
-public class MNodeContainers {
+public class MemMNodeContainer extends ConcurrentHashMap<String, IMemMNode>
+    implements IMNodeContainer<IMemMNode> {
 
-  @SuppressWarnings("rawtypes")
-  private static final IMNodeContainer EMPTY_CONTAINER = new EmptyContainer();
+  private static final IMNodeContainer<IMemMNode> EMPTY_CONTAINER =
+      new MemMNodeContainer.EmptyContainer();
 
-  @SuppressWarnings("unchecked")
-  public static IMNodeContainer emptyMNodeContainer() {
+  public static IMNodeContainer<IMemMNode> emptyMNodeContainer() {
     return EMPTY_CONTAINER;
   }
 
-  public static boolean isEmptyContainer(IMNodeContainer container) {
-    return EMPTY_CONTAINER.equals(container);
-  }
-
-  public static IMNodeContainer getNewMNodeContainer() {
-    if (IoTDBDescriptor.getInstance()
-        .getConfig()
-        .getSchemaEngineMode()
-        .equals(SchemaEngineMode.Schema_File.toString())) {
-      return new CachedMNodeContainer();
-    } else {
-      return new MNodeContainerMapImpl();
-    }
-  }
-
-  private static class EmptyContainer extends AbstractMap<String, IMNode>
-      implements IMNodeContainer {
+  private static class EmptyContainer extends AbstractMap<String, IMemMNode>
+      implements IMNodeContainer<IMemMNode> {
 
     @Override
     public int size() {
@@ -81,7 +65,7 @@ public class MNodeContainers {
     }
 
     @Override
-    public IMNode get(Object key) {
+    public IMemMNode get(Object key) {
       return null;
     }
 
@@ -93,12 +77,12 @@ public class MNodeContainers {
 
     @Override
     @NotNull
-    public Collection<IMNode> values() {
+    public Collection<IMemMNode> values() {
       return emptySet();
     }
 
     @NotNull
-    public Set<Map.Entry<String, IMNode>> entrySet() {
+    public Set<Map.Entry<String, IMemMNode>> entrySet() {
       return emptySet();
     }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainerMapImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/estimator/MNodeSizeEstimator.java
similarity index 75%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainerMapImpl.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/estimator/MNodeSizeEstimator.java
index a7a043b6de..37f6201fb6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/container/MNodeContainerMapImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/estimator/MNodeSizeEstimator.java
@@ -16,11 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.container;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+package org.apache.iotdb.db.metadata.mnode.mem.estimator;
 
-import java.util.concurrent.ConcurrentHashMap;
+public class MNodeSizeEstimator {
+  private static int ALIAS_BASE_SIZE = 52;
 
-public class MNodeContainerMapImpl extends ConcurrentHashMap<String, IMNode>
-    implements IMNodeContainer {}
+  public static int getAliasBaseSize() {
+    return ALIAS_BASE_SIZE;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/factory/MemMNodeFactory.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/factory/MemMNodeFactory.java
new file mode 100644
index 0000000000..24215c1e15
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/factory/MemMNodeFactory.java
@@ -0,0 +1,84 @@
+/*
+ * 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.mem.factory;
+
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.impl.AboveDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.impl.BasicInternalMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.impl.DatabaseDeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.impl.DatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.impl.DeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.impl.MeasurementMNode;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+public class MemMNodeFactory implements IMNodeFactory<IMemMNode> {
+
+  private MemMNodeFactory() {}
+
+  private static class MemMNodeFactoryHolder {
+    private static final MemMNodeFactory INSTANCE = new MemMNodeFactory();
+
+    private MemMNodeFactoryHolder() {}
+  }
+
+  public static MemMNodeFactory getInstance() {
+    return MemMNodeFactory.MemMNodeFactoryHolder.INSTANCE;
+  }
+
+  @Override
+  public IMeasurementMNode<IMemMNode> createMeasurementMNode(
+      IDeviceMNode<IMemMNode> parent, String name, IMeasurementSchema schema, String alias) {
+    return new MeasurementMNode(parent, name, schema, alias);
+  }
+
+  @Override
+  public IDeviceMNode<IMemMNode> createDeviceMNode(IMemMNode parent, String name) {
+    return new DeviceMNode(parent, name);
+  }
+
+  @Override
+  public IDatabaseMNode<IMemMNode> createDatabaseMNode(IMemMNode parent, String name) {
+    return new DatabaseMNode(parent, name);
+  }
+
+  @Override
+  public IDatabaseMNode<IMemMNode> createDatabaseMNode(
+      IMemMNode parent, String name, long dataTTL) {
+    return new DatabaseMNode(parent, name, dataTTL);
+  }
+
+  @Override
+  public IMemMNode createDatabaseDeviceMNode(IMemMNode parent, String name, long dataTTL) {
+    return new DatabaseDeviceMNode(parent, name, dataTTL);
+  }
+
+  @Override
+  public IMemMNode createAboveDatabaseMNode(IMemMNode parent, String name) {
+    return new AboveDatabaseMNode(parent, name);
+  }
+
+  @Override
+  public IMemMNode createInternalMNode(IMemMNode parent, String name) {
+    return new BasicInternalMNode(parent, name);
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeMemoryModeTest.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/AboveDatabaseMNode.java
similarity index 60%
rename from server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeMemoryModeTest.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/AboveDatabaseMNode.java
index 0156e6c6f0..46c47cd530 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeMemoryModeTest.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/AboveDatabaseMNode.java
@@ -16,20 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mtree.multimode;
+package org.apache.iotdb.db.metadata.mnode.mem.impl;
 
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.metadata.mtree.MTreeBelowSGTest;
-import org.apache.iotdb.db.metadata.schemaregion.SchemaEngineMode;
+import org.apache.iotdb.commons.schema.node.common.AbstractAboveDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.basic.BasicMNode;
 
-public class MTreeMemoryModeTest extends MTreeBelowSGTest {
-  @Override
-  protected void setConfig() {
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setSchemaEngineMode(SchemaEngineMode.Memory.toString());
+public class AboveDatabaseMNode extends AbstractAboveDatabaseMNode<IMemMNode, BasicMNode>
+    implements IMemMNode {
+  public AboveDatabaseMNode(IMemMNode parent, String name) {
+    super(new BasicInternalMNode(parent, name));
   }
 
   @Override
-  protected void rollBackConfig() {}
+  public IMemMNode getAsMNode() {
+    return this;
+  }
 }
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/mem/impl/BasicInternalMNode.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/BasicInternalMNode.java
index 47e95f0b9c..b790fbf6c5 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/BasicInternalMNode.java
@@ -16,44 +16,29 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.db.metadata.mnode.mem.impl;
 
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.container.MNodeContainers;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-
-import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.basic.BasicMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.container.MemMNodeContainer;
 
 /**
  * This class is the implementation of Metadata Node. One MNode instance represents one node in the
  * Metadata Tree
  */
-public class InternalMNode extends MNode {
-
-  private static final long serialVersionUID = -770028375899514063L;
+public class BasicInternalMNode extends BasicMNode {
 
   /**
-   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
-   * synchronized check
+   * suppress warnings reason: volatile for double synchronized check
    *
    * <p>This will be a ConcurrentHashMap instance
    */
   @SuppressWarnings("squid:S3077")
-  protected transient volatile IMNodeContainer children = null;
-
-  /**
-   * This field is mainly used in cluster schema template features. In InternalMNode of ConfigMTree,
-   * this field represents the template set on this node. In EntityMNode of MTree in SchemaRegion,
-   * this field represents the template activated on this node. The normal usage value range is [0,
-   * Int.MaxValue], since this is implemented as auto inc id. The default value -1 means
-   * NON_TEMPLATE. This value will be set negative to implement some pre-delete features.
-   */
-  protected int schemaTemplateId = NON_TEMPLATE;
-
-  private volatile boolean useTemplate = false;
+  private transient volatile IMNodeContainer<IMemMNode> children = null;
 
   /** Constructor of MNode. */
-  public InternalMNode(IMNode parent, String name) {
+  public BasicInternalMNode(IMemMNode parent, String name) {
     super(parent, name);
   }
 
@@ -65,8 +50,8 @@ public class InternalMNode extends MNode {
 
   /** get the child with the name */
   @Override
-  public IMNode getChild(String name) {
-    IMNode child = null;
+  public IMemMNode getChild(String name) {
+    IMemMNode child = null;
     if (children != null) {
       child = children.get(name);
     }
@@ -81,7 +66,7 @@ public class InternalMNode extends MNode {
    * @return the child of this node after addChild
    */
   @Override
-  public IMNode addChild(String name, IMNode child) {
+  public IMemMNode addChild(String name, IMemMNode child) {
     /* use cpu time to exchange memory
      * measurementNode's children should be null to save memory
      * add child method will only be called when writing MTree, which is not a frequent operation
@@ -90,12 +75,12 @@ public class InternalMNode extends MNode {
       // double check, children is volatile
       synchronized (this) {
         if (children == null) {
-          children = MNodeContainers.getNewMNodeContainer();
+          children = new MemMNodeContainer();
         }
       }
     }
     child.setParent(this);
-    IMNode existingChild = children.putIfAbsent(name, child);
+    IMemMNode existingChild = children.putIfAbsent(name, child);
     return existingChild == null ? child : existingChild;
   }
 
@@ -111,7 +96,7 @@ public class InternalMNode extends MNode {
    * @return return the MNode already added
    */
   @Override
-  public IMNode addChild(IMNode child) {
+  public IMemMNode addChild(IMemMNode child) {
     /* use cpu time to exchange memory
      * measurementNode's children should be null to save memory
      * add child method will only be called when writing MTree, which is not a frequent operation
@@ -120,7 +105,7 @@ public class InternalMNode extends MNode {
       // double check, children is volatile
       synchronized (this) {
         if (children == null) {
-          children = MNodeContainers.getNewMNodeContainer();
+          children = new MemMNodeContainer();
         }
       }
     }
@@ -132,7 +117,7 @@ public class InternalMNode extends MNode {
 
   /** delete a child */
   @Override
-  public IMNode deleteChild(String name) {
+  public IMemMNode deleteChild(String name) {
     if (children != null) {
       return children.remove(name);
     }
@@ -146,11 +131,11 @@ public class InternalMNode extends MNode {
    * @param newChildNode new child node
    */
   @Override
-  public synchronized void replaceChild(String oldChildName, IMNode newChildNode) {
+  public synchronized void replaceChild(String oldChildName, IMemMNode newChildNode) {
     if (!oldChildName.equals(newChildNode.getName())) {
       throw new RuntimeException("New child's name must be the same as old child's name!");
     }
-    IMNode oldChildNode = this.getChild(oldChildName);
+    IMemMNode oldChildNode = this.getChild(oldChildName);
     if (oldChildNode == null) {
       return;
     }
@@ -161,12 +146,9 @@ public class InternalMNode extends MNode {
   }
 
   @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
+  public void moveDataToNewMNode(IMemMNode newMNode) {
     super.moveDataToNewMNode(newMNode);
 
-    newMNode.setUseTemplate(useTemplate);
-    newMNode.setSchemaTemplateId(schemaTemplateId);
-
     if (children != null) {
       newMNode.setChildren(children);
       children.forEach((childName, childNode) -> childNode.setParent(newMNode));
@@ -174,86 +156,26 @@ public class InternalMNode extends MNode {
   }
 
   @Override
-  public IMNodeContainer getChildren() {
+  public IMNodeContainer<IMemMNode> getChildren() {
     if (children == null) {
-      return MNodeContainers.emptyMNodeContainer();
+      return MemMNodeContainer.emptyMNodeContainer();
     }
     return children;
   }
 
   @Override
-  public void setChildren(IMNodeContainer children) {
+  public void setChildren(IMNodeContainer<IMemMNode> children) {
     this.children = children;
   }
 
+  /** MNodeContainer reference and basic occupation, 8 + 80B */
   @Override
-  public int getSchemaTemplateId() {
-    return schemaTemplateId >= -1 ? schemaTemplateId : -schemaTemplateId - 2;
-  }
-
-  @Override
-  public int getSchemaTemplateIdWithState() {
-    return schemaTemplateId;
-  }
-
-  @Override
-  public void setSchemaTemplateId(int schemaTemplateId) {
-    this.schemaTemplateId = schemaTemplateId;
-  }
-
-  /**
-   * In InternalMNode, schemaTemplateId represents the template set on this node. The pre unset
-   * mechanism is implemented by making this value negative. Since value 0 and -1 are all occupied,
-   * the available negative value range is [Int.MIN_VALUE, -2]. The value of a pre unset case equals
-   * the negative normal value minus 2. For example, if the id of set template is 0, then - 0 - 2 =
-   * -2 represents the pre unset operation of this template on this node.
-   */
-  @Override
-  public void preUnsetSchemaTemplate() {
-    if (this.schemaTemplateId > -1) {
-      this.schemaTemplateId = -schemaTemplateId - 2;
-    }
-  }
-
-  @Override
-  public void rollbackUnsetSchemaTemplate() {
-    if (schemaTemplateId < -1) {
-      schemaTemplateId = -schemaTemplateId - 2;
-    }
-  }
-
-  @Override
-  public boolean isSchemaTemplatePreUnset() {
-    return schemaTemplateId < -1;
-  }
-
-  @Override
-  public void unsetSchemaTemplate() {
-    this.schemaTemplateId = -1;
-  }
-
-  @Override
-  public boolean isAboveDatabase() {
-    return false;
-  }
-
-  @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return isConfig ? MNodeType.SG_INTERNAL : MNodeType.INTERNAL;
-  }
-
-  @Override
-  public boolean isUseTemplate() {
-    return useTemplate;
-  }
-
-  @Override
-  public void setUseTemplate(boolean useTemplate) {
-    this.useTemplate = useTemplate;
+  public int estimateSize() {
+    return 8 + 80 + super.estimateSize();
   }
 
   @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    return visitor.visitInternalMNode(this, context);
+  public IMemMNode getAsMNode() {
+    return this;
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeDiskModeTest.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DatabaseDeviceMNode.java
similarity index 55%
rename from server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeDiskModeTest.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DatabaseDeviceMNode.java
index 7629d8de12..c21ded3b82 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeDiskModeTest.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DatabaseDeviceMNode.java
@@ -16,27 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mtree.multimode;
+package org.apache.iotdb.db.metadata.mnode.mem.impl;
 
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.metadata.mtree.MTreeBelowSGTest;
-import org.apache.iotdb.db.metadata.schemaregion.SchemaEngineMode;
+import org.apache.iotdb.commons.schema.node.common.AbstractDatabaseDeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.basic.BasicMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.info.DatabaseDeviceInfo;
 
-public abstract class MTreeDiskModeTest extends MTreeBelowSGTest {
-  @Override
-  protected void setConfig() {
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setSchemaEngineMode(SchemaEngineMode.Schema_File.toString());
-    setMemSize();
+public class DatabaseDeviceMNode extends AbstractDatabaseDeviceMNode<IMemMNode, BasicMNode>
+    implements IMemMNode {
+  public DatabaseDeviceMNode(IMemMNode parent, String name, long dataTTL) {
+    super(new BasicInternalMNode(parent, name), new DatabaseDeviceInfo<>());
+    setDataTTL(dataTTL);
   }
 
   @Override
-  protected void rollBackConfig() {
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setSchemaEngineMode(SchemaEngineMode.Memory.toString());
+  public IMemMNode getAsMNode() {
+    return this;
   }
-
-  protected abstract void setMemSize();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DatabaseMNode.java
similarity index 52%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DatabaseMNode.java
index 77b9d7e23f..6f43d665f4 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DatabaseMNode.java
@@ -16,33 +16,28 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.metadata.mnode.mem.impl;
 
-package org.apache.iotdb.db.metadata.mnode.iterator;
+import org.apache.iotdb.commons.schema.node.common.AbstractDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.basic.BasicMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.info.DatabaseInfo;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+public class DatabaseMNode extends AbstractDatabaseMNode<IMemMNode, BasicMNode>
+    implements IMemMNode {
 
-import java.util.Iterator;
-
-public class MNodeIterator implements IMNodeIterator {
-
-  private Iterator<IMNode> iterator;
-
-  public MNodeIterator(Iterator<IMNode> iterator) {
-    this.iterator = iterator;
+  public DatabaseMNode(IMemMNode parent, String name) {
+    super(new BasicInternalMNode(parent, name), new DatabaseInfo<>());
   }
 
-  @Override
-  public boolean hasNext() {
-    return iterator.hasNext();
-  }
-
-  @Override
-  public IMNode next() {
-    return iterator.next();
+  // TODO: @yukun, remove this constructor
+  public DatabaseMNode(IMemMNode parent, String name, long dataTTL) {
+    this(parent, name);
+    setDataTTL(dataTTL);
   }
 
   @Override
-  public void close() {
-    iterator = null;
+  public IMemMNode getAsMNode() {
+    return this;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DeviceMNode.java
similarity index 58%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DeviceMNode.java
index 77b9d7e23f..9915b8a499 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/DeviceMNode.java
@@ -16,33 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.metadata.mnode.mem.impl;
 
-package org.apache.iotdb.db.metadata.mnode.iterator;
+import org.apache.iotdb.commons.schema.node.common.AbstractDeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.basic.BasicMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.info.DeviceInfo;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+public class DeviceMNode extends AbstractDeviceMNode<IMemMNode, BasicMNode> implements IMemMNode {
 
-import java.util.Iterator;
-
-public class MNodeIterator implements IMNodeIterator {
-
-  private Iterator<IMNode> iterator;
-
-  public MNodeIterator(Iterator<IMNode> iterator) {
-    this.iterator = iterator;
-  }
-
-  @Override
-  public boolean hasNext() {
-    return iterator.hasNext();
-  }
-
-  @Override
-  public IMNode next() {
-    return iterator.next();
+  public DeviceMNode(IMemMNode parent, String name) {
+    super(new BasicInternalMNode(parent, name), new DeviceInfo<>());
   }
 
   @Override
-  public void close() {
-    iterator = null;
+  public IMemMNode getAsMNode() {
+    return this;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/MeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/MeasurementMNode.java
new file mode 100644
index 0000000000..79e66a366b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/impl/MeasurementMNode.java
@@ -0,0 +1,49 @@
+/*
+ * 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.mem.impl;
+
+import org.apache.iotdb.commons.schema.node.common.AbstractMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.IMemMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.basic.BasicMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.container.MemMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.info.MeasurementInfo;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+public class MeasurementMNode extends AbstractMeasurementMNode<IMemMNode, BasicMNode>
+    implements IMemMNode {
+
+  public MeasurementMNode(
+      IDeviceMNode<IMemMNode> parent, String name, IMeasurementSchema schema, String alias) {
+    super(
+        new BasicMNode(parent == null ? null : parent.getAsMNode(), name),
+        new MeasurementInfo(schema, alias));
+  }
+
+  @Override
+  public IMemMNode getAsMNode() {
+    return this;
+  }
+
+  @Override
+  public IMNodeContainer<IMemMNode> getChildren() {
+    return MemMNodeContainer.emptyMNodeContainer();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/AboveDatabaseMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/BasicMNodeInfo.java
similarity index 65%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/AboveDatabaseMNode.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/BasicMNodeInfo.java
index 9b2251d40f..49deac759a 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/AboveDatabaseMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/BasicMNodeInfo.java
@@ -16,22 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.db.metadata.mnode.mem.info;
 
-/** Used to fill the link list of MNode above database in IMTreeBelowSG */
-public class AboveDatabaseMNode extends InternalMNode {
-  /**
-   * Constructor of MNode.
-   *
-   * @param parent
-   * @param name
-   */
-  public AboveDatabaseMNode(IMNode parent, String name) {
-    super(parent, name);
+public class BasicMNodeInfo {
+  private String name;
+
+  public BasicMNodeInfo(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
   }
 
-  @Override
-  public boolean isAboveDatabase() {
-    return true;
+  public int estimateSize() {
+    // object header, 8B
+    // name reference, name length and name hash code, 8 + 4 + 4 = 16B
+    return 8 + 16;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DatabaseDeviceInfo.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DatabaseDeviceInfo.java
new file mode 100644
index 0000000000..587436dba7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DatabaseDeviceInfo.java
@@ -0,0 +1,59 @@
+/*
+ * 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.mem.info;
+
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.info.IDatabaseDeviceInfo;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+
+public class DatabaseDeviceInfo<N extends IMNode<N>> extends DeviceInfo<N>
+    implements IDatabaseDeviceInfo<N> {
+  /**
+   * when the data file in a database is older than dataTTL, it is considered invalid and will be
+   * eventually deleted.
+   */
+  private long dataTTL;
+
+  @Override
+  public void moveDataToNewMNode(IDatabaseMNode<N> newMNode) {
+    newMNode.setDataTTL(dataTTL);
+  }
+
+  @Override
+  public long getDataTTL() {
+    return dataTTL;
+  }
+
+  @Override
+  public void setDataTTL(long dataTTL) {
+    this.dataTTL = dataTTL;
+  }
+
+  /**
+   * The memory occupied by an DatabaseDeviceInfo based occupation
+   *
+   * <ol>
+   *   <li>long dataTTL, 8B
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return super.estimateSize() + 8;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DatabaseInfo.java
similarity index 52%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DatabaseInfo.java
index 77b9d7e23f..02ba001888 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DatabaseInfo.java
@@ -16,33 +16,43 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.metadata.mnode.mem.info;
 
-package org.apache.iotdb.db.metadata.mnode.iterator;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.info.IDatabaseInfo;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+public class DatabaseInfo<N extends IMNode<N>> implements IDatabaseInfo<N> {
 
-import java.util.Iterator;
+  private long dataTTL;
 
-public class MNodeIterator implements IMNodeIterator {
+  public DatabaseInfo() {}
 
-  private Iterator<IMNode> iterator;
-
-  public MNodeIterator(Iterator<IMNode> iterator) {
-    this.iterator = iterator;
+  @Override
+  public void moveDataToNewMNode(IDatabaseMNode<N> newMNode) {
+    newMNode.setDataTTL(dataTTL);
   }
 
   @Override
-  public boolean hasNext() {
-    return iterator.hasNext();
+  public long getDataTTL() {
+    return dataTTL;
   }
 
   @Override
-  public IMNode next() {
-    return iterator.next();
+  public void setDataTTL(long dataTTL) {
+    this.dataTTL = dataTTL;
   }
 
+  /**
+   * The memory occupied by an DatabaseDeviceInfo based occupation
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>long dataTTL, 8B
+   * </ol>
+   */
   @Override
-  public void close() {
-    iterator = null;
+  public int estimateSize() {
+    return 8 + 8;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/EntityMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DeviceInfo.java
similarity index 57%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/EntityMNode.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DeviceInfo.java
index 31d8112051..3418299646 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/EntityMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/DeviceInfo.java
@@ -16,67 +16,52 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.db.metadata.mnode.mem.info;
 
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.info.IDeviceInfo;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
 
 import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-public class EntityMNode extends InternalMNode implements IEntityMNode {
+import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
+
+public class DeviceInfo<N extends IMNode<N>> implements IDeviceInfo<N> {
 
   /**
-   * suppress warnings reason: volatile for double synchronized check
-   *
-   * <p>This will be a ConcurrentHashMap instance
+   * In EntityMNode of MTree in SchemaRegion, this field represents the template activated on this
+   * node. The normal usage value range is [0, Int.MaxValue], since this is implemented as auto inc
+   * id. The default value -1 means NON_TEMPLATE. This value will be set negative to implement some
+   * pre-delete features.
    */
-  @SuppressWarnings("squid:S3077")
-  private transient volatile Map<String, IMeasurementMNode> aliasChildren = null;
+  protected int schemaTemplateId = NON_TEMPLATE;
 
-  private volatile boolean isAligned = false;
-
-  @Override
-  public String getFullPath() {
-    if (fullPath == null) {
-      fullPath = concatFullPath().intern();
-    }
-    return fullPath;
-  }
+  private volatile boolean useTemplate = false;
 
   /**
-   * Constructor of MNode.
+   * suppress warnings reason: volatile for double synchronized check
    *
-   * @param parent
-   * @param name
+   * <p>This will be a ConcurrentHashMap instance
    */
-  public EntityMNode(IMNode parent, String name) {
-    super(parent, name);
-  }
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMeasurementMNode<N>> aliasChildren = null;
 
-  /** check whether the MNode has a child with the name */
-  @Override
-  public boolean hasChild(String name) {
-    return (children != null && children.containsKey(name))
-        || (aliasChildren != null && aliasChildren.containsKey(name));
-  }
+  private volatile boolean isAligned = false;
 
-  /** get the child with the name */
   @Override
-  public IMNode getChild(String name) {
-    IMNode child = null;
-    if (children != null) {
-      child = children.get(name);
-    }
-    if (child != null) {
-      return child;
-    }
-    return aliasChildren == null ? null : aliasChildren.get(name);
+  public void moveDataToNewMNode(IDeviceMNode<N> newMNode) {
+    newMNode.setSchemaTemplateId(schemaTemplateId);
+    newMNode.setUseTemplate(useTemplate);
+    newMNode.setAliasChildren(aliasChildren);
+    newMNode.setAligned(isAligned);
   }
 
   /** add an alias */
   @Override
-  public boolean addAlias(String alias, IMeasurementMNode child) {
+  public boolean addAlias(String alias, IMeasurementMNode<N> child) {
     if (aliasChildren == null) {
       // double check, alias children volatile
       synchronized (this) {
@@ -98,7 +83,7 @@ public class EntityMNode extends InternalMNode implements IEntityMNode {
   }
 
   @Override
-  public Map<String, IMeasurementMNode> getAliasChildren() {
+  public Map<String, IMeasurementMNode<N>> getAliasChildren() {
     if (aliasChildren == null) {
       return Collections.emptyMap();
     }
@@ -106,10 +91,24 @@ public class EntityMNode extends InternalMNode implements IEntityMNode {
   }
 
   @Override
-  public void setAliasChildren(Map<String, IMeasurementMNode> aliasChildren) {
+  public void setAliasChildren(Map<String, IMeasurementMNode<N>> aliasChildren) {
     this.aliasChildren = aliasChildren;
   }
 
+  @Override
+  public boolean hasAliasChild(String name) {
+    return aliasChildren != null && aliasChildren.containsKey(name);
+  }
+
+  @Override
+  public N getAliasChild(String name) {
+    if (aliasChildren != null) {
+      IMeasurementMNode<N> child = aliasChildren.get(name);
+      return child == null ? null : child.getAsMNode();
+    }
+    return null;
+  }
+
   /**
    * In EntityMNode(device node), schemaTemplateId represents the template activated on this node.
    * The pre deactivation mechanism is implemented by making this value negative. Since value 0 and
@@ -123,6 +122,11 @@ public class EntityMNode extends InternalMNode implements IEntityMNode {
     return schemaTemplateId >= -1 ? schemaTemplateId : -schemaTemplateId - 2;
   }
 
+  @Override
+  public int getSchemaTemplateIdWithState() {
+    return schemaTemplateId;
+  }
+
   @Override
   public boolean isPreDeactivateTemplate() {
     return schemaTemplateId < -1;
@@ -149,40 +153,43 @@ public class EntityMNode extends InternalMNode implements IEntityMNode {
   }
 
   @Override
-  public boolean isAligned() {
-    return isAligned;
+  public boolean isUseTemplate() {
+    return useTemplate;
   }
 
   @Override
-  public void setAligned(boolean isAligned) {
-    this.isAligned = isAligned;
+  public void setUseTemplate(boolean useTemplate) {
+    this.useTemplate = useTemplate;
   }
 
   @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
-    super.moveDataToNewMNode(newMNode);
-
-    if (newMNode.isEntity()) {
-      IEntityMNode newEntityMNode = newMNode.getAsEntityMNode();
-      newEntityMNode.setAligned(isAligned);
-      if (aliasChildren != null) {
-        newEntityMNode.setAliasChildren(aliasChildren);
-      }
-    }
+  public void setSchemaTemplateId(int schemaTemplateId) {
+    this.schemaTemplateId = schemaTemplateId;
   }
 
   @Override
-  public boolean isEntity() {
-    return true;
+  public boolean isAligned() {
+    return isAligned;
   }
 
   @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return MNodeType.DEVICE;
+  public void setAligned(boolean isAligned) {
+    this.isAligned = isAligned;
   }
 
+  /**
+   * The memory occupied by an DeviceInfo based occupation
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>int schemaTemplateId, 4B
+   *   <li>boolean useTemplate, 1B
+   *   <li>boolean isAligned, 1B
+   *   <li>aliasChildren reference, 8B
+   * </ol>
+   */
   @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    return visitor.visitEntityMNode(this, context);
+  public int estimateSize() {
+    return 8 + 4 + 1 + 1 + 8;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/MeasurementInfo.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/MeasurementInfo.java
new file mode 100644
index 0000000000..9df799726d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/info/MeasurementInfo.java
@@ -0,0 +1,113 @@
+/*
+ * 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.mem.info;
+
+import org.apache.iotdb.commons.schema.node.info.IMeasurementInfo;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+public class MeasurementInfo implements IMeasurementInfo {
+
+  /** alias name of this measurement */
+  protected String alias;
+  /** tag/attribute's start offset in tag file */
+  private long offset = -1;
+  /** measurement's Schema for one timeseries represented by current leaf node */
+  private IMeasurementSchema schema;
+  /** whether this measurement is pre deleted and considered in black list */
+  private boolean preDeleted = false;
+
+  // alias length, hashCode and occupation in aliasMap, 4 + 4 + 44 = 52B
+  private static final int ALIAS_BASE_SIZE = 52;
+
+  public MeasurementInfo(IMeasurementSchema schema, String alias) {
+    this.schema = schema;
+    this.alias = alias;
+  }
+
+  @Override
+  public void moveDataToNewMNode(IMeasurementMNode<?> newMNode) {
+    newMNode.setSchema(schema);
+    newMNode.setAlias(alias);
+    newMNode.setOffset(offset);
+    newMNode.setPreDeleted(preDeleted);
+  }
+
+  @Override
+  public IMeasurementSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public void setSchema(IMeasurementSchema schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  public TSDataType getDataType() {
+    return schema.getType();
+  }
+
+  @Override
+  public String getAlias() {
+    return alias;
+  }
+
+  @Override
+  public void setAlias(String alias) {
+    this.alias = alias;
+  }
+
+  @Override
+  public long getOffset() {
+    return offset;
+  }
+
+  @Override
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  @Override
+  public boolean isPreDeleted() {
+    return preDeleted;
+  }
+
+  @Override
+  public void setPreDeleted(boolean preDeleted) {
+    this.preDeleted = preDeleted;
+  }
+
+  /**
+   * The memory occupied by an MeasurementInfo based occupation
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>alias reference, 8B
+   *   <li>long tagOffset, 8B
+   *   <li>boolean preDeleted, 1B
+   *   <li>estimated schema size, 32B
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + 8 + 1 + 32 + (alias == null ? 0 : ALIAS_BASE_SIZE + alias.length());
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/AbstractTraverserIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/AbstractTraverserIterator.java
similarity index 77%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/AbstractTraverserIterator.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/AbstractTraverserIterator.java
index 026fd03660..dddd3e3744 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/AbstractTraverserIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/AbstractTraverserIterator.java
@@ -16,14 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.iterator;
+package org.apache.iotdb.db.metadata.mnode.mem.iterator;
 
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeIterator;
+import org.apache.iotdb.db.metadata.mnode.utils.MNodeUtils;
 import org.apache.iotdb.db.metadata.mtree.store.IMTreeStore;
 import org.apache.iotdb.db.metadata.template.Template;
-import org.apache.iotdb.db.metadata.template.TemplateMNodeGenerator;
 
 import java.util.Iterator;
 import java.util.Map;
@@ -35,22 +37,25 @@ import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
  * TraverserIterator for traversing device node. The iterator returns the filtered child nodes in
  * the mtree and the child nodes in the template.
  */
-public abstract class AbstractTraverserIterator implements IMNodeIterator {
-  private final IMNodeIterator directChildrenIterator;
-  private Iterator<IMNode> templateChildrenIterator;
-  protected IMNode nextMatchedNode;
+public abstract class AbstractTraverserIterator<N extends IMNode<N>> implements IMNodeIterator<N> {
+  private final IMNodeIterator<N> directChildrenIterator;
+  private Iterator<N> templateChildrenIterator;
+  protected N nextMatchedNode;
   protected boolean usingDirectChildrenIterator = true;
   // if true, the pre deleted measurement or pre deactivated template won't be processed
   private boolean skipPreDeletedSchema = false;
 
   protected AbstractTraverserIterator(
-      IMTreeStore store, IEntityMNode parent, Map<Integer, Template> templateMap)
+      IMTreeStore<N> store,
+      IDeviceMNode<N> parent,
+      Map<Integer, Template> templateMap,
+      IMNodeFactory<N> nodeFactory)
       throws MetadataException {
-    this.directChildrenIterator = store.getChildrenIterator(parent);
+    this.directChildrenIterator = store.getChildrenIterator(parent.getAsMNode());
     if (templateMap != null && parent.isUseTemplate()) {
       Template template = getActivatedSchemaTemplate(parent, templateMap);
       if (template != null) {
-        templateChildrenIterator = TemplateMNodeGenerator.getChildren(template);
+        templateChildrenIterator = MNodeUtils.getChildren(template, nodeFactory);
       }
     }
   }
@@ -60,10 +65,10 @@ public abstract class AbstractTraverserIterator implements IMNodeIterator {
   }
 
   private Template getActivatedSchemaTemplate(
-      IEntityMNode node, Map<Integer, Template> templateMap) {
+      IDeviceMNode<N> node, Map<Integer, Template> templateMap) {
     // new cluster, the used template is directly recorded as template id in device mnode
     if (node.getSchemaTemplateId() != NON_TEMPLATE) {
-      if (skipPreDeletedSchema && node.getAsEntityMNode().isPreDeactivateTemplate()) {
+      if (skipPreDeletedSchema && node.getAsDeviceMNode().isPreDeactivateTemplate()) {
         // skip this pre deactivated template, the invoker will skip this
         return null;
       }
@@ -99,11 +104,11 @@ public abstract class AbstractTraverserIterator implements IMNodeIterator {
   }
 
   @Override
-  public IMNode next() {
+  public N next() {
     if (!hasNext()) {
       throw new NoSuchElementException();
     }
-    IMNode result = nextMatchedNode;
+    N result = nextMatchedNode;
     nextMatchedNode = null;
     return result;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/MNodeIterator.java
similarity index 75%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/MNodeIterator.java
index 77b9d7e23f..e5864e5b1c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/MNodeIterator.java
@@ -17,17 +17,18 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.metadata.mnode.iterator;
+package org.apache.iotdb.db.metadata.mnode.mem.iterator;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeIterator;
 
 import java.util.Iterator;
 
-public class MNodeIterator implements IMNodeIterator {
+public class MNodeIterator<N extends IMNode<?>> implements IMNodeIterator<N> {
 
-  private Iterator<IMNode> iterator;
+  private Iterator<N> iterator;
 
-  public MNodeIterator(Iterator<IMNode> iterator) {
+  public MNodeIterator(Iterator<N> iterator) {
     this.iterator = iterator;
   }
 
@@ -37,7 +38,7 @@ public class MNodeIterator implements IMNodeIterator {
   }
 
   @Override
-  public IMNode next() {
+  public N next() {
     return iterator.next();
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MemoryTraverserIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/MemoryTraverserIterator.java
similarity index 65%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MemoryTraverserIterator.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/MemoryTraverserIterator.java
index 9a8d9be7bc..503d0e4620 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MemoryTraverserIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/mem/iterator/MemoryTraverserIterator.java
@@ -16,19 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.iterator;
+package org.apache.iotdb.db.metadata.mnode.mem.iterator;
 
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
 import org.apache.iotdb.db.metadata.mtree.store.IMTreeStore;
 import org.apache.iotdb.db.metadata.template.Template;
 
 import java.util.Map;
 
-public class MemoryTraverserIterator extends AbstractTraverserIterator {
+// only use for IConfigMNode and IMemMNode
+public class MemoryTraverserIterator<N extends IMNode<N>> extends AbstractTraverserIterator<N> {
   public MemoryTraverserIterator(
-      IMTreeStore store, IEntityMNode parent, Map<Integer, Template> templateMap)
+      IMTreeStore<N> store,
+      IDeviceMNode<N> parent,
+      Map<Integer, Template> templateMap,
+      IMNodeFactory<N> nodeFactory)
       throws MetadataException {
-    super(store, parent, templateMap);
+    super(store, parent, templateMap, nodeFactory);
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeNonMemoryTest.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/ICachedMNode.java
similarity index 72%
rename from server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeNonMemoryTest.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/ICachedMNode.java
index 2de16968af..8c322603a4 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/multimode/MTreeNonMemoryTest.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/ICachedMNode.java
@@ -16,13 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mtree.multimode;
+package org.apache.iotdb.db.metadata.mnode.schemafile;
 
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
 
-public class MTreeNonMemoryTest extends MTreeDiskModeTest {
-  @Override
-  protected void setMemSize() {
-    IoTDBDescriptor.getInstance().getConfig().setCachedMNodeSizeInSchemaFileMode(0);
-  }
+public interface ICachedMNode extends IMNode<ICachedMNode> {
+  CacheEntry getCacheEntry();
+
+  void setCacheEntry(CacheEntry cacheEntry);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/basic/CachedBasicMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/basic/CachedBasicMNode.java
new file mode 100644
index 0000000000..e6f582e5f5
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/basic/CachedBasicMNode.java
@@ -0,0 +1,264 @@
+/*
+ * 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.schemafile.basic;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.MNodeType;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.commons.schema.node.visitor.MNodeVisitor;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.container.CachedMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.schemafile.info.CacheMNodeInfo;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class CachedBasicMNode implements ICachedMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  private ICachedMNode parent;
+  private final CacheMNodeInfo cacheMNodeInfo;
+
+  /** from root to this node, only be set when used once for InternalMNode */
+  private String fullPath;
+
+  /** Constructor of MNode. */
+  public CachedBasicMNode(ICachedMNode parent, String name) {
+    this.parent = parent;
+    this.cacheMNodeInfo = new CacheMNodeInfo(name);
+  }
+
+  @Override
+  public String getName() {
+    return cacheMNodeInfo.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    cacheMNodeInfo.setName(name);
+  }
+
+  @Override
+  public ICachedMNode getParent() {
+    return parent;
+  }
+
+  @Override
+  public void setParent(ICachedMNode parent) {
+    this.parent = parent;
+  }
+
+  @Override
+  public String getFullPath() {
+    if (fullPath == null) {
+      fullPath = concatFullPath();
+    }
+    return fullPath;
+  }
+
+  String concatFullPath() {
+    StringBuilder builder = new StringBuilder(getName());
+    ICachedMNode curr = this;
+    while (curr.getParent() != null) {
+      curr = curr.getParent();
+      builder.insert(0, IoTDBConstant.PATH_SEPARATOR).insert(0, curr.getName());
+    }
+    return builder.toString();
+  }
+
+  @Override
+  public void setFullPath(String fullPath) {
+    this.fullPath = fullPath;
+  }
+
+  @Override
+  public PartialPath getPartialPath() {
+    List<String> detachedPath = new ArrayList<>();
+    ICachedMNode temp = this;
+    detachedPath.add(temp.getName());
+    while (temp.getParent() != null) {
+      temp = temp.getParent();
+      detachedPath.add(0, temp.getName());
+    }
+    return new PartialPath(detachedPath.toArray(new String[0]));
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  /** get the child with the name */
+  @Override
+  public ICachedMNode getChild(String name) {
+    return null;
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   * @return the child of this node after addChild
+   */
+  @Override
+  public ICachedMNode addChild(String name, ICachedMNode child) {
+    return null;
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  @Override
+  public ICachedMNode addChild(ICachedMNode child) {
+    return null;
+  }
+
+  /** delete a child */
+  @Override
+  public ICachedMNode deleteChild(String name) {
+    return null;
+  }
+
+  /**
+   * Replace a child of this mnode. New child's name must be the same as old child's name.
+   *
+   * @param oldChildName measurement name
+   * @param newChildNode new child node
+   */
+  @Override
+  public synchronized void replaceChild(String oldChildName, ICachedMNode newChildNode) {}
+
+  @Override
+  public void moveDataToNewMNode(ICachedMNode newMNode) {
+    newMNode.setParent(parent);
+    newMNode.setCacheEntry(getCacheEntry());
+  }
+
+  @Override
+  public IMNodeContainer<ICachedMNode> getChildren() {
+    return CachedMNodeContainer.emptyMNodeContainer();
+  }
+
+  @Override
+  public void setChildren(IMNodeContainer<ICachedMNode> children) {}
+
+  @Override
+  public boolean isAboveDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDatabase() {
+    return false;
+  }
+
+  @Override
+  public boolean isDevice() {
+    return false;
+  }
+
+  @Override
+  public boolean isMeasurement() {
+    return false;
+  }
+
+  @Override
+  public MNodeType getMNodeType(Boolean isConfig) {
+    return isConfig ? MNodeType.SG_INTERNAL : MNodeType.INTERNAL;
+  }
+
+  @Override
+  public IDatabaseMNode<ICachedMNode> getAsDatabaseMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IDeviceMNode<ICachedMNode> getAsDeviceMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public IMeasurementMNode<ICachedMNode> getAsMeasurementMNode() {
+    throw new UnsupportedOperationException("Wrong MNode Type");
+  }
+
+  @Override
+  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
+    return visitor.visitBasicMNode(this, context);
+  }
+
+  @Override
+  public CacheEntry getCacheEntry() {
+    return cacheMNodeInfo.getCacheEntry();
+  }
+
+  @Override
+  public void setCacheEntry(CacheEntry cacheEntry) {
+    cacheMNodeInfo.setCacheEntry(cacheEntry);
+  }
+
+  /**
+   * The basic memory occupied by any CacheBasicMNode object
+   *
+   * <ol>
+   *   <li>object header, 8B
+   *   <li>node attributes
+   *       <ol>
+   *         <li>basicMNodeInfo reference, 8B
+   *         <li>parent reference, 8B
+   *         <li>fullPath reference, 8B
+   *       </ol>
+   *   <li>MapEntry in parent
+   *       <ol>
+   *         <li>key reference, 8B
+   *         <li>value reference, 8B
+   *         <li>entry size, see ConcurrentHashMap.Node, 28
+   *       </ol>
+   * </ol>
+   */
+  @Override
+  public int estimateSize() {
+    return 8 + 8 + 8 + 8 + 8 + 8 + 28 + cacheMNodeInfo.estimateSize();
+  }
+
+  @Override
+  public ICachedMNode getAsMNode() {
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/CachedMNodeContainer.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/container/CachedMNodeContainer.java
similarity index 67%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/CachedMNodeContainer.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/container/CachedMNodeContainer.java
index 47fecfe9da..aed2ae4f8d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/CachedMNodeContainer.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/container/CachedMNodeContainer.java
@@ -16,13 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mtree.store.disk;
+package org.apache.iotdb.db.metadata.mnode.schemafile.container;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+
+import org.jetbrains.annotations.NotNull;
 
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -33,20 +37,29 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 
+import static java.util.Collections.emptySet;
+
 public class CachedMNodeContainer implements ICachedMNodeContainer {
 
   private long segmentAddress = -1;
 
-  private Map<String, IMNode> childCache = null;
-  private Map<String, IMNode> newChildBuffer = null;
-  private Map<String, IMNode> updatedChildBuffer = null;
+  private Map<String, ICachedMNode> childCache = null;
+  private Map<String, ICachedMNode> newChildBuffer = null;
+  private Map<String, ICachedMNode> updatedChildBuffer = null;
+
+  private static final IMNodeContainer<ICachedMNode> EMPTY_CONTAINER =
+      new CachedMNodeContainer.EmptyContainer();
+
+  public static IMNodeContainer<ICachedMNode> emptyMNodeContainer() {
+    return EMPTY_CONTAINER;
+  }
 
   @Override
   public int size() {
     return getSize(childCache) + getSize(newChildBuffer) + getSize(updatedChildBuffer);
   }
 
-  private int getSize(Map<String, IMNode> map) {
+  private int getSize(Map<String, ICachedMNode> map) {
     return map == null ? 0 : map.size();
   }
 
@@ -55,7 +68,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     return isEmpty(childCache) && isEmpty(newChildBuffer) && isEmpty(updatedChildBuffer);
   }
 
-  private boolean isEmpty(Map<String, IMNode> map) {
+  private boolean isEmpty(Map<String, ICachedMNode> map) {
     return map == null || map.isEmpty();
   }
 
@@ -66,7 +79,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
         || containsKey(updatedChildBuffer, key);
   }
 
-  private boolean containsKey(Map<String, IMNode> map, Object key) {
+  private boolean containsKey(Map<String, ICachedMNode> map, Object key) {
     return map != null && map.containsKey(key);
   }
 
@@ -77,13 +90,13 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
         || containsValue(updatedChildBuffer, value);
   }
 
-  private boolean containsValue(Map<String, IMNode> map, Object value) {
+  private boolean containsValue(Map<String, ICachedMNode> map, Object value) {
     return map != null && map.containsValue(value);
   }
 
   @Override
-  public IMNode get(Object key) {
-    IMNode result = get(childCache, key);
+  public ICachedMNode get(Object key) {
+    ICachedMNode result = get(childCache, key);
     if (result != null) {
       return result;
     }
@@ -94,13 +107,13 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     return get(updatedChildBuffer, key);
   }
 
-  private IMNode get(Map<String, IMNode> map, Object key) {
+  private ICachedMNode get(Map<String, ICachedMNode> map, Object key) {
     return map == null ? null : map.get(key);
   }
 
   @Nullable
   @Override
-  public synchronized IMNode put(String key, IMNode value) {
+  public synchronized ICachedMNode put(String key, ICachedMNode value) {
     if (newChildBuffer == null) {
       newChildBuffer = new ConcurrentHashMap<>();
     }
@@ -109,9 +122,9 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
 
   @Nullable
   @Override
-  public synchronized IMNode putIfAbsent(String key, IMNode value) {
+  public synchronized ICachedMNode putIfAbsent(String key, ICachedMNode value) {
 
-    IMNode node = get(key);
+    ICachedMNode node = get(key);
     if (node == null) {
       if (newChildBuffer == null) {
         newChildBuffer = new ConcurrentHashMap<>();
@@ -123,8 +136,8 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
   }
 
   @Override
-  public synchronized IMNode remove(Object key) {
-    IMNode result = remove(childCache, key);
+  public synchronized ICachedMNode remove(Object key) {
+    ICachedMNode result = remove(childCache, key);
     if (result == null) {
       result = remove(newChildBuffer, key);
     }
@@ -134,12 +147,12 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     return result;
   }
 
-  private IMNode remove(Map<String, IMNode> map, Object key) {
+  private ICachedMNode remove(Map<String, ICachedMNode> map, Object key) {
     return map == null ? null : map.remove(key);
   }
 
   @Override
-  public synchronized void putAll(@Nonnull Map<? extends String, ? extends IMNode> m) {
+  public synchronized void putAll(@Nonnull Map<? extends String, ? extends ICachedMNode> m) {
     if (newChildBuffer == null) {
       newChildBuffer = new ConcurrentHashMap<>();
     }
@@ -163,42 +176,42 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     return result;
   }
 
-  private Set<String> keySet(Map<String, IMNode> map) {
+  private Set<String> keySet(Map<String, ICachedMNode> map) {
     return map == null ? Collections.emptySet() : map.keySet();
   }
 
   @Nonnull
   @Override
-  public Collection<IMNode> values() {
-    Collection<IMNode> result = new ArrayList<>();
+  public Collection<ICachedMNode> values() {
+    Collection<ICachedMNode> result = new ArrayList<>();
     result.addAll(getValues(childCache));
     result.addAll(getValues(newChildBuffer));
     result.addAll(getValues(updatedChildBuffer));
     return result;
   }
 
-  private Collection<IMNode> getValues(Map<String, IMNode> map) {
+  private Collection<ICachedMNode> getValues(Map<String, ICachedMNode> map) {
     return map == null ? Collections.emptyList() : map.values();
   }
 
   @Nonnull
   @Override
-  public Set<Entry<String, IMNode>> entrySet() {
-    Set<Entry<String, IMNode>> result = new HashSet<>();
+  public Set<Entry<String, ICachedMNode>> entrySet() {
+    Set<Entry<String, ICachedMNode>> result = new HashSet<>();
     result.addAll(entrySet(childCache));
     result.addAll(entrySet(newChildBuffer));
     result.addAll(entrySet(updatedChildBuffer));
     return result;
   }
 
-  private Set<Entry<String, IMNode>> entrySet(Map<String, IMNode> map) {
+  private Set<Entry<String, ICachedMNode>> entrySet(Map<String, ICachedMNode> map) {
     return map == null ? Collections.emptySet() : map.entrySet();
   }
 
   @Nullable
   @Override
-  public synchronized IMNode replace(String key, IMNode value) {
-    IMNode replacedOne = replace(childCache, key, value);
+  public synchronized ICachedMNode replace(String key, ICachedMNode value) {
+    ICachedMNode replacedOne = replace(childCache, key, value);
     if (replacedOne == null) {
       replacedOne = replace(newChildBuffer, key, value);
     }
@@ -208,7 +221,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     return replacedOne;
   }
 
-  private IMNode replace(Map<String, IMNode> map, String key, IMNode value) {
+  private ICachedMNode replace(Map<String, ICachedMNode> map, String key, ICachedMNode value) {
     return map == null ? null : map.replace(key, value);
   }
 
@@ -251,37 +264,37 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
   }
 
   @Override
-  public Iterator<IMNode> getChildrenIterator() {
+  public Iterator<ICachedMNode> getChildrenIterator() {
     return new CachedMNodeContainerIterator();
   }
 
   @Override
-  public Iterator<IMNode> getChildrenBufferIterator() {
+  public Iterator<ICachedMNode> getChildrenBufferIterator() {
     return new BufferIterator();
   }
 
   @Override
-  public Iterator<IMNode> getNewChildBufferIterator() {
+  public Iterator<ICachedMNode> getNewChildBufferIterator() {
     return getNewChildBuffer().values().iterator();
   }
 
   @Override
-  public Map<String, IMNode> getChildCache() {
+  public Map<String, ICachedMNode> getChildCache() {
     return childCache == null ? Collections.emptyMap() : childCache;
   }
 
   @Override
-  public Map<String, IMNode> getNewChildBuffer() {
+  public Map<String, ICachedMNode> getNewChildBuffer() {
     return newChildBuffer == null ? Collections.emptyMap() : newChildBuffer;
   }
 
   @Override
-  public Map<String, IMNode> getUpdatedChildBuffer() {
+  public Map<String, ICachedMNode> getUpdatedChildBuffer() {
     return updatedChildBuffer == null ? Collections.emptyMap() : updatedChildBuffer;
   }
 
   @Override
-  public synchronized void loadChildrenFromDisk(Map<String, IMNode> children) {
+  public synchronized void loadChildrenFromDisk(Map<String, ICachedMNode> children) {
     if (childCache == null) {
       childCache = new ConcurrentHashMap<>();
     }
@@ -289,7 +302,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
   }
 
   @Override
-  public synchronized void addChildToCache(IMNode node) {
+  public synchronized void addChildToCache(ICachedMNode node) {
     String name = node.getName();
     if (containsKey(name)) {
       return;
@@ -301,7 +314,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
   }
 
   @Override
-  public synchronized void appendMNode(IMNode node) {
+  public synchronized void appendMNode(ICachedMNode node) {
     if (newChildBuffer == null) {
       newChildBuffer = new ConcurrentHashMap<>();
     }
@@ -310,7 +323,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
 
   @Override
   public synchronized void updateMNode(String name) {
-    IMNode node = remove(childCache, name);
+    ICachedMNode node = remove(childCache, name);
     if (node != null) {
       if (updatedChildBuffer == null) {
         updatedChildBuffer = new ConcurrentHashMap<>();
@@ -321,7 +334,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
 
   @Override
   public synchronized void moveMNodeToCache(String name) {
-    IMNode node = remove(newChildBuffer, name);
+    ICachedMNode node = remove(newChildBuffer, name);
     if (node == null) {
       node = remove(updatedChildBuffer, name);
     }
@@ -340,17 +353,17 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     StringBuilder builder = new StringBuilder();
     builder.append("CachedMNodeContainer:{");
     builder.append("childCache:[");
-    for (IMNode node : getValues(childCache)) {
+    for (ICachedMNode node : getValues(childCache)) {
       builder.append(node.getName()).append(",");
     }
     builder.append("];");
     builder.append("newChildBuffer:[");
-    for (IMNode node : getValues(newChildBuffer)) {
+    for (ICachedMNode node : getValues(newChildBuffer)) {
       builder.append(node.getName()).append(",");
     }
     builder.append("];");
     builder.append("updateChildBuffer:[");
-    for (IMNode node : getValues(updatedChildBuffer)) {
+    for (ICachedMNode node : getValues(updatedChildBuffer)) {
       builder.append(node.getName()).append(",");
     }
     builder.append("];");
@@ -358,9 +371,9 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     return builder.toString();
   }
 
-  private class CachedMNodeContainerIterator implements Iterator<IMNode> {
+  private class CachedMNodeContainerIterator implements Iterator<ICachedMNode> {
 
-    Iterator<IMNode> iterator;
+    Iterator<ICachedMNode> iterator;
     byte status = 0;
 
     CachedMNodeContainerIterator() {
@@ -381,7 +394,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     }
 
     @Override
-    public IMNode next() {
+    public ICachedMNode next() {
       return iterator.next();
     }
 
@@ -401,10 +414,10 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     }
   }
 
-  private class BufferIterator implements Iterator<IMNode> {
-    Iterator<IMNode> iterator;
-    Iterator<IMNode> newBufferIterator;
-    Iterator<IMNode> updateBufferIterator;
+  private class BufferIterator implements Iterator<ICachedMNode> {
+    Iterator<ICachedMNode> iterator;
+    Iterator<ICachedMNode> newBufferIterator;
+    Iterator<ICachedMNode> updateBufferIterator;
     byte status = 0;
 
     BufferIterator() {
@@ -427,7 +440,7 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
     }
 
     @Override
-    public IMNode next() {
+    public ICachedMNode next() {
       return iterator.next();
     }
 
@@ -440,4 +453,55 @@ public class CachedMNodeContainer implements ICachedMNodeContainer {
       return false;
     }
   }
+
+  private static class EmptyContainer extends AbstractMap<String, ICachedMNode>
+      implements IMNodeContainer<ICachedMNode> {
+
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return true;
+    }
+
+    @Override
+    public boolean containsKey(Object key) {
+      return false;
+    }
+
+    @Override
+    public boolean containsValue(Object value) {
+      return false;
+    }
+
+    @Override
+    public ICachedMNode get(Object key) {
+      return null;
+    }
+
+    @Override
+    @NotNull
+    public Set<String> keySet() {
+      return emptySet();
+    }
+
+    @Override
+    @NotNull
+    public Collection<ICachedMNode> values() {
+      return emptySet();
+    }
+
+    @NotNull
+    public Set<Map.Entry<String, ICachedMNode>> entrySet() {
+      return emptySet();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return o == this;
+    }
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/ICachedMNodeContainer.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/container/ICachedMNodeContainer.java
similarity index 58%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/ICachedMNodeContainer.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/container/ICachedMNodeContainer.java
index 1579f98b79..22dcbb8b9e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/ICachedMNodeContainer.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/container/ICachedMNodeContainer.java
@@ -16,16 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mtree.store.disk;
+package org.apache.iotdb.db.metadata.mnode.schemafile.container;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.container.MNodeContainers;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
 
 import java.util.Iterator;
 import java.util.Map;
 
-public interface ICachedMNodeContainer extends IMNodeContainer {
+public interface ICachedMNodeContainer extends IMNodeContainer<ICachedMNode> {
 
   long getSegmentAddress();
 
@@ -41,23 +40,23 @@ public interface ICachedMNodeContainer extends IMNodeContainer {
 
   boolean hasChildInBuffer(String name);
 
-  Iterator<IMNode> getChildrenIterator();
+  Iterator<ICachedMNode> getChildrenIterator();
 
-  Iterator<IMNode> getChildrenBufferIterator();
+  Iterator<ICachedMNode> getChildrenBufferIterator();
 
-  Iterator<IMNode> getNewChildBufferIterator();
+  Iterator<ICachedMNode> getNewChildBufferIterator();
 
-  Map<String, IMNode> getChildCache();
+  Map<String, ICachedMNode> getChildCache();
 
-  Map<String, IMNode> getNewChildBuffer();
+  Map<String, ICachedMNode> getNewChildBuffer();
 
-  Map<String, IMNode> getUpdatedChildBuffer();
+  Map<String, ICachedMNode> getUpdatedChildBuffer();
 
-  void loadChildrenFromDisk(Map<String, IMNode> children);
+  void loadChildrenFromDisk(Map<String, ICachedMNode> children);
 
-  void addChildToCache(IMNode node);
+  void addChildToCache(ICachedMNode node);
 
-  void appendMNode(IMNode node);
+  void appendMNode(ICachedMNode node);
 
   void updateMNode(String name);
 
@@ -65,16 +64,16 @@ public interface ICachedMNodeContainer extends IMNodeContainer {
 
   void evictMNode(String name);
 
-  static ICachedMNodeContainer getCachedMNodeContainer(IMNode node) {
-    IMNodeContainer container = node.getChildren();
-    if (container.equals(MNodeContainers.emptyMNodeContainer())) {
+  static ICachedMNodeContainer getCachedMNodeContainer(ICachedMNode node) {
+    IMNodeContainer<ICachedMNode> container = node.getChildren();
+    if (container.equals(CachedMNodeContainer.emptyMNodeContainer())) {
       container = new CachedMNodeContainer();
       node.setChildren(container);
     }
     return (ICachedMNodeContainer) container;
   }
 
-  static ICachedMNodeContainer getBelongedContainer(IMNode node) {
+  static ICachedMNodeContainer getBelongedContainer(ICachedMNode node) {
     return (ICachedMNodeContainer) node.getParent().getChildren();
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/factory/CacheMNodeFactory.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/factory/CacheMNodeFactory.java
new file mode 100644
index 0000000000..5e79e03258
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/factory/CacheMNodeFactory.java
@@ -0,0 +1,84 @@
+/*
+ * 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.schemafile.factory;
+
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.impl.CachedAboveDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.impl.CachedBasicInternalMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.impl.CachedDatabaseDeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.impl.CachedDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.impl.CachedDeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.impl.CachedMeasurementMNode;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+public class CacheMNodeFactory implements IMNodeFactory<ICachedMNode> {
+
+  private CacheMNodeFactory() {}
+
+  private static class CacheMNodeFactoryHolder {
+    private static final CacheMNodeFactory INSTANCE = new CacheMNodeFactory();
+
+    private CacheMNodeFactoryHolder() {}
+  }
+
+  public static CacheMNodeFactory getInstance() {
+    return CacheMNodeFactory.CacheMNodeFactoryHolder.INSTANCE;
+  }
+
+  @Override
+  public IMeasurementMNode<ICachedMNode> createMeasurementMNode(
+      IDeviceMNode<ICachedMNode> parent, String name, IMeasurementSchema schema, String alias) {
+    return new CachedMeasurementMNode(parent, name, schema, alias);
+  }
+
+  @Override
+  public IDeviceMNode<ICachedMNode> createDeviceMNode(ICachedMNode parent, String name) {
+    return new CachedDeviceMNode(parent, name);
+  }
+
+  @Override
+  public IDatabaseMNode<ICachedMNode> createDatabaseMNode(ICachedMNode parent, String name) {
+    return new CachedDatabaseMNode(parent, name);
+  }
+
+  @Override
+  public IDatabaseMNode<ICachedMNode> createDatabaseMNode(
+      ICachedMNode parent, String name, long dataTTL) {
+    return new CachedDatabaseMNode(parent, name, dataTTL);
+  }
+
+  @Override
+  public ICachedMNode createDatabaseDeviceMNode(ICachedMNode parent, String name, long dataTTL) {
+    return new CachedDatabaseDeviceMNode(parent, name, dataTTL);
+  }
+
+  @Override
+  public ICachedMNode createAboveDatabaseMNode(ICachedMNode parent, String name) {
+    return new CachedAboveDatabaseMNode(parent, name);
+  }
+
+  @Override
+  public ICachedMNode createInternalMNode(ICachedMNode parent, String name) {
+    return new CachedBasicInternalMNode(parent, name);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedAboveDatabaseMNode.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
copy to server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedAboveDatabaseMNode.java
index 77b9d7e23f..f8ab57603c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedAboveDatabaseMNode.java
@@ -16,33 +16,32 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.metadata.mnode.schemafile.impl;
 
-package org.apache.iotdb.db.metadata.mnode.iterator;
+import org.apache.iotdb.commons.schema.node.common.AbstractAboveDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.basic.CachedBasicMNode;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+public class CachedAboveDatabaseMNode
+    extends AbstractAboveDatabaseMNode<ICachedMNode, CachedBasicMNode> implements ICachedMNode {
 
-import java.util.Iterator;
-
-public class MNodeIterator implements IMNodeIterator {
-
-  private Iterator<IMNode> iterator;
-
-  public MNodeIterator(Iterator<IMNode> iterator) {
-    this.iterator = iterator;
+  public CachedAboveDatabaseMNode(ICachedMNode parent, String name) {
+    super(new CachedBasicInternalMNode(parent, name));
   }
 
   @Override
-  public boolean hasNext() {
-    return iterator.hasNext();
+  public CacheEntry getCacheEntry() {
+    return basicMNode.getCacheEntry();
   }
 
   @Override
-  public IMNode next() {
-    return iterator.next();
+  public void setCacheEntry(CacheEntry cacheEntry) {
+    basicMNode.setCacheEntry(cacheEntry);
   }
 
   @Override
-  public void close() {
-    iterator = null;
+  public ICachedMNode getAsMNode() {
+    return this;
   }
 }
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/schemafile/impl/CachedBasicInternalMNode.java
similarity index 51%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedBasicInternalMNode.java
index 47e95f0b9c..b06a98180a 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedBasicInternalMNode.java
@@ -16,44 +16,29 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode;
+package org.apache.iotdb.db.metadata.mnode.schemafile.impl;
 
-import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer;
-import org.apache.iotdb.db.metadata.mnode.container.MNodeContainers;
-import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor;
-
-import static org.apache.iotdb.db.metadata.MetadataConstant.NON_TEMPLATE;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.basic.CachedBasicMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.container.CachedMNodeContainer;
 
 /**
  * This class is the implementation of Metadata Node. One MNode instance represents one node in the
  * Metadata Tree
  */
-public class InternalMNode extends MNode {
-
-  private static final long serialVersionUID = -770028375899514063L;
+public class CachedBasicInternalMNode extends CachedBasicMNode {
 
   /**
-   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
-   * synchronized check
+   * suppress warnings reason: volatile for double synchronized check
    *
    * <p>This will be a ConcurrentHashMap instance
    */
   @SuppressWarnings("squid:S3077")
-  protected transient volatile IMNodeContainer children = null;
-
-  /**
-   * This field is mainly used in cluster schema template features. In InternalMNode of ConfigMTree,
-   * this field represents the template set on this node. In EntityMNode of MTree in SchemaRegion,
-   * this field represents the template activated on this node. The normal usage value range is [0,
-   * Int.MaxValue], since this is implemented as auto inc id. The default value -1 means
-   * NON_TEMPLATE. This value will be set negative to implement some pre-delete features.
-   */
-  protected int schemaTemplateId = NON_TEMPLATE;
-
-  private volatile boolean useTemplate = false;
+  private transient volatile IMNodeContainer<ICachedMNode> children = null;
 
   /** Constructor of MNode. */
-  public InternalMNode(IMNode parent, String name) {
+  public CachedBasicInternalMNode(ICachedMNode parent, String name) {
     super(parent, name);
   }
 
@@ -65,8 +50,8 @@ public class InternalMNode extends MNode {
 
   /** get the child with the name */
   @Override
-  public IMNode getChild(String name) {
-    IMNode child = null;
+  public ICachedMNode getChild(String name) {
+    ICachedMNode child = null;
     if (children != null) {
       child = children.get(name);
     }
@@ -81,7 +66,7 @@ public class InternalMNode extends MNode {
    * @return the child of this node after addChild
    */
   @Override
-  public IMNode addChild(String name, IMNode child) {
+  public ICachedMNode addChild(String name, ICachedMNode child) {
     /* use cpu time to exchange memory
      * measurementNode's children should be null to save memory
      * add child method will only be called when writing MTree, which is not a frequent operation
@@ -90,12 +75,12 @@ public class InternalMNode extends MNode {
       // double check, children is volatile
       synchronized (this) {
         if (children == null) {
-          children = MNodeContainers.getNewMNodeContainer();
+          children = new CachedMNodeContainer();
         }
       }
     }
     child.setParent(this);
-    IMNode existingChild = children.putIfAbsent(name, child);
+    ICachedMNode existingChild = children.putIfAbsent(name, child);
     return existingChild == null ? child : existingChild;
   }
 
@@ -111,7 +96,7 @@ public class InternalMNode extends MNode {
    * @return return the MNode already added
    */
   @Override
-  public IMNode addChild(IMNode child) {
+  public ICachedMNode addChild(ICachedMNode child) {
     /* use cpu time to exchange memory
      * measurementNode's children should be null to save memory
      * add child method will only be called when writing MTree, which is not a frequent operation
@@ -120,7 +105,7 @@ public class InternalMNode extends MNode {
       // double check, children is volatile
       synchronized (this) {
         if (children == null) {
-          children = MNodeContainers.getNewMNodeContainer();
+          children = new CachedMNodeContainer();
         }
       }
     }
@@ -132,7 +117,7 @@ public class InternalMNode extends MNode {
 
   /** delete a child */
   @Override
-  public IMNode deleteChild(String name) {
+  public ICachedMNode deleteChild(String name) {
     if (children != null) {
       return children.remove(name);
     }
@@ -146,11 +131,11 @@ public class InternalMNode extends MNode {
    * @param newChildNode new child node
    */
   @Override
-  public synchronized void replaceChild(String oldChildName, IMNode newChildNode) {
+  public synchronized void replaceChild(String oldChildName, ICachedMNode newChildNode) {
     if (!oldChildName.equals(newChildNode.getName())) {
       throw new RuntimeException("New child's name must be the same as old child's name!");
     }
-    IMNode oldChildNode = this.getChild(oldChildName);
+    ICachedMNode oldChildNode = this.getChild(oldChildName);
     if (oldChildNode == null) {
       return;
     }
@@ -161,12 +146,9 @@ public class InternalMNode extends MNode {
   }
 
   @Override
-  public void moveDataToNewMNode(IMNode newMNode) {
+  public void moveDataToNewMNode(ICachedMNode newMNode) {
     super.moveDataToNewMNode(newMNode);
 
-    newMNode.setUseTemplate(useTemplate);
-    newMNode.setSchemaTemplateId(schemaTemplateId);
-
     if (children != null) {
       newMNode.setChildren(children);
       children.forEach((childName, childNode) -> childNode.setParent(newMNode));
@@ -174,86 +156,36 @@ public class InternalMNode extends MNode {
   }
 
   @Override
-  public IMNodeContainer getChildren() {
+  public IMNodeContainer<ICachedMNode> getChildren() {
     if (children == null) {
-      return MNodeContainers.emptyMNodeContainer();
+      return CachedMNodeContainer.emptyMNodeContainer();
     }
     return children;
   }
 
   @Override
-  public void setChildren(IMNodeContainer children) {
+  public void setChildren(IMNodeContainer<ICachedMNode> children) {
     this.children = children;
   }
 
-  @Override
-  public int getSchemaTemplateId() {
-    return schemaTemplateId >= -1 ? schemaTemplateId : -schemaTemplateId - 2;
-  }
-
-  @Override
-  public int getSchemaTemplateIdWithState() {
-    return schemaTemplateId;
-  }
-
-  @Override
-  public void setSchemaTemplateId(int schemaTemplateId) {
-    this.schemaTemplateId = schemaTemplateId;
-  }
-
   /**
-   * In InternalMNode, schemaTemplateId represents the template set on this node. The pre unset
-   * mechanism is implemented by making this value negative. Since value 0 and -1 are all occupied,
-   * the available negative value range is [Int.MIN_VALUE, -2]. The value of a pre unset case equals
-   * the negative normal value minus 2. For example, if the id of set template is 0, then - 0 - 2 =
-   * -2 represents the pre unset operation of this template on this node.
+   * MNodeContainer reference and basic occupation, 8 + 80B.
+   *
+   * <ol>
+   *   Estimated delta size of CachedMNodeContainer compared with other MNodeContainer.
+   *   <li>address, 8B
+   *   <li>three map reference (1 cache and 2 buffer), 8 * 3 = 24B
+   *   <li>estimate occupation of map implementation, minus the basic container occupation, 80 * 3 -
+   *       80 = 160B
+   * </ol>
    */
   @Override
-  public void preUnsetSchemaTemplate() {
-    if (this.schemaTemplateId > -1) {
-      this.schemaTemplateId = -schemaTemplateId - 2;
-    }
-  }
-
-  @Override
-  public void rollbackUnsetSchemaTemplate() {
-    if (schemaTemplateId < -1) {
-      schemaTemplateId = -schemaTemplateId - 2;
-    }
-  }
-
-  @Override
-  public boolean isSchemaTemplatePreUnset() {
-    return schemaTemplateId < -1;
-  }
-
-  @Override
-  public void unsetSchemaTemplate() {
-    this.schemaTemplateId = -1;
-  }
-
-  @Override
-  public boolean isAboveDatabase() {
-    return false;
-  }
-
-  @Override
-  public MNodeType getMNodeType(Boolean isConfig) {
-    return isConfig ? MNodeType.SG_INTERNAL : MNodeType.INTERNAL;
-  }
-
-  @Override
-  public boolean isUseTemplate() {
-    return useTemplate;
-  }
-
-  @Override
-  public void setUseTemplate(boolean useTemplate) {
-    this.useTemplate = useTemplate;
+  public int estimateSize() {
+    return 8 + 80 + 192 + super.estimateSize();
   }
 
   @Override
-  public <R, C> R accept(MNodeVisitor<R, C> visitor, C context) {
-    return visitor.visitInternalMNode(this, context);
+  public ICachedMNode getAsMNode() {
+    return this;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDatabaseDeviceMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDatabaseDeviceMNode.java
new file mode 100644
index 0000000000..dc4ba9283f
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDatabaseDeviceMNode.java
@@ -0,0 +1,49 @@
+/*
+ * 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.schemafile.impl;
+
+import org.apache.iotdb.commons.schema.node.common.AbstractDatabaseDeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.info.DatabaseDeviceInfo;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.basic.CachedBasicMNode;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
+
+public class CachedDatabaseDeviceMNode
+    extends AbstractDatabaseDeviceMNode<ICachedMNode, CachedBasicMNode> implements ICachedMNode {
+
+  public CachedDatabaseDeviceMNode(ICachedMNode parent, String name, long dataTTL) {
+    super(new CachedBasicInternalMNode(parent, name), new DatabaseDeviceInfo<>());
+    setDataTTL(dataTTL);
+  }
+
+  @Override
+  public CacheEntry getCacheEntry() {
+    return basicMNode.getCacheEntry();
+  }
+
+  @Override
+  public void setCacheEntry(CacheEntry cacheEntry) {
+    basicMNode.setCacheEntry(cacheEntry);
+  }
+
+  @Override
+  public ICachedMNode getAsMNode() {
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDatabaseMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDatabaseMNode.java
new file mode 100644
index 0000000000..58952882fd
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDatabaseMNode.java
@@ -0,0 +1,54 @@
+/*
+ * 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.schemafile.impl;
+
+import org.apache.iotdb.commons.schema.node.common.AbstractDatabaseMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.info.DatabaseInfo;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.basic.CachedBasicMNode;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
+
+public class CachedDatabaseMNode extends AbstractDatabaseMNode<ICachedMNode, CachedBasicMNode>
+    implements ICachedMNode {
+
+  public CachedDatabaseMNode(ICachedMNode parent, String name) {
+    super(new CachedBasicInternalMNode(parent, name), new DatabaseInfo<>());
+  }
+
+  // TODO: @yukun, remove this constructor
+  public CachedDatabaseMNode(ICachedMNode parent, String name, long dataTTL) {
+    this(parent, name);
+    setDataTTL(dataTTL);
+  }
+
+  @Override
+  public CacheEntry getCacheEntry() {
+    return basicMNode.getCacheEntry();
+  }
+
+  @Override
+  public void setCacheEntry(CacheEntry cacheEntry) {
+    basicMNode.setCacheEntry(cacheEntry);
+  }
+
+  @Override
+  public ICachedMNode getAsMNode() {
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDeviceMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDeviceMNode.java
new file mode 100644
index 0000000000..a9a331c476
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedDeviceMNode.java
@@ -0,0 +1,48 @@
+/*
+ * 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.schemafile.impl;
+
+import org.apache.iotdb.commons.schema.node.common.AbstractDeviceMNode;
+import org.apache.iotdb.db.metadata.mnode.mem.info.DeviceInfo;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.basic.CachedBasicMNode;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
+
+public class CachedDeviceMNode extends AbstractDeviceMNode<ICachedMNode, CachedBasicMNode>
+    implements ICachedMNode {
+
+  public CachedDeviceMNode(ICachedMNode parent, String name) {
+    super(new CachedBasicInternalMNode(parent, name), new DeviceInfo<>());
+  }
+
+  @Override
+  public CacheEntry getCacheEntry() {
+    return basicMNode.getCacheEntry();
+  }
+
+  @Override
+  public void setCacheEntry(CacheEntry cacheEntry) {
+    basicMNode.setCacheEntry(cacheEntry);
+  }
+
+  @Override
+  public ICachedMNode getAsMNode() {
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedMeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedMeasurementMNode.java
new file mode 100644
index 0000000000..df10e21f74
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/impl/CachedMeasurementMNode.java
@@ -0,0 +1,60 @@
+/*
+ * 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.schemafile.impl;
+
+import org.apache.iotdb.commons.schema.node.common.AbstractMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeContainer;
+import org.apache.iotdb.db.metadata.mnode.mem.info.MeasurementInfo;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.basic.CachedBasicMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.container.CachedMNodeContainer;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+public class CachedMeasurementMNode extends AbstractMeasurementMNode<ICachedMNode, CachedBasicMNode>
+    implements ICachedMNode {
+
+  public CachedMeasurementMNode(
+      IDeviceMNode<ICachedMNode> parent, String name, IMeasurementSchema schema, String alias) {
+    super(
+        new CachedBasicMNode(parent == null ? null : parent.getAsMNode(), name),
+        new MeasurementInfo(schema, alias));
+  }
+
+  @Override
+  public CacheEntry getCacheEntry() {
+    return basicMNode.getCacheEntry();
+  }
+
+  @Override
+  public void setCacheEntry(CacheEntry cacheEntry) {
+    basicMNode.setCacheEntry(cacheEntry);
+  }
+
+  @Override
+  public ICachedMNode getAsMNode() {
+    return this;
+  }
+
+  @Override
+  public IMNodeContainer<ICachedMNode> getChildren() {
+    return CachedMNodeContainer.emptyMNodeContainer();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/info/CacheMNodeInfo.java
similarity index 58%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/info/CacheMNodeInfo.java
index 77b9d7e23f..d89dba9bca 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/MNodeIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/info/CacheMNodeInfo.java
@@ -16,33 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.metadata.mnode.schemafile.info;
 
-package org.apache.iotdb.db.metadata.mnode.iterator;
+import org.apache.iotdb.db.metadata.mnode.mem.info.BasicMNodeInfo;
+import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry;
 
-import org.apache.iotdb.db.metadata.mnode.IMNode;
+public class CacheMNodeInfo extends BasicMNodeInfo {
 
-import java.util.Iterator;
+  private CacheEntry cacheEntry;
 
-public class MNodeIterator implements IMNodeIterator {
-
-  private Iterator<IMNode> iterator;
-
-  public MNodeIterator(Iterator<IMNode> iterator) {
-    this.iterator = iterator;
+  public CacheMNodeInfo(String name) {
+    super(name);
   }
 
-  @Override
-  public boolean hasNext() {
-    return iterator.hasNext();
+  public CacheEntry getCacheEntry() {
+    return cacheEntry;
   }
 
-  @Override
-  public IMNode next() {
-    return iterator.next();
+  public void setCacheEntry(CacheEntry cacheEntry) {
+    this.cacheEntry = cacheEntry;
   }
 
   @Override
-  public void close() {
-    iterator = null;
+  public int estimateSize() {
+    // Estimated size of CacheEntry = 40
+    return super.estimateSize() + 40;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/CachedTraverserIterator.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/iterator/CachedTraverserIterator.java
similarity index 67%
rename from server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/CachedTraverserIterator.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/iterator/CachedTraverserIterator.java
index 501f748315..32ff2d28d3 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/iterator/CachedTraverserIterator.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/schemafile/iterator/CachedTraverserIterator.java
@@ -16,22 +16,28 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata.mnode.iterator;
+package org.apache.iotdb.db.metadata.mnode.schemafile.iterator;
 
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.db.metadata.mnode.mem.iterator.AbstractTraverserIterator;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
 import org.apache.iotdb.db.metadata.mtree.store.IMTreeStore;
 import org.apache.iotdb.db.metadata.template.Template;
 
 import java.util.Map;
 
-public class CachedTraverserIterator extends AbstractTraverserIterator {
-  private final IMTreeStore store;
+public class CachedTraverserIterator extends AbstractTraverserIterator<ICachedMNode> {
+  private final IMTreeStore<ICachedMNode> store;
 
   public CachedTraverserIterator(
-      IMTreeStore store, IEntityMNode parent, Map<Integer, Template> templateMap)
+      IMTreeStore<ICachedMNode> store,
+      IDeviceMNode<ICachedMNode> parent,
+      Map<Integer, Template> templateMap,
+      IMNodeFactory<ICachedMNode> nodeFactory)
       throws MetadataException {
-    super(store, parent, templateMap);
+    super(store, parent, templateMap, nodeFactory);
     this.store = store;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/utils/MNodeUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/utils/MNodeUtils.java
new file mode 100644
index 0000000000..f64f270cce
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/utils/MNodeUtils.java
@@ -0,0 +1,123 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.schema.node.IMNode;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.Iterator;
+
+public class MNodeUtils {
+
+  /**
+   * When a measurement, represented by template or MeasurementMNode, is going to be added to a
+   * node, the node should be set to entity. Before invoking this method, lock the related MTree
+   * part first.
+   *
+   * @param node node to be transformed
+   * @return generated entityMNode
+   */
+  public static <N extends IMNode<N>> IDeviceMNode<N> setToEntity(
+      IMNode<N> node, IMNodeFactory<N> nodeFactory) {
+    IDeviceMNode<N> entityMNode;
+    if (node.isDevice()) {
+      entityMNode = node.getAsDeviceMNode();
+    } else {
+      if (node.isDatabase()) {
+        entityMNode =
+            nodeFactory
+                .createDatabaseDeviceMNode(
+                    node.getParent(), node.getName(), node.getAsDatabaseMNode().getDataTTL())
+                .getAsDeviceMNode();
+        node.moveDataToNewMNode(entityMNode.getAsMNode());
+      } else {
+        // basic node
+        entityMNode = nodeFactory.createDeviceMNode(node.getParent(), node.getName());
+        if (node.getParent() != null) {
+          node.getParent().replaceChild(node.getName(), entityMNode.getAsMNode());
+        } else {
+          node.moveDataToNewMNode(entityMNode.getAsMNode());
+        }
+      }
+    }
+    return entityMNode;
+  }
+
+  /**
+   * When there's no measurement, represented by template or MeasurementMNode, is under this
+   * entityMNode, it should not act as entity anymore. Before invoking this method, lock related
+   * MTree structure first.
+   *
+   * @param entityMNode node to be transformed
+   * @return generated NoEntity node
+   */
+  public static <N extends IMNode<N>> N setToInternal(
+      IDeviceMNode<N> entityMNode, IMNodeFactory<N> nodeFactor) {
+    N node;
+    N parent = entityMNode.getParent();
+    if (entityMNode.isDatabase()) {
+      IDatabaseMNode<N> databaseMNode =
+          nodeFactor.createDatabaseMNode(parent, entityMNode.getName());
+      databaseMNode.setDataTTL(entityMNode.getAsDatabaseMNode().getDataTTL());
+      node = databaseMNode.getAsMNode();
+    } else {
+      node = nodeFactor.createInternalMNode(parent, entityMNode.getName());
+    }
+
+    if (parent != null) {
+      parent.replaceChild(entityMNode.getName(), node);
+    }
+    return node;
+  }
+
+  public static <N extends IMNode<N>> N getChild(
+      Template template, String name, IMNodeFactory<N> nodeFactor) {
+    IMeasurementSchema schema = template.getSchema(name);
+    return schema == null
+        ? null
+        : nodeFactor
+            .createMeasurementMNode(null, name, template.getSchema(name), null)
+            .getAsMNode();
+  }
+
+  public static <N extends IMNode<N>> Iterator<N> getChildren(
+      Template template, IMNodeFactory<N> nodeFactor) {
+    return new Iterator<N>() {
+      private final Iterator<IMeasurementSchema> schemas =
+          template.getSchemaMap().values().iterator();
+
+      @Override
+      public boolean hasNext() {
+        return schemas.hasNext();
+      }
+
+      @Override
+      public N next() {
+        IMeasurementSchema schema = schemas.next();
+        return nodeFactor
+            .createMeasurementMNode(null, schema.getMeasurementId(), schema, null)
+            .getAsMNode();
+      }
+    };
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/visitor/MNodeVisitor.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/visitor/MNodeVisitor.java
deleted file mode 100644
index 53ccc1f059..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/visitor/MNodeVisitor.java
+++ /dev/null
@@ -1,39 +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.visitor;
-
-import org.apache.iotdb.db.metadata.mnode.EntityMNode;
-import org.apache.iotdb.db.metadata.mnode.InternalMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.StorageGroupEntityMNode;
-import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
-
-public abstract class MNodeVisitor<R, C> {
-
-  public abstract R visitInternalMNode(InternalMNode node, C context);
-
-  public abstract R visitStorageGroupMNode(StorageGroupMNode node, C context);
-
-  public abstract R visitStorageGroupEntityMNode(StorageGroupEntityMNode node, C context);
-
-  public abstract R visitEntityMNode(EntityMNode node, C context);
-
-  public abstract R visitMeasurementMNode(MeasurementMNode node, C context);
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java
index e427680ade..a20b453307 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/ConfigMTree.java
@@ -24,16 +24,16 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeIterator;
 import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
 import org.apache.iotdb.db.exception.metadata.DatabaseAlreadySetException;
 import org.apache.iotdb.db.exception.metadata.DatabaseNotSetException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.InternalMNode;
-import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.iterator.IMNodeIterator;
-import org.apache.iotdb.db.metadata.mtree.store.MemMTreeStore;
+import org.apache.iotdb.db.metadata.mnode.config.IConfigMNode;
+import org.apache.iotdb.db.metadata.mnode.config.factory.ConfigMNodeFactory;
+import org.apache.iotdb.db.metadata.mtree.store.ConfigMTreeStore;
 import org.apache.iotdb.db.metadata.mtree.traverser.collector.DatabaseCollector;
 import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveDBCollector;
 import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeCollector;
@@ -74,12 +74,14 @@ public class ConfigMTree {
 
   private final Logger logger = LoggerFactory.getLogger(ConfigMTree.class);
 
-  private IMNode root;
+  private IConfigMNode root;
   // this store is only used for traverser invoking
-  private MemMTreeStore store;
+  private final ConfigMTreeStore store;
+
+  private final IMNodeFactory<IConfigMNode> nodeFactory = ConfigMNodeFactory.getInstance();
 
   public ConfigMTree() throws MetadataException {
-    store = new MemMTreeStore(new PartialPath(PATH_ROOT), false);
+    store = new ConfigMTreeStore(nodeFactory);
     root = store.getRoot();
   }
 
@@ -103,39 +105,39 @@ public class ConfigMTree {
     if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
       throw new IllegalPathException(path.getFullPath());
     }
-    IMNode cur = root;
+    IConfigMNode cur = root;
     int i = 1;
     // e.g., path = root.a.b.sg, create internal nodes for a, b
     while (i < nodeNames.length - 1) {
-      IMNode temp = cur.getChild(nodeNames[i]);
+      IConfigMNode temp = store.getChild(cur, nodeNames[i]);
       if (temp == null) {
-        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
-      } else if (temp.isStorageGroup()) {
+        store.addChild(cur, nodeNames[i], nodeFactory.createInternalMNode(cur, nodeNames[i]));
+      } else if (temp.isDatabase()) {
         // before create database, check whether the database already exists
         throw new DatabaseAlreadySetException(temp.getFullPath());
       }
-      cur = cur.getChild(nodeNames[i]);
+      cur = store.getChild(cur, nodeNames[i]);
       i++;
     }
 
     // synchronize check and add, we need addChild operation be atomic.
     // only write operations on mtree will be synchronized
     synchronized (this) {
-      if (cur.hasChild(nodeNames[i])) {
+      if (store.hasChild(cur, nodeNames[i])) {
         // node b has child sg
-        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+        if (store.getChild(cur, nodeNames[i]).isDatabase()) {
           throw new DatabaseAlreadySetException(path.getFullPath());
         } else {
           throw new DatabaseAlreadySetException(path.getFullPath(), true);
         }
       } else {
-        IStorageGroupMNode storageGroupMNode =
-            new StorageGroupMNode(
-                cur, nodeNames[i], CommonDescriptor.getInstance().getConfig().getDefaultTTLInMs());
+        IDatabaseMNode<IConfigMNode> databaseMNode =
+            nodeFactory.createDatabaseMNode(cur, nodeNames[i]);
+        databaseMNode.setDataTTL(CommonDescriptor.getInstance().getConfig().getDefaultTTLInMs());
 
-        IMNode result = cur.addChild(nodeNames[i], storageGroupMNode);
+        IConfigMNode result = store.addChild(cur, nodeNames[i], databaseMNode.getAsMNode());
 
-        if (result != storageGroupMNode) {
+        if (result != databaseMNode) {
           throw new DatabaseAlreadySetException(path.getFullPath(), true);
         }
       }
@@ -144,11 +146,11 @@ public class ConfigMTree {
 
   /** Delete a database */
   public void deleteDatabase(PartialPath path) throws MetadataException {
-    IStorageGroupMNode storageGroupMNode = getDatabaseNodeByDatabasePath(path);
-    IMNode cur = storageGroupMNode.getParent();
+    IDatabaseMNode<IConfigMNode> databaseMNode = getDatabaseNodeByDatabasePath(path);
+    IConfigMNode cur = databaseMNode.getParent();
     // Suppose current system has root.a.b.sg1, root.a.sg2, and delete root.a.b.sg1
     // delete the database node sg1
-    cur.deleteChild(storageGroupMNode.getName());
+    store.deleteChild(cur, databaseMNode.getName());
 
     // delete node a while retain root.a.sg2
     while (cur.getParent() != null && cur.getChildren().size() == 0) {
@@ -189,10 +191,12 @@ public class ConfigMTree {
       PartialPath pathPattern, boolean isPrefixMatch, boolean collectInternal)
       throws MetadataException {
     List<PartialPath> result = new LinkedList<>();
-    try (DatabaseCollector<List<PartialPath>> collector =
-        new DatabaseCollector<List<PartialPath>>(root, pathPattern, store, isPrefixMatch) {
+    try (DatabaseCollector<?, ?> collector =
+        new DatabaseCollector<List<PartialPath>, IConfigMNode>(
+            root, pathPattern, store, isPrefixMatch) {
+
           @Override
-          protected void collectDatabase(IStorageGroupMNode node) {
+          protected void collectDatabase(IDatabaseMNode<IConfigMNode> node) {
             result.add(node.getPartialPath());
           }
         }) {
@@ -209,11 +213,11 @@ public class ConfigMTree {
    */
   public List<PartialPath> getAllDatabasePaths() {
     List<PartialPath> res = new ArrayList<>();
-    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    Deque<IConfigMNode> nodeStack = new ArrayDeque<>();
     nodeStack.add(root);
     while (!nodeStack.isEmpty()) {
-      IMNode current = nodeStack.pop();
-      if (current.isStorageGroup()) {
+      IConfigMNode current = nodeStack.pop();
+      if (current.isDatabase()) {
         res.add(current.getPartialPath());
       } else {
         nodeStack.addAll(current.getChildren().values());
@@ -231,7 +235,8 @@ public class ConfigMTree {
    */
   public int getDatabaseNum(PartialPath pathPattern, boolean isPrefixMatch)
       throws MetadataException {
-    try (DatabaseCounter counter = new DatabaseCounter(root, pathPattern, store, isPrefixMatch)) {
+    try (DatabaseCounter<IConfigMNode> counter =
+        new DatabaseCounter<>(root, pathPattern, store, isPrefixMatch)) {
       return (int) counter.count();
     }
   }
@@ -239,31 +244,31 @@ public class ConfigMTree {
   /**
    * E.g., root.sg is database given [root, sg], if the give path is not a database, throw exception
    */
-  public IStorageGroupMNode getDatabaseNodeByDatabasePath(PartialPath storageGroupPath)
+  public IDatabaseMNode<IConfigMNode> getDatabaseNodeByDatabasePath(PartialPath databasePath)
       throws MetadataException {
-    String[] nodes = storageGroupPath.getNodes();
+    String[] nodes = databasePath.getNodes();
     if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(storageGroupPath.getFullPath());
+      throw new IllegalPathException(databasePath.getFullPath());
     }
-    IMNode cur = root;
+    IConfigMNode cur = root;
     for (int i = 1; i < nodes.length - 1; i++) {
       cur = cur.getChild(nodes[i]);
       if (cur == null) {
-        throw new DatabaseNotSetException(storageGroupPath.getFullPath());
+        throw new DatabaseNotSetException(databasePath.getFullPath());
       }
-      if (cur.isStorageGroup()) {
+      if (cur.isDatabase()) {
         throw new DatabaseAlreadySetException(cur.getFullPath());
       }
     }
 
     cur = cur.getChild(nodes[nodes.length - 1]);
     if (cur == null) {
-      throw new DatabaseNotSetException(storageGroupPath.getFullPath());
+      throw new DatabaseNotSetException(databasePath.getFullPath());
     }
-    if (cur.isStorageGroup()) {
-      return cur.getAsStorageGroupMNode();
+    if (cur.isDatabase()) {
+      return cur.getAsDatabaseMNode();
     } else {
-      throw new DatabaseAlreadySetException(storageGroupPath.getFullPath(), true);
+      throw new DatabaseAlreadySetException(databasePath.getFullPath(), true);
     }
   }
 
@@ -272,19 +277,20 @@ public class ConfigMTree {
    * device], return the MNode of root.sg Get database node, the give path don't need to be database
    * path.
    */
-  public IStorageGroupMNode getDatabaseNodeByPath(PartialPath path) throws MetadataException {
+  public IDatabaseMNode<IConfigMNode> getDatabaseNodeByPath(PartialPath path)
+      throws MetadataException {
     String[] nodes = path.getNodes();
     if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
       throw new IllegalPathException(path.getFullPath());
     }
-    IMNode cur = root;
+    IConfigMNode cur = root;
     for (int i = 1; i < nodes.length; i++) {
       cur = cur.getChild(nodes[i]);
       if (cur == null) {
         break;
       }
-      if (cur.isStorageGroup()) {
-        return cur.getAsStorageGroupMNode();
+      if (cur.isDatabase()) {
+        return cur.getAsDatabaseMNode();
       }
     }
     throw new DatabaseNotSetException(path.getFullPath());
@@ -298,16 +304,16 @@ public class ConfigMTree {
    */
   public boolean isDatabaseAlreadySet(PartialPath path) {
     String[] nodeNames = path.getNodes();
-    IMNode cur = root;
+    IConfigMNode cur = root;
     if (!nodeNames[0].equals(root.getName())) {
       return false;
     }
     for (int i = 1; i < nodeNames.length; i++) {
-      if (!cur.hasChild(nodeNames[i])) {
+      if (!store.hasChild(cur, nodeNames[i])) {
         return false;
       }
-      cur = cur.getChild(nodeNames[i]);
-      if (cur.isStorageGroup()) {
+      cur = store.getChild(cur, nodeNames[i]);
+      if (cur.isDatabase()) {
         return true;
       }
     }
@@ -322,16 +328,16 @@ public class ConfigMTree {
    */
   public void checkDatabaseAlreadySet(PartialPath path) throws DatabaseAlreadySetException {
     String[] nodeNames = path.getNodes();
-    IMNode cur = root;
+    IConfigMNode cur = root;
     if (!nodeNames[0].equals(root.getName())) {
       return;
     }
     for (int i = 1; i < nodeNames.length; i++) {
-      if (!cur.hasChild(nodeNames[i])) {
+      if (!store.hasChild(cur, nodeNames[i])) {
         return;
       }
-      cur = cur.getChild(nodeNames[i]);
-      if (cur.isStorageGroup()) {
+      cur = store.getChild(cur, nodeNames[i]);
+      if (cur.isDatabase()) {
         throw new DatabaseAlreadySetException(cur.getFullPath());
       }
     }
@@ -342,20 +348,21 @@ public class ConfigMTree {
 
   // region MTree Node Management
 
-  public IMNode getNodeWithAutoCreate(PartialPath path) throws DatabaseNotSetException {
+  public IConfigMNode getNodeWithAutoCreate(PartialPath path) throws DatabaseNotSetException {
     String[] nodeNames = path.getNodes();
-    IMNode cur = root;
-    IMNode child;
+    IConfigMNode cur = root;
+    IConfigMNode child;
     boolean hasStorageGroup = false;
     for (int i = 1; i < nodeNames.length; i++) {
-      child = cur.getChild(nodeNames[i]);
+      child = store.getChild(cur, nodeNames[i]);
       if (child == null) {
         if (hasStorageGroup) {
-          child = cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+          child =
+              store.addChild(cur, nodeNames[i], nodeFactory.createInternalMNode(cur, nodeNames[i]));
         } else {
           throw new DatabaseNotSetException(path.getFullPath());
         }
-      } else if (child.isStorageGroup()) {
+      } else if (child.isDatabase()) {
         hasStorageGroup = true;
       }
 
@@ -371,10 +378,10 @@ public class ConfigMTree {
   public Pair<List<PartialPath>, Set<PartialPath>> getNodesListInGivenLevel(
       PartialPath pathPattern, int nodeLevel, boolean isPrefixMatch) throws MetadataException {
     List<PartialPath> result = new LinkedList<>();
-    try (MNodeAboveDBCollector<?> collector =
-        new MNodeAboveDBCollector<Void>(root, pathPattern, store, isPrefixMatch) {
+    try (MNodeAboveDBCollector<Void, IConfigMNode> collector =
+        new MNodeAboveDBCollector<Void, IConfigMNode>(root, pathPattern, store, isPrefixMatch) {
           @Override
-          protected Void collectMNode(IMNode node) {
+          protected Void collectMNode(IConfigMNode node) {
             result.add(getPartialPathFromRootToNode(node));
             return null;
           }
@@ -402,11 +409,11 @@ public class ConfigMTree {
   public Pair<Set<TSchemaNode>, Set<PartialPath>> getChildNodePathInNextLevel(
       PartialPath pathPattern) throws MetadataException {
     Set<TSchemaNode> result = new TreeSet<>();
-    try (MNodeAboveDBCollector<?> collector =
-        new MNodeAboveDBCollector<Void>(
+    try (MNodeAboveDBCollector<Void, IConfigMNode> collector =
+        new MNodeAboveDBCollector<Void, IConfigMNode>(
             root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD), store, false) {
           @Override
-          protected Void collectMNode(IMNode node) {
+          protected Void collectMNode(IConfigMNode node) {
             result.add(
                 new TSchemaNode(
                     getPartialPathFromRootToNode(node).getFullPath(),
@@ -431,8 +438,8 @@ public class ConfigMTree {
    */
   public void checkTemplateOnPath(PartialPath path) throws MetadataException {
     String[] nodeNames = path.getNodes();
-    IMNode cur = root;
-    IMNode child;
+    IConfigMNode cur = root;
+    IConfigMNode child;
 
     if (cur.getSchemaTemplateId() != NON_TEMPLATE) {
       throw new MetadataException("Template already exists on " + cur.getFullPath());
@@ -456,12 +463,12 @@ public class ConfigMTree {
   }
 
   // traverse  all the  descendant of the given path node
-  private void checkTemplateOnSubtree(IMNode node) throws MetadataException {
+  private void checkTemplateOnSubtree(IConfigMNode node) throws MetadataException {
     if (node.isMeasurement()) {
       return;
     }
-    IMNode child;
-    IMNodeIterator iterator = store.getChildrenIterator(node);
+    IConfigMNode child;
+    IMNodeIterator<IConfigMNode> iterator = store.getChildrenIterator(node);
     while (iterator.hasNext()) {
       child = iterator.next();
 
@@ -478,10 +485,11 @@ public class ConfigMTree {
   public List<String> getPathsSetOnTemplate(int templateId, boolean filterPreUnset)
       throws MetadataException {
     List<String> resSet = new ArrayList<>();
-    try (MNodeCollector<?> collector =
-        new MNodeCollector<Void>(root, new PartialPath(ALL_RESULT_NODES), store, false) {
+    try (MNodeCollector<Void, IConfigMNode> collector =
+        new MNodeCollector<Void, IConfigMNode>(
+            root, new PartialPath(ALL_RESULT_NODES), store, false) {
           @Override
-          protected boolean acceptFullMatchedNode(IMNode node) {
+          protected boolean acceptFullMatchedNode(IConfigMNode node) {
             if (super.acceptFullMatchedNode(node)) {
               // if node not set template, go on traversing
               if (node.getSchemaTemplateId() != NON_TEMPLATE) {
@@ -497,20 +505,20 @@ public class ConfigMTree {
           }
 
           @Override
-          protected Void collectMNode(IMNode node) {
+          protected Void collectMNode(IConfigMNode node) {
             resSet.add(node.getFullPath());
             return null;
           }
 
           @Override
-          protected boolean shouldVisitSubtreeOfFullMatchedNode(IMNode node) {
+          protected boolean shouldVisitSubtreeOfFullMatchedNode(IConfigMNode node) {
             // descendants of the node cannot set another template, exit from this branch
             return (node.getSchemaTemplateId() == NON_TEMPLATE)
                 && super.shouldVisitSubtreeOfFullMatchedNode(node);
           }
 
           @Override
-          protected boolean shouldVisitSubtreeOfInternalMatchedNode(IMNode node) {
+          protected boolean shouldVisitSubtreeOfInternalMatchedNode(IConfigMNode node) {
             // descendants of the node cannot set another template, exit from this branch
             return (node.getSchemaTemplateId() == NON_TEMPLATE)
                 && super.shouldVisitSubtreeOfFullMatchedNode(node);
@@ -525,22 +533,22 @@ public class ConfigMTree {
   public Map<Integer, Set<PartialPath>> getTemplateSetInfo(PartialPath pathPattern)
       throws MetadataException {
     Map<Integer, Set<PartialPath>> result = new HashMap<>();
-    try (MNodeCollector<?> collector =
-        new MNodeCollector<Void>(root, pathPattern, store, false) {
+    try (MNodeCollector<Void, IConfigMNode> collector =
+        new MNodeCollector<Void, IConfigMNode>(root, pathPattern, store, false) {
           @Override
-          protected boolean acceptFullMatchedNode(IMNode node) {
+          protected boolean acceptFullMatchedNode(IConfigMNode node) {
             return (node.getSchemaTemplateId() != NON_TEMPLATE)
                 || super.acceptFullMatchedNode(node);
           }
 
           @Override
-          protected boolean acceptInternalMatchedNode(IMNode node) {
+          protected boolean acceptInternalMatchedNode(IConfigMNode node) {
             return (node.getSchemaTemplateId() != NON_TEMPLATE)
                 || super.acceptInternalMatchedNode(node);
           }
 
           @Override
-          protected Void collectMNode(IMNode node) {
+          protected Void collectMNode(IConfigMNode node) {
             result
                 .computeIfAbsent(node.getSchemaTemplateId(), k -> new HashSet<>())
                 .add(getPartialPathFromRootToNode(node));
@@ -548,14 +556,14 @@ public class ConfigMTree {
           }
 
           @Override
-          protected boolean shouldVisitSubtreeOfFullMatchedNode(IMNode node) {
+          protected boolean shouldVisitSubtreeOfFullMatchedNode(IConfigMNode node) {
             // descendants of the node cannot set another template, exit from this branch
             return (node.getSchemaTemplateId() == NON_TEMPLATE)
                 && super.shouldVisitSubtreeOfFullMatchedNode(node);
           }
 
           @Override
-          protected boolean shouldVisitSubtreeOfInternalMatchedNode(IMNode node) {
+          protected boolean shouldVisitSubtreeOfInternalMatchedNode(IConfigMNode node) {
             // descendants of the node cannot set another template, exit from this branch
             return (node.getSchemaTemplateId() == NON_TEMPLATE)
                 && super.shouldVisitSubtreeOfFullMatchedNode(node);
@@ -578,9 +586,10 @@ public class ConfigMTree {
     getNodeSetTemplate(templateId, path).unsetSchemaTemplate();
   }
 
-  private IMNode getNodeSetTemplate(int templateId, PartialPath path) throws MetadataException {
+  private IConfigMNode getNodeSetTemplate(int templateId, PartialPath path)
+      throws MetadataException {
     String[] nodeNames = path.getNodes();
-    IMNode cur = root;
+    IConfigMNode cur = root;
     for (int i = 1; i < nodeNames.length; i++) {
       cur = cur.getChild(nodeNames[i]);
       if (cur == null) {
@@ -599,10 +608,10 @@ public class ConfigMTree {
   // region Serialization and Deserialization
 
   public void serialize(OutputStream outputStream) throws IOException {
-    serializeInternalNode((InternalMNode) this.root, outputStream);
+    serializeConfigBasicMNode(this.root, outputStream);
   }
 
-  private void serializeInternalNode(InternalMNode node, OutputStream outputStream)
+  private void serializeConfigBasicMNode(IConfigMNode node, OutputStream outputStream)
       throws IOException {
     serializeChildren(node, outputStream);
 
@@ -612,25 +621,25 @@ public class ConfigMTree {
     ReadWriteIOUtils.write(node.getChildren().size(), outputStream);
   }
 
-  private void serializeChildren(InternalMNode node, OutputStream outputStream) throws IOException {
-    for (IMNode child : node.getChildren().values()) {
-      if (child.isStorageGroup()) {
-        serializeStorageGroupNode((StorageGroupMNode) child, outputStream);
+  private void serializeChildren(IConfigMNode node, OutputStream outputStream) throws IOException {
+    for (IConfigMNode child : node.getChildren().values()) {
+      if (child.isDatabase()) {
+        serializeDatabaseNode(child.getAsDatabaseMNode(), outputStream);
       } else {
-        serializeInternalNode((InternalMNode) child, outputStream);
+        serializeConfigBasicMNode(child, outputStream);
       }
     }
   }
 
-  private void serializeStorageGroupNode(
-      StorageGroupMNode storageGroupNode, OutputStream outputStream) throws IOException {
-    serializeChildren(storageGroupNode, outputStream);
+  private void serializeDatabaseNode(
+      IDatabaseMNode<IConfigMNode> storageGroupNode, OutputStream outputStream) throws IOException {
+    serializeChildren(storageGroupNode.getAsMNode(), outputStream);
 
     ReadWriteIOUtils.write(STORAGE_GROUP_MNODE_TYPE, outputStream);
     ReadWriteIOUtils.write(storageGroupNode.getName(), outputStream);
-    ReadWriteIOUtils.write(storageGroupNode.getSchemaTemplateId(), outputStream);
-    ThriftConfigNodeSerDeUtils.serializeTStorageGroupSchema(
-        storageGroupNode.getStorageGroupSchema(), outputStream);
+    ReadWriteIOUtils.write(storageGroupNode.getAsMNode().getSchemaTemplateId(), outputStream);
+    ThriftConfigNodeSerDeUtils.serializeTDatabaseSchema(
+        storageGroupNode.getAsMNode().getDatabaseSchema(), outputStream);
   }
 
   public void deserialize(InputStream inputStream) throws IOException {
@@ -638,14 +647,14 @@ public class ConfigMTree {
 
     String name = null;
     int childNum = 0;
-    Stack<Pair<InternalMNode, Boolean>> stack = new Stack<>();
-    StorageGroupMNode storageGroupMNode;
-    InternalMNode internalMNode;
+    Stack<Pair<IConfigMNode, Boolean>> stack = new Stack<>();
+    IConfigMNode databaseMNode;
+    IConfigMNode internalMNode;
 
     if (type == STORAGE_GROUP_MNODE_TYPE) {
-      storageGroupMNode = deserializeDatabaseMNode(inputStream);
-      name = storageGroupMNode.getName();
-      stack.push(new Pair<>(storageGroupMNode, true));
+      databaseMNode = deserializeDatabaseMNode(inputStream);
+      name = databaseMNode.getName();
+      stack.push(new Pair<>(databaseMNode, true));
     } else {
       internalMNode = deserializeInternalMNode(inputStream);
       childNum = ReadWriteIOUtils.readInt(inputStream);
@@ -669,13 +678,13 @@ public class ConfigMTree {
           name = internalMNode.getName();
           break;
         case STORAGE_GROUP_MNODE_TYPE:
-          storageGroupMNode = deserializeDatabaseMNode(inputStream);
+          databaseMNode = deserializeDatabaseMNode(inputStream).getAsMNode();
           childNum = 0;
           while (!stack.isEmpty() && !stack.peek().right) {
-            storageGroupMNode.addChild(stack.pop().left);
+            databaseMNode.addChild(stack.pop().left);
           }
-          stack.push(new Pair<>(storageGroupMNode, true));
-          name = storageGroupMNode.getName();
+          stack.push(new Pair<>(databaseMNode, true));
+          name = databaseMNode.getName();
           break;
         default:
           logger.error("Unrecognized node type. Cannot deserialize MTreeAboveSG from given buffer");
@@ -685,19 +694,21 @@ public class ConfigMTree {
     this.root = stack.peek().left;
   }
 
-  private InternalMNode deserializeInternalMNode(InputStream inputStream) throws IOException {
-    InternalMNode internalMNode = new InternalMNode(null, ReadWriteIOUtils.readString(inputStream));
-    internalMNode.setSchemaTemplateId(ReadWriteIOUtils.readInt(inputStream));
-    return internalMNode;
+  private IConfigMNode deserializeInternalMNode(InputStream inputStream) throws IOException {
+    IConfigMNode basicMNode =
+        nodeFactory.createInternalMNode(null, ReadWriteIOUtils.readString(inputStream));
+    basicMNode.setSchemaTemplateId(ReadWriteIOUtils.readInt(inputStream));
+    return basicMNode;
   }
 
-  private StorageGroupMNode deserializeDatabaseMNode(InputStream inputStream) throws IOException {
-    StorageGroupMNode storageGroupMNode =
-        new StorageGroupMNode(null, ReadWriteIOUtils.readString(inputStream));
-    storageGroupMNode.setSchemaTemplateId(ReadWriteIOUtils.readInt(inputStream));
-    storageGroupMNode.setStorageGroupSchema(
-        ThriftConfigNodeSerDeUtils.deserializeTStorageGroupSchema(inputStream));
-    return storageGroupMNode;
+  private IConfigMNode deserializeDatabaseMNode(InputStream inputStream) throws IOException {
+    IDatabaseMNode<IConfigMNode> databaseMNode =
+        nodeFactory.createDatabaseMNode(null, ReadWriteIOUtils.readString(inputStream));
+    databaseMNode.getAsMNode().setSchemaTemplateId(ReadWriteIOUtils.readInt(inputStream));
+    databaseMNode
+        .getAsMNode()
+        .setDatabaseSchema(ThriftConfigNodeSerDeUtils.deserializeTDatabaseSchema(inputStream));
+    return databaseMNode.getAsMNode();
   }
 
   // endregion
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/IMTreeBelowSG.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/IMTreeBelowSG.java
deleted file mode 100644
index de0aa5b8e0..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/IMTreeBelowSG.java
+++ /dev/null
@@ -1,195 +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.mtree;
-
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.template.Template;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-
-import java.io.File;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public interface IMTreeBelowSG {
-  void clear();
-
-  /**
-   * Create MTree snapshot
-   *
-   * @param snapshotDir specify snapshot directory
-   * @return false if failed to create snapshot; true if success
-   */
-  boolean createSnapshot(File snapshotDir);
-
-  IMeasurementMNode createTimeseries(
-      PartialPath path,
-      TSDataType dataType,
-      TSEncoding encoding,
-      CompressionType compressor,
-      Map<String, String> props,
-      String alias)
-      throws MetadataException;
-
-  /**
-   * Create aligned timeseries with full paths from root to one leaf node. Before creating
-   * timeseries, the * database should be set first, throw exception otherwise
-   *
-   * @param devicePath device path
-   * @param measurements measurements list
-   * @param dataTypes data types list
-   * @param encodings encodings list
-   * @param compressors compressor
-   */
-  List<IMeasurementMNode> createAlignedTimeseries(
-      PartialPath devicePath,
-      List<String> measurements,
-      List<TSDataType> dataTypes,
-      List<TSEncoding> encodings,
-      List<CompressionType> compressors,
-      List<String> aliasList)
-      throws MetadataException;
-
-  /**
-   * Check if measurements under device exists in MTree
-   *
-   * @param devicePath device full path
-   * @param measurementList measurements list
-   * @param aliasList alias of measurement
-   * @return If all measurements not exists, return empty map. Otherwise, return a map whose key is
-   *     index of measurement in list and value is exception.
-   */
-  Map<Integer, MetadataException> checkMeasurementExistence(
-      PartialPath devicePath, List<String> measurementList, List<String> aliasList);
-
-  /**
-   * Delete path. The path should be a full path from root to leaf node
-   *
-   * @param path Format: root.node(.node)+
-   */
-  IMeasurementMNode deleteTimeseries(PartialPath path) throws MetadataException;
-
-  boolean isEmptyInternalMNode(IMNode node) throws MetadataException;
-
-  /**
-   * Construct schema black list via setting matched timeseries to pre deleted.
-   *
-   * @param pathPattern path pattern
-   * @return PartialPath of timeseries that has been set to pre deleted
-   */
-  List<PartialPath> constructSchemaBlackList(PartialPath pathPattern) throws MetadataException;
-
-  /**
-   * Rollback schema black list via setting matched timeseries to not pre deleted.
-   *
-   * @param pathPattern path pattern
-   * @return PartialPath of timeseries that has been set to not pre deleted
-   */
-  List<PartialPath> rollbackSchemaBlackList(PartialPath pathPattern) throws MetadataException;
-
-  /**
-   * Get all pre-deleted timeseries matched by given pathPattern. For example, given path pattern
-   * root.sg.*.s1 and pre-deleted timeseries root.sg.d1.s1, root.sg.d2.s1, then the result set is
-   * {root.sg.d1.s1, root.sg.d2.s1}.
-   *
-   * @param pathPattern path pattern
-   * @return all pre-deleted timeseries matched by given pathPattern
-   */
-  List<PartialPath> getPreDeletedTimeseries(PartialPath pathPattern) throws MetadataException;
-
-  /**
-   * Get all devices of pre-deleted timeseries matched by given pathPattern. For example, given path
-   * pattern root.sg.*.s1 and pre-deleted timeseries root.sg.d1.s1, root.sg.d2.s1, then the result
-   * set is {root.sg.d1, root.sg.d2}.
-   *
-   * @param pathPattern path pattern
-   * @return all devices of pre-deleted timeseries matched by given pathPattern
-   */
-  Set<PartialPath> getDevicesOfPreDeletedTimeseries(PartialPath pathPattern)
-      throws MetadataException;
-
-  void setAlias(IMeasurementMNode measurementMNode, String alias) throws MetadataException;
-
-  /**
-   * Add an interval path to MTree. This is only used for automatically creating schema
-   *
-   * <p>e.g., get root.sg.d1, get or create all internal nodes and return the node of d1
-   */
-  IMNode getDeviceNodeWithAutoCreating(PartialPath deviceId) throws MetadataException;
-
-  /**
-   * Fetch all measurement path
-   *
-   * @param pathPattern a path pattern or a full path, may contain wildcard
-   * @param templateMap <TemplateId, Template>
-   * @param withTags whether returns all the tags of each timeseries as well.
-   * @return schema
-   */
-  List<MeasurementPath> fetchSchema(
-      PartialPath pathPattern, Map<Integer, Template> templateMap, boolean withTags)
-      throws MetadataException;
-
-  /**
-   * Get node by the path
-   *
-   * @return last node in given seriesPath
-   */
-  IMNode getNodeByPath(PartialPath path) throws MetadataException;
-
-  IMeasurementMNode getMeasurementMNode(PartialPath path) throws MetadataException;
-
-  long countAllMeasurement() throws MetadataException;
-
-  void activateTemplate(PartialPath activatePath, Template template) throws MetadataException;
-
-  /**
-   * constructSchemaBlackListWithTemplate
-   *
-   * @param templateSetInfo PathPattern and templateId to pre-deactivate
-   * @return Actual full path and templateId that has been pre-deactivated
-   */
-  Map<PartialPath, List<Integer>> constructSchemaBlackListWithTemplate(
-      Map<PartialPath, List<Integer>> templateSetInfo) throws MetadataException;
-
-  /**
-   * rollbackSchemaBlackListWithTemplate
-   *
-   * @param templateSetInfo PathPattern and templateId to rollback pre-deactivate
-   * @return Actual full path and templateId that has been rolled back
-   */
-  Map<PartialPath, List<Integer>> rollbackSchemaBlackListWithTemplate(
-      Map<PartialPath, List<Integer>> templateSetInfo) throws MetadataException;
-
-  /**
-   * deactivateTemplateInBlackList
-   *
-   * @param templateSetInfo PathPattern and templateId to rollback deactivate
-   * @return Actual full path and templateId that has been deactivated
-   */
-  Map<PartialPath, List<Integer>> deactivateTemplateInBlackList(
-      Map<PartialPath, List<Integer>> templateSetInfo) throws MetadataException;
-
-  long countPathsUsingTemplate(PartialPath pathPattern, int templateId) throws MetadataException;
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
index 60d70848da..e754bbf616 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
@@ -23,6 +23,11 @@ import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
+import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
+import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
+import org.apache.iotdb.commons.schema.node.utils.IMNodeIterator;
 import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
 import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
@@ -34,13 +39,8 @@ import org.apache.iotdb.db.exception.metadata.template.DifferentTemplateExceptio
 import org.apache.iotdb.db.exception.metadata.template.TemplateImcompatibeException;
 import org.apache.iotdb.db.exception.metadata.template.TemplateIsInUseException;
 import org.apache.iotdb.db.metadata.MetadataConstant;
-import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.InternalMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.iterator.IMNodeIterator;
+import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
+import org.apache.iotdb.db.metadata.mnode.schemafile.factory.CacheMNodeFactory;
 import org.apache.iotdb.db.metadata.mtree.store.CachedMTreeStore;
 import org.apache.iotdb.db.metadata.mtree.traverser.Traverser;
 import org.apache.iotdb.db.metadata.mtree.traverser.TraverserWithLimitOffsetWrapper;
@@ -101,36 +101,37 @@ import java.util.function.Function;
  *   <li>Interfaces and Implementation for Template check
  * </ol>
  */
-public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
+public class MTreeBelowSGCachedImpl {
 
   private final CachedMTreeStore store;
-  private volatile IStorageGroupMNode storageGroupMNode;
-  private final IMNode rootNode;
-  private final Function<IMeasurementMNode, Map<String, String>> tagGetter;
+  private volatile ICachedMNode storageGroupMNode;
+  private final ICachedMNode rootNode;
+  private final Function<IMeasurementMNode<ICachedMNode>, Map<String, String>> tagGetter;
+  private final IMNodeFactory<ICachedMNode> nodeFactory = CacheMNodeFactory.getInstance();
   private final int levelOfSG;
 
   // region MTree initialization, clear and serialization
   public MTreeBelowSGCachedImpl(
       PartialPath storageGroupPath,
-      Function<IMeasurementMNode, Map<String, String>> tagGetter,
+      Function<IMeasurementMNode<ICachedMNode>, Map<String, String>> tagGetter,
       Runnable flushCallback,
-      Consumer<IMeasurementMNode> measurementProcess,
+      Consumer<IMeasurementMNode<ICachedMNode>> measurementProcess,
       int schemaRegionId,
       CachedSchemaRegionStatistics regionStatistics)
       throws MetadataException, IOException {
     this.tagGetter = tagGetter;
     store = new CachedMTreeStore(storageGroupPath, schemaRegionId, regionStatistics, flushCallback);
-    this.storageGroupMNode = store.getRoot().getAsStorageGroupMNode();
+    this.storageGroupMNode = store.getRoot();
     this.storageGroupMNode.setParent(storageGroupMNode.getParent());
     this.rootNode = store.generatePrefix(storageGroupPath);
     levelOfSG = storageGroupPath.getNodeLength() - 1;
 
     // recover measurement
-    try (MeasurementCollector<?> collector =
-        new MeasurementCollector<Void>(
+    try (MeasurementCollector<Void, ICachedMNode> collector =
+        new MeasurementCollector<Void, ICachedMNode>(
             this.rootNode, new PartialPath(storageGroupMNode.getFullPath()), this.store, true) {
-          @Override
-          protected Void collectMeasurement(IMeasurementMNode node) {
+
+          protected Void collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
             measurementProcess.accept(node);
             regionStatistics.addTimeseries(1L);
             return null;
@@ -144,21 +145,21 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
   private MTreeBelowSGCachedImpl(
       PartialPath storageGroupPath,
       CachedMTreeStore store,
-      Consumer<IMeasurementMNode> measurementProcess,
-      Function<IMeasurementMNode, Map<String, String>> tagGetter)
+      Consumer<IMeasurementMNode<ICachedMNode>> measurementProcess,
+      Function<IMeasurementMNode<ICachedMNode>, Map<String, String>> tagGetter)
       throws MetadataException {
     this.store = store;
-    this.storageGroupMNode = store.getRoot().getAsStorageGroupMNode();
+    this.storageGroupMNode = store.getRoot();
     this.rootNode = store.generatePrefix(storageGroupPath);
     levelOfSG = storageGroupMNode.getPartialPath().getNodeLength() - 1;
     this.tagGetter = tagGetter;
 
     // recover measurement
-    try (MeasurementCollector<?> collector =
-        new MeasurementCollector<Void>(
+    try (MeasurementCollector<Void, ICachedMNode> collector =
+        new MeasurementCollector<Void, ICachedMNode>(
             this.rootNode, new PartialPath(storageGroupMNode.getFullPath()), this.store, true) {
-          @Override
-          protected Void collectMeasurement(IMeasurementMNode node) {
+
+          protected Void collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
             measurementProcess.accept(node);
             return null;
           }
@@ -167,18 +168,18 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     }
   }
 
-  @Override
   public void clear() {
     store.clear();
     storageGroupMNode = null;
   }
 
-  protected void replaceStorageGroupMNode(IStorageGroupMNode newMNode) {
-    this.storageGroupMNode.getParent().replaceChild(this.storageGroupMNode.getName(), newMNode);
-    this.storageGroupMNode = newMNode;
+  protected void replaceStorageGroupMNode(IDatabaseMNode<ICachedMNode> newMNode) {
+    this.storageGroupMNode
+        .getParent()
+        .replaceChild(this.storageGroupMNode.getName(), newMNode.getAsMNode());
+    this.storageGroupMNode = newMNode.getAsMNode();
   }
 
-  @Override
   public boolean createSnapshot(File snapshotDir) {
     return store.createSnapshot(snapshotDir);
   }
@@ -188,8 +189,8 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
       String storageGroupFullPath,
       int schemaRegionId,
       CachedSchemaRegionStatistics regionStatistics,
-      Consumer<IMeasurementMNode> measurementProcess,
-      Function<IMeasurementMNode, Map<String, String>> tagGetter,
+      Consumer<IMeasurementMNode<ICachedMNode>> measurementProcess,
+      Function<IMeasurementMNode<ICachedMNode>, Map<String, String>> tagGetter,
       Runnable flushCallback)
       throws IOException, MetadataException {
     return new MTreeBelowSGCachedImpl(
@@ -204,8 +205,7 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
 
   // region Timeseries operation, including create and delete
 
-  @Override
-  public IMeasurementMNode createTimeseries(
+  public IMeasurementMNode<ICachedMNode> createTimeseries(
       PartialPath path,
       TSDataType dataType,
       TSEncoding encoding,
@@ -213,9 +213,9 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
       Map<String, String> props,
       String alias)
       throws MetadataException {
-    IMeasurementMNode measurementMNode =
+    IMeasurementMNode<ICachedMNode> measurementMNode =
         createTimeseriesWithPinnedReturn(path, dataType, encoding, compressor, props, alias);
-    unPinMNode(measurementMNode);
+    unPinMNode(measurementMNode.getAsMNode());
     return measurementMNode;
   }
 
@@ -230,7 +230,7 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
    * @param props props
    * @param alias alias of measurement
    */
-  public IMeasurementMNode createTimeseriesWithPinnedReturn(
+  public IMeasurementMNode<ICachedMNode> createTimeseriesWithPinnedReturn(
       PartialPath path,
       TSDataType dataType,
       TSEncoding encoding,
@@ -244,13 +244,13 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     }
     MetaFormatUtils.checkTimeseries(path);
     PartialPath devicePath = path.getDevicePath();
-    IMNode deviceParent = checkAndAutoCreateInternalPath(devicePath);
+    ICachedMNode deviceParent = checkAndAutoCreateInternalPath(devicePath);
 
     try {
       // synchronize check and add, we need addChild and add Alias become atomic operation
       // only write on mtree will be synchronized
       synchronized (this) {
-        IMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent);
+        ICachedMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent);
 
         try {
           MetaFormatUtils.checkTimeseriesProps(path.getFullPath(), props);
@@ -265,30 +265,30 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
             throw new PathAlreadyExistException(path.getFullPath());
           }
 
-          if (device.isEntity() && device.getAsEntityMNode().isAligned()) {
+          if (device.isDevice() && device.getAsDeviceMNode().isAligned()) {
             throw new AlignedTimeseriesException(
                 "timeseries under this entity is aligned, please use createAlignedTimeseries or change entity.",
                 device.getFullPath());
           }
 
-          IEntityMNode entityMNode;
-          if (device.isEntity()) {
-            entityMNode = device.getAsEntityMNode();
+          IDeviceMNode<ICachedMNode> entityMNode;
+          if (device.isDevice()) {
+            entityMNode = device.getAsDeviceMNode();
           } else {
             entityMNode = store.setToEntity(device);
-            if (entityMNode.isStorageGroup()) {
-              replaceStorageGroupMNode(entityMNode.getAsStorageGroupMNode());
+            if (entityMNode.isDatabase()) {
+              replaceStorageGroupMNode(entityMNode.getAsDatabaseMNode());
             }
-            device = entityMNode;
+            device = entityMNode.getAsMNode();
           }
 
-          IMeasurementMNode measurementMNode =
-              MeasurementMNode.getMeasurementMNode(
+          IMeasurementMNode<ICachedMNode> measurementMNode =
+              nodeFactory.createMeasurementMNode(
                   entityMNode,
                   leafName,
                   new MeasurementSchema(leafName, dataType, encoding, compressor, props),
                   alias);
-          store.addChild(entityMNode, leafName, measurementMNode);
+          store.addChild(entityMNode.getAsMNode(), leafName, measurementMNode.getAsMNode());
           // link alias to LeafMNode
           if (alias != null) {
             entityMNode.addAlias(alias, measurementMNode);
@@ -315,8 +315,7 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
    * @param encodings encodings list
    * @param compressors compressor
    */
-  @Override
-  public List<IMeasurementMNode> createAlignedTimeseries(
+  public List<IMeasurementMNode<ICachedMNode>> createAlignedTimeseries(
       PartialPath devicePath,
       List<String> measurements,
       List<TSDataType> dataTypes,
@@ -324,15 +323,15 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
       List<CompressionType> compressors,
       List<String> aliasList)
       throws MetadataException {
-    List<IMeasurementMNode> measurementMNodeList = new ArrayList<>();
+    List<IMeasurementMNode<ICachedMNode>> measurementMNodeList = new ArrayList<>();
     MetaFormatUtils.checkSchemaMeasurementNames(measurements);
-    IMNode deviceParent = checkAndAutoCreateInternalPath(devicePath);
+    ICachedMNode deviceParent = checkAndAutoCreateInternalPath(devicePath);
 
     try {
       // synchronize check and add, we need addChild operation be atomic.
       // only write operations on mtree will be synchronized
       synchronized (this) {
-        IMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent);
+        ICachedMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent);
 
         try {
           for (int i = 0; i < measurements.size(); i++) {
@@ -348,27 +347,27 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
             }
           }
 
-          if (device.isEntity() && !device.getAsEntityMNode().isAligned()) {
+          if (device.isDevice() && !device.getAsDeviceMNode().isAligned()) {
             throw new AlignedTimeseriesException(
                 "Timeseries under this entity is not aligned, please use createTimeseries or change entity.",
                 devicePath.getFullPath());
           }
 
-          IEntityMNode entityMNode;
-          if (device.isEntity()) {
-            entityMNode = device.getAsEntityMNode();
+          IDeviceMNode<ICachedMNode> entityMNode;
+          if (device.isDevice()) {
+            entityMNode = device.getAsDeviceMNode();
           } else {
             entityMNode = store.setToEntity(device);
             entityMNode.setAligned(true);
-            if (entityMNode.isStorageGroup()) {
-              replaceStorageGroupMNode(entityMNode.getAsStorageGroupMNode());
+            if (entityMNode.isDatabase()) {
+              replaceStorageGroupMNode(entityMNode.getAsDatabaseMNode());
             }
-            device = entityMNode;
+            device = entityMNode.getAsMNode();
           }
 
           for (int i = 0; i < measurements.size(); i++) {
-            IMeasurementMNode measurementMNode =
-                MeasurementMNode.getMeasurementMNode(
+            IMeasurementMNode<ICachedMNode> measurementMNode =
+                nodeFactory.createMeasurementMNode(
                     entityMNode,
                     measurements.get(i),
                     new MeasurementSchema(
@@ -377,7 +376,8 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
                         encodings.get(i),
                         compressors.get(i)),
                     aliasList == null ? null : aliasList.get(i));
-            store.addChild(entityMNode, measurements.get(i), measurementMNode);
+            store.addChild(
+                entityMNode.getAsMNode(), measurements.get(i), measurementMNode.getAsMNode());
             if (aliasList != null && aliasList.get(i) != null) {
               entityMNode.addAlias(aliasList.get(i), measurementMNode);
             }
@@ -395,22 +395,21 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     }
   }
 
-  @Override
   public Map<Integer, MetadataException> checkMeasurementExistence(
       PartialPath devicePath, List<String> measurementList, List<String> aliasList) {
-    IMNode device;
+    ICachedMNode device;
     try {
       device = getNodeByPath(devicePath);
     } catch (MetadataException e) {
       return Collections.emptyMap();
     }
     try {
-      if (!device.isEntity()) {
+      if (!device.isDevice()) {
         return Collections.emptyMap();
       }
       Map<Integer, MetadataException> failingMeasurementMap = new HashMap<>();
       for (int i = 0; i < measurementList.size(); i++) {
-        IMNode node = null;
+        ICachedMNode node = null;
         try {
           node = store.getChild(device, measurementList.get(i));
           if (node != null) {
@@ -456,14 +455,15 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     }
   }
 
-  private IMNode checkAndAutoCreateInternalPath(PartialPath devicePath) throws MetadataException {
+  private ICachedMNode checkAndAutoCreateInternalPath(PartialPath devicePath)
+      throws MetadataException {
     String[] nodeNames = devicePath.getNodes();
     MetaFormatUtils.checkTimeseries(devicePath);
     if (nodeNames.length == levelOfSG + 1) {
       return null;
     }
-    IMNode cur = storageGroupMNode;
-    IMNode child;
+    ICachedMNode cur = storageGroupMNode;
+    ICachedMNode child;
     String childName;
     try {
       // e.g, path = root.sg.d1.s1,  create internal nodes and set cur to sg node, parent of d1
@@ -471,7 +471,7 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
         childName = nodeNames[i];
         child = store.getChild(cur, childName);
         if (child == null) {
-          child = store.addChild(cur, childName, new InternalMNode(cur, childName));
+          child = store.addChild(cur, childName, nodeFactory.createInternalMNode(cur, childName));
         }
         cur = child;
 
@@ -486,17 +486,18 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     }
   }
 
-  private IMNode checkAndAutoCreateDeviceNode(String deviceName, IMNode deviceParent)
+  private ICachedMNode checkAndAutoCreateDeviceNode(String deviceName, ICachedMNode deviceParent)
       throws MetadataException {
     if (deviceParent == null) {
       // device is sg
       pinMNode(storageGroupMNode);
       return storageGroupMNode;
     }
-    IMNode device = store.getChild(deviceParent, deviceName);
+    ICachedMNode device = store.getChild(deviceParent, deviceName);
     if (device == null) {
       device =
-          store.addChild(deviceParent, deviceName, new InternalMNode(deviceParent, deviceName));
+          store.addChild(
+              deviceParent, deviceName, nodeFactory.createInternalMNode(deviceParent, deviceName));
     }
 
     if (device.isMeasurement()) {
@@ -510,21 +511,21 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
    *
    * @param path Format: root.node(.node)+
    */
-  @Override
-  public IMeasurementMNode deleteTimeseries(PartialPath path) throws MetadataException {
+  public IMeasurementMNode<ICachedMNode> deleteTimeseries(PartialPath path)
+      throws MetadataException {
     String[] nodes = path.getNodes();
     if (nodes.length == 0) {
       throw new IllegalPathException(path.getFullPath());
     }
 
-    IMeasurementMNode deletedNode = getMeasurementMNode(path);
-    IEntityMNode parent = deletedNode.getParent();
+    IMeasurementMNode<ICachedMNode> deletedNode = getMeasurementMNode(path);
+    ICachedMNode parent = deletedNode.getParent();
     // delete the last node of path
     store.deleteChild(parent, path.getMeasurement());
     if (deletedNode.getAlias() != null) {
-      parent.deleteAliasChild(deletedNode.getAlias());
+      parent.getAsDeviceMNode().deleteAliasChild(deletedNode.getAlias());
     }
-    deleteAndUnpinEmptyInternalMNode(parent);
+    deleteAndUnpinEmptyInternalMNode(parent.getAsDeviceMNode());
     return deletedNode;
   }
 
@@ -534,12 +535,13 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
    *
    * @param entityMNode delete empty InternalMNode from entityMNode to storageGroupMNode
    */
-  private void deleteAndUnpinEmptyInternalMNode(IEntityMNode entityMNode) throws MetadataException {
-    IMNode curNode = entityMNode;
+  private void deleteAndUnpinEmptyInternalMNode(IDeviceMNode<ICachedMNode> entityMNode)
+      throws MetadataException {
+    ICachedMNode curNode = entityMNode.getAsMNode();
     if (!entityMNode.isUseTemplate()) {
       boolean hasMeasurement = false;
-      IMNode child;
-      IMNodeIterator iterator = store.getChildrenIterator(entityMNode);
+      ICachedMNode child;
+      IMNodeIterator<ICachedMNode> iterator = store.getChildrenIterator(curNode);
       try {
         while (iterator.hasNext()) {
           child = iterator.next();
@@ -556,8 +558,8 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
       if (!hasMeasurement) {
         synchronized (this) {
           curNode = store.setToInternal(entityMNode);
-          if (curNode.isStorageGroup()) {
-            replaceStorageGroupMNode(curNode.getAsStorageGroupMNode());
+          if (curNode.isDatabase()) {
+            replaceStorageGroupMNode(curNode.getAsDatabaseMNode());
           }
         }
       }
@@ -566,7 +568,7 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     // delete all empty ancestors except database and MeasurementMNode
     while (isEmptyInternalMNode(curNode)) {
       // if current database has no time series, return the database name
-      if (curNode.isStorageGroup()) {
+      if (curNode.isDatabase()) {
         return;
       }
       store.deleteChild(curNode.getParent(), curNode.getName());
@@ -575,30 +577,29 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     unPinMNode(curNode);
   }
 
-  @Override
-  public boolean isEmptyInternalMNode(IMNode node) throws MetadataException {
-    IMNodeIterator iterator = store.getChildrenIterator(node);
+  private boolean isEmptyInternalMNode(ICachedMNode node) throws MetadataException {
+    IMNodeIterator<ICachedMNode> iterator = store.getChildrenIterator(node);
     try {
       return !IoTDBConstant.PATH_ROOT.equals(node.getName())
           && !node.isMeasurement()
-          && !node.isUseTemplate()
+          && !(node.isDevice() && node.getAsDeviceMNode().isUseTemplate())
           && !iterator.hasNext();
     } finally {
       iterator.close();
     }
   }
 
-  @Override
   public List<PartialPath> constructSchemaBlackList(PartialPath pathPattern)
       throws MetadataException {
     List<PartialPath> result = new ArrayList<>();
-    try (MeasurementUpdater updater =
-        new MeasurementUpdater(rootNode, pathPattern, store, false) {
-          @Override
-          protected void updateMeasurement(IMeasurementMNode node) throws MetadataException {
+    try (MeasurementUpdater<ICachedMNode> updater =
+        new MeasurementUpdater<ICachedMNode>(rootNode, pathPattern, store, false) {
+
+          protected void updateMeasurement(IMeasurementMNode<ICachedMNode> node)
+              throws MetadataException {
             node.setPreDeleted(true);
-            store.updateMNode(node);
-            result.add(getPartialPathFromRootToNode(node));
+            store.updateMNode(node.getAsMNode());
+            result.add(getPartialPathFromRootToNode(node.getAsMNode()));
           }
         }) {
       updater.update();
@@ -606,17 +607,17 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     return result;
   }
 
-  @Override
   public List<PartialPath> rollbackSchemaBlackList(PartialPath pathPattern)
       throws MetadataException {
     List<PartialPath> result = new ArrayList<>();
-    try (MeasurementUpdater updater =
-        new MeasurementUpdater(rootNode, pathPattern, store, false) {
-          @Override
-          protected void updateMeasurement(IMeasurementMNode node) throws MetadataException {
+    try (MeasurementUpdater<ICachedMNode> updater =
+        new MeasurementUpdater<ICachedMNode>(rootNode, pathPattern, store, false) {
+
+          protected void updateMeasurement(IMeasurementMNode<ICachedMNode> node)
+              throws MetadataException {
             node.setPreDeleted(false);
-            store.updateMNode(node);
-            result.add(getPartialPathFromRootToNode(node));
+            store.updateMNode(node.getAsMNode());
+            result.add(getPartialPathFromRootToNode(node.getAsMNode()));
           }
         }) {
       updater.update();
@@ -624,16 +625,15 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     return result;
   }
 
-  @Override
   public List<PartialPath> getPreDeletedTimeseries(PartialPath pathPattern)
       throws MetadataException {
     List<PartialPath> result = new LinkedList<>();
-    try (MeasurementCollector<Void> collector =
-        new MeasurementCollector<Void>(rootNode, pathPattern, store, false) {
-          @Override
-          protected Void collectMeasurement(IMeasurementMNode node) {
+    try (MeasurementCollector<Void, ICachedMNode> collector =
+        new MeasurementCollector<Void, ICachedMNode>(rootNode, pathPattern, store, false) {
+
+          protected Void collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
             if (node.isPreDeleted()) {
-              result.add(getPartialPathFromRootToNode(node));
+              result.add(getPartialPathFromRootToNode(node.getAsMNode()));
             }
             return null;
           }
@@ -643,16 +643,15 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     return result;
   }
 
-  @Override
   public Set<PartialPath> getDevicesOfPreDeletedTimeseries(PartialPath pathPattern)
       throws MetadataException {
     Set<PartialPath> result = new HashSet<>();
-    try (MeasurementCollector<Void> collector =
-        new MeasurementCollector<Void>(rootNode, pathPattern, store, false) {
-          @Override
-          protected Void collectMeasurement(IMeasurementMNode node) {
+    try (MeasurementCollector<Void, ICachedMNode> collector =
+        new MeasurementCollector<Void, ICachedMNode>(rootNode, pathPattern, store, false) {
+
+          protected Void collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
             if (node.isPreDeleted()) {
-              result.add(getPartialPathFromRootToNode(node).getDevicePath());
+              result.add(getPartialPathFromRootToNode(node.getAsMNode()).getDevicePath());
             }
             return null;
           }
@@ -663,8 +662,8 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
     return result;
   }
 
-  @Override
-  public void setAlias(IMeasurementMNode measurementMNode, String alias) throws MetadataException {
+  public void setAlias(IMeasurementMNode<ICachedMNode> measurementMNode, String alias)
+      throws MetadataException {
     store.setAlias(measurementMNode, alias);
   }
 
@@ -677,17 +676,17 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
    *
    * <p>e.g., get root.sg.d1, get or create all internal nodes and return the node of d1
    */
-  @Override
-  public IMNode getDeviceNodeWithAutoCreating(PartialPath deviceId) throws MetadataException {
+  public ICachedMNode getDeviceNodeWithAutoCreating(PartialPath deviceId) throws MetadataException {
     String[] nodeNames = deviceId.getNodes();
     MetaFormatUtils.checkTimeseries(deviceId);
-    IMNode cur = storageGroupMNode;
-    IMNode child;
+    ICachedMNode cur = storageGroupMNode;
+    ICachedMNode child;
     try {
       for (int i = levelOfSG + 1; i < nodeNames.length; i++) {
         child = store.getChild(cur, nodeNames[i]);
         if (child == null) {
-          child = store.addChild(cur, nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+          child =
+              store.addChild(cur, nodeNames[i], nodeFactory.createInternalMNode(cur, nodeNames[i]));
         }
         cur = child;
       }
@@ -701,15 +700,14 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
 
   // region Interfaces and Implementation for metadata info Query
 
-  @Override
   public List<MeasurementPath> fetchSchema(
       PartialPath pathPattern, Map<Integer, Template> templateMap, boolean withTags)
       throws MetadataException {
     List<MeasurementPath> result = new LinkedList<>();
-    try (MeasurementCollector<Void> collector =
-        new MeasurementCollector<Void>(rootNode, pathPattern, store, false) {
... 8431 lines suppressed ...