You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hx...@apache.org on 2020/07/28 16:51:02 UTC

[incubator-iotdb] 01/01: [not finished] extract MManager as interface

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

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

commit 3872fdce471a282e0da59ecc83028717c4b30cf2
Author: xiangdong huang <sa...@gmail.com>
AuthorDate: Wed Jul 29 00:50:33 2020 +0800

    [not finished] extract MManager as interface
---
 .../engine/storagegroup/StorageGroupProcessor.java |   3 +-
 .../apache/iotdb/db/metadata/ISchemaManager.java   | 291 +++++++++++++++++++++
 .../org/apache/iotdb/db/metadata/MManager.java     |  59 ++++-
 .../org/apache/iotdb/db/monitor/StatMonitor.java   |  16 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  44 ++--
 .../iotdb/db/engine/MetadataManagerHelper.java     |   4 +-
 .../storagegroup/FileNodeManagerBenchmark.java     |   4 +-
 .../iotdb/db/metadata/MManagerAdvancedTest.java    |   2 +-
 .../iotdb/db/metadata/MManagerBasicTest.java       |   6 +-
 .../iotdb/db/metadata/MManagerImproveTest.java     |  32 +--
 .../java/org/apache/iotdb/db/qp/PlannerTest.java   |  32 +--
 .../db/sync/receiver/load/FileLoaderTest.java      |   4 +-
 .../recover/SyncReceiverLogAnalyzerTest.java       |   4 +-
 .../db/writelog/recover/DeviceStringTest.java      |  12 +-
 14 files changed, 429 insertions(+), 84 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index f384496..571c30a 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -76,6 +76,7 @@ import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.OutOfTTLException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
@@ -1564,7 +1565,7 @@ public class StorageGroupProcessor {
       long endTime) throws WriteProcessException {
     MNode node = null;
     try {
-      MManager manager = MManager.getInstance();
+      ISchemaManager manager = MManager.getInstance();
       node = manager.getDeviceNodeWithAutoCreateAndReadLock(deviceId);
 
       MNode measurementNode = manager.getChild(node, measurementId);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/ISchemaManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/ISchemaManager.java
new file mode 100644
index 0000000..3e66be1
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/ISchemaManager.java
@@ -0,0 +1,291 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
+import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+public interface ISchemaManager {
+
+  /**
+   * init the schema manager.
+   * This function will be called after a manamger instance is created.
+   * 1. NOTICE that the implementation of this method should be thread-safety.
+   * 2. the implementataion should be idempotent (means the method can be called multiple times)
+   */
+  void init();
+
+  /**
+   * clear the schema in this manager.
+   * all schema in memory and on disk should be cleared
+   */
+  void clear();
+
+  /**
+   * execute the cmd to operate schema
+   * @param cmd
+   * TODO command format:
+   * @throws IOException if saving the schema failed on disk
+   * @throws MetadataException if cmd has error, or the cmd is conflict with existing schema.
+   */
+  void operation(String cmd) throws IOException, MetadataException;
+
+  /**
+   * create a new time series.
+   * @param plan the create time series plan.
+   * @throws MetadataException if the plan has error or it is conflict with existing schema.
+   */
+  void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException;
+
+  /**
+   * create a new time series.
+   * @param path the full path of the time series
+   * @param dataType the data type of the series
+   * @param encoding the encoding method of the series
+   * @param compressor the compression method of the series
+   * @param props the customized properties of the series
+   * @throws MetadataException if something has error or it is conflict with existing schema.
+   */
+  void createTimeseries(String path, TSDataType dataType, TSEncoding encoding,
+      CompressionType compressor, Map<String, String> props) throws MetadataException;
+
+  /**
+   * Delete all timeseries under the given path, may cross different storage group
+   *
+   * @param prefixPath path to be deleted, could be root or a prefix path or a full path
+   * @return the deletion failed Timeseries, which are contacted by ","
+   */
+  String deleteTimeseries(String prefixPath) throws MetadataException;
+
+  /**
+   * add a new storage group
+   * @param storageGroup the full path of the storage group
+   * @throws MetadataException if the path has error or is conflict with existing storage groups.
+   */
+  void setStorageGroup(String storageGroup) throws MetadataException;
+
+  /**
+   * delete given storage groups
+   * @param storageGroups  full paths of storage groups
+   * @throws MetadataException if deletion failed (e.g., the sg does not exist)
+   */
+  void deleteStorageGroups(List<String> storageGroups) throws MetadataException;
+
+  /**
+   * get the data type of a time series
+   * @param path full path of a series
+   * @return data type of the time series
+   * @throws MetadataException query failed or the time series does not exist
+   */
+  TSDataType getSeriesType(String path) throws MetadataException;
+
+  /**
+   * get the detail schema of given time series
+   * @param deviceId the prefix path of the given time series
+   * @param measurements the suffix paths of the given time series (does not contains ".")
+   * @return the detail schemas.
+   * @throws MetadataException
+   */
+  MeasurementSchema[] getSchemas(String deviceId, String[] measurements)
+      throws MetadataException;
+
+  /**
+   * get all devices whose prefix paths match the prefixPath
+   * @param prefixPath a prefix path, must start with root. The prefix path can contain *.
+   * @return all devices whose prefix paths match the prefixPath
+   * @throws MetadataException
+   */
+  Set<String> getDevices(String prefixPath) throws MetadataException;
+
+  /**
+   * Get all paths from the given level which have the given prefix path.
+   *
+   * @param prefixPath a prefix path that starts with root and does not allow having *.
+   * @param nodeLevel  the level of the path that will be returned. The level should >= the level of
+   *                   the given prefix path.
+   * @return A List instance which stores all node at given level
+   */
+  List<String> getNodesList(String prefixPath, int nodeLevel) throws MetadataException;
+
+  /**
+   * given a prefix path or full path, find which storage group it belongs to
+   * @param path a prefix path or full path
+   * @return
+   * @throws StorageGroupNotSetException
+   */
+  String getStorageGroupName(String path) throws StorageGroupNotSetException;
+
+  /**
+   * @return all storage group names in the system
+   */
+  List<String> getAllStorageGroupNames();
+
+  /**
+   *
+   * @return all storage group nodes in the system
+   */
+  List<StorageGroupMNode> getAllStorageGroupNodes();
+
+  /**
+   * get all full paths which match the given prefix Path.
+   * @param prefixPath a prefix path which starts with root, and can contain *.
+   * @return all full paths
+   * @throws MetadataException
+   */
+  List<String> getAllTimeseriesName(String prefixPath) throws MetadataException;
+
+  /**
+   * get all full paths which match the given prefix Path.
+   * @param prefixPath a prefix path which starts with root, and can contain *.
+   * @return all full paths
+   * @throws MetadataException
+   */
+  List<Path> getAllTimeseriesPath(String prefixPath) throws MetadataException;
+
+  /**
+   * get the number of all full paths which match the given prefix Path.
+   * @param prefixPath a prefix path which starts with root, and can contain *.
+   * @return
+   * @throws MetadataException
+   */
+  int getAllTimeseriesCount(String prefixPath) throws MetadataException;
+
+  /**
+   * get the number of all  paths which match the given prefix Path, and whose level equal to the
+   * given level
+   * @param prefixPath a prefix path which starts with root, and can contain *.
+   * @param level a given level, must >= the level ofthe given prefixPath
+   * @return
+   * @throws MetadataException
+   */
+  int getNodesCountInGivenLevel(String prefixPath, int level) throws MetadataException;
+
+  List<ShowTimeSeriesResult> showTimeseries(ShowTimeSeriesPlan plan, QueryContext context)
+      throws MetadataException;
+
+  /**
+   * get the detail schema of given time series
+   * @param device the prefix path of the given time series
+   * @param measurement the suffix path of the given time series (does not contains ".")
+   * @return the detail schema.
+   * @throws MetadataException
+   */
+  MeasurementSchema getSeriesSchema(String device, String measurement)
+          throws MetadataException;
+
+  Set<String> getChildNodePathInNextLevel(String path) throws MetadataException;
+
+  boolean isPathExist(String path);
+
+  MNode getNodeByPath(String path) throws MetadataException;
+
+  StorageGroupMNode getStorageGroupNode(String path) throws MetadataException;
+
+  MNode getDeviceNodeWithAutoCreateAndReadLock(
+      String path, boolean autoCreateSchema, int sgLevel) throws MetadataException;
+
+  MNode getDeviceNodeWithAutoCreateAndReadLock(String path) throws MetadataException;
+
+  MNode getDeviceNode(String path) throws MetadataException;
+
+  String getDeviceId(String path);
+
+  MNode getChild(MNode parent, String child);
+
+  String getMetadataInString();
+
+  @TestOnly
+  void setMaxSeriesNumberAmongStorageGroup(long maxSeriesNumberAmongStorageGroup);
+
+  long getMaximalSeriesNumberAmongStorageGroups();
+
+  void setTTL(String storageGroup, long dataTTL) throws MetadataException, IOException;
+
+  Map<String, Long> getStorageGroupsTTL();
+
+  void changeOffset(String path, long offset) throws MetadataException;
+
+  void changeAlias(String path, String alias) throws MetadataException;
+
+  void upsertTagsAndAttributes(String alias, Map<String, String> tagsMap,
+      Map<String, String> attributesMap, String fullPath) throws MetadataException, IOException;
+
+  void addAttributes(Map<String, String> attributesMap, String fullPath)
+          throws MetadataException, IOException;
+
+  void addTags(Map<String, String> tagsMap, String fullPath)
+              throws MetadataException, IOException;
+
+  void dropTagsOrAttributes(Set<String> keySet, String fullPath)
+                  throws MetadataException, IOException;
+
+  void setTagsOrAttributesValue(Map<String, String> alterMap, String fullPath)
+                      throws MetadataException, IOException;
+
+  void renameTagOrAttributeKey(String oldKey, String newKey, String fullPath)
+                          throws MetadataException, IOException;
+
+  void collectTimeseriesSchema(MNode startingNode,
+      Collection<TimeseriesSchema> timeseriesSchemas);
+
+  void collectMeasurementSchema(MNode startingNode,
+      Collection<MeasurementSchema> timeseriesSchemas);
+
+  void collectSeries(String startingPath, List<MeasurementSchema> measurementSchemas);
+
+  Map<String, String> determineStorageGroup(String path) throws IllegalPathException;
+
+  void cacheMeta(String path, MeasurementMeta meta);
+
+  void updateLastCache(String seriesPath, TimeValuePair timeValuePair,
+      boolean highPriorityUpdate, Long latestFlushedTime,
+      MeasurementMNode node);
+
+  TimeValuePair getLastCache(String seriesPath);
+
+  void createMTreeSnapshot();
+
+  MeasurementSchema[] getSeriesSchemasAndReadLockDevice(String deviceId,
+      String[] measurementList, InsertPlan plan) throws MetadataException;
+
+  void unlockDeviceReadLock(String deviceId);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 3e0f029..4f8eced 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -90,7 +90,7 @@ import org.slf4j.LoggerFactory;
  * into files. This class contains all the interfaces to modify the metadata for delta system. All
  * the operations will be insert into the logs temporary in case the downtime of the delta system.
  */
-public class MManager {
+public class MManager implements ISchemaManager {
 
   private static final Logger logger = LoggerFactory.getLogger(MManager.class);
   private static final String TIME_SERIES_TREE_HEADER = "===  Timeseries Tree  ===\n\n";
@@ -191,6 +191,7 @@ public class MManager {
   }
 
   // Because the writer will be used later and should not be closed here.
+  @Override
   @SuppressWarnings("squid:S2093")
   public synchronized void init() {
     if (initialized) {
@@ -274,6 +275,7 @@ public class MManager {
   /**
    * function for clearing MTree
    */
+  @Override
   public void clear() {
     lock.writeLock().lock();
     try {
@@ -302,6 +304,7 @@ public class MManager {
     }
   }
 
+  @Override
   public void operation(String cmd) throws IOException, MetadataException {
     // see createTimeseries() to get the detailed format of the cmd
     String[] args = cmd.trim().split(",", -1);
@@ -362,11 +365,12 @@ public class MManager {
     }
   }
 
+  @Override
   public void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException {
     createTimeseries(plan, -1);
   }
 
-  public void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws MetadataException {
+  private void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws MetadataException {
     lock.writeLock().lock();
     String path = plan.getPath().getFullPath();
     try {
@@ -439,6 +443,7 @@ public class MManager {
    * @return whether the measurement occurs for the first time in this storage group (if true, the
    * measurement should be registered to the StorageEngine too)
    */
+  @Override
   public void createTimeseries(String path, TSDataType dataType, TSEncoding encoding,
       CompressionType compressor, Map<String, String> props) throws MetadataException {
     createTimeseries(
@@ -452,6 +457,7 @@ public class MManager {
    * @param prefixPath path to be deleted, could be root or a prefix path or a full path
    * @return The String is the deletion failed Timeseries
    */
+  @Override
   public String deleteTimeseries(String prefixPath) throws MetadataException {
     lock.writeLock().lock();
 
@@ -575,6 +581,7 @@ public class MManager {
    *
    * @param storageGroup root.node.(node)*
    */
+  @Override
   public void setStorageGroup(String storageGroup) throws MetadataException {
     lock.writeLock().lock();
     try {
@@ -603,6 +610,7 @@ public class MManager {
    *
    * @param storageGroups list of paths to be deleted. Format: root.node
    */
+  @Override
   public void deleteStorageGroups(List<String> storageGroups) throws MetadataException {
     lock.writeLock().lock();
     try {
@@ -662,6 +670,7 @@ public class MManager {
    *
    * @param path full path
    */
+  @Override
   public TSDataType getSeriesType(String path) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -675,6 +684,7 @@ public class MManager {
     }
   }
 
+  @Override
   public MeasurementSchema[] getSchemas(String deviceId, String[] measurements)
       throws MetadataException {
     lock.readLock().lock();
@@ -701,6 +711,7 @@ public class MManager {
    *                   wildcard can only match one level, otherwise it can match to the tail.
    * @return A HashSet instance which stores devices names with given prefixPath.
    */
+  @Override
   public Set<String> getDevices(String prefixPath) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -719,11 +730,12 @@ public class MManager {
    * @param nodeLevel  the level can not be smaller than the level of the prefixPath
    * @return A List instance which stores all node at given level
    */
+  @Override
   public List<String> getNodesList(String prefixPath, int nodeLevel) throws MetadataException {
     return getNodesList(prefixPath, nodeLevel, null);
   }
 
-  public List<String> getNodesList(String prefixPath, int nodeLevel, StorageGroupFilter filter)
+  private List<String> getNodesList(String prefixPath, int nodeLevel, StorageGroupFilter filter)
       throws MetadataException {
     lock.readLock().lock();
     try {
@@ -740,6 +752,7 @@ public class MManager {
    *
    * @return storage group in the given path
    */
+  @Override
   public String getStorageGroupName(String path) throws StorageGroupNotSetException {
     lock.readLock().lock();
     try {
@@ -752,6 +765,7 @@ public class MManager {
   /**
    * Get all storage group names
    */
+  @Override
   public List<String> getAllStorageGroupNames() {
     lock.readLock().lock();
     try {
@@ -764,6 +778,7 @@ public class MManager {
   /**
    * Get all storage group MNodes
    */
+  @Override
   public List<StorageGroupMNode> getAllStorageGroupNodes() {
     lock.readLock().lock();
     try {
@@ -780,6 +795,7 @@ public class MManager {
    * @param prefixPath can be a prefix or a full path. if the wildcard is not at the tail, then each
    *                   wildcard can only match one level, otherwise it can match to the tail.
    */
+  @Override
   public List<String> getAllTimeseriesName(String prefixPath) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -793,6 +809,7 @@ public class MManager {
    * Similar to method getAllTimeseriesName(), but return Path instead of String in order to include
    * alias.
    */
+  @Override
   public List<Path> getAllTimeseriesPath(String prefixPath) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -805,6 +822,7 @@ public class MManager {
   /**
    * To calculate the count of timeseries for given prefix path.
    */
+  @Override
   public int getAllTimeseriesCount(String prefixPath) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -820,6 +838,7 @@ public class MManager {
    * @param prefixPath a prefix path or a full path, can not contain '*'
    * @param level      the level can not be smaller than the level of the prefixPath
    */
+  @Override
   public int getNodesCountInGivenLevel(String prefixPath, int level) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -923,6 +942,7 @@ public class MManager {
     return true;
   }
 
+  @Override
   public List<ShowTimeSeriesResult> showTimeseries(ShowTimeSeriesPlan plan, QueryContext context)
       throws MetadataException {
     // show timeseries with index
@@ -975,6 +995,7 @@ public class MManager {
     }
   }
 
+  @Override
   public MeasurementSchema getSeriesSchema(String device, String measurement)
       throws MetadataException {
     lock.readLock().lock();
@@ -1001,6 +1022,7 @@ public class MManager {
    *
    * @return All child nodes' seriesPath(s) of given seriesPath.
    */
+  @Override
   public Set<String> getChildNodePathInNextLevel(String path) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -1015,6 +1037,7 @@ public class MManager {
    *
    * @param path a full path or a prefix path
    */
+  @Override
   public boolean isPathExist(String path) {
     lock.readLock().lock();
     try {
@@ -1027,6 +1050,7 @@ public class MManager {
   /**
    * Get node by path
    */
+  @Override
   public MNode getNodeByPath(String path) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -1040,6 +1064,7 @@ public class MManager {
    * Get storage group node by path. If storage group is not set, StorageGroupNotSetException will
    * be thrown
    */
+  @Override
   public StorageGroupMNode getStorageGroupNode(String path) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -1057,6 +1082,7 @@ public class MManager {
    *
    * @param path path
    */
+  @Override
   public MNode getDeviceNodeWithAutoCreateAndReadLock(
       String path, boolean autoCreateSchema, int sgLevel) throws MetadataException {
     lock.readLock().lock();
@@ -1106,11 +1132,13 @@ public class MManager {
   /**
    * !!!!!!Attention!!!!! must call the return node's readUnlock() if you call this method.
    */
+  @Override
   public MNode getDeviceNodeWithAutoCreateAndReadLock(String path) throws MetadataException {
     return getDeviceNodeWithAutoCreateAndReadLock(
         path, config.isAutoCreateSchemaEnabled(), config.getDefaultStorageGroupLevel());
   }
 
+  @Override
   public MNode getDeviceNode(String path) throws MetadataException {
     lock.readLock().lock();
     MNode node;
@@ -1131,6 +1159,7 @@ public class MManager {
    * @param path read from disk
    * @return deviceId
    */
+  @Override
   public String getDeviceId(String path) {
     MNode deviceNode = null;
     try {
@@ -1142,6 +1171,7 @@ public class MManager {
     return path;
   }
 
+  @Override
   public MNode getChild(MNode parent, String child) {
     lock.readLock().lock();
     try {
@@ -1154,6 +1184,7 @@ public class MManager {
   /**
    * Get metadata in string
    */
+  @Override
   public String getMetadataInString() {
     lock.readLock().lock();
     try {
@@ -1163,15 +1194,18 @@ public class MManager {
     }
   }
 
+  @Override
   @TestOnly
   public void setMaxSeriesNumberAmongStorageGroup(long maxSeriesNumberAmongStorageGroup) {
     this.maxSeriesNumberAmongStorageGroup = maxSeriesNumberAmongStorageGroup;
   }
 
+  @Override
   public long getMaximalSeriesNumberAmongStorageGroups() {
     return maxSeriesNumberAmongStorageGroup;
   }
 
+  @Override
   public void setTTL(String storageGroup, long dataTTL) throws MetadataException, IOException {
     lock.writeLock().lock();
     try {
@@ -1189,6 +1223,7 @@ public class MManager {
    *
    * @return key-> storageGroupName, value->ttl
    */
+  @Override
   public Map<String, Long> getStorageGroupsTTL() {
     Map<String, Long> storageGroupsTTL = new HashMap<>();
     try {
@@ -1210,6 +1245,7 @@ public class MManager {
    * @param path   timeseries
    * @param offset offset in the tag file
    */
+  @Override
   public void changeOffset(String path, long offset) throws MetadataException {
     lock.writeLock().lock();
     try {
@@ -1219,6 +1255,7 @@ public class MManager {
     }
   }
 
+  @Override
   public void changeAlias(String path, String alias) throws MetadataException {
     lock.writeLock().lock();
     try {
@@ -1242,6 +1279,7 @@ public class MManager {
    * @param attributesMap newly added attributes map
    * @param fullPath      timeseries
    */
+  @Override
   public void upsertTagsAndAttributes(String alias, Map<String, String> tagsMap,
       Map<String, String> attributesMap, String fullPath) throws MetadataException, IOException {
     lock.writeLock().lock();
@@ -1345,6 +1383,7 @@ public class MManager {
    * @param attributesMap newly added attributes map
    * @param fullPath      timeseries
    */
+  @Override
   public void addAttributes(Map<String, String> attributesMap, String fullPath)
       throws MetadataException, IOException {
     lock.writeLock().lock();
@@ -1388,6 +1427,7 @@ public class MManager {
    * @param tagsMap  newly added tags map
    * @param fullPath timeseries
    */
+  @Override
   public void addTags(Map<String, String> tagsMap, String fullPath)
       throws MetadataException, IOException {
     lock.writeLock().lock();
@@ -1441,6 +1481,7 @@ public class MManager {
    * @param keySet   tags key or attributes key
    * @param fullPath timeseries path
    */
+  @Override
   public void dropTagsOrAttributes(Set<String> keySet, String fullPath)
       throws MetadataException, IOException {
     lock.writeLock().lock();
@@ -1512,6 +1553,7 @@ public class MManager {
    * @param alterMap the new tags or attributes key-value
    * @param fullPath timeseries
    */
+  @Override
   public void setTagsOrAttributesValue(Map<String, String> alterMap, String fullPath)
       throws MetadataException, IOException {
     lock.writeLock().lock();
@@ -1591,6 +1633,7 @@ public class MManager {
    * @param newKey   new key of tag or attribute
    * @param fullPath timeseries
    */
+  @Override
   public void renameTagOrAttributeKey(String oldKey, String newKey, String fullPath)
       throws MetadataException, IOException {
     lock.writeLock().lock();
@@ -1687,6 +1730,7 @@ public class MManager {
     }
   }
 
+  @Override
   public void collectTimeseriesSchema(MNode startingNode,
       Collection<TimeseriesSchema> timeseriesSchemas) {
     Deque<MNode> nodeDeque = new ArrayDeque<>();
@@ -1703,6 +1747,7 @@ public class MManager {
     }
   }
 
+  @Override
   public void collectMeasurementSchema(MNode startingNode,
       Collection<MeasurementSchema> timeseriesSchemas) {
     Deque<MNode> nodeDeque = new ArrayDeque<>();
@@ -1725,6 +1770,7 @@ public class MManager {
    * @param startingPath
    * @param measurementSchemas
    */
+  @Override
   public void collectSeries(String startingPath, List<MeasurementSchema> measurementSchemas) {
     MNode mNode;
     try {
@@ -1759,6 +1805,7 @@ public class MManager {
    * @param path can be a prefix or a full path.
    * @return StorageGroupName-FullPath pairs
    */
+  @Override
   public Map<String, String> determineStorageGroup(String path) throws IllegalPathException {
     lock.readLock().lock();
     try {
@@ -1782,10 +1829,12 @@ public class MManager {
    * if the path is in local mtree, nothing needed to do (because mtree is in the memory); Otherwise
    * cache the path to mRemoteSchemaCache
    */
+  @Override
   public void cacheMeta(String path, MeasurementMeta meta) {
     // do nothing
   }
 
+  @Override
   public void updateLastCache(String seriesPath, TimeValuePair timeValuePair,
       boolean highPriorityUpdate, Long latestFlushedTime,
       MeasurementMNode node) {
@@ -1801,6 +1850,7 @@ public class MManager {
     }
   }
 
+  @Override
   public TimeValuePair getLastCache(String seriesPath) {
     try {
       MeasurementMNode node = (MeasurementMNode) mtree.getNodeByPath(seriesPath);
@@ -1833,6 +1883,7 @@ public class MManager {
     }
   }
 
+  @Override
   public void createMTreeSnapshot() {
     lock.readLock().lock();
     long time = System.currentTimeMillis();
@@ -1868,6 +1919,7 @@ public class MManager {
    *
    * @throws MetadataException
    */
+  @Override
   public MeasurementSchema[] getSeriesSchemasAndReadLockDevice(String deviceId,
       String[] measurementList, InsertPlan plan) throws MetadataException {
     MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
@@ -2001,6 +2053,7 @@ public class MManager {
    *
    * @param deviceId
    */
+  @Override
   public void unlockDeviceReadLock(String deviceId) {
     try {
       MNode mNode = getDeviceNode(deviceId);
diff --git a/server/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java b/server/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java
index a602663..f43aa7a 100644
--- a/server/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java
@@ -28,7 +28,7 @@ import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.monitor.MonitorConstants.FileNodeManagerStatConstants;
 import org.apache.iotdb.db.monitor.MonitorConstants.FileNodeProcessorStatConstants;
 import org.apache.iotdb.db.monitor.collector.FileSize;
@@ -77,7 +77,7 @@ public class StatMonitor implements IService {
 
   private StatMonitor() {
     initTemporaryStatList();
-    MManager mmanager = IoTDB.metaManager;
+    ISchemaManager mmanager = IoTDB.metaManager;
     statisticMap = new HashMap<>();
     IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
     statMonitorDetectFreqSec = config.getStatMonitorDetectFreqSec();
@@ -140,11 +140,11 @@ public class StatMonitor implements IService {
   }
 
   void registerStatStorageGroup() {
-    MManager mManager = IoTDB.metaManager;
+    ISchemaManager ISchemaManager = IoTDB.metaManager;
     String prefix = MonitorConstants.STAT_STORAGE_GROUP_PREFIX;
     try {
-      if (!mManager.isPathExist(prefix)) {
-        mManager.setStorageGroup(prefix);
+      if (!ISchemaManager.isPathExist(prefix)) {
+        ISchemaManager.setStorageGroup(prefix);
       }
     } catch (Exception e) {
       logger.error("MManager cannot set storage group to MTree.", e);
@@ -157,15 +157,15 @@ public class StatMonitor implements IService {
    * @param hashMap series path and data type pair, for example: [root.stat.file.size.DATA, INT64]
    */
   public synchronized void registerStatStorageGroup(Map<String, String> hashMap) {
-    MManager mManager = IoTDB.metaManager;
+    ISchemaManager ISchemaManager = IoTDB.metaManager;
     try {
       for (Map.Entry<String, String> entry : hashMap.entrySet()) {
         if (entry.getValue() == null) {
           logger.error("Registering metadata but data type of {} is null", entry.getKey());
         }
 
-        if (!mManager.isPathExist(entry.getKey())) {
-          mManager.createTimeseries(entry.getKey(), TSDataType.valueOf(entry.getValue()),
+        if (!ISchemaManager.isPathExist(entry.getKey())) {
+          ISchemaManager.createTimeseries(entry.getKey(), TSDataType.valueOf(entry.getValue()),
               TSEncoding.valueOf("RLE"),
               TSFileDescriptor.getInstance().getConfig().getCompressor(),
               Collections.emptyMap());
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 7420dcd..c1f93fa 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -73,7 +73,7 @@ import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
@@ -150,13 +150,13 @@ public class PlanExecutor implements IPlanExecutor {
   // for data query
   protected IQueryRouter queryRouter;
   // for system schema
-  private MManager mManager;
+  private ISchemaManager ISchemaManager;
   // for administration
   private IAuthorizer authorizer;
 
   public PlanExecutor() throws QueryProcessException {
     queryRouter = new QueryRouter();
-    mManager = IoTDB.metaManager;
+    ISchemaManager = IoTDB.metaManager;
     try {
       authorizer = BasicAuthorizer.getInstance();
     } catch (AuthException e) {
@@ -287,7 +287,7 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   private void operateCreateSnapshot() {
-    mManager.createMTreeSnapshot();
+    ISchemaManager.createMTreeSnapshot();
   }
 
   private void operateTracing(TracingPlan plan) {
@@ -754,7 +754,7 @@ public class PlanExecutor implements IPlanExecutor {
       String device = chunkGroupMetadata.getDevice();
       MNode node = null;
       try {
-        node = mManager.getDeviceNodeWithAutoCreateAndReadLock(device, true, sgLevel);
+        node = ISchemaManager.getDeviceNodeWithAutoCreateAndReadLock(device, true, sgLevel);
         for (ChunkMetadata chunkMetadata : chunkGroupMetadata.getChunkMetadataList()) {
           Path series = new Path(chunkGroupMetadata.getDevice(), chunkMetadata.getMeasurementUid());
           if (!registeredSeries.contains(series)) {
@@ -767,7 +767,7 @@ public class PlanExecutor implements IPlanExecutor {
                       chunkMetadata.getMeasurementUid()));
             }
             if (!node.hasChild(chunkMetadata.getMeasurementUid())) {
-              mManager.createTimeseries(
+              ISchemaManager.createTimeseries(
                   series.getFullPath(),
                   schema.getType(),
                   schema.getEncodingType(),
@@ -839,11 +839,11 @@ public class PlanExecutor implements IPlanExecutor {
     String deviceId = path.getDevice();
     String measurementId = path.getMeasurement();
     try {
-      if (!mManager.isPathExist(path.getFullPath())) {
+      if (!ISchemaManager.isPathExist(path.getFullPath())) {
         throw new QueryProcessException(
             String.format("Time series %s does not exist.", path.getFullPath()));
       }
-      mManager.getStorageGroupName(path.getFullPath());
+      ISchemaManager.getStorageGroupName(path.getFullPath());
       StorageEngine.getInstance().delete(deviceId, measurementId, startTime, endTime);
     } catch (MetadataException | StorageEngineException e) {
       throw new QueryProcessException(e);
@@ -852,7 +852,7 @@ public class PlanExecutor implements IPlanExecutor {
 
   protected MeasurementSchema[] getSeriesSchemas(InsertPlan insertPlan)
       throws MetadataException {
-    return mManager
+    return ISchemaManager
         .getSeriesSchemasAndReadLockDevice(insertPlan.getDeviceId(), insertPlan.getMeasurements(),
             insertPlan);
   }
@@ -870,7 +870,7 @@ public class PlanExecutor implements IPlanExecutor {
     } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
     } finally {
-      mManager.unlockDeviceReadLock(insertRowPlan.getDeviceId());
+      ISchemaManager.unlockDeviceReadLock(insertRowPlan.getDeviceId());
     }
   }
 
@@ -887,7 +887,7 @@ public class PlanExecutor implements IPlanExecutor {
     } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
     } finally {
-      mManager.unlockDeviceReadLock(insertTabletPlan.getDeviceId());
+      ISchemaManager.unlockDeviceReadLock(insertTabletPlan.getDeviceId());
     }
   }
 
@@ -966,7 +966,7 @@ public class PlanExecutor implements IPlanExecutor {
   private boolean createTimeSeries(CreateTimeSeriesPlan createTimeSeriesPlan)
       throws QueryProcessException {
     try {
-      mManager.createTimeseries(createTimeSeriesPlan);
+      ISchemaManager.createTimeseries(createTimeSeriesPlan);
     } catch (MetadataException e) {
       throw new QueryProcessException(e);
     }
@@ -980,7 +980,7 @@ public class PlanExecutor implements IPlanExecutor {
       deleteDataOfTimeSeries(deletePathList);
       List<String> failedNames = new LinkedList<>();
       for (Path path : deletePathList) {
-        String failedTimeseries = mManager.deleteTimeseries(path.toString());
+        String failedTimeseries = ISchemaManager.deleteTimeseries(path.toString());
         if (!failedTimeseries.isEmpty()) {
           failedNames.add(failedTimeseries);
         }
@@ -1003,22 +1003,22 @@ public class PlanExecutor implements IPlanExecutor {
         case RENAME:
           String beforeName = alterMap.keySet().iterator().next();
           String currentName = alterMap.get(beforeName);
-          mManager.renameTagOrAttributeKey(beforeName, currentName, path.getFullPath());
+          ISchemaManager.renameTagOrAttributeKey(beforeName, currentName, path.getFullPath());
           break;
         case SET:
-          mManager.setTagsOrAttributesValue(alterMap, path.getFullPath());
+          ISchemaManager.setTagsOrAttributesValue(alterMap, path.getFullPath());
           break;
         case DROP:
-          mManager.dropTagsOrAttributes(alterMap.keySet(), path.getFullPath());
+          ISchemaManager.dropTagsOrAttributes(alterMap.keySet(), path.getFullPath());
           break;
         case ADD_TAGS:
-          mManager.addTags(alterMap, path.getFullPath());
+          ISchemaManager.addTags(alterMap, path.getFullPath());
           break;
         case ADD_ATTRIBUTES:
-          mManager.addAttributes(alterMap, path.getFullPath());
+          ISchemaManager.addAttributes(alterMap, path.getFullPath());
           break;
         case UPSERT:
-          mManager.upsertTagsAndAttributes(alterTimeSeriesPlan.getAlias(),
+          ISchemaManager.upsertTagsAndAttributes(alterTimeSeriesPlan.getAlias(),
               alterTimeSeriesPlan.getTagsMap(), alterTimeSeriesPlan.getAttributesMap(),
               path.getFullPath());
           break;
@@ -1037,7 +1037,7 @@ public class PlanExecutor implements IPlanExecutor {
       throws QueryProcessException {
     Path path = setStorageGroupPlan.getPath();
     try {
-      mManager.setStorageGroup(path.getFullPath());
+      ISchemaManager.setStorageGroup(path.getFullPath());
     } catch (MetadataException e) {
       throw new QueryProcessException(e);
     }
@@ -1052,7 +1052,7 @@ public class PlanExecutor implements IPlanExecutor {
         StorageEngine.getInstance().deleteStorageGroup(storageGroupPath.getFullPath());
         deletePathList.add(storageGroupPath.getFullPath());
       }
-      mManager.deleteStorageGroups(deletePathList);
+      ISchemaManager.deleteStorageGroups(deletePathList);
     } catch (MetadataException e) {
       throw new QueryProcessException(e);
     }
@@ -1267,7 +1267,7 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   protected String deleteTimeSeries(String path) throws MetadataException {
-    return mManager.deleteTimeseries(path);
+    return ISchemaManager.deleteTimeseries(path);
   }
 
   @SuppressWarnings("unused") // for the distributed version
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/MetadataManagerHelper.java b/server/src/test/java/org/apache/iotdb/db/engine/MetadataManagerHelper.java
index ac5c489..d98cbb4 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/MetadataManagerHelper.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/MetadataManagerHelper.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.engine;
 
-import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -30,7 +30,7 @@ import java.util.Collections;
 public class MetadataManagerHelper {
 
   public static void initMetadata() {
-    MManager mmanager = IoTDB.metaManager;
+    ISchemaManager mmanager = IoTDB.metaManager;
     mmanager.init();
     try {
       mmanager.setStorageGroup("root.vehicle.d0");
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
index edb05fe..3fbe805 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
@@ -21,8 +21,8 @@ package org.apache.iotdb.db.engine.storagegroup;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.db.utils.RandomNum;
@@ -67,7 +67,7 @@ public class FileNodeManagerBenchmark {
 
   private static void prepare()
       throws MetadataException {
-    MManager manager = IoTDB.metaManager;
+    ISchemaManager manager = IoTDB.metaManager;
     manager.setStorageGroup(prefix);
     for (String device : devices) {
       for (String measurement : measurements) {
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerAdvancedTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerAdvancedTest.java
index 93fc3d0..bdb9412 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerAdvancedTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerAdvancedTest.java
@@ -41,7 +41,7 @@ import static org.junit.Assert.fail;
 
 public class MManagerAdvancedTest {
 
-  private static MManager mmanager = null;
+  private static ISchemaManager mmanager = null;
 
   @Before
   public void setUp() throws Exception {
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
index ea00cf1..4839020 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
@@ -275,7 +275,7 @@ public class MManagerBasicTest {
 
   @Test
   public void testMaximalSeriesNumberAmongStorageGroup() throws MetadataException {
-    MManager manager = IoTDB.metaManager;
+    ISchemaManager manager = IoTDB.metaManager;
     assertEquals(0, manager.getMaximalSeriesNumberAmongStorageGroups());
     manager.setStorageGroup("root.laptop");
     assertEquals(0, manager.getMaximalSeriesNumberAmongStorageGroups());
@@ -329,7 +329,7 @@ public class MManagerBasicTest {
 
   @Test
   public void testGetDevicesWithGivenPrefix() {
-    MManager manager = IoTDB.metaManager;
+    ISchemaManager manager = IoTDB.metaManager;
 
     try {
       manager.setStorageGroup("root.laptop");
@@ -356,7 +356,7 @@ public class MManagerBasicTest {
 
   @Test
   public void testGetChildNodePathInNextLevel() {
-    MManager manager = IoTDB.metaManager;
+    ISchemaManager manager = IoTDB.metaManager;
     String[] res = new String[]{
         "[root.laptop, root.vehicle]",
         "[root.laptop.b1, root.laptop.b2]",
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
index dc6172f..86f77c8 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
@@ -44,18 +44,18 @@ public class MManagerImproveTest {
 
   private static final int TIMESERIES_NUM = 1000;
   private static final int DEVICE_NUM = 10;
-  private static MManager mManager = null;
+  private static ISchemaManager ISchemaManager = null;
 
   @Before
   public void setUp() throws Exception {
     EnvironmentUtils.envSetUp();
-    mManager = IoTDB.metaManager;
-    mManager.setStorageGroup("root.t1.v2");
+    ISchemaManager = IoTDB.metaManager;
+    ISchemaManager.setStorageGroup("root.t1.v2");
 
     for (int j = 0; j < DEVICE_NUM; j++) {
       for (int i = 0; i < TIMESERIES_NUM; i++) {
         String p = "root.t1.v2.d" + j + ".s" + i;
-        mManager.createTimeseries(p, TSDataType.TEXT, TSEncoding.RLE,
+        ISchemaManager.createTimeseries(p, TSDataType.TEXT, TSEncoding.RLE,
             TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap());
       }
     }
@@ -65,16 +65,16 @@ public class MManagerImproveTest {
 
   @Test
   public void checkSetUp() {
-    mManager = IoTDB.metaManager;
+    ISchemaManager = IoTDB.metaManager;
 
-    assertTrue(mManager.isPathExist("root.t1.v2.d3.s5"));
-    assertFalse(mManager.isPathExist("root.t1.v2.d9.s" + TIMESERIES_NUM));
-    assertFalse(mManager.isPathExist("root.t10"));
+    assertTrue(ISchemaManager.isPathExist("root.t1.v2.d3.s5"));
+    assertFalse(ISchemaManager.isPathExist("root.t1.v2.d9.s" + TIMESERIES_NUM));
+    assertFalse(ISchemaManager.isPathExist("root.t10"));
   }
 
   @Test
   public void analyseTimeCost() throws MetadataException {
-    mManager = IoTDB.metaManager;
+    ISchemaManager = IoTDB.metaManager;
 
     long startTime, endTime;
     long string_combine, path_exist, list_init, check_filelevel, get_seriestype;
@@ -86,7 +86,7 @@ public class MManagerImproveTest {
 
     startTime = System.currentTimeMillis();
     for (int i = 0; i < 100000; i++) {
-      assertTrue(mManager.isPathExist(path));
+      assertTrue(ISchemaManager.isPathExist(path));
     }
     endTime = System.currentTimeMillis();
     path_exist += endTime - startTime;
@@ -97,7 +97,7 @@ public class MManagerImproveTest {
 
     startTime = System.currentTimeMillis();
     for (int i = 0; i < 100000; i++) {
-      TSDataType dataType = mManager.getSeriesType(path);
+      TSDataType dataType = ISchemaManager.getSeriesType(path);
       assertEquals(TSDataType.TEXT, dataType);
     }
     endTime = System.currentTimeMillis();
@@ -114,8 +114,8 @@ public class MManagerImproveTest {
       throws MetadataException {
     for (String measurement : measurementList) {
       String path = deviceId + "." + measurement;
-      assertTrue(mManager.isPathExist(path));
-      TSDataType dataType = mManager.getSeriesType(path);
+      assertTrue(ISchemaManager.isPathExist(path));
+      TSDataType dataType = ISchemaManager.getSeriesType(path);
       assertEquals(TSDataType.TEXT, dataType);
     }
   }
@@ -124,7 +124,7 @@ public class MManagerImproveTest {
       throws MetadataException {
     for (String measurement : measurementList) {
       String path = deviceId + "." + measurement;
-      TSDataType dataType = mManager.getSeriesType(path);
+      TSDataType dataType = ISchemaManager.getSeriesType(path);
       assertEquals(TSDataType.TEXT, dataType);
     }
   }
@@ -132,7 +132,7 @@ public class MManagerImproveTest {
   private void doCacheTest(String deviceId, List<String> measurementList) throws MetadataException {
     MNode node = null;
     try {
-      node = mManager.getDeviceNodeWithAutoCreateAndReadLock(deviceId);
+      node = ISchemaManager.getDeviceNodeWithAutoCreateAndReadLock(deviceId);
       for (String s : measurementList) {
         assertTrue(node.hasChild(s));
         MeasurementMNode measurementNode = (MeasurementMNode) node.getChild(s);
@@ -148,7 +148,7 @@ public class MManagerImproveTest {
 
   @Test
   public void improveTest() throws MetadataException {
-    mManager = IoTDB.metaManager;
+    ISchemaManager = IoTDB.metaManager;
 
     String[] deviceIdList = new String[DEVICE_NUM];
     for (int i = 0; i < DEVICE_NUM; i++) {
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java b/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
index 763b90c..db344d7 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
@@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.Collections;
 import org.antlr.v4.runtime.misc.ParseCancellationException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
@@ -42,7 +42,7 @@ public class PlannerTest {
 
   private CompressionType compressionType =
       TSFileDescriptor.getInstance().getConfig().getCompressor();
-  private MManager mManager = IoTDB.metaManager;
+  private ISchemaManager ISchemaManager = IoTDB.metaManager;
   private Planner processor = new Planner();
 
   static {
@@ -52,42 +52,42 @@ public class PlannerTest {
   @Before
   public void setUp() throws Exception {
     EnvironmentUtils.envSetUp();
-    mManager.setStorageGroup("root.vehicle");
-    mManager.setStorageGroup("root.vehicle1");
-    mManager.createTimeseries("root.vehicle.device1.sensor1", TSDataType.valueOf("INT32"),
+    ISchemaManager.setStorageGroup("root.vehicle");
+    ISchemaManager.setStorageGroup("root.vehicle1");
+    ISchemaManager.createTimeseries("root.vehicle.device1.sensor1", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle.device1.sensor2", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle.device1.sensor2", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle.device1.sensor3", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle.device1.sensor3", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle.device2.sensor1", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle.device2.sensor1", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle.device2.sensor2", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle.device2.sensor2", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle.device2.sensor3", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle.device2.sensor3", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle1.device1.sensor1", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle1.device1.sensor1", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle1.device1.sensor2", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle1.device1.sensor2", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle1.device1.sensor3", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle1.device1.sensor3", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle1.device2.sensor1", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle1.device2.sensor1", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle1.device2.sensor2", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle1.device2.sensor2", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
-    mManager.createTimeseries("root.vehicle1.device2.sensor3", TSDataType.valueOf("INT32"),
+    ISchemaManager.createTimeseries("root.vehicle1.device2.sensor3", TSDataType.valueOf("INT32"),
         TSEncoding.valueOf("RLE"), compressionType, Collections
             .emptyMap());
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderTest.java b/server/src/test/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderTest.java
index 16ed72d..60a77e5 100644
--- a/server/src/test/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderTest.java
@@ -28,7 +28,7 @@ import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.sync.conf.SyncConstant;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -63,7 +63,7 @@ public class FileLoaderTest {
   }
 
   private void initMetadata() throws MetadataException {
-    MManager mmanager = IoTDB.metaManager;
+    ISchemaManager mmanager = IoTDB.metaManager;
     mmanager.init();
     mmanager.setStorageGroup("root.sg0");
     mmanager.setStorageGroup("root.sg1");
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzerTest.java
index c9fea93..345fca3 100644
--- a/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzerTest.java
@@ -27,7 +27,7 @@ import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.sync.conf.SyncConstant;
 import org.apache.iotdb.db.sync.receiver.load.FileLoader;
@@ -70,7 +70,7 @@ public class SyncReceiverLogAnalyzerTest {
   }
 
   private void initMetadata() throws MetadataException {
-    MManager mmanager = IoTDB.metaManager;
+    ISchemaManager mmanager = IoTDB.metaManager;
     mmanager.init();
     mmanager.setStorageGroup("root.sg0");
     mmanager.setStorageGroup("root.sg1");
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/DeviceStringTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/DeviceStringTest.java
index fa85ca3..082737f 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/DeviceStringTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/DeviceStringTest.java
@@ -25,7 +25,7 @@ import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.ISchemaManager;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -55,7 +55,7 @@ public class DeviceStringTest {
   private String logNodePrefix = TestConstant.OUTPUT_DATA_DIR.concat("testNode/0");
   private Schema schema;
   private TsFileResource resource;
-  private MManager mManager = IoTDB.metaManager;
+  private ISchemaManager ISchemaManager = IoTDB.metaManager;
 
   @Before
   public void setup() throws IOException, WriteProcessException, MetadataException {
@@ -66,16 +66,16 @@ public class DeviceStringTest {
     schema = new Schema();
     schema.registerTimeseries(new Path(("root.sg.device99"), ("sensor4")),
         new MeasurementSchema("sensor4", TSDataType.INT64, TSEncoding.PLAIN));
-    mManager.createTimeseries("root.sg.device99.sensor4", TSDataType.INT64, TSEncoding.PLAIN,
+    ISchemaManager.createTimeseries("root.sg.device99.sensor4", TSDataType.INT64, TSEncoding.PLAIN,
             TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap());
     schema.registerTimeseries(new Path(("root.sg.device99"), ("sensor2")),
         new MeasurementSchema("sensor2", TSDataType.INT64, TSEncoding.PLAIN));
-    mManager
+    ISchemaManager
         .createTimeseries("root.sg.device99.sensor2", TSDataType.INT64, TSEncoding.PLAIN,
             TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap());
     schema.registerTimeseries(new Path(("root.sg.device99"), ("sensor1")),
         new MeasurementSchema("sensor1", TSDataType.INT64, TSEncoding.PLAIN));
-    mManager
+    ISchemaManager
         .createTimeseries("root.sg.device99.sensor1", TSDataType.INT64, TSEncoding.PLAIN,
             TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap());
     writer = new TsFileWriter(tsF, schema);
@@ -109,7 +109,7 @@ public class DeviceStringTest {
     resource.deserialize();
     assertTrue(!resource.getDeviceToIndexMap().keySet().isEmpty());
     for (String device : resource.getDeviceToIndexMap().keySet()) {
-      assertTrue(device == mManager.getDeviceId(device));
+      assertTrue(device == ISchemaManager.getDeviceId(device));
     }
   }
 }
\ No newline at end of file