You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2021/09/26 11:03:39 UTC

[iotdb] branch master updated: [IOTDB-1026] Support wildcard ** in Path And Replace PrefixPath usage with PathPattern in IOTDB-SQL (#3918)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new bc3b736  [IOTDB-1026] Support wildcard ** in Path And Replace PrefixPath usage with PathPattern in IOTDB-SQL (#3918)
bc3b736 is described below

commit bc3b736834b364ce4fddeed3780de8655ec33e03
Author: Marcos_Zyk <38...@users.noreply.github.com>
AuthorDate: Sun Sep 26 19:02:48 2021 +0800

    [IOTDB-1026] Support wildcard ** in Path And Replace PrefixPath usage with PathPattern in IOTDB-SQL (#3918)
---
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |    2 +-
 .../iotdb/cluster/coordinator/Coordinator.java     |    2 +-
 .../cluster/log/applier/AsyncDataLogApplier.java   |    8 +-
 .../iotdb/cluster/log/applier/DataLogApplier.java  |    6 +-
 .../log/manage/PartitionedSnapshotLogManager.java  |   11 +-
 .../apache/iotdb/cluster/metadata/CMManager.java   |   79 +-
 .../iotdb/cluster/partition/PartitionTable.java    |    4 +-
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |   19 +-
 .../iotdb/cluster/query/ClusterPlanRouter.java     |   15 +-
 .../iotdb/cluster/query/LocalQueryExecutor.java    |    2 +-
 .../cluster/server/member/MetaGroupMember.java     |    2 +-
 .../apache/iotdb/cluster/utils/ClusterUtils.java   |    4 +-
 .../cluster/server/member/DataGroupMemberTest.java |    2 +-
 .../tests/tools/importCsv/ImportCsvTestIT.java     |    8 +-
 docs/SystemDesign/SchemaManager/SchemaManager.md   |    2 +-
 docs/UserGuide/Appendix/SQL-Reference.md           |  121 +-
 .../Data-Concept/Data-Model-and-Terminology.md     |   23 +-
 .../DDL-Data-Definition-Language.md                |   69 +-
 .../DML-Data-Manipulation-Language.md              |    2 +-
 .../zh/SystemDesign/SchemaManager/SchemaManager.md |    2 +-
 docs/zh/UserGuide/Appendix/SQL-Reference.md        |  127 +-
 .../Data-Concept/Data-Model-and-Terminology.md     |   28 +-
 .../DDL-Data-Definition-Language.md                |   72 +-
 .../DML-Data-Manipulation-Language.md              |   17 +-
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |    3 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |    6 +-
 .../iotdb/db/engine/merge/task/MergeTask.java      |    3 +-
 .../engine/storagegroup/StorageGroupProcessor.java |    2 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |  986 +++++-----
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 1963 --------------------
 .../org/apache/iotdb/db/metadata/PartialPath.java  |   47 +-
 .../org/apache/iotdb/db/metadata/mnode/IMNode.java |    3 -
 .../iotdb/db/metadata/mnode/InternalMNode.java     |   24 -
 .../iotdb/db/metadata/mnode/MeasurementMNode.java  |    6 -
 .../org/apache/iotdb/db/metadata/mtree/MTree.java  | 1280 +++++++++++++
 .../mtree/traverser/PathGrouperByStorageGroup.java |  103 +
 .../db/metadata/mtree/traverser/Traverser.java     |  235 +++
 .../traverser/collector/CollectorTraverser.java    |   93 +
 .../traverser/collector/EntityPathCollector.java   |   65 +
 .../mtree/traverser/collector/MNodeCollector.java  |   76 +
 .../traverser/collector/MeasurementCollector.java  |  134 ++
 .../collector/MeasurementPathCollector.java        |   60 +
 .../collector/MeasurementSchemaCollector.java      |  114 ++
 .../collector/StorageGroupPathCollector.java       |   65 +
 .../traverser/collector/TSEntityPathCollector.java |   84 +
 .../mtree/traverser/counter/CounterTraverser.java  |   38 +
 .../mtree/traverser/counter/EntityCounter.java     |   44 +
 .../mtree/traverser/counter/MNodeLevelCounter.java |   51 +
 .../traverser/counter/MeasurementCounter.java      |   79 +
 .../traverser/counter/StorageGroupCounter.java     |   46 +
 .../apache/iotdb/db/metadata/utils/MetaUtils.java  |    4 +-
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |    2 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |   14 +-
 .../iotdb/db/qp/logical/crud/QueryOperator.java    |    2 +-
 .../iotdb/db/query/dataset/ShowDevicesDataSet.java |    3 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |    2 +-
 .../db/tools/virtualsg/DeviceMappingViewer.java    |    2 +-
 .../org/apache/iotdb/db/utils/FilePathUtils.java   |    4 +-
 .../iotdb/db/writelog/recover/LogReplayer.java     |    2 +-
 .../iotdb/db/integration/IoTDBAddSubDeviceIT.java  |  249 ---
 .../iotdb/db/integration/IoTDBAlignByDeviceIT.java |    9 +-
 .../db/integration/IoTDBAutoCreateSchemaIT.java    |   50 -
 .../apache/iotdb/db/integration/IoTDBCloseIT.java  |    2 +-
 .../iotdb/db/integration/IoTDBCompleteIT.java      |   26 +-
 .../apache/iotdb/db/integration/IoTDBDaemonIT.java |    2 +-
 .../db/integration/IoTDBDeleteStorageGroupIT.java  |    2 +-
 .../db/integration/IoTDBDeleteTimeseriesIT.java    |    4 +-
 .../iotdb/db/integration/IoTDBDeletionIT.java      |    6 +-
 .../iotdb/db/integration/IoTDBDisableAlignIT.java  |    6 +-
 .../iotdb/db/integration/IoTDBExecuteBatchIT.java  |    4 +-
 .../db/integration/IoTDBFloatPrecisionIT.java      |    4 +-
 .../db/integration/IoTDBFlushQueryMergeIT.java     |    2 +-
 .../db/integration/IoTDBInsertWithQueryIT.java     |    4 +-
 .../iotdb/db/integration/IoTDBLargeDataIT.java     |    4 +-
 .../apache/iotdb/db/integration/IoTDBLastIT.java   |    2 +-
 .../db/integration/IoTDBLoadExternalTsfileIT.java  |    4 +-
 .../iotdb/db/integration/IoTDBMetadataFetchIT.java |   10 +-
 .../iotdb/db/integration/IoTDBMultiDeviceIT.java   |   14 +-
 .../iotdb/db/integration/IoTDBMultiSeriesIT.java   |    7 +-
 .../db/integration/IoTDBMultiStatementsIT.java     |    3 +-
 .../iotdb/db/integration/IoTDBQueryDemoIT.java     |   13 +-
 .../db/integration/IoTDBQueryMemoryControlIT.java  |    6 +-
 .../iotdb/db/integration/IoTDBRestartIT.java       |    4 +-
 .../db/integration/IoTDBRpcCompressionIT.java      |    6 +-
 .../IoTDBSameMeasurementsDifferentTypesIT.java     |    3 +-
 .../iotdb/db/integration/IoTDBSensorUpdateIT.java  |    4 +-
 .../db/integration/IoTDBSequenceDataQueryIT.java   |    2 +-
 .../IoTDBSetSystemReadOnlyWritableIT.java          |    2 +-
 .../iotdb/db/integration/IoTDBSimpleQueryIT.java   |   20 +-
 .../apache/iotdb/db/integration/IoTDBTagIT.java    |    9 +-
 .../iotdb/db/integration/IoTDBTimeZoneIT.java      |    4 +-
 .../apache/iotdb/db/integration/IoTDBTtlIT.java    |    2 +-
 .../aggregation/IoTDBAggregationSmallDataIT.java   |    2 +-
 .../db/integration/auth/IoTDBAuthorizationIT.java  |    6 +-
 .../iotdb/db/metadata/MManagerAdvancedTest.java    |   12 +-
 .../iotdb/db/metadata/MManagerBasicTest.java       |  139 +-
 .../org/apache/iotdb/db/metadata/MTreeTest.java    |  276 ++-
 .../apache/iotdb/db/metadata/PartialPathTest.java  |    8 +
 .../java/org/apache/iotdb/db/qp/PlannerTest.java   |    3 +-
 .../iotdb/db/qp/logical/LogicalPlanSmallTest.java  |    6 +-
 .../db/qp/physical/InsertTabletMultiPlanTest.java  |    3 +-
 .../iotdb/db/qp/physical/InsertTabletPlanTest.java |   15 +-
 .../iotdb/db/qp/physical/PhysicalPlanTest.java     |    2 +-
 .../iotdb/db/query/dataset/SingleDataSetTest.java  |    2 +-
 .../apache/iotdb/db/sink/LocalIoTDBSinkTest.java   |    4 +-
 .../org/apache/iotdb/db/sink/MQTTSinkTest.java     |    4 +-
 .../db/utils/datastructure/PrecisionTest.java      |    8 +-
 .../iotdb/session/IoTDBSessionComplexIT.java       |    6 +-
 .../apache/iotdb/session/IoTDBSessionSimpleIT.java |    6 +-
 .../apache/iotdb/session/IoTDBSessionVectorIT.java |    8 +-
 .../org/apache/iotdb/spark/db/IoTDBTest.scala      |   14 +-
 .../test/java/org/apache/iotdb/db/sql/Cases.java   |    4 +-
 112 files changed, 3950 insertions(+), 3339 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index eff457e..e3e37ad 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -1356,7 +1356,7 @@ LOCK
 //============================
 COMMA : ',';
 
-STAR : '*';
+STAR : '*' | '**';
 
 OPERATOR_EQ : '=' | '==';
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
index b388f76..67deb27 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
@@ -207,7 +207,7 @@ public class Coordinator {
       throws MetadataException, CheckConsistencyException {
     if (plan instanceof SetSchemaTemplatePlan) {
       try {
-        IoTDB.metaManager.getStorageGroupPath(
+        IoTDB.metaManager.getBelongedStorageGroup(
             new PartialPath(((SetSchemaTemplatePlan) plan).getPrefixPath()));
       } catch (IllegalPathException e) {
         // the plan has been checked
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/AsyncDataLogApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/AsyncDataLogApplier.java
index 89a8519..b0a6f0d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/AsyncDataLogApplier.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/AsyncDataLogApplier.java
@@ -154,16 +154,16 @@ public class AsyncDataLogApplier implements LogApplier {
     PartialPath sgPath = null;
     if (plan instanceof InsertMultiTabletPlan) {
       PartialPath deviceId = ((InsertMultiTabletPlan) plan).getFirstDeviceId();
-      sgPath = IoTDB.metaManager.getStorageGroupPath(deviceId);
+      sgPath = IoTDB.metaManager.getBelongedStorageGroup(deviceId);
     } else if (plan instanceof InsertRowsPlan) {
       PartialPath path = ((InsertRowsPlan) plan).getFirstDeviceId();
-      sgPath = IoTDB.metaManager.getStorageGroupPath(path);
+      sgPath = IoTDB.metaManager.getBelongedStorageGroup(path);
     } else if (plan instanceof InsertPlan) {
       PartialPath deviceId = ((InsertPlan) plan).getPrefixPath();
-      sgPath = IoTDB.metaManager.getStorageGroupPath(deviceId);
+      sgPath = IoTDB.metaManager.getBelongedStorageGroup(deviceId);
     } else if (plan instanceof CreateTimeSeriesPlan) {
       PartialPath path = ((CreateTimeSeriesPlan) plan).getPath();
-      sgPath = IoTDB.metaManager.getStorageGroupPath(path);
+      sgPath = IoTDB.metaManager.getBelongedStorageGroup(path);
     }
     return sgPath;
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
index c0ff907..14abaa6 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/DataLogApplier.java
@@ -127,7 +127,7 @@ public class DataLogApplier extends BaseApplier {
     boolean hasSync = false;
     for (InsertTabletPlan insertTabletPlan : plan.getInsertTabletPlanList()) {
       try {
-        IoTDB.metaManager.getStorageGroupPath(insertTabletPlan.getPrefixPath());
+        IoTDB.metaManager.getBelongedStorageGroup(insertTabletPlan.getPrefixPath());
       } catch (StorageGroupNotSetException e) {
         try {
           if (!hasSync) {
@@ -149,7 +149,7 @@ public class DataLogApplier extends BaseApplier {
     boolean hasSync = false;
     for (InsertRowPlan insertRowPlan : plan.getInsertRowPlanList()) {
       try {
-        IoTDB.metaManager.getStorageGroupPath(insertRowPlan.getPrefixPath());
+        IoTDB.metaManager.getBelongedStorageGroup(insertRowPlan.getPrefixPath());
       } catch (StorageGroupNotSetException e) {
         try {
           if (!hasSync) {
@@ -169,7 +169,7 @@ public class DataLogApplier extends BaseApplier {
   private void applyInsert(InsertPlan plan)
       throws StorageGroupNotSetException, QueryProcessException, StorageEngineException {
     try {
-      IoTDB.metaManager.getStorageGroupPath(plan.getPrefixPath());
+      IoTDB.metaManager.getBelongedStorageGroup(plan.getPrefixPath());
     } catch (StorageGroupNotSetException e) {
       // the sg may not exist because the node does not catch up with the leader, retry after
       // synchronization
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
index 04d7c07..20a6944 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/PartitionedSnapshotLogManager.java
@@ -29,8 +29,7 @@ import org.apache.iotdb.cluster.partition.PartitionTable;
 import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.server.member.DataGroupMember;
-import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
 
@@ -97,11 +96,11 @@ public abstract class PartitionedSnapshotLogManager<T extends Snapshot> extends
 
   void collectTimeseriesSchemas(List<Integer> requiredSlots) {
     slotTimeseries.clear();
-    List<IStorageGroupMNode> allSgNodes = IoTDB.metaManager.getAllStorageGroupNodes();
+    List<PartialPath> allSgPaths = IoTDB.metaManager.getAllStorageGroupPaths();
 
     Set<Integer> requiredSlotsSet = new HashSet<Integer>(requiredSlots);
-    for (IMNode sgNode : allSgNodes) {
-      String storageGroupName = sgNode.getFullPath();
+    for (PartialPath sgPath : allSgPaths) {
+      String storageGroupName = sgPath.getFullPath();
       int slot =
           SlotPartitionTable.getSlotStrategy()
               .calculateSlotByTime(storageGroupName, 0, ClusterConstant.SLOT_NUM);
@@ -111,7 +110,7 @@ public abstract class PartitionedSnapshotLogManager<T extends Snapshot> extends
       }
       Collection<TimeseriesSchema> schemas =
           slotTimeseries.computeIfAbsent(slot, s -> new HashSet<>());
-      IoTDB.metaManager.collectTimeseriesSchema(sgNode, schemas);
+      IoTDB.metaManager.collectTimeseriesSchema(sgPath, schemas);
       logger.debug("{}: {} timeseries are snapshot in slot {}", getName(), schemas.size(), slot);
     }
   }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
index e5f056d..e19b1f9 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
@@ -38,6 +38,7 @@ import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.MManager;
@@ -182,25 +183,25 @@ public class CMManager extends MManager {
   }
 
   @Override
-  public TSDataType getSeriesType(PartialPath path) throws MetadataException {
+  public TSDataType getSeriesType(PartialPath fullPath) throws MetadataException {
 
-    if (path.equals(SQLConstant.TIME_PATH)) {
+    if (fullPath.equals(SQLConstant.TIME_PATH)) {
       return TSDataType.INT64;
     }
 
-    String measurement = path.getMeasurement();
-    if (path instanceof VectorPartialPath) {
-      if (((VectorPartialPath) path).getSubSensorsList().size() != 1) {
+    String measurement = fullPath.getMeasurement();
+    if (fullPath instanceof VectorPartialPath) {
+      if (((VectorPartialPath) fullPath).getSubSensorsList().size() != 1) {
         return TSDataType.VECTOR;
       } else {
-        measurement = ((VectorPartialPath) path).getSubSensor(0);
+        measurement = ((VectorPartialPath) fullPath).getSubSensor(0);
       }
     }
 
     // try remote cache first
     try {
       cacheLock.readLock().lock();
-      IMeasurementMNode measurementMNode = mRemoteMetaCache.get(path);
+      IMeasurementMNode measurementMNode = mRemoteMetaCache.get(fullPath);
       if (measurementMNode != null) {
         return measurementMNode.getDataType(measurement);
       }
@@ -211,11 +212,11 @@ public class CMManager extends MManager {
     // try local MTree
     TSDataType seriesType;
     try {
-      seriesType = super.getSeriesType(path);
+      seriesType = super.getSeriesType(fullPath);
     } catch (PathNotExistException e) {
       // pull from remote node
       List<IMeasurementSchema> schemas =
-          metaPuller.pullMeasurementSchemas(Collections.singletonList(path));
+          metaPuller.pullMeasurementSchemas(Collections.singletonList(fullPath));
       if (!schemas.isEmpty()) {
         IMeasurementSchema measurementSchema = schemas.get(0);
         IMeasurementMNode measurementMNode =
@@ -223,16 +224,17 @@ public class CMManager extends MManager {
                 null, measurementSchema.getMeasurementId(), measurementSchema, null);
         if (measurementSchema instanceof VectorMeasurementSchema) {
           for (int i = 0; i < measurementSchema.getSubMeasurementsList().size(); i++) {
-            cacheMeta(((VectorPartialPath) path).getPathWithSubSensor(i), measurementMNode, false);
+            cacheMeta(
+                ((VectorPartialPath) fullPath).getPathWithSubSensor(i), measurementMNode, false);
           }
           cacheMeta(
-              new PartialPath(path.getDevice(), measurementSchema.getMeasurementId()),
+              new PartialPath(fullPath.getDevice(), measurementSchema.getMeasurementId()),
               measurementMNode,
               true);
         } else {
-          cacheMeta(path, measurementMNode, true);
+          cacheMeta(fullPath, measurementMNode, true);
         }
-        return measurementMNode.getDataType(path.getMeasurement());
+        return measurementMNode.getDataType(fullPath.getMeasurement());
       } else {
         throw e;
       }
@@ -948,12 +950,7 @@ public class CMManager extends MManager {
    * @return all paths after removing wildcards in the path
    */
   public Set<PartialPath> getMatchedDevices(PartialPath originPath) throws MetadataException {
-    // get all storage groups this path may belong to
-    // the key is the storage group name and the value is the path to be queried with storage group
-    // added, e.g:
-    // "root.*" will be translated into:
-    // "root.group1" -> "root.group1.*", "root.group2" -> "root.group2.*" ...
-    Map<String, String> sgPathMap = determineStorageGroup(originPath);
+    Map<String, String> sgPathMap = groupPathByStorageGroup(originPath);
     Set<PartialPath> ret = getMatchedDevices(sgPathMap);
     logger.debug("The devices of path {} are {}", originPath, ret);
     return ret;
@@ -1131,7 +1128,7 @@ public class CMManager extends MManager {
         } catch (CheckConsistencyException e) {
           logger.warn("Failed to check consistency.", e);
         }
-        Set<PartialPath> allDevices = getDevices(pathUnderSG);
+        Set<PartialPath> allDevices = getDevicesByPrefix(pathUnderSG);
         logger.debug(
             "{}: get matched paths of {} locally, result {}",
             metaGroupMember.getName(),
@@ -1219,14 +1216,8 @@ public class CMManager extends MManager {
   /** Similar to method getAllTimeseriesPath(), but return Path with alias alias. */
   @Override
   public Pair<List<PartialPath>, Integer> getAllTimeseriesPathWithAlias(
-      PartialPath prefixPath, int limit, int offset) throws MetadataException {
-
-    // get all storage groups this path may belong to
-    // the key is the storage group name and the value is the path to be queried with storage group
-    // added, e.g:
-    // "root.*" will be translated into:
-    // "root.group1" -> "root.group1.*", "root.group2" -> "root.group2.*" ...
-    Map<String, String> sgPathMap = determineStorageGroup(prefixPath);
+      PartialPath pathPattern, int limit, int offset) throws MetadataException {
+    Map<String, String> sgPathMap = groupPathByStorageGroup(pathPattern);
     List<PartialPath> result = getMatchedPaths(sgPathMap, true);
 
     int skippedOffset = 0;
@@ -1241,7 +1232,7 @@ public class CMManager extends MManager {
     if (limit > 0 && result.size() > limit) {
       result = result.subList(0, limit);
     }
-    logger.debug("The paths of path {} are {}", prefixPath, result);
+    logger.debug("The paths of path {} are {}", pathPattern, result);
 
     return new Pair<>(result, skippedOffset);
   }
@@ -1253,12 +1244,7 @@ public class CMManager extends MManager {
    * @return all paths after removing wildcards in the path
    */
   public List<PartialPath> getMatchedPaths(PartialPath originPath) throws MetadataException {
-    // get all storage groups this path may belong to
-    // the key is the storage group name and the value is the path to be queried with storage group
-    // added, e.g:
-    // "root.*" will be translated into:
-    // "root.group1" -> "root.group1.*", "root.group2" -> "root.group2.*" ...
-    Map<String, String> sgPathMap = determineStorageGroup(originPath);
+    Map<String, String> sgPathMap = groupPathByStorageGroup(originPath);
     List<PartialPath> ret = getMatchedPaths(sgPathMap, false);
     logger.debug("The paths of path {} are {}", originPath, ret);
     return ret;
@@ -1327,7 +1313,7 @@ public class CMManager extends MManager {
   public Set<String> getAllDevices(List<String> paths) throws MetadataException {
     Set<String> results = new HashSet<>();
     for (String path : paths) {
-      getDevices(new PartialPath(path)).stream()
+      this.getMatchedDevices(new PartialPath(path)).stream()
           .map(PartialPath::getFullPath)
           .forEach(results::add);
     }
@@ -1342,13 +1328,13 @@ public class CMManager extends MManager {
    * @param nodeLevel
    */
   public List<String> getNodeList(String path, int nodeLevel) throws MetadataException {
-    return getNodesList(new PartialPath(path), nodeLevel).stream()
+    return getNodesListInGivenLevel(new PartialPath(path), nodeLevel).stream()
         .map(PartialPath::getFullPath)
         .collect(Collectors.toList());
   }
 
   public Set<String> getChildNodeInNextLevel(String path) throws MetadataException {
-    return getChildNodeInNextLevel(new PartialPath(path));
+    return getChildNodeNameInNextLevel(new PartialPath(path));
   }
 
   public Set<String> getChildNodePathInNextLevel(String path) throws MetadataException {
@@ -1376,8 +1362,9 @@ public class CMManager extends MManager {
   }
 
   @Override
-  public IMNode getMNode(IMNode deviceMNode, String measurementName) {
-    IMNode child = deviceMNode.getChild(measurementName);
+  protected IMeasurementMNode getMeasurementMNode(IMNode deviceMNode, String measurementName)
+      throws PathAlreadyExistException {
+    IMeasurementMNode child = super.getMeasurementMNode(deviceMNode, measurementName);
     if (child == null) {
       child = mRemoteMetaCache.get(deviceMNode.getPartialPath().concatNode(measurementName));
     }
@@ -1390,11 +1377,11 @@ public class CMManager extends MManager {
   }
 
   public List<ShowDevicesResult> getLocalDevices(ShowDevicesPlan plan) throws MetadataException {
-    return super.getDevices(plan);
+    return super.getMatchedDevices(plan);
   }
 
   @Override
-  public List<ShowDevicesResult> getDevices(ShowDevicesPlan plan) throws MetadataException {
+  public List<ShowDevicesResult> getMatchedDevices(ShowDevicesPlan plan) throws MetadataException {
     ConcurrentSkipListSet<ShowDevicesResult> resultSet = new ConcurrentSkipListSet<>();
     ExecutorService pool =
         new ThreadPoolExecutor(
@@ -1560,7 +1547,7 @@ public class CMManager extends MManager {
         metaGroupMember.getLocalDataMember(group.getHeader(), group.getId());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
-      List<ShowDevicesResult> localResult = super.getDevices(plan);
+      List<ShowDevicesResult> localResult = super.getMatchedDevices(plan);
       resultSet.addAll(localResult);
       logger.debug("Fetched {} devices of {} from {}", localResult.size(), plan.getPath(), group);
     } catch (MetadataException e) {
@@ -1744,16 +1731,16 @@ public class CMManager extends MManager {
   }
 
   @Override
-  public PartialPath getStorageGroupPath(PartialPath path) throws StorageGroupNotSetException {
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
     try {
-      return super.getStorageGroupPath(path);
+      return super.getBelongedStorageGroup(path);
     } catch (StorageGroupNotSetException e) {
       try {
         metaGroupMember.syncLeader(null);
       } catch (CheckConsistencyException ex) {
         logger.warn("Failed to check consistency.", e);
       }
-      return super.getStorageGroupPath(path);
+      return super.getBelongedStorageGroup(path);
     }
   }
 }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
index 98b8b84..4b4129a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/PartitionTable.java
@@ -124,7 +124,7 @@ public interface PartitionTable {
    */
   default PartitionGroup partitionByPathTime(PartialPath path, long timestamp)
       throws MetadataException {
-    PartialPath storageGroup = IoTDB.metaManager.getStorageGroupPath(path);
+    PartialPath storageGroup = IoTDB.metaManager.getBelongedStorageGroup(path);
     return this.route(storageGroup.getFullPath(), timestamp);
   }
 
@@ -138,7 +138,7 @@ public interface PartitionTable {
     long partitionInterval = StorageEngine.getTimePartitionInterval();
 
     MultiKeyMap<Long, PartitionGroup> timeRangeMapRaftGroup = new MultiKeyMap<>();
-    PartialPath storageGroup = IoTDB.metaManager.getStorageGroupPath(path);
+    PartialPath storageGroup = IoTDB.metaManager.getBelongedStorageGroup(path);
     startTime = StorageEngine.convertMilliWithPrecision(startTime);
     endTime = StorageEngine.convertMilliWithPrecision(endTime);
     while (startTime <= endTime) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
index 2dd5043..d36b9aa 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
@@ -134,12 +134,8 @@ public class ClusterPlanExecutor extends PlanExecutor {
     } catch (CheckConsistencyException e) {
       throw new MetadataException(e);
     }
-    // get all storage groups this path may belong to
-    // the key is the storage group name and the value is the path to be queried with storage group
-    // added, e.g:
-    // "root.*" will be translated into:
-    // "root.group1" -> "root.group1.*", "root.group2" -> "root.group2.*" ...
-    Map<String, String> sgPathMap = IoTDB.metaManager.determineStorageGroup(path);
+
+    Map<String, String> sgPathMap = IoTDB.metaManager.groupPathByStorageGroup(path);
     if (sgPathMap.isEmpty()) {
       throw new PathNotExistException(path.getFullPath());
     }
@@ -286,13 +282,6 @@ public class ClusterPlanExecutor extends PlanExecutor {
   }
 
   @Override
-  protected Set<PartialPath> getDevices(PartialPath path) throws MetadataException {
-    // make sure this node knows all storage groups
-    ((CMManager) IoTDB.metaManager).syncMetaLeader();
-    return ((CMManager) IoTDB.metaManager).getMatchedDevices(path);
-  }
-
-  @Override
   protected List<PartialPath> getNodesList(PartialPath schemaPattern, int level)
       throws MetadataException {
 
@@ -334,7 +323,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
     DataGroupMember localDataMember = metaGroupMember.getLocalDataMember(group.getHeader());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
-      return IoTDB.metaManager.getNodesList(
+      return IoTDB.metaManager.getNodesListInGivenLevel(
           schemaPattern,
           level,
           new SlotSgFilter(
@@ -433,7 +422,7 @@ public class ClusterPlanExecutor extends PlanExecutor {
         metaGroupMember.getLocalDataMember(group.getHeader(), group.getId());
     localDataMember.syncLeaderWithConsistencyCheck(false);
     try {
-      return IoTDB.metaManager.getChildNodeInNextLevel(path);
+      return IoTDB.metaManager.getChildNodeNameInNextLevel(path);
     } catch (MetadataException e) {
       logger.error("Cannot not get next children nodes of {} from {} locally", path, group);
       return Collections.emptySet();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
index f34597a..6905019 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
@@ -29,7 +29,6 @@ import org.apache.iotdb.cluster.partition.PartitionTable;
 import org.apache.iotdb.cluster.rpc.thrift.Node;
 import org.apache.iotdb.cluster.utils.PartitionUtils;
 import org.apache.iotdb.db.engine.StorageEngine;
-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;
@@ -111,7 +110,7 @@ public class ClusterPlanRouter {
   private PartitionGroup routePlan(ShowChildPathsPlan plan) {
     try {
       return partitionTable.route(
-          getMManager().getStorageGroupPath(plan.getPath()).getFullPath(), 0);
+          getMManager().getBelongedStorageGroup(plan.getPath()).getFullPath(), 0);
     } catch (MetadataException e) {
       // the path is too short to have no a storage group name, e.g., "root"
       // so we can do it locally.
@@ -223,7 +222,7 @@ public class ClusterPlanRouter {
         InsertTabletPlan tmpPlan = (InsertTabletPlan) entry.getKey();
         PartitionGroup tmpPg = entry.getValue();
         // 1.1 the sg that the plan(actually calculated based on device) belongs to
-        PartialPath tmpSgPath = IoTDB.metaManager.getStorageGroupPath(tmpPlan.getPrefixPath());
+        PartialPath tmpSgPath = IoTDB.metaManager.getBelongedStorageGroup(tmpPlan.getPrefixPath());
         Map<PartialPath, InsertMultiTabletPlan> sgPathPlanMap = pgSgPathPlanMap.get(tmpPg);
         if (sgPathPlanMap == null) {
           // 2.1 construct the InsertMultiTabletPlan
@@ -284,7 +283,7 @@ public class ClusterPlanRouter {
     Map<PartitionGroup, InsertRowsPlan> groupPlanMap = new HashMap<>();
     for (int i = 0; i < insertRowsPlan.getInsertRowPlanList().size(); i++) {
       InsertRowPlan rowPlan = insertRowsPlan.getInsertRowPlanList().get(i);
-      PartialPath storageGroup = getMManager().getStorageGroupPath(rowPlan.getPrefixPath());
+      PartialPath storageGroup = getMManager().getBelongedStorageGroup(rowPlan.getPrefixPath());
       PartitionGroup group = partitionTable.route(storageGroup.getFullPath(), rowPlan.getTime());
       if (groupPlanMap.containsKey(group)) {
         InsertRowsPlan tmpPlan = groupPlanMap.get(group);
@@ -305,7 +304,7 @@ public class ClusterPlanRouter {
   @SuppressWarnings("SuspiciousSystemArraycopy")
   private Map<PhysicalPlan, PartitionGroup> splitAndRoutePlan(InsertTabletPlan plan)
       throws MetadataException {
-    PartialPath storageGroup = getMManager().getStorageGroupPath(plan.getPrefixPath());
+    PartialPath storageGroup = getMManager().getBelongedStorageGroup(plan.getPrefixPath());
     Map<PhysicalPlan, PartitionGroup> result = new HashMap<>();
     long[] times = plan.getTimes();
     if (times.length == 0) {
@@ -398,10 +397,10 @@ public class ClusterPlanRouter {
   }
 
   private Map<PhysicalPlan, PartitionGroup> splitAndRoutePlan(CountPlan plan)
-      throws StorageGroupNotSetException, IllegalPathException {
+      throws MetadataException {
     // CountPlan is quite special because it has the behavior of wildcard at the tail of the path
     // even though there is no wildcard
-    Map<String, String> sgPathMap = getMManager().determineStorageGroup(plan.getPath());
+    Map<String, String> sgPathMap = getMManager().groupPathByStorageGroup(plan.getPath());
     if (sgPathMap.isEmpty()) {
       throw new StorageGroupNotSetException(plan.getPath().getFullPath());
     }
@@ -518,7 +517,7 @@ public class ClusterPlanRouter {
     Map<PhysicalPlan, PartitionGroup> result = new HashMap<>();
     Map<PartitionGroup, List<InsertRowPlan>> groupPlanMap = new HashMap<>();
     Map<PartitionGroup, List<Integer>> groupPlanIndexMap = new HashMap<>();
-    PartialPath storageGroup = getMManager().getStorageGroupPath(plan.getPrefixPath());
+    PartialPath storageGroup = getMManager().getBelongedStorageGroup(plan.getPrefixPath());
     for (int i = 0; i < plan.getRowPlans().length; i++) {
       InsertRowPlan p = plan.getRowPlans()[i];
       PartitionGroup group = partitionTable.route(storageGroup.getFullPath(), p.getTime());
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
index d8b1e96..8365d64 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
@@ -529,7 +529,7 @@ public class LocalQueryExecutor {
             .computeIfAbsent(slotPreviousHolderMap.get(slot), s -> new ArrayList<>())
             .add(prefixPath);
       } else {
-        getCMManager().collectTimeseriesSchema(prefixPath, timeseriesSchemas);
+        getCMManager().collectTimeseriesSchema(new PartialPath(prefixPath), timeseriesSchemas);
       }
     }
 
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
index 3da9dd8..bacacab 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
@@ -1494,7 +1494,7 @@ public class MetaGroupMember extends RaftMember {
     List<PartitionGroup> partitionGroups = new ArrayList<>();
     PartialPath storageGroupName;
     try {
-      storageGroupName = IoTDB.metaManager.getStorageGroupPath(path);
+      storageGroupName = IoTDB.metaManager.getBelongedStorageGroup(path);
     } catch (MetadataException e) {
       throw new StorageEngineException(e);
     }
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/ClusterUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/ClusterUtils.java
index 398eb3d..9b22972 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/utils/ClusterUtils.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/ClusterUtils.java
@@ -372,7 +372,7 @@ public class ClusterUtils {
       PartialPath prefixPath, MetaGroupMember metaGroupMember) throws MetadataException {
     int slot;
     try {
-      PartialPath storageGroup = IoTDB.metaManager.getStorageGroupPath(prefixPath);
+      PartialPath storageGroup = IoTDB.metaManager.getBelongedStorageGroup(prefixPath);
       slot =
           SlotPartitionTable.getSlotStrategy()
               .calculateSlotByPartitionNum(storageGroup.getFullPath(), 0, ClusterConstant.SLOT_NUM);
@@ -384,7 +384,7 @@ public class ClusterUtils {
       } catch (CheckConsistencyException checkConsistencyException) {
         throw new MetadataException(checkConsistencyException.getMessage());
       }
-      PartialPath storageGroup = IoTDB.metaManager.getStorageGroupPath(prefixPath);
+      PartialPath storageGroup = IoTDB.metaManager.getBelongedStorageGroup(prefixPath);
       slot =
           SlotPartitionTable.getSlotStrategy()
               .calculateSlotByPartitionNum(storageGroup.getFullPath(), 0, ClusterConstant.SLOT_NUM);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
index 293549c..a925ada 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
@@ -989,7 +989,7 @@ public class DataGroupMemberTest extends BaseMember {
   @Test
   public void testGetPaths() {
     System.out.println("Start testGetPaths()");
-    String path = TestUtils.getTestSg(0);
+    String path = TestUtils.getTestSg(0) + ".**";
     AtomicReference<GetAllPathsResult> pathResult = new AtomicReference<>();
     GenericHandler<GetAllPathsResult> handler =
         new GenericHandler<>(TestUtils.getNode(0), pathResult);
diff --git a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java
index 29facf7..345de2f 100644
--- a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java
+++ b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java
@@ -180,7 +180,7 @@ public class ImportCsvTestIT extends AbstractScript {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      if (statement.execute("select * from root")) {
+      if (statement.execute("select * from root.**")) {
         ResultSet resultSet = statement.getResultSet();
         testResult(resultSet, 6, 3);
         resultSet.close();
@@ -209,7 +209,7 @@ public class ImportCsvTestIT extends AbstractScript {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      if (statement.execute("select * from root")) {
+      if (statement.execute("select * from root.**")) {
         ResultSet resultSet = statement.getResultSet();
         testResult(resultSet, 6, 3);
         resultSet.close();
@@ -239,7 +239,7 @@ public class ImportCsvTestIT extends AbstractScript {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      if (statement.execute("select * from root")) {
+      if (statement.execute("select * from root.**")) {
         ResultSet resultSet = statement.getResultSet();
         testResult(resultSet, 6, 3);
         resultSet.close();
@@ -269,7 +269,7 @@ public class ImportCsvTestIT extends AbstractScript {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      if (statement.execute("select * from root")) {
+      if (statement.execute("select ** from root")) {
         ResultSet resultSet = statement.getResultSet();
         testResult(resultSet, 6, 3);
         resultSet.close();
diff --git a/docs/SystemDesign/SchemaManager/SchemaManager.md b/docs/SystemDesign/SchemaManager/SchemaManager.md
index 27d8882..1d9269f 100644
--- a/docs/SystemDesign/SchemaManager/SchemaManager.md
+++ b/docs/SystemDesign/SchemaManager/SchemaManager.md
@@ -143,7 +143,7 @@ Same as above, at the beginning of each operation, it will try to obatin the wri
 
 ## MTree
 
-* org.apache.iotdb.db.metadata.MTree
+* org.apache.iotdb.db.metadata.mtree.MTree
 
 There three types of nodes in MTree: StorageGroupMNode态InternalMNode(Non-leaf node)态LeafMNode(leaf node), they all extend to MNode.
 
diff --git a/docs/UserGuide/Appendix/SQL-Reference.md b/docs/UserGuide/Appendix/SQL-Reference.md
index 3c43f20..9c1a0b9 100644
--- a/docs/UserGuide/Appendix/SQL-Reference.md
+++ b/docs/UserGuide/Appendix/SQL-Reference.md
@@ -53,17 +53,17 @@ It costs 0.417s
 ``` SQL
 SET STORAGE GROUP TO <FullPath>
 Eg: IoTDB > SET STORAGE GROUP TO root.ln.wf01.wt01
-Note: FullPath can not include `*`
+Note: FullPath can not include wildcard `*` or `**`
 ```
 
 * Delete Storage Group
 
 ```
-DELETE STORAGE GROUP <FullPath> [COMMA <FullPath>]*
+DELETE STORAGE GROUP <PathPattern> [COMMA <PathPattern>]*
 Eg: IoTDB > DELETE STORAGE GROUP root.ln.wf01.wt01
 Eg: IoTDB > DELETE STORAGE GROUP root.ln.wf01.wt01, root.ln.wf01.wt02
 Eg: IoTDB > DELETE STORAGE GROUP root.ln.wf01.*
-Eg: IoTDB > DELETE STORAGE GROUP root.*
+Eg: IoTDB > DELETE STORAGE GROUP root.**
 ```
 
 * Create Timeseries Statement
@@ -128,7 +128,7 @@ Eg: set schema template temp1 to root.beijing
 * Delete Timeseries Statement
 
 ```
-DELETE TIMESERIES <PrefixPath> [COMMA <PrefixPath>]*
+DELETE TIMESERIES <PathPattern> [COMMA <PathPattern>]*
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.status
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.status, root.ln.wf01.wt01.temperature
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.*
@@ -171,18 +171,18 @@ Note: This statement can only be used in IoTDB Client. If you need to show all t
 
 ```
 SHOW TIMESERIES <Path>
-Eg: IoTDB > SHOW TIMESERIES root
-Eg: IoTDB > SHOW TIMESERIES root.ln
+Eg: IoTDB > SHOW TIMESERIES root.**
+Eg: IoTDB > SHOW TIMESERIES root.ln.**
 Eg: IoTDB > SHOW TIMESERIES root.ln.*.*.status
 Eg: IoTDB > SHOW TIMESERIES root.ln.wf01.wt01.status
-Note: The path can be prefix path, star path or timeseries path
+Note: The path can be timeseries path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
 * Show Specific Timeseries Statement with where clause
 
 ```
-SHOW TIMESERIES prefixPath? showWhereClause?
+SHOW TIMESERIES pathPattern? showWhereClause?
 showWhereClause
     : WHERE (property | containsExpression)
     ;
@@ -190,14 +190,14 @@ containsExpression
     : name=ID OPERATOR_CONTAINS value=propertyValue
     ;
 
-Eg: show timeseries root.ln where unit='c'
-Eg: show timeseries root.ln where description contains 'test1'
+Eg: show timeseries root.ln.** where unit='c'
+Eg: show timeseries root.ln.** where description contains 'test1'
 ```
 
 * Show Specific Timeseries Statement with where clause start from offset and limit the total number of result
 
 ```
-SHOW TIMESERIES prefixPath? showWhereClause? limitClause?
+SHOW TIMESERIES pathPattern? showWhereClause? limitClause?
 
 showWhereClause
     : WHERE (property | containsExpression)
@@ -210,9 +210,9 @@ limitClause
     | offsetClause? LIMIT INT
     ;
     
-Eg: show timeseries root.ln where unit='c'
-Eg: show timeseries root.ln where description contains 'test1'
-Eg: show timeseries root.ln where unit='c' limit 10 offset 10
+Eg: show timeseries root.ln.** where unit='c'
+Eg: show timeseries root.ln.** where description contains 'test1'
+Eg: show timeseries root.ln.** where unit='c' limit 10 offset 10
 ```
 
 * Show Storage Group Statement
@@ -226,10 +226,10 @@ Note: This statement can be used in IoTDB Client and JDBC.
 * Show Specific Storage Group Statement
 
 ```
-SHOW STORAGE GROUP <PrefixPath>
+SHOW STORAGE GROUP <Path>
 Eg: IoTDB > SHOW STORAGE GROUP root.*
 Eg: IoTDB > SHOW STORAGE GROUP root.ln
-Note: The path can be prefix path or star path.
+Note: The path can be full path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -245,20 +245,20 @@ Note: This statement can be used in IoTDB Client and JDBC.
 
 ```
 COUNT TIMESERIES <Path>
-Eg: IoTDB > COUNT TIMESERIES root
-Eg: IoTDB > COUNT TIMESERIES root.ln
+Eg: IoTDB > COUNT TIMESERIES root.**
+Eg: IoTDB > COUNT TIMESERIES root.ln.**
 Eg: IoTDB > COUNT TIMESERIES root.ln.*.*.status
 Eg: IoTDB > COUNT TIMESERIES root.ln.wf01.wt01.status
-Note: The path can be prefix path, star path or timeseries path.
+Note: The path can be timeseries path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
 ```
 COUNT TIMESERIES <Path> GROUP BY LEVEL=<INTEGER>
-Eg: IoTDB > COUNT TIMESERIES root GROUP BY LEVEL=1
-Eg: IoTDB > COUNT TIMESERIES root.ln GROUP BY LEVEL=2
-Eg: IoTDB > COUNT TIMESERIES root.ln.wf01 GROUP BY LEVEL=3
-Note: The path can be prefix path or timeseries path.
+Eg: IoTDB > COUNT TIMESERIES root.** GROUP BY LEVEL=1
+Eg: IoTDB > COUNT TIMESERIES root.ln.** GROUP BY LEVEL=2
+Eg: IoTDB > COUNT TIMESERIES root.ln.wf01.* GROUP BY LEVEL=3
+Note: The path can be timeseries path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -266,11 +266,11 @@ Note: This statement can be used in IoTDB Client and JDBC.
 
 ```
 COUNT NODES <Path> LEVEL=<INTEGER>
-Eg: IoTDB > COUNT NODES root LEVEL=2
-Eg: IoTDB > COUNT NODES root.ln LEVEL=2
+Eg: IoTDB > COUNT NODES root.** LEVEL=2
+Eg: IoTDB > COUNT NODES root.ln.** LEVEL=2
 Eg: IoTDB > COUNT NODES root.ln.* LEVEL=3
 Eg: IoTDB > COUNT NODES root.ln.wf01 LEVEL=3
-Note: The path can be prefix path or timeseries path.
+Note: The path can be full path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -286,13 +286,12 @@ Note: This statement can be used in IoTDB Client and JDBC.
 * Show Specific Devices Statement
 
 ```
-SHOW DEVICES <PrefixPath> (WITH STORAGE GROUP)? limitClause?
-Eg: IoTDB > SHOW DEVICES root
-Eg: IoTDB > SHOW DEVICES root.ln
+SHOW DEVICES <PathPattern> (WITH STORAGE GROUP)? limitClause?
+Eg: IoTDB > SHOW DEVICES root.**
+Eg: IoTDB > SHOW DEVICES root.ln.**
 Eg: IoTDB > SHOW DEVICES root.*.wf01
 Eg: IoTDB > SHOW DEVICES root.ln WITH STORAGE GROUP
 Eg: IoTDB > SHOW DEVICES root.*.wf01 WITH STORAGE GROUP
-Note: The path can be prefix path or star path.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -305,12 +304,11 @@ Note: This statement can be used in IoTDB Client and JDBC.
 
 * Show Child Paths Statement
 ```
-SHOW CHILD PATHS <Path>
+SHOW CHILD PATHS <PathPattern>
 Eg: IoTDB > SHOW CHILD PATHS root
 Eg: IoTDB > SHOW CHILD PATHS root.ln
 Eg: IoTDB > SHOW CHILD PATHS root.*.wf01
-Eg: IoTDB > SHOW CHILD PATHS root.ln.wf*
-Note: The path can be prefix path or star path, the nodes can be in a "prefix + star" format. 
+Eg: IoTDB > SHOW CHILD PATHS root.ln.wf* 
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -338,7 +336,7 @@ Note: The order of Sensor and PointValue need one-to-one correspondence
 * Delete Record Statement
 
 ```
-DELETE FROM <PrefixPath> [COMMA <PrefixPath>]* [WHERE <WhereClause>]?
+DELETE FROM <PathPattern> [COMMA <PathPattern>]* [WHERE <WhereClause>]?
 WhereClause : <Condition> [(AND) <Condition>]*
 Condition  : <TimeExpr> [(AND) <TimeExpr>]*
 TimeExpr : TIME PrecedenceEqualOperator (<TimeValue> | <RelativeTime>)
@@ -363,7 +361,8 @@ RelativeTimeDurationUnit = Integer ('Y'|'MO'|'W'|'D'|'H'|'M'|'S'|'MS'|'US'|'NS')
 RelativeTime : (now() | <TimeValue>) [(+|-) RelativeTimeDurationUnit]+
 SensorExpr : (<Timeseries> | <Path>) PrecedenceEqualOperator <PointValue>
 Eg: IoTDB > SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-1 0:13:00
-Eg. IoTDB > SELECT * FROM root
+Eg. IoTDB > SELECT ** FROM root
+Eg. IoTDB > SELECT * FROM root.**
 Eg. IoTDB > SELECT * FROM root where time > now() - 5m
 Eg. IoTDB > SELECT * FROM root.ln.*.wf*
 Eg. IoTDB > SELECT COUNT(temperature) FROM root.ln.wf01.wt01 WHERE root.ln.wf01.wt01.temperature < 25
@@ -477,10 +476,10 @@ SELECT <SelectClause> FROM <FromClause> WHERE  <WhereClause> GROUP BY <GroupByCl
 orderByTimeClause: order by time (asc | desc)?
 
 Eg: SELECT last_value(temperature) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (float[PREVIOUS]) order by time desc
-Eg: SELECT * from root order by time desc
-Eg: SELECT * from root order by time desc align by device 
-Eg: SELECT * from root order by time desc disable align
-Eg: SELECT last * from root order by time desc
+Eg: SELECT * from root.** order by time desc
+Eg: SELECT * from root.** order by time desc align by device 
+Eg: SELECT * from root.** order by time desc disable align
+Eg: SELECT last * from root.** order by time desc
 ```
 
 * Limit Statement
@@ -519,18 +518,18 @@ AlignbyDeviceClause : ALIGN BY DEVICE
 
 Rules:  
 1. Both uppercase and lowercase are ok.  
-Correct example: select * from root.sg1 align by device  
-Correct example: select * from root.sg1 ALIGN BY DEVICE  
+Correct example: select * from root.sg1.* align by device  
+Correct example: select * from root.sg1.* ALIGN BY DEVICE  
 
 2. AlignbyDeviceClause can only be used at the end of a query statement.  
-Correct example: select * from root.sg1 where time > 10 align by device  
-Wrong example: select * from root.sg1 align by device where time > 10  
+Correct example: select * from root.sg1.* where time > 10 align by device  
+Wrong example: select * from root.sg1.* align by device where time > 10  
 
 3. The paths of the SELECT clause can only be single level. In other words, the paths of the SELECT clause can only be measurements or STAR, without DOT.
 Correct example: select s0,s1 from root.sg1.* align by device  
 Correct example: select s0,s1 from root.sg1.d0, root.sg1.d1 align by device  
 Correct example: select * from root.sg1.* align by device  
-Correct example: select * from root align by device  
+Correct example: select * from root.** align by device  
 Correct example: select s0,s1,* from root.*.* align by device  
 Wrong example: select d0.s1, d0.s2, d1.s0 from root.sg1 align by device  
 Wrong example: select *.s0, *.s1 from root.* align by device  
@@ -582,16 +581,16 @@ For example, "select s0,s0,s1 from root.sg.* align by device" is not equal to "s
 - select * from root.sg.d0 where root.sg.d0.s0 = 15 align by device
 
 9. More correct examples:
-   - select * from root.vehicle align by device
+   - select * from root.vehicle.* align by device
    - select s0,s0,s1 from root.vehicle.* align by device
    - select s0,s1 from root.vehicle.* limit 10 offset 1 align by device
-   - select * from root.vehicle slimit 10 soffset 2 align by device
-   - select * from root.vehicle where time > 10 align by device
+   - select * from root.vehicle.* slimit 10 soffset 2 align by device
+   - select * from root.vehicle.* where time > 10 align by device
    - select * from root.vehicle.* where time < 10 AND s0 > 25 align by device
-   - select * from root.vehicle where root.vehicle.d0.s0>0 align by device
+   - select * from root.vehicle.* where root.vehicle.d0.s0>0 align by device
    - select count(*) from root.vehicle align by device
-   - select sum(*) from root.vehicle GROUP BY (20ms,0,[2,50]) align by device
-   - select * from root.vehicle where time = 3 Fill(int32[previous, 5ms]) align by device
+   - select sum(*) from root.vehicle.* GROUP BY (20ms,0,[2,50]) align by device
+   - select * from root.vehicle.* where time = 3 Fill(int32[previous, 5ms]) align by device
 ```
 * Disable Align Statement
 
@@ -600,19 +599,19 @@ Disable Align Clause: DISABLE ALIGN
 
 Rules:  
 1. Both uppercase and lowercase are ok.  
-Correct example: select * from root.sg1 disable align  
-Correct example: select * from root.sg1 DISABLE ALIGN  
+Correct example: select * from root.sg1.* disable align  
+Correct example: select * from root.sg1.* DISABLE ALIGN  
 
 2. Disable Align Clause can only be used at the end of a query statement.  
-Correct example: select * from root.sg1 where time > 10 disable align 
-Wrong example: select * from root.sg1 disable align where time > 10 
+Correct example: select * from root.sg1.* where time > 10 disable align 
+Wrong example: select * from root.sg1.* disable align where time > 10 
 
 3. Disable Align Clause cannot be used with Aggregation, Fill Statements, Group By or Group By Device Statements, but can with Limit Statements.
-Correct example: select * from root.sg1 limit 3 offset 2 disable align
-Correct example: select * from root.sg1 slimit 3 soffset 2 disable align
+Correct example: select * from root.sg1.* limit 3 offset 2 disable align
+Correct example: select * from root.sg1.* slimit 3 soffset 2 disable align
 Wrong example: select count(s0),count(s1) from root.sg1.d1 disable align
-Wrong example: select * from root.vehicle where root.vehicle.d0.s0>0 disable align
-Wrong example: select * from root.vehicle align by device disable align
+Wrong example: select * from root.vehicle.* where root.vehicle.d0.s0>0 disable align
+Wrong example: select * from root.vehicle.* align by device disable align
 
 4. The display principle of the result table is that only when the column (or row) has existing data will the column (or row) be shown, with nonexistent cells being empty.
 
@@ -625,11 +624,11 @@ You could expect a table like:
 |      |               |      |               | 900  | 8000          |
 
 5. More correct examples: 
-   - select * from root.vehicle disable align
+   - select * from root.vehicle.* disable align
    - select s0,s0,s1 from root.vehicle.* disable align
    - select s0,s1 from root.vehicle.* limit 10 offset 1 disable align
-   - select * from root.vehicle slimit 10 soffset 2 disable align
-   - select * from root.vehicle where time > 10 disable align
+   - select * from root.vehicle.* slimit 10 soffset 2 disable align
+   - select * from root.vehicle.* where time > 10 disable align
 
 ```
 
diff --git a/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md b/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md
index 041e78a..96ca3fe 100644
--- a/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md
+++ b/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md
@@ -70,7 +70,7 @@ path: LayerName (DOT LayerName)+
 LayerName: Identifier | STAR
 ```
 
-Among them, STAR is "*" and DOT is ".".
+Among them, STAR is `*` or `**` and DOT is `.`.
 
 We call the middle part of a path between two "." as a layer, and thus `root.A.B.C` is a path with four layers. 
 
@@ -92,26 +92,15 @@ The characters supported in LayerName without double quotes are as below:
 > Besides, if deploy on Windows system, the LayerName is case-insensitive, which means it's not allowed to set storage groups `root.ln` and `root.LN` at the same time.
 
 
-* Prefix Path
+* Path Pattern
 
-The prefix path refers to the path where the prefix of a timeseries path is located. A prefix path contains all timeseries paths prefixed by the path. For example, suppose that we have three sensors: `root.vehicle.device1.sensor1`, `root.vehicle.device1.sensor2`, `root.vehicle.device2.sensor1`, the prefix path `root.vehicle.device1` contains two timeseries paths `root.vehicle.device1.sensor1` and `root.vehicle.device1.sensor2` while `root.vehicle.device2.sensor1` is excluded.
+In order to make it easier and faster to express multiple timeseries paths, IoTDB provides users with the path pattern. Users can construct a path pattern by using wildcard `*` and `**`. Wildcard can appear in any layer of the path. 
 
-* Path With Star
+`*` represents one layer. For example, `root.vehicle.*.sensor1` represents a 4-layer path which is prefixed with `root.vehicle` and suffixed with `sensor1`.
 
-In order to make it easier and faster to express multiple timeseries paths or prefix paths, IoTDB provides users with the path pith star. `*` can appear in any layer of the path. According to the position where `*` appears, the path with star can be divided into two types:
-
-`*` appears at the end of the path;
-
-`*` appears in the middle of the path;
-
-When `*` appears at the end of the path, it represents (`*`)+, which is one or more layers of `*`. For example, `root.vehicle.device1.*` represents all paths prefixed by `root.vehicle.device1` with layers greater than or equal to 4, like `root.vehicle.device1.*`, `root.vehicle.device1.*.*`, `root.vehicle.device1.*.*.*`, etc.
-
-When `*` appears in the middle of the path, it represents `*` itself, i.e., a layer. For example, `root.vehicle.*.sensor1` represents a 4-layer path which is prefixed with `root.vehicle` and suffixed with `sensor1`.   
-
-> Note1: `*` cannot be placed at the beginning of the path.
-
-> Note2: A path with `*` at the end has the same meaning as a prefix path, e.g., `root.vehicle.*` and `root.vehicle` is the same.
+`**` represents (`*`)+, which is one or more layers of `*`. For example, `root.vehicle.device1.*` represents all paths prefixed by `root.vehicle.device1` with layers greater than or equal to 4, like `root.vehicle.device1.*`, `root.vehicle.device1.*.*`, `root.vehicle.device1.*.*.*`, etc; `root.vehicle.**.sensor1` represents a path which is prefixed with `root.vehicle` and suffixed with `sensor1` and has at least 4 layers.
 
+> Note1: Wildcard `*` and `**` cannot be placed at the beginning of the path.
 
 
 ### Timeseries
diff --git a/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md b/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
index 9c38ab4..c8fe797 100644
--- a/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
+++ b/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
@@ -47,11 +47,11 @@ Besides, if deploy on Windows system, the LayerName is case-insensitive, which m
 
 ### Show Storage Group
 
-After creating the storage group, we can use the [SHOW STORAGE GROUP](../Appendix/SQL-Reference.md) statement and [SHOW STORAGE GROUP \<PrefixPath>](../Appendix/SQL-Reference.md) to view the storage groups. The SQL statements are as follows:
+After creating the storage group, we can use the [SHOW STORAGE GROUP](../Appendix/SQL-Reference.md) statement and [SHOW STORAGE GROUP \<PathPattern>](../Appendix/SQL-Reference.md) to view the storage groups. The SQL statements are as follows:
 
 ```
 IoTDB> show storage group
-IoTDB> show storage group root.ln
+IoTDB> show storage group root.**
 ```
 
 The result is as follows:
@@ -69,13 +69,13 @@ It costs 0.060s
 
 ### Delete Storage Group
 
-User can use the `DELETE STORAGE GROUP <PrefixPath>` statement to delete all storage groups under the prefixPath. Please note the data in the storage group will also be deleted. 
+User can use the `DELETE STORAGE GROUP <PathPattern>` statement to delete all storage groups matching the pathPattern. Please note the data in the storage group will also be deleted. 
 
 ```
 IoTDB > DELETE STORAGE GROUP root.ln
 IoTDB > DELETE STORAGE GROUP root.sgcc
 // delete all data, all timeseries and all storage groups
-IoTDB > DELETE STORAGE GROUP root.*
+IoTDB > DELETE STORAGE GROUP root.**
 ```
 ## Timeseries Management
 ### Create Timeseries
@@ -102,7 +102,7 @@ Please refer to [Encoding](../Data-Concept/Encoding.md) for correspondence betwe
 
 ### Delete Timeseries
 
-To delete the timeseries we created before, we are able to use `DELETE TimeSeries <PrefixPath>` statement.
+To delete the timeseries we created before, we are able to use `DELETE TimeSeries <PathPattern>` statement.
 
 The usage are as follows:
 
@@ -127,7 +127,7 @@ error: Not support deleting part of aligned timeseies!
 
 ### Show Timeseries
 
-* SHOW LATEST? TIMESERIES prefixPath? showWhereClause? limitClause?
+* SHOW LATEST? TIMESERIES pathPattern? showWhereClause? limitClause?
 
   There are four optional clauses added in SHOW TIMESERIES, return information of time series 
   
@@ -139,13 +139,13 @@ Examples:
 
   presents all timeseries information in JSON form
  
-* SHOW TIMESERIES <`Path`> 
+* SHOW TIMESERIES <`PathPattern`> 
   
-  returns all timeseries information under the given <`Path`>.  <`Path`> needs to be a prefix path or a path with star or a timeseries path. SQL statements are as follows:
+  returns all timeseries information matching the given <`PathPattern`>. SQL statements are as follows:
 
 ```
-IoTDB> show timeseries root
-IoTDB> show timeseries root.ln
+IoTDB> show timeseries root.**
+IoTDB> show timeseries root.ln.**
 ```
 
 The results are shown below respectively:
@@ -177,15 +177,15 @@ Total line number = 4
 It costs 0.004s
 ```
 
-* SHOW TIMESERIES (<`PrefixPath`>)? WhereClause
+* SHOW TIMESERIES (<`PathPattern`>)? WhereClause
  
-  returns all the timeseries information that satisfy the where condition and start with the prefixPath SQL statements are as follows:
+  returns all the timeseries information that satisfy the where condition and match the pathPattern. SQL statements are as follows:
 
 ```
 ALTER timeseries root.ln.wf02.wt02.hardware ADD TAGS unit=c
 ALTER timeseries root.ln.wf02.wt02.status ADD TAGS description=test1
-show timeseries root.ln where unit=c
-show timeseries root.ln where description contains 'test1'
+show timeseries root.ln.** where unit=c
+show timeseries root.ln.** where description contains 'test1'
 ```
 
 The results are shown below respectly:
@@ -223,11 +223,11 @@ It is worth noting that when the queried path does not exist, the system will re
 
 ### Count Timeseries
 
-IoTDB is able to use `COUNT TIMESERIES <Path>` to count the number of timeseries in the path. SQL statements are as follows:
+IoTDB is able to use `COUNT TIMESERIES <Path>` to count the number of timeseries matching the path. SQL statements are as follows:
 
 ```
-IoTDB > COUNT TIMESERIES root
-IoTDB > COUNT TIMESERIES root.ln
+IoTDB > COUNT TIMESERIES root.**
+IoTDB > COUNT TIMESERIES root.ln.**
 IoTDB > COUNT TIMESERIES root.ln.*.*.status
 IoTDB > COUNT TIMESERIES root.ln.wf01.wt01.status
 ```
@@ -259,9 +259,9 @@ Then the Metadata Tree will be as below:
 As can be seen, `root` is considered as `LEVEL=0`. So when you enter statements such as:
 
 ```
-IoTDB > COUNT TIMESERIES root GROUP BY LEVEL=1
-IoTDB > COUNT TIMESERIES root.ln GROUP BY LEVEL=2
-IoTDB > COUNT TIMESERIES root.ln.wf01 GROUP BY LEVEL=2
+IoTDB > COUNT TIMESERIES root.** GROUP BY LEVEL=1
+IoTDB > COUNT TIMESERIES root.ln.** GROUP BY LEVEL=2
+IoTDB > COUNT TIMESERIES root.ln.wf01.* GROUP BY LEVEL=2
 ```
 
 You will get following results:
@@ -346,10 +346,10 @@ ALTER timeseries root.turbine.d1.s1 UPSERT ALIAS=newAlias TAGS(tag3=v3, tag4=v4)
 ### Show Child Paths
 
 ```
-SHOW CHILD PATHS prefixPath
+SHOW CHILD PATHS pathPattern
 ```
 
-Return all child paths of the prefixPath, the prefixPath could contains *.
+Return all child paths of all the paths matching pathPattern.
 
 Exampleļ¼š
 
@@ -371,10 +371,10 @@ It costs 0.002s
 ### Show Child Nodes
 
 ```
-SHOW CHILD NODES prefixPath
+SHOW CHILD NODES pathPattern
 ```
 
-Return all child nodes of the prefixPath.
+Return all child nodes of the pathPattern.
 
 Exampleļ¼š
 
@@ -401,12 +401,12 @@ Exampleļ¼š
 
 ### Count Nodes
 
-IoTDB is able to use `COUNT NODES <PrefixPath> LEVEL=<INTEGER>` to count the number of nodes at the given level in current Metadata Tree. This could be used to query the number of devices. The usage are as follows:
+IoTDB is able to use `COUNT NODES <PathPattern> LEVEL=<INTEGER>` to count the number of nodes at the given level in current Metadata Tree. This could be used to query the number of devices. The usage are as follows:
 
 ```
-IoTDB > COUNT NODES root LEVEL=2
-IoTDB > COUNT NODES root.ln LEVEL=2
-IoTDB > COUNT NODES root.ln.wf01 LEVEL=3
+IoTDB > COUNT NODES root.** LEVEL=2
+IoTDB > COUNT NODES root.ln.** LEVEL=2
+IoTDB > COUNT NODES root.ln.wf01.** LEVEL=3
 ```
 
 As for the above mentioned example and Metadata tree, you can get following results:
@@ -438,22 +438,21 @@ It costs 0.002s
 ```
 
 > Note: The path of timeseries is just a filter condition, which has no relationship with the definition of level.
-`PrefixPath` could contains `*`, but all nodes after `*` would be ignored. Only the prefix path before `*` is valid.
 
 ### Show Devices
 
-* SHOW DEVICES prefixPath? (WITH STORAGE GROUP)? limitClause? #showDevices
+* SHOW DEVICES pathPattern? (WITH STORAGE GROUP)? limitClause? #showDevices
 
 Similar to `Show Timeseries`, IoTDB also supports two ways of viewing devices:
 
-* `SHOW DEVICES` statement presents all devices' information, which is equal to `SHOW DEVICES root`.
-* `SHOW DEVICES <PrefixPath>` statement specifies the `PrefixPath` and returns the devices information under the given level.
+* `SHOW DEVICES` statement presents all devices' information, which is equal to `SHOW DEVICES root.**`.
+* `SHOW DEVICES <PathPattern>` statement specifies the `PathPattern` and returns the devices information matching the pathPattern and under the given level.
 
 SQL statement is as follows:
 
 ```
 IoTDB> show devices
-IoTDB> show devices root.ln
+IoTDB> show devices root.ln.**
 ```
 
 You can get results below:
@@ -483,14 +482,14 @@ It costs 0.001s
 To view devices' information with storage group, we can use `SHOW DEVICES WITH STORAGE GROUP` statement.
 
 * `SHOW DEVICES WITH STORAGE GROUP` statement presents all devices' information with their storage group.
-* `SHOW DEVICES <PrefixPath> WITH STORAGE GROUP` statement specifies the `PrefixPath` and returns the 
+* `SHOW DEVICES <PathPattern> WITH STORAGE GROUP` statement specifies the `PathPattern` and returns the 
 devices' information under the given level with their storage group information.
 
 SQL statement is as follows:
 
 ```
 IoTDB> show devices with storage group
-IoTDB> show devices root.ln with storage group
+IoTDB> show devices root.ln.** with storage group
 ```
 
 You can get results below:
diff --git a/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md b/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
index 76e1bc9..14bf1d2 100644
--- a/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
+++ b/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
@@ -207,7 +207,7 @@ IoTDB supports the 'order by time' statement since 0.11, it's used to display re
 For example, the SQL statement is:
 
 ```sql
-select * from root.ln where time > 1 order by time desc limit 10;
+select * from root.ln.** where time > 1 order by time desc limit 10;
 ```
 The execution result of this SQL statement is as follows:
 
diff --git a/docs/zh/SystemDesign/SchemaManager/SchemaManager.md b/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
index 167696f..348fae2 100644
--- a/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
+++ b/docs/zh/SystemDesign/SchemaManager/SchemaManager.md
@@ -136,7 +136,7 @@ IoTDB ēš„å…ƒę•°ę®ē»Ÿäø€ē”± MManger ē®”ē†ļ¼ŒåŒ…ꋬ仄äø‹å‡ äøŖéƒØ分ļ¼š
 
 ## å…ƒę•°ę®ę ‘
 
-* org.apache.iotdb.db.metadata.MTree
+* org.apache.iotdb.db.metadata.mtree.MTree
 
 ꠑäø­åŒ…ꋬäø‰ē§čŠ‚ē‚¹ļ¼šStorageGroupMNode态InternalMNodeļ¼ˆéžå¶å­čŠ‚ē‚¹ļ¼‰ć€LeafMNodeļ¼ˆå¶å­čŠ‚ē‚¹ļ¼‰ļ¼Œä»–们都ę˜Æ MNode ēš„子ē±»ć€‚
 
diff --git a/docs/zh/UserGuide/Appendix/SQL-Reference.md b/docs/zh/UserGuide/Appendix/SQL-Reference.md
index 396a336..286b7ea 100644
--- a/docs/zh/UserGuide/Appendix/SQL-Reference.md
+++ b/docs/zh/UserGuide/Appendix/SQL-Reference.md
@@ -44,14 +44,15 @@ It costs 0.417s
 ``` SQL
 SET STORAGE GROUP TO <FullPath>
 Eg: IoTDB > SET STORAGE GROUP TO root.ln.wf01.wt01
-Note: FullPath can not include `*`
+Note: FullPath can not include wildcard `*` or `**`
 ```
 * 删除存å‚Øē»„
 
 ```
-DELETE STORAGE GROUP <FullPath> [COMMA <FullPath>]*
+DELETE STORAGE GROUP <PathPattern> [COMMA <PathPattern>]*
 Eg: IoTDB > DELETE STORAGE GROUP root.ln
 Eg: IoTDB > DELETE STORAGE GROUP root.*
+Eg: IoTDB > DELETE STORAGE GROUP root.**
 ```
 
 * 创å»ŗꗶ闓åŗåˆ—čƭ叄
@@ -116,7 +117,7 @@ Eg: set schema template temp1 to root.beijing
 * åˆ é™¤ę—¶é—“åŗåˆ—čƭ叄
 
 ```
-DELETE TIMESERIES <PrefixPath> [COMMA <PrefixPath>]*
+DELETE TIMESERIES <PathPattern> [COMMA <PathPattern>]*
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.status
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.status, root.ln.wf01.wt01.temperature
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.*
@@ -160,18 +161,18 @@ Note: This statement can only be used in IoTDB Client. If you need to show all t
 
 ```
 SHOW TIMESERIES <Path>
-Eg: IoTDB > SHOW TIMESERIES root
-Eg: IoTDB > SHOW TIMESERIES root.ln
+Eg: IoTDB > SHOW TIMESERIES root.**
+Eg: IoTDB > SHOW TIMESERIES root.ln.**
 Eg: IoTDB > SHOW TIMESERIES root.ln.*.*.status
 Eg: IoTDB > SHOW TIMESERIES root.ln.wf01.wt01.status
-Note: The path can be prefix path, star path or timeseries path
+Note: The path can be timeseries path or path pattern
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
 * ę˜¾ē¤ŗę»”č¶³ę”ä»¶ēš„ꗶ闓åŗåˆ—čƭ叄
 
 ```
-SHOW TIMESERIES prefixPath? showWhereClause?
+SHOW TIMESERIES pathPattern? showWhereClause?
 showWhereClause
     : WHERE (property | containsExpression)
     ;
@@ -179,14 +180,14 @@ containsExpression
     : name=ID OPERATOR_CONTAINS value=propertyValue
     ;
 
-Eg: show timeseries root.ln where unit='c'
-Eg: show timeseries root.ln where description contains 'test1'
+Eg: show timeseries root.ln.** where unit='c'
+Eg: show timeseries root.ln.** where description contains 'test1'
 ```
 
 * åˆ†é”µę˜¾ē¤ŗę»”č¶³ę”ä»¶ēš„ꗶ闓åŗåˆ—čƭ叄
 
 ```
-SHOW TIMESERIES prefixPath? showWhereClause? limitClause?
+SHOW TIMESERIES pathPattern? showWhereClause? limitClause?
 
 showWhereClause
     : WHERE (property | containsExpression)
@@ -199,9 +200,9 @@ limitClause
     | offsetClause? LIMIT INT
     ;
     
-Eg: show timeseries root.ln where unit='c'
-Eg: show timeseries root.ln where description contains 'test1'
-Eg: show timeseries root.ln where unit='c' limit 10 offset 10
+Eg: show timeseries root.ln.** where unit='c'
+Eg: show timeseries root.ln.** where description contains 'test1'
+Eg: show timeseries root.ln.** where unit='c' limit 10 offset 10
 ```
 
 * ę˜¾ē¤ŗ存å‚Øē»„čƭ叄
@@ -215,10 +216,10 @@ Note: This statement can be used in IoTDB Client and JDBC.
 * ę˜¾ē¤ŗē‰¹å®šå­˜å‚Øē»„čƭ叄
 
 ```
-SHOW STORAGE GROUP <PrefixPath>
+SHOW STORAGE GROUP <PathPattern>
 Eg: IoTDB > SHOW STORAGE GROUP root.*
+Eg: IoTDB > SHOW STORAGE GROUP root.**
 Eg: IoTDB > SHOW STORAGE GROUP root.ln
-Note: The path can be prefix path or star path.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -234,20 +235,20 @@ Note: This statement can be used in IoTDB Client and JDBC.
 
 ```
 COUNT TIMESERIES <Path>
-Eg: IoTDB > COUNT TIMESERIES root
-Eg: IoTDB > COUNT TIMESERIES root.ln
+Eg: IoTDB > COUNT TIMESERIES root.**
+Eg: IoTDB > COUNT TIMESERIES root.ln.**
 Eg: IoTDB > COUNT TIMESERIES root.ln.*.*.status
 Eg: IoTDB > COUNT TIMESERIES root.ln.wf01.wt01.status
-Note: The path can be prefix path, star path or timeseries path.
+Note: The path can be timeseries path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
 ```
 COUNT TIMESERIES <Path> GROUP BY LEVEL=<INTEGER>
-Eg: IoTDB > COUNT TIMESERIES root GROUP BY LEVEL=1
-Eg: IoTDB > COUNT TIMESERIES root.ln GROUP BY LEVEL=2
-Eg: IoTDB > COUNT TIMESERIES root.ln.wf01 GROUP BY LEVEL=3
-Note: The path can be prefix path or timeseries path.
+Eg: IoTDB > COUNT TIMESERIES root.** GROUP BY LEVEL=1
+Eg: IoTDB > COUNT TIMESERIES root.ln.** GROUP BY LEVEL=2
+Eg: IoTDB > COUNT TIMESERIES root.ln.wf01.* GROUP BY LEVEL=3
+Note: The path can be timeseries path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -255,11 +256,11 @@ Note: This statement can be used in IoTDB Client and JDBC.
 
 ```
 COUNT NODES <Path> LEVEL=<INTEGER>
-Eg: IoTDB > COUNT NODES root LEVEL=2
-Eg: IoTDB > COUNT NODES root.ln LEVEL=2
-Eg: IoTDB > COUNT NODES root.ln.* LEVEL=3
-Eg: IoTDB > COUNT NODES root.ln.wf01 LEVEL=3
-Note: The path can be prefix path or timeseries path.
+Eg: IoTDB > COUNT NODES root.** LEVEL=2
+Eg: IoTDB > COUNT NODES root.ln.** LEVEL=2
+Eg: IoTDB > COUNT NODES root.ln.*.* LEVEL=3
+Eg: IoTDB > COUNT NODES root.ln.wf01.* LEVEL=3
+Note: The path can be timeseries path or path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -275,13 +276,13 @@ Note: This statement can be used in IoTDB Client and JDBC.
 * ę˜¾ē¤ŗē‰¹å®šč®¾å¤‡čƭ叄
 
 ```
-SHOW DEVICES <PrefixPath> (WITH STORAGE GROUP)? limitClause?
-Eg: IoTDB > SHOW DEVICES root
-Eg: IoTDB > SHOW DEVICES root.ln
+SHOW DEVICES <PathPattern> (WITH STORAGE GROUP)? limitClause?
+Eg: IoTDB > SHOW DEVICES root.**
+Eg: IoTDB > SHOW DEVICES root.ln.*
 Eg: IoTDB > SHOW DEVICES root.*.wf01
-Eg: IoTDB > SHOW DEVICES root.ln WITH STORAGE GROUP
+Eg: IoTDB > SHOW DEVICES root.ln.* WITH STORAGE GROUP
 Eg: IoTDB > SHOW DEVICES root.*.wf01 WITH STORAGE GROUP
-Note: The path can be prefix path or star path.
+Note: The path can be path pattern.
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -296,12 +297,11 @@ Note: This statement can be used in IoTDB Client and JDBC.
 * ę˜¾ē¤ŗå­čŠ‚ē‚¹åē§°čƭ叄
 
 ```
-SHOW CHILD PATHS <Path>
+SHOW CHILD PATHS <PathPattern>
 Eg: IoTDB > SHOW CHILD PATHS root
 Eg: IoTDB > SHOW CHILD PATHS root.ln
 Eg: IoTDB > SHOW CHILD PATHS root.*.wf01
 Eg: IoTDB > SHOW CHILD PATHS root.ln.wf*
-Note: The path can be prefix path or star path, the nodes can be in a "prefix + star" format. 
 Note: This statement can be used in IoTDB Client and JDBC.
 ```
 
@@ -330,7 +330,7 @@ Note: The order of Sensor and PointValue need one-to-one correspondence
 * åˆ é™¤č®°å½•čƭ叄
 
 ```
-DELETE FROM <PrefixPath> [COMMA <PrefixPath>]* [WHERE <WhereClause>]?
+DELETE FROM <PathPattern> [COMMA <PathPattern>]* [WHERE <WhereClause>]?
 WhereClause : <Condition> [(AND) <Condition>]*
 Condition  : <TimeExpr> [(AND) <TimeExpr>]*
 TimeExpr : TIME PrecedenceEqualOperator (<TimeValue> | <RelativeTime>)
@@ -355,8 +355,9 @@ RelativeTimeDurationUnit = Integer ('Y'|'MO'|'W'|'D'|'H'|'M'|'S'|'MS'|'US'|'NS')
 RelativeTime : (now() | <TimeValue>) [(+|-) RelativeTimeDurationUnit]+
 SensorExpr : (<Timeseries> | <Path>) PrecedenceEqualOperator <PointValue>
 Eg: IoTDB > SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-01 00:13:00
-Eg. IoTDB > SELECT * FROM root
-Eg. IoTDB > SELECT * FROM root where time > now() - 5m
+Eg. IoTDB > SELECT ** FROM root
+Eg. IoTDB > SELECT * FROM root.**
+Eg. IoTDB > SELECT * FROM root.** where time > now() - 5m
 Eg. IoTDB > SELECT * FROM root.ln.*.wf*
 Eg. IoTDB > SELECT COUNT(temperature) FROM root.ln.wf01.wt01 WHERE root.ln.wf01.wt01.temperature < 25
 Eg. IoTDB > SELECT MIN_TIME(temperature) FROM root.ln.wf01.wt01 WHERE root.ln.wf01.wt01.temperature < 25
@@ -471,9 +472,9 @@ SELECT <SelectClause> FROM <FromClause> WHERE  <WhereClause> GROUP BY <GroupByCl
 orderByTimeClause: order by time (asc | desc)?
 
 Eg: SELECT last_value(temperature) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (float[PREVIOUS]) order by time desc
-Eg: SELECT * from root order by time desc
-Eg: SELECT * from root order by time desc align by device 
-Eg: SELECT * from root order by time desc disable align
+Eg: SELECT * from root.** order by time desc
+Eg: SELECT * from root.** order by time desc align by device 
+Eg: SELECT * from root.** order by time desc disable align
 Eg: SELECT last * from root order by time desc
 ```
 
@@ -513,18 +514,18 @@ AlignbyDeviceClause : ALIGN BY DEVICE
 
 č§„åˆ™ļ¼š
 1. 大小写äøę•ę„Ÿć€‚
-ę­£ä¾‹ļ¼šselect * from root.sg1 align by device
-ę­£ä¾‹ļ¼šselect * from root.sg1 ALIGN BY DEVICE
+ę­£ä¾‹ļ¼šselect * from root.sg1.** align by device
+ę­£ä¾‹ļ¼šselect * from root.sg1.** ALIGN BY DEVICE
 
 2. AlignbyDeviceClause åŖčƒ½ę”¾åœØęœ«å°¾ć€‚
-ę­£ä¾‹ļ¼šselect * from root.sg1 where time > 10 align by device  
-错例ļ¼šselect * from root.sg1 align by device where time > 10  
+ę­£ä¾‹ļ¼šselect * from root.sg1.** where time > 10 align by device  
+错例ļ¼šselect * from root.sg1.** align by device where time > 10  
 
 3. Select 子叄äø­ēš„ path åŖčƒ½ę˜Æ单层ļ¼Œęˆ–č€…é€šé…ē¬¦ļ¼Œäøå…č®ø꜉ path 分隔ē¬¦"."怂
 ę­£ä¾‹ļ¼šselect s0,s1 from root.sg1.* align by device  
 ę­£ä¾‹ļ¼šselect s0,s1 from root.sg1.d0, root.sg1.d1 align by device  
 ę­£ä¾‹ļ¼šselect * from root.sg1.* align by device  
-ę­£ä¾‹ļ¼šselect * from root align by device  
+ę­£ä¾‹ļ¼šselect * from root.** align by device  
 ę­£ä¾‹ļ¼šselect s0,s1,* from root.*.* align by device  
 错例ļ¼šselect d0.s1, d0.s2, d1.s0 from root.sg1 align by device  
 错例ļ¼šselect *.s0, *.s1 from root.* align by device  
@@ -573,16 +574,16 @@ root.sg1.d0.s0 is INT32 while root.sg2.d3.s0 is FLOAT.
 - select * from root.sg.d0 where root.sg.d0.s0 = 15 align by device
 
 9. ę›“å¤šę­£ä¾‹ļ¼š
-   - select * from root.vehicle align by device
+   - select * from root.vehicle.* align by device
    - select s0,s0,s1 from root.vehicle.* align by device
    - select s0,s1 from root.vehicle.* limit 10 offset 1 align by device
-   - select * from root.vehicle slimit 10 soffset 2 align by device
-   - select * from root.vehicle where time > 10 align by device
+   - select * from root.vehicle.* slimit 10 soffset 2 align by device
+   - select * from root.vehicle.* where time > 10 align by device
    - select * from root.vehicle.* where time < 10 AND s0 > 25 align by device
-   - select * from root.vehicle where root.vehicle.d0.s0>0 align by device
-   - select count(*) from root.vehicle align by device
-   - select sum(*) from root.vehicle GROUP BY (20ms,0,[2,50]) align by device
-   - select * from root.vehicle where time = 3 Fill(int32[previous, 5ms]) align by device
+   - select * from root.vehicle.* where root.vehicle.d0.s0>0 align by device
+   - select count(*) from root.vehicle.* align by device
+   - select sum(*) from root.vehicle.* GROUP BY (20ms,0,[2,50]) align by device
+   - select * from root.vehicle.* where time = 3 Fill(int32[previous, 5ms]) align by device
 ```
 
 * Disable align čƭ叄
@@ -590,19 +591,19 @@ root.sg1.d0.s0 is INT32 while root.sg2.d3.s0 is FLOAT.
 ```
 č§„åˆ™ļ¼š
 1. 大小写均åÆ怂
-ę­£ä¾‹ļ¼šselect * from root.sg1 disable align  
-ę­£ä¾‹ļ¼šselect * from root.sg1 DISABLE ALIGN  
+ę­£ä¾‹ļ¼šselect * from root.sg1.* disable align  
+ę­£ä¾‹ļ¼šselect * from root.sg1.* DISABLE ALIGN  
 
 2. Disable Align åŖčƒ½ē”ØäŗŽęŸ„čÆ¢čÆ­å„å„å°¾ć€‚
-ę­£ä¾‹ļ¼šselect * from root.sg1 where time > 10 disable align 
-错例ļ¼šselect * from root.sg1 disable align where time > 10 
+ę­£ä¾‹ļ¼šselect * from root.sg1.* where time > 10 disable align 
+错例ļ¼šselect * from root.sg1.* disable align where time > 10 
 
 3. Disable Align äøčƒ½ē”ØäŗŽčšåˆęŸ„čÆ¢ć€Fill čÆ­å„ć€Group by ꈖ Group by device čƭ叄ļ¼Œä½†åÆē”ØäŗŽ Limit čÆ­å„ć€‚
-ę­£ä¾‹ļ¼šselect * from root.sg1 limit 3 offset 2 disable align
-ę­£ä¾‹ļ¼šselect * from root.sg1 slimit 3 soffset 2 disable align
+ę­£ä¾‹ļ¼šselect * from root.sg1.* limit 3 offset 2 disable align
+ę­£ä¾‹ļ¼šselect * from root.sg1.* slimit 3 soffset 2 disable align
 错例ļ¼šselect count(s0),count(s1) from root.sg1.d1 disable align
-错例ļ¼šselect * from root.vehicle where root.vehicle.d0.s0>0 disable align
-错例ļ¼šselect * from root.vehicle align by device disable align
+错例ļ¼šselect * from root.vehicle.* where root.vehicle.d0.s0>0 disable align
+错例ļ¼šselect * from root.vehicle.* align by device disable align
 
 4. ē»“ęžœę˜¾ē¤ŗč‹„ę— ę•°ę®ę˜¾ē¤ŗäøŗē©ŗē™½ć€‚
 
@@ -615,11 +616,11 @@ root.sg1.d0.s0 is INT32 while root.sg2.d3.s0 is FLOAT.
 |      |               |      |               | 900  | 8000          |
 
 5. äø€äŗ›ę­£ē”®ä½æē”Øę ·ä¾‹ļ¼š
-   - select * from root.vehicle disable align
+   - select * from root.vehicle.* disable align
    - select s0,s0,s1 from root.vehicle.* disable align
    - select s0,s1 from root.vehicle.* limit 10 offset 1 disable align
-   - select * from root.vehicle slimit 10 soffset 2 disable align
-   - select * from root.vehicle where time > 10 disable align
+   - select * from root.vehicle.* slimit 10 soffset 2 disable align
+   - select * from root.vehicle.* where time > 10 disable align
 
 ```
 
diff --git a/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md b/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md
index 981ef7f..8c2658f 100644
--- a/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md
+++ b/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md
@@ -63,7 +63,7 @@ IoTDB ęؔ型ē»“ęž„ę¶‰åŠēš„åŸŗęœ¬ę¦‚åæµåœØäø‹ę–‡å°†åščƦē»†å™čæ°ć€‚
 
 存å‚Øē»„节ē‚¹ååŖę”Æꌁäø­č‹±ę–‡å­—ē¬¦ć€ę•°å­—态äø‹åˆ’ēŗæ和äø­åˆ’ēŗæēš„ē»„åˆć€‚ä¾‹å¦‚`root. 存å‚Øē»„_1-ē»„1` 怂
 
-* č·Æ径
+* č·Æ径ļ¼ˆPathļ¼‰
 
 åœØ IoTDB äø­ļ¼Œč·Æ径ę˜Æꌇē¬¦åˆä»„äø‹ēŗ¦ęŸēš„č”Øč¾¾å¼ļ¼š
 
@@ -72,7 +72,7 @@ path: LayerName (DOT LayerName)+
 LayerName: Identifier | STAR
 ```
 
-其äø­ STAR äøŗā€œ*ā€ļ¼ŒDOT äøŗā€œ.ā€ć€‚
+其äø­ STAR äøŗ `*` ꈖ `**`ļ¼ŒDOT äøŗ `.`怂
 
 ęˆ‘ä»¬ē§°äø€äøŖč·Æ径äø­åœØäø¤äøŖā€œ.ā€äø­é—“ēš„éƒØ分叫做äø€äøŖ层ēŗ§ļ¼Œåˆ™`root.a.b.c`äøŗäø€äøŖ层ēŗ§äøŗ 4 ēš„č·Æå¾„ć€‚
 
@@ -90,27 +90,15 @@ LayerName: Identifier | STAR
 
 > ę³Øꄏļ¼šstorage group äø­ēš„ LayerName åŖę”ÆęŒę•°å­—ļ¼Œå­—ęƍļ¼Œę±‰å­—ļ¼Œäø‹åˆ’ēŗæ和äø­åˆ’ēŗæć€‚å¦å¤–ļ¼Œå¦‚ęžœåœØ Windows ē³»ē»ŸäøŠéƒØē½²ļ¼Œå­˜å‚Øē»„层ēŗ§åē§°ę˜Æ大小写äøę•ę„Ÿēš„ć€‚ä¾‹å¦‚åŒę—¶åˆ›å»ŗ`root.ln` 和 `root.LN` ę˜Æäøč¢«å…č®øēš„怂
 
-* 前ē¼€č·Æ径
+* č·Æ径ęؔ式ļ¼ˆPath Patternļ¼‰
+  
+äøŗäŗ†ä½æ得åœØč”Øč¾¾å¤šäøŖꗶ闓åŗåˆ—ēš„ę—¶å€™ę›“åŠ ę–¹ä¾æåæ«ę·ļ¼ŒIoTDB äøŗē”Øęˆ·ęä¾›åø¦é€šé…ē¬¦`*`ꈖ`**`ēš„č·Æå¾„ć€‚ē”ØꈷåÆ仄利ē”Øäø¤ē§é€šé…ē¬¦ęž„造å‡ŗꜟꜛēš„č·Æ径ęØ”å¼ć€‚é€šé…ē¬¦åÆ仄å‡ŗēŽ°åœØč·Æ径äø­ēš„ä»»ä½•å±‚ć€‚
 
-前ē¼€č·Æ径ę˜Æꌇäø€äøŖꗶ闓åŗåˆ—ēš„前ē¼€ę‰€åœØēš„č·Æ径ļ¼Œäø€äøŖ前ē¼€č·Æ径包含仄čÆ„č·Æ径äøŗ前ē¼€ēš„ę‰€ęœ‰ę—¶é—“åŗåˆ—ć€‚ä¾‹å¦‚å½“å‰ęˆ‘ä»¬ęœ‰`root.vehicle.device1.sensor1`, `root.vehicle.device1.sensor2`, `root.vehicle.device2.sensor1`äø‰äøŖä¼ ę„Ÿå™Øļ¼Œåˆ™`root.vehicle.device1`前ē¼€č·Æ径包含`root.vehicle.device1.sensor1`态`root.vehicle.device1.sensor2`äø¤äøŖꗶ闓åŗåˆ—ļ¼Œč€ŒäøåŒ…含`root.vehicle.device2.sensor1`怂
+`*`åœØč·Æ径äø­č”Øē¤ŗäø€å±‚ć€‚ä¾‹å¦‚`root.vehicle.*.sensor1`代č”Øēš„ę˜Æ仄`root.vehicle`äøŗ前ē¼€ļ¼Œä»„`sensor1`äøŗ后ē¼€ļ¼Œå±‚ꬔē­‰äŗŽ 4 层ēš„č·Æå¾„ć€‚
 
-* åø¦`*`č·Æ径
-  äøŗäŗ†ä½æ得åœØč”Øč¾¾å¤šäøŖꗶ闓åŗåˆ—ęˆ–č”Øč¾¾å‰ē¼€č·Æ径ēš„ę—¶å€™ę›“åŠ ę–¹ä¾æåæ«ę·ļ¼ŒIoTDB äøŗē”Øęˆ·ęä¾›åø¦`*`č·Æå¾„ć€‚`*`åÆ仄å‡ŗēŽ°åœØč·Æ径äø­ēš„ä»»ä½•å±‚ć€‚ęŒ‰ē…§`*`å‡ŗēŽ°ēš„位ē½®ļ¼Œåø¦`*`č·Æ径åÆ仄分äøŗäø¤ē§ļ¼š
-
-`*`å‡ŗēŽ°åœØč·Æ径ēš„ē»“å°¾ļ¼›
-
-`*`å‡ŗēŽ°åœØč·Æ径ēš„äø­é—“ļ¼›
-
-当`*`å‡ŗēŽ°åœØč·Æ径ēš„ē»“å°¾ę—¶ļ¼Œå…¶ä»£č”Øēš„ę˜Æļ¼ˆ`*`ļ¼‰+ļ¼Œå³äøŗäø€å±‚ęˆ–å¤šå±‚`*`ć€‚ä¾‹å¦‚`root.vehicle.device1.*`代č”Øēš„ę˜Æ`root.vehicle.device1.*`, `root.vehicle.device1.*.*`, `root.vehicle.device1.*.*.*`ē­‰ę‰€ęœ‰ä»„`root.vehicle.device1`äøŗ前ē¼€č·Æ径ēš„大äŗŽē­‰äŗŽ 4 层ēš„č·Æå¾„ć€‚
-
-当`*`å‡ŗēŽ°åœØč·Æ径ēš„äø­é—“ļ¼Œå…¶ä»£č”Øēš„ę˜Æ`*`ęœ¬čŗ«ļ¼Œå³äøŗäø€å±‚ć€‚ä¾‹å¦‚`root.vehicle.*.sensor1`代č”Øēš„ę˜Æ仄`root.vehicle`äøŗ前ē¼€ļ¼Œä»„`sensor1`äøŗ后ē¼€ļ¼Œå±‚ꬔē­‰äŗŽ 4 层ēš„č·Æå¾„ć€‚
-
-> ę³Øꄏļ¼š`*`äøčƒ½ę”¾åœØč·Æå¾„å¼€å¤“ć€‚
-
-> ę³Øꄏļ¼š`*`ę”¾åœØęœ«å°¾ę—¶äøŽå‰ē¼€č·Æ径č”Øꄏē›ø同ļ¼Œä¾‹å¦‚`root.vehicle.*`äøŽ`root.vehicle`äøŗē›øåŒå«ä¹‰ć€‚
-
-> ę³Øꄏļ¼š`*`create 创å»ŗę—¶ļ¼ŒåŽé¢ēš„č·Æå¾„åŒę—¶äøčƒ½å«ęœ‰`*`怂 
+`**`åœØč·Æ径äø­č”Øē¤ŗę˜Æļ¼ˆ`*`ļ¼‰+ļ¼Œå³äøŗäø€å±‚ęˆ–å¤šå±‚`*`ć€‚ä¾‹å¦‚`root.vehicle.device1.**`代č”Øēš„ę˜Æ`root.vehicle.device1.*`, `root.vehicle.device1.*.*`, `root.vehicle.device1.*.*.*`ē­‰ę‰€ęœ‰ä»„`root.vehicle.device1`äøŗ前ē¼€č·Æ径ēš„大äŗŽē­‰äŗŽ 4 层ēš„č·Æ径ļ¼›`root.vehicle.**.sensor1`代č”Øēš„ę˜Æ仄`root.vehicle`äøŗ前ē¼€ļ¼Œä»„`sensor1`äøŗ后ē¼€ļ¼Œå±‚ꬔ大äŗŽē­‰äŗŽ 4 层ēš„č·Æå¾„ć€‚
 
+> ę³Øꄏļ¼š`*`和`**`äøčƒ½ę”¾åœØč·Æå¾„å¼€å¤“ć€‚
 
 
 ### äø€å…ƒć€å¤šå…ƒę—¶é—“åŗåˆ—
diff --git a/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md b/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
index e7177cd..aa23819 100644
--- a/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
+++ b/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
@@ -48,11 +48,12 @@ Msg: 300: root.ln has already been set to storage group.
 
 ### ęŸ„ēœ‹å­˜å‚Øē»„
 
-åœØ存å‚Øē»„创å»ŗ后ļ¼Œęˆ‘们åÆ仄ä½æē”Ø [SHOW STORAGE GROUP](../Appendix/SQL-Reference.md) čƭ叄和 [SHOW STORAGE GROUP \<PrefixPath>](../Appendix/SQL-Reference.md) ę„ęŸ„ēœ‹å­˜å‚Øē»„ļ¼ŒSQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
+åœØ存å‚Øē»„创å»ŗ后ļ¼Œęˆ‘们åÆ仄ä½æē”Ø [SHOW STORAGE GROUP](../Appendix/SQL-Reference.md) čƭ叄和 [SHOW STORAGE GROUP \<PathPattern>](../Appendix/SQL-Reference.md) ę„ęŸ„ēœ‹å­˜å‚Øē»„ļ¼ŒSQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
 
 ```
 IoTDB> show storage group
-IoTDB> show storage group root.ln
+IoTDB> show storage group root.*
+IoTDB> show storage group root.**
 ```
 
 ę‰§č”Œē»“ęžœäøŗļ¼š
@@ -70,13 +71,13 @@ It costs 0.060s
 
 ### 删除存å‚Øē»„
 
-ē”ØꈷåÆ仄ä½æē”Ø`DELETE STORAGE GROUP <PrefixPath>`čƭ叄删除čƄ前ē¼€č·Æ径äø‹ę‰€ęœ‰ēš„å­˜å‚Øē»„怂åœØ删除ēš„čæ‡ē؋äø­ļ¼Œéœ€č¦ę³Øꄏēš„ę˜Æ存å‚Øē»„ēš„ę•°ę®ä¹Ÿä¼šč¢«åˆ é™¤ć€‚
+ē”ØꈷåÆ仄ä½æē”Ø`DELETE STORAGE GROUP <PathPattern>`čƭ叄删除čÆ„č·Æ径ęؔ式匹配ēš„ę‰€ęœ‰ēš„å­˜å‚Øē»„怂åœØ删除ēš„čæ‡ē؋äø­ļ¼Œéœ€č¦ę³Øꄏēš„ę˜Æ存å‚Øē»„ēš„ę•°ę®ä¹Ÿä¼šč¢«åˆ é™¤ć€‚
 
 ```
 IoTDB > DELETE STORAGE GROUP root.ln
 IoTDB > DELETE STORAGE GROUP root.sgcc
 // åˆ é™¤ę‰€ęœ‰ę•°ę®ļ¼Œę—¶é—“åŗåˆ—仄及存å‚Øē»„
-IoTDB > DELETE STORAGE GROUP root.*
+IoTDB > DELETE STORAGE GROUP root.**
 ```
 ## ꗶ闓åŗåˆ—ē®”ē†
 
@@ -103,7 +104,7 @@ error: encoding TS_2DIFF does not support BOOLEAN
 
 ### åˆ é™¤ę—¶é—“åŗåˆ—
 
-ęˆ‘ä»¬åÆ仄ä½æē”Ø`DELETE TimeSeries <PrefixPath>`čÆ­å„ę„åˆ é™¤ęˆ‘ä»¬ä¹‹å‰åˆ›å»ŗēš„ꗶ闓åŗåˆ—怂SQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
+ęˆ‘ä»¬åÆ仄ä½æē”Ø`DELETE TimeSeries <PathPattern>`čÆ­å„ę„åˆ é™¤ęˆ‘ä»¬ä¹‹å‰åˆ›å»ŗēš„ꗶ闓åŗåˆ—怂SQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
 
 ```
 IoTDB> delete timeseries root.ln.wf01.wt01.status
@@ -126,7 +127,7 @@ error: Not support deleting part of aligned timeseies!
 
 ### ęŸ„ēœ‹ę—¶é—“åŗåˆ—
 
-* SHOW LATEST? TIMESERIES prefixPath? showWhereClause? limitClause?
+* SHOW LATEST? TIMESERIES pathPattern? showWhereClause? limitClause?
 
   SHOW TIMESERIES äø­åÆä»„ęœ‰å››ē§åÆ选ēš„子叄ļ¼ŒęŸ„čÆ¢ē»“ęžœäøŗčæ™äŗ›ę—¶é—“åŗåˆ—ēš„ę‰€ęœ‰äæ”ęÆ
 
@@ -140,11 +141,11 @@ error: Not support deleting part of aligned timeseies!
 
 * SHOW TIMESERIES <`Path`>
 
-  čæ”回ē»™å®šč·Æ径ēš„äø‹ēš„ę‰€ęœ‰ę—¶é—“åŗåˆ—äæ”ęÆć€‚å…¶äø­ `Path` éœ€č¦äøŗäø€äøŖ前ē¼€č·Æå¾„ć€åø¦ę˜Ÿč·Æå¾„ęˆ–ę—¶é—“åŗåˆ—č·Æå¾„ć€‚ä¾‹å¦‚ļ¼Œåˆ†åˆ«ęŸ„ēœ‹`root`č·Æ径和`root.ln`č·Æ径äø‹ēš„ꗶ闓åŗåˆ—ļ¼ŒSQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
+  čæ”回ē»™å®šč·Æ径ēš„äø‹ēš„ę‰€ęœ‰ę—¶é—“åŗåˆ—äæ”ęÆć€‚å…¶äø­ `Path` éœ€č¦äøŗäø€äøŖꗶ闓åŗåˆ—č·Æå¾„ęˆ–č·Æ径ęØ”å¼ć€‚ä¾‹å¦‚ļ¼Œåˆ†åˆ«ęŸ„ēœ‹`root`č·Æ径和`root.ln`č·Æ径äø‹ēš„ꗶ闓åŗåˆ—ļ¼ŒSQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
 
 ```
-IoTDB> show timeseries root
-IoTDB> show timeseries root.ln
+IoTDB> show timeseries root.**
+IoTDB> show timeseries root.ln.**
 ```
 
 ę‰§č”Œē»“ęžœåˆ†åˆ«äøŗļ¼š
@@ -176,15 +177,15 @@ Total line number = 4
 It costs 0.004s
 ```
 
-* SHOW TIMESERIES (<`PrefixPath`>)? WhereClause 
+* SHOW TIMESERIES (<`PathPattern`>)? WhereClause 
   
   čæ”回ē»™å®šč·Æ径ēš„äø‹ēš„ę‰€ęœ‰ę»”č¶³ę”ä»¶ēš„ꗶ闓åŗåˆ—äæ”ęÆļ¼ŒSQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
 
 ```
 ALTER timeseries root.ln.wf02.wt02.hardware ADD TAGS unit=c
 ALTER timeseries root.ln.wf02.wt02.status ADD TAGS description=test1
-show timeseries root.ln where unit=c
-show timeseries root.ln where description contains 'test1'
+show timeseries root.ln.** where unit=c
+show timeseries root.ln.** where description contains 'test1'
 ```
 
 ę‰§č”Œē»“ęžœåˆ†åˆ«äøŗļ¼š
@@ -223,8 +224,8 @@ It costs 0.004s
 
 IoTDB ę”Æꌁä½æē”Ø`COUNT TIMESERIES<Path>`ę„ē»Ÿč®”äø€ę”č·Æ径äø­ēš„ꗶ闓åŗåˆ—äøŖꕰ怂SQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
 ```
-IoTDB > COUNT TIMESERIES root
-IoTDB > COUNT TIMESERIES root.ln
+IoTDB > COUNT TIMESERIES root.**
+IoTDB > COUNT TIMESERIES root.ln.**
 IoTDB > COUNT TIMESERIES root.ln.*.*.status
 IoTDB > COUNT TIMESERIES root.ln.wf01.wt01.status
 ```
@@ -256,15 +257,15 @@ It costs 0.004s
 åÆ仄ēœ‹åˆ°ļ¼Œ`root`č¢«å®šä¹‰äøŗ`LEVEL=0`ć€‚é‚£ä¹ˆå½“ä½ č¾“å…„å¦‚äø‹čÆ­å„ę—¶ļ¼š
 
 ```
-IoTDB > COUNT TIMESERIES root GROUP BY LEVEL=1
-IoTDB > COUNT TIMESERIES root.ln GROUP BY LEVEL=2
-IoTDB > COUNT TIMESERIES root.ln.wf01 GROUP BY LEVEL=2
+IoTDB > COUNT TIMESERIES root.** GROUP BY LEVEL=1
+IoTDB > COUNT TIMESERIES root.ln.** GROUP BY LEVEL=2
+IoTDB > COUNT TIMESERIES root.ln.wf01.* GROUP BY LEVEL=2
 ```
 
 你将得到仄äø‹ē»“ęžœļ¼š
 
 ```
-IoTDB> COUNT TIMESERIES root GROUP BY LEVEL=1
+IoTDB> COUNT TIMESERIES root.** GROUP BY LEVEL=1
 +------------+-----+
 |      column|count|
 +------------+-----+
@@ -275,7 +276,7 @@ IoTDB> COUNT TIMESERIES root GROUP BY LEVEL=1
 Total line number = 3
 It costs 0.002s
 
-IoTDB > COUNT TIMESERIES root.ln GROUP BY LEVEL=2
+IoTDB > COUNT TIMESERIES root.ln.** GROUP BY LEVEL=2
 +------------+-----+
 |      column|count|
 +------------+-----+
@@ -285,7 +286,7 @@ IoTDB > COUNT TIMESERIES root.ln GROUP BY LEVEL=2
 Total line number = 2
 It costs 0.002s
 
-IoTDB > COUNT TIMESERIES root.ln.wf01 GROUP BY LEVEL=2
+IoTDB > COUNT TIMESERIES root.ln.wf01.* GROUP BY LEVEL=2
 +------------+-----+
 |      column|count|
 +------------+-----+
@@ -348,10 +349,10 @@ ALTER timeseries root.turbine.d1.s1 UPSERT ALIAS=newAlias TAGS(tag2=newV2, tag3=
 ### ęŸ„ēœ‹å­č·Æ径
 
 ```
-SHOW CHILD PATHS prefixPath
+SHOW CHILD PATHS pathPattern
 ```
 
-åÆä»„ęŸ„ēœ‹ę­¤å‰ē¼€č·Æ径ēš„äø‹äø€å±‚ēš„ę‰€ęœ‰č·Æ径ļ¼Œå‰ē¼€č·Æ径允č®øä½æē”Ø * 通配ē¬¦ć€‚
+åÆä»„ęŸ„ēœ‹ę­¤č·Æ径ęØ”å¼ę‰€åŒ¹é…ēš„ę‰€ęœ‰č·Æ径ēš„äø‹äø€å±‚ēš„ę‰€ęœ‰č·Æ径ļ¼Œå³pathPattern.*ę‰€åŒ¹é…ēš„č·Æå¾„ć€‚
 
 ē¤ŗ例ļ¼š
 
@@ -380,10 +381,10 @@ SHOW CHILD PATHS prefixPath
 ### ęŸ„ēœ‹å­čŠ‚ē‚¹
 
 ```
-SHOW CHILD NODES prefixPath
+SHOW CHILD NODES pathPattern
 ```
 
-åÆä»„ęŸ„ēœ‹ę­¤å‰ē¼€č·Æ径ēš„äø‹äø€å±‚ēš„ę‰€ęœ‰čŠ‚ē‚¹ć€‚
+åÆä»„ęŸ„ēœ‹ę­¤č·Æ径ęØ”å¼ę‰€åŒ¹é…ēš„节ē‚¹ēš„äø‹äø€å±‚ēš„ę‰€ęœ‰čŠ‚ē‚¹ć€‚
 
 ē¤ŗ例ļ¼š
 
@@ -410,12 +411,12 @@ SHOW CHILD NODES prefixPath
 
 ### ē»Ÿč®”节ē‚¹ę•°
 
-IoTDB ę”Æꌁä½æē”Ø`COUNT NODES <PrefixPath> LEVEL=<INTEGER>`ę„ē»Ÿč®”当前 Metadata ꠑäø‹ęŒ‡å®šå±‚ēŗ§ēš„节ē‚¹äøŖꕰļ¼Œčæ™ę”čƭ叄åÆ仄ē”Øę„ē»Ÿč®”č®¾å¤‡ę•°ć€‚ä¾‹å¦‚ļ¼š
+IoTDB ę”Æꌁä½æē”Ø`COUNT NODES <PathPattern> LEVEL=<INTEGER>`ę„ē»Ÿč®”当前 Metadata ꠑäø‹ęŒ‡å®šå±‚ēŗ§ēš„节ē‚¹äøŖꕰļ¼Œčæ™ę”čƭ叄åÆ仄ē”Øę„ē»Ÿč®”č®¾å¤‡ę•°ć€‚ä¾‹å¦‚ļ¼š
 
 ```
-IoTDB > COUNT NODES root LEVEL=2
-IoTDB > COUNT NODES root.ln LEVEL=2
-IoTDB > COUNT NODES root.ln.wf01 LEVEL=3
+IoTDB > COUNT NODES root.** LEVEL=2
+IoTDB > COUNT NODES root.ln.** LEVEL=2
+IoTDB > COUNT NODES root.ln.wf01.* LEVEL=3
 ```
 
 åƹäŗŽäøŠé¢ęåˆ°ēš„例子和 Metadata Treeļ¼Œä½ åÆä»„čŽ·å¾—å¦‚äø‹ē»“ęžœļ¼š
@@ -447,22 +448,21 @@ It costs 0.002s
 ```
 
 > ę³Øꄏļ¼šę—¶é—“åŗåˆ—ēš„č·Æ径åŖę˜Æčæ‡ę»¤ę”件ļ¼ŒäøŽ level ēš„å®šä¹‰ę— å…³ć€‚
-其äø­`PrefixPath`åÆ仄包含`*`ļ¼Œä½†ę˜Æ`*`及其后ēš„ę‰€ęœ‰čŠ‚ē‚¹å°†č¢«åæ½ē•„ļ¼Œä»…åœØ`*`前ēš„前ē¼€č·Æå¾„ęœ‰ę•ˆć€‚
 
 ### ęŸ„ēœ‹č®¾å¤‡
 
-* SHOW DEVICES prefixPath? (WITH STORAGE GROUP)? limitClause? #showDevices
+* SHOW DEVICES pathPattern? (WITH STORAGE GROUP)? limitClause? #showDevices
 
 äøŽ `Show Timeseries` ē›øä¼¼ļ¼ŒIoTDB ē›®å‰ä¹Ÿę”Æꌁäø¤ē§ę–¹å¼ęŸ„ēœ‹č®¾å¤‡ć€‚
 
-* `SHOW DEVICES` čÆ­å„ę˜¾ē¤ŗå½“å‰ę‰€ęœ‰ēš„č®¾å¤‡äæ”ęÆļ¼Œē­‰ä»·äŗŽ `SHOW DEVICES root`怂
-* `SHOW DEVICES <PrefixPath>` čÆ­å„č§„å®šäŗ† `PrefixPath`ļ¼Œčæ”回åœØē»™å®šēš„前ē¼€č·Æ径äø‹ēš„č®¾å¤‡äæ”ęÆ怂
+* `SHOW DEVICES` čÆ­å„ę˜¾ē¤ŗå½“å‰ę‰€ęœ‰ēš„č®¾å¤‡äæ”ęÆļ¼Œē­‰ä»·äŗŽ `SHOW DEVICES root.**`怂
+* `SHOW DEVICES <PathPattern>` čÆ­å„č§„å®šäŗ† `PathPattern`ļ¼Œčæ”回ē»™å®šēš„č·Æ径ęØ”å¼ę‰€åŒ¹é…ēš„č®¾å¤‡äæ”ęÆ怂
 
 SQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
 
 ```
 IoTDB> show devices
-IoTDB> show devices root.ln
+IoTDB> show devices root.ln.**
 ```
 
 ä½ åÆä»„čŽ·å¾—å¦‚äø‹ę•°ę®ļ¼š
@@ -491,14 +491,14 @@ It costs 0.001s
 
 ęŸ„ēœ‹č®¾å¤‡åŠå…¶å­˜å‚Øē»„äæ”ęÆļ¼ŒåÆ仄ä½æē”Ø `SHOW DEVICES WITH STORAGE GROUP` čÆ­å„ć€‚
 
-* `SHOW DEVICES WITH STORAGE GROUP` čÆ­å„ę˜¾ē¤ŗå½“å‰ę‰€ęœ‰ēš„č®¾å¤‡äæ”ęÆå’Œå…¶ę‰€åœØēš„å­˜å‚Øē»„ļ¼Œē­‰ä»·äŗŽ `SHOW DEVICES root`怂
-* `SHOW DEVICES <PrefixPath> WITH STORAGE GROUP` čÆ­å„č§„å®šäŗ† `PrefixPath`ļ¼Œčæ”回åœØē»™å®šēš„前ē¼€č·Æ径äø‹ēš„č®¾å¤‡äæ”ęÆå’Œå…¶ę‰€åœØēš„å­˜å‚Øē»„怂
+* `SHOW DEVICES WITH STORAGE GROUP` čÆ­å„ę˜¾ē¤ŗå½“å‰ę‰€ęœ‰ēš„č®¾å¤‡äæ”ęÆå’Œå…¶ę‰€åœØēš„å­˜å‚Øē»„ļ¼Œē­‰ä»·äŗŽ `SHOW DEVICES root.**`怂
+* `SHOW DEVICES <PathPattern> WITH STORAGE GROUP` čÆ­å„č§„å®šäŗ† `PathPattern`ļ¼Œčæ”回ē»™å®šēš„č·Æ径ęØ”å¼ę‰€åŒ¹é…ēš„č®¾å¤‡äæ”ęÆå’Œå…¶ę‰€åœØēš„å­˜å‚Øē»„怂
 
 SQL čƭ叄如äø‹ę‰€ē¤ŗļ¼š
 
 ```
 IoTDB> show devices with storage group
-IoTDB> show devices root.ln with storage group
+IoTDB> show devices root.ln.** with storage group
 ```
 
 ä½ åÆä»„čŽ·å¾—å¦‚äø‹ę•°ę®ļ¼š
diff --git a/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md b/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
index 4795de2..2c3f21d 100644
--- a/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
+++ b/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
@@ -213,7 +213,7 @@ It costs 0.014s
 IoTDB åœØ 0.11 ē‰ˆęœ¬å¼€å§‹ę”Æꌁ 'order by time' čƭ叄ļ¼Œē”ØäŗŽåƹē»“ęžœęŒ‰ē…§ę—¶é—“čæ›č”Œé™åŗå±•ē¤ŗć€‚ä¾‹å¦‚ļ¼ŒSQL čƭ叄äøŗļ¼š
 
 ```sql
-select * from root.ln where time > 1 order by time desc limit 10;
+select * from root.ln.** where time > 1 order by time desc limit 10;
 ```
 
 čÆ­å„ę‰§č”Œēš„ē»“ęžœäøŗļ¼š
@@ -1616,13 +1616,13 @@ It costs 0.005s
 * å¦‚ęžœē»“ęžœé›†äø­ļ¼Œä»»ę„äø€åˆ—äøŗ nullļ¼Œåˆ™čæ‡ę»¤ęŽ‰čÆ„č”Œļ¼›å³čŽ·å¾—ēš„ē»“ęžœé›†äøåŒ…含任何ē©ŗ值
 
 ```sql
-select * from root.ln.* where time <= 2017-11-01T00:01:00 WITHOUT NULL ANY
+select * from root.ln.** where time <= 2017-11-01T00:01:00 WITHOUT NULL ANY
 ```
 
 * åœØé™é‡‡ę ·ęŸ„čÆ¢äø­ļ¼Œå¦‚ęžœē»“ęžœé›†ēš„ęŸäø€č”Œę‰€ęœ‰åˆ—都äøŗ nullļ¼Œåˆ™čæ‡ę»¤ęŽ‰čÆ„č”Œļ¼›å³čŽ·å¾—ēš„ē»“ęžœé›†äøåŒ…å«ę‰€ęœ‰å€¼éƒ½äøŗ null ēš„č”Œ
 
 ```sql
-select * from root.ln.* where time <= 2017-11-01T00:01:00 WITHOUT NULL ALL
+select * from root.ln.** where time <= 2017-11-01T00:01:00 WITHOUT NULL ALL
 ```
 
 ### 其他åÆ¹é½ę–¹å¼ēš„ē»“ęžœé›†
@@ -1634,7 +1634,7 @@ IoTDB ę”ÆęŒå¦å¤–äø¤ē§ē»“ęžœčæ”回形式ļ¼šęŒ‰č®¾å¤‡ę—¶é—“åƹ齐 'align by dev
 'align by device' åÆ¹é½ę–¹å¼äø‹ļ¼Œč®¾å¤‡ ID 会单ē‹¬ä½œäøŗäø€åˆ—å‡ŗēŽ°ć€‚åœØ select 子叄äø­å†™äŗ†å¤šå°‘列ļ¼Œęœ€ē»ˆē»“ęžœå°±ä¼šęœ‰čÆ„åˆ—ę•°+2 ļ¼ˆę—¶é—“åˆ—å’Œč®¾å¤‡åå­—åˆ—ļ¼‰ć€‚SQL 形如ļ¼š
 
 ```sql
-select * from root.ln.* where time <= 2017-11-01T00:01:00 align by device
+select * from root.ln.** where time <= 2017-11-01T00:01:00 align by device
 ```
 
 ē»“ęžœå¦‚äø‹ļ¼š
@@ -1661,7 +1661,7 @@ It costs 0.012s
 SQL 形如ļ¼š
 
 ```sql
-select * from root.ln.* where time <= 2017-11-01T00:01:00 disable align
+select * from root.ln.** where time <= 2017-11-01T00:01:00 disable align
 ```
 
 ē»“ęžœå¦‚äø‹ļ¼š
@@ -1727,13 +1727,8 @@ delete from root.ln.wf02.status
 
 ### å¤šä¼ ę„Ÿå™Øꗶ闓åŗåˆ—值删除    
 
-当 ln 集团 wf02 子ē«™ēš„ wt02 č®¾å¤‡åœØ 2017-11-01 16:26:00 之前ēš„ä¾›ē”µēŠ¶ę€å’Œč®¾å¤‡ē”¬ä»¶ē‰ˆęœ¬éƒ½éœ€č¦åˆ é™¤ļ¼Œę­¤ę—¶åÆ仄ä½æē”Øå«ä¹‰ę›“å¹æēš„ [前ē¼€č·Æå¾„ęˆ–åø¦`*`č·Æ径](../Data-Concept/Data-Model-and-Terminology.md) čæ›č”Œåˆ é™¤ę“ä½œļ¼Œčæ›č”Œę­¤ę“ä½œēš„ SQL čƭ叄äøŗļ¼š
+当 ln 集团 wf02 子ē«™ēš„ wt02 č®¾å¤‡åœØ 2017-11-01 16:26:00 之前ēš„ä¾›ē”µēŠ¶ę€å’Œč®¾å¤‡ē”¬ä»¶ē‰ˆęœ¬éƒ½éœ€č¦åˆ é™¤ļ¼Œę­¤ę—¶åÆ仄ä½æē”Øå«ä¹‰ę›“å¹æēš„ [č·Æ径ęؔ式ļ¼ˆPath Patternļ¼‰](../Data-Concept/Data-Model-and-Terminology.md) čæ›č”Œåˆ é™¤ę“ä½œļ¼Œčæ›č”Œę­¤ę“ä½œēš„ SQL čƭ叄äøŗļ¼š
 
-```sql
-delete from root.ln.wf02.wt02 where time <= 2017-11-01T16:26:00;
-```
-
-ꈖ
 
 ```sql
 delete from root.ln.wf02.wt02.* where time <= 2017-11-01T16:26:00;
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
index a943896..5f132c6 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
@@ -119,7 +119,8 @@ public class IoTDBConstant {
   public static final String COLUMN_TRIGGER_STATUS_STARTED = "started";
   public static final String COLUMN_TRIGGER_STATUS_STOPPED = "stopped";
 
-  public static final String PATH_WILDCARD = "*";
+  public static final String ONE_LEVEL_PATH_WILDCARD = "*";
+  public static final String MULTI_LEVEL_PATH_WILDCARD = "**";
   public static final String TIME = "time";
 
   // sdt parameters
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index c06645e..1c8eb7e 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -715,7 +715,7 @@ public class StorageEngine implements IService {
       TimePartitionFilter timePartitionFilter)
       throws StorageEngineException {
     try {
-      List<PartialPath> sgPaths = IoTDB.metaManager.searchAllRelatedStorageGroups(path);
+      List<PartialPath> sgPaths = IoTDB.metaManager.getBelongedStorageGroups(path);
       for (PartialPath storageGroupPath : sgPaths) {
         // storage group has no data
         if (!processorMap.containsKey(storageGroupPath)) {
@@ -737,7 +737,7 @@ public class StorageEngine implements IService {
       PartialPath path, long planIndex, TimePartitionFilter timePartitionFilter)
       throws StorageEngineException {
     try {
-      List<PartialPath> sgPaths = IoTDB.metaManager.searchAllRelatedStorageGroups(path);
+      List<PartialPath> sgPaths = IoTDB.metaManager.getBelongedStorageGroups(path);
       for (PartialPath storageGroupPath : sgPaths) {
         // storage group has no data
         if (!processorMap.containsKey(storageGroupPath)) {
@@ -877,7 +877,7 @@ public class StorageEngine implements IService {
     }
     String device = deviceSet.iterator().next();
     PartialPath devicePath = new PartialPath(device);
-    PartialPath storageGroupPath = IoTDB.metaManager.getStorageGroupPath(devicePath);
+    PartialPath storageGroupPath = IoTDB.metaManager.getBelongedStorageGroup(devicePath);
     getProcessorDirectly(storageGroupPath).loadNewTsFile(newTsFileResource);
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
index bcc774a..e3416db 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
@@ -145,7 +145,8 @@ public class MergeTask implements Callable<Void> {
 
     mergeLogger.logFiles(resource);
 
-    Set<PartialPath> devices = IoTDB.metaManager.getDevices(new PartialPath(storageGroupName));
+    Set<PartialPath> devices =
+        IoTDB.metaManager.getDevicesByPrefix(new PartialPath(storageGroupName));
     Map<PartialPath, IMeasurementSchema> measurementSchemaMap = new HashMap<>();
     List<PartialPath> unmergedSeries = new ArrayList<>();
     for (PartialPath device : devices) {
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 d4d6977..717a971 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
@@ -1897,7 +1897,7 @@ public class StorageGroupProcessor {
     List<ModificationFile> updatedModFiles = new ArrayList<>();
 
     try {
-      Set<PartialPath> devicePaths = IoTDB.metaManager.getDevices(path.getDevicePath());
+      Set<PartialPath> devicePaths = IoTDB.metaManager.getBelongedDevices(path);
       for (PartialPath device : devicePaths) {
         Long lastUpdateTime = null;
         for (Map<String, Long> latestTimeMap : latestTimeForEachDevice.values()) {
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 8767af7..dacdda7 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
@@ -27,7 +27,6 @@ import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
 import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
@@ -41,6 +40,7 @@ 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.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTree;
 import org.apache.iotdb.db.metadata.tag.TagManager;
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.metadata.template.TemplateManager;
@@ -112,6 +112,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
 import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
 import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
 
@@ -119,6 +120,37 @@ import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARA
  * This class takes the responsibility of serialization of all the metadata info and persistent it
  * into files. This class contains all the interfaces to modify the metadata for delta system. All
  * the operations will be insert into the logs temporary in case the downtime of the delta system.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>MManager Singleton
+ *   <li>Interfaces and Implementation of MManager initialization态snapshot态recover and clear
+ *   <li>Interfaces for CQ
+ *   <li>Interfaces and Implementation for Timeseries operation
+ *   <li>Interfaces and Implementation for StorageGroup and TTL operation
+ *   <li>Interfaces for get and auto create device
+ *   <li>Interfaces for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for metadata count
+ *         <li>Interfaces for level Node info Query
+ *         <li>Interfaces for StorageGroup and TTL info Query
+ *         <li>Interfaces for Entity/Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *       </ol>
+ *   <li>Interfaces and methods for MNode query
+ *   <li>Interfaces for alias and tag/attribute operations
+ *   <li>Interfaces only for Cluster module usage
+ *   <li>Interfaces for lastCache operations
+ *   <li>Interfaces and Implementation for InsertPlan process
+ *   <li>Interfaces and Implementation for Template operations
+ *   <li>TestOnly Interfaces
+ * </ol>
  */
 @SuppressWarnings("java:S1135") // ignore todos
 public class MManager {
@@ -158,6 +190,7 @@ public class MManager {
   private TagManager tagManager = TagManager.getInstance();
   private TemplateManager templateManager = TemplateManager.getInstance();
 
+  // region MManager Singleton
   private static class MManagerHolder {
 
     private MManagerHolder() {
@@ -167,6 +200,13 @@ public class MManager {
     private static final MManager INSTANCE = new MManager();
   }
 
+  /** we should not use this function in other place, but only in IoTDB class */
+  public static MManager getInstance() {
+    return MManagerHolder.INSTANCE;
+  }
+  // endregion
+
+  // region Interfaces and Implementation of MManager initialization态snapshot态recover and clear
   protected MManager() {
     mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
     mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
@@ -210,11 +250,6 @@ public class MManager {
     }
   }
 
-  /** we should not use this function in other place, but only in IoTDB class */
-  public static MManager getInstance() {
-    return MManagerHolder.INSTANCE;
-  }
-
   // Because the writer will be used later and should not be closed here.
   @SuppressWarnings("squid:S2093")
   public synchronized void init() {
@@ -242,18 +277,6 @@ public class MManager {
     initialized = true;
   }
 
-  /**
-   * Attention!!!!!, this method could only be used for Tests involving multiple mmanagers. The
-   * singleton of templateManager and tagManager will cause interference between mmanagers if one of
-   * the mmanagers invoke init method or clear method
-   */
-  @TestOnly
-  public void initForMultiMManagerTest() {
-    templateManager = TemplateManager.getNewInstanceForTest();
-    tagManager = TagManager.getNewInstanceForTest();
-    init();
-  }
-
   /** @return line number of the logFile */
   @SuppressWarnings("squid:S3776")
   private int initFromLog(File logFile) throws IOException {
@@ -263,7 +286,7 @@ public class MManager {
       int idx = 0;
       try (MLogReader mLogReader =
           new MLogReader(config.getSchemaDir(), MetadataConstant.METADATA_LOG); ) {
-        idx = applyMlog(mLogReader);
+        idx = applyMLog(mLogReader);
         logger.debug(
             "spend {} ms to deserialize mtree from mlog.bin", System.currentTimeMillis() - time);
         return idx;
@@ -275,7 +298,7 @@ public class MManager {
     }
   }
 
-  private int applyMlog(MLogReader mLogReader) {
+  private int applyMLog(MLogReader mLogReader) {
     int idx = 0;
     PhysicalPlan plan;
     while (mLogReader.hasNext()) {
@@ -409,7 +432,9 @@ public class MManager {
         logger.error("Unrecognizable command {}", plan.getOperatorType());
     }
   }
+  // endregion
 
+  // region Interfaces for CQ
   public void createContinuousQuery(CreateContinuousQueryPlan plan) throws IOException {
     logWriter.createContinuousQuery(plan);
   }
@@ -417,24 +442,15 @@ public class MManager {
   public void dropContinuousQuery(DropContinuousQueryPlan plan) throws IOException {
     logWriter.dropContinuousQuery(plan);
   }
+  // endregion
+
+  // region Interfaces and Implementation for Timeseries operation
+  // including create and delete
 
   public void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException {
     createTimeseries(plan, -1);
   }
 
-  private void ensureStorageGroup(PartialPath path) throws MetadataException {
-    try {
-      mtree.getStorageGroupPath(path);
-    } catch (StorageGroupNotSetException e) {
-      if (!config.isAutoCreateSchemaEnabled()) {
-        throw e;
-      }
-      PartialPath storageGroupPath =
-          MetaUtils.getStorageGroupPathByLevel(path, config.getDefaultStorageGroupLevel());
-      setStorageGroup(storageGroupPath);
-    }
-  }
-
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws MetadataException {
     if (!allowToCreateNewSeries) {
@@ -582,8 +598,27 @@ public class MManager {
     }
   }
 
+  private void ensureStorageGroup(PartialPath path) throws MetadataException {
+    try {
+      mtree.getBelongedStorageGroup(path);
+    } catch (StorageGroupNotSetException e) {
+      if (!config.isAutoCreateSchemaEnabled()) {
+        throw e;
+      }
+      PartialPath storageGroupPath =
+          MetaUtils.getStorageGroupPathByLevel(path, config.getDefaultStorageGroupLevel());
+      try {
+        setStorageGroup(storageGroupPath);
+      } catch (StorageGroupAlreadySetException storageGroupAlreadySetException) {
+        // do nothing
+        // concurrent timeseries creation may result concurrent ensureStorageGroup
+        // it's ok that the storageGroup has already been set
+      }
+    }
+  }
+
   /**
-   * Delete all timeseries under the given path, may cross different storage group
+   * Delete all timeseries matching the given path pattern, may cross different storage group
    *
    * @param prefixPath path to be deleted, could be root or a prefix path or a full path
    * @return deletion failed Timeseries
@@ -646,12 +681,6 @@ public class MManager {
     }
   }
 
-  /** remove the node from the tag inverted index */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void removeFromTagInvertedIndex(IMeasurementMNode node) throws IOException {
-    tagManager.removeFromTagInvertedIndex(node);
-  }
-
   /**
    * @param path full path from root to leaf node
    * @return After delete if the storage group is empty, return its path, otherwise return null
@@ -684,6 +713,10 @@ public class MManager {
     }
     return storageGroupPath;
   }
+  // endregion
+
+  // region Interfaces and Implementation for StorageGroup and TTL operation
+  // including sg set and delete, and ttl set
 
   /**
    * Set storage group of the given path to MTree.
@@ -712,7 +745,8 @@ public class MManager {
   public void deleteStorageGroups(List<PartialPath> storageGroups) throws MetadataException {
     try {
       for (PartialPath storageGroup : storageGroups) {
-        totalSeriesNumber.addAndGet(-mtree.getAllTimeseriesCount(storageGroup));
+        totalSeriesNumber.addAndGet(
+            -mtree.getAllTimeseriesCount(storageGroup.concatNode(MULTI_LEVEL_PATH_WILDCARD)));
         // clear cached MNode
         if (!allowToCreateNewSeries
             && totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE < MTREE_SIZE_THRESHOLD) {
@@ -744,88 +778,197 @@ public class MManager {
     }
   }
 
-  /**
-   * Check if the given path is storage group or not.
-   *
-   * @param path Format: root.node.(node)*
-   * @apiNote :for cluster
-   */
-  public boolean isStorageGroup(PartialPath path) {
-    return mtree.isStorageGroup(path);
+  public void setTTL(PartialPath storageGroup, long dataTTL) throws MetadataException, IOException {
+    getStorageGroupNodeByStorageGroupPath(storageGroup).setDataTTL(dataTTL);
+    if (!isRecovering) {
+      logWriter.setTTL(storageGroup, dataTTL);
+    }
   }
+  // endregion
 
+  // region Interfaces for get and auto create device
   /**
-   * Get series type for given seriesPath.
+   * get device node, if the storage group is not set, create it when autoCreateSchema is true
+   *
+   * <p>(we develop this method as we need to get the node's lock after we get the lock.writeLock())
    *
-   * @param path full path
+   * @param path path
+   * @param allowCreateSg The stand-alone version can create an sg at will, but the cluster version
+   *     needs to make the Meta group aware of the creation of an SG, so an exception needs to be
+   *     thrown here
    */
-  public TSDataType getSeriesType(PartialPath path) throws MetadataException {
-    if (path.equals(SQLConstant.TIME_PATH)) {
-      return TSDataType.INT64;
-    }
-
-    IMeasurementSchema schema = mtree.getSchema(path);
-    if (schema instanceof MeasurementSchema) {
-      return schema.getType();
-    } else {
-      if (((VectorPartialPath) path).getSubSensorsList().size() != 1) {
-        return TSDataType.VECTOR;
-      } else {
-        String subSensor = ((VectorPartialPath) path).getSubSensor(0);
-        List<String> measurements = schema.getSubMeasurementsList();
-        return schema.getSubMeasurementsTSDataTypeList().get(measurements.indexOf(subSensor));
+  public IMNode getDeviceNodeWithAutoCreate(
+      PartialPath path, boolean autoCreateSchema, boolean allowCreateSg, int sgLevel)
+      throws IOException, MetadataException {
+    IMNode node;
+    boolean shouldSetStorageGroup;
+    try {
+      node = mNodeCache.get(path);
+      return node;
+    } catch (CacheException e) {
+      if (!autoCreateSchema) {
+        throw new PathNotExistException(path.getFullPath());
       }
+      shouldSetStorageGroup = e.getCause() instanceof StorageGroupNotSetException;
     }
-  }
 
-  public IMeasurementMNode[] getMNodes(PartialPath deviceId, String[] measurements)
-      throws MetadataException {
-    IMeasurementMNode[] mNodes = new IMeasurementMNode[measurements.length];
-    for (int i = 0; i < mNodes.length; i++) {
-      try {
-        mNodes[i] = (IMeasurementMNode) getNodeByPath(deviceId.concatNode(measurements[i]));
-      } catch (PathNotExistException ignored) {
-        logger.warn("{} does not exist in {}", measurements[i], deviceId);
+    try {
+      if (shouldSetStorageGroup) {
+        if (allowCreateSg) {
+          PartialPath storageGroupPath = MetaUtils.getStorageGroupPathByLevel(path, sgLevel);
+          setStorageGroup(storageGroupPath);
+        } else {
+          throw new StorageGroupNotSetException(path.getFullPath());
+        }
       }
-      if (mNodes[i] == null && !IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
-        throw new MetadataException(measurements[i] + " does not exist in " + deviceId);
+      node = mtree.getDeviceNodeWithAutoCreating(path, sgLevel);
+      if (!(node.isStorageGroup())) {
+        logWriter.autoCreateDeviceMNode(new AutoCreateDeviceMNodePlan(node.getPartialPath()));
       }
+      return node;
+    } catch (StorageGroupAlreadySetException e) {
+      if (e.isHasChild()) {
+        // if setStorageGroup failure is because of child, the deviceNode should not be created.
+        // Timeseries can't be create under a deviceNode without storageGroup.
+        throw e;
+      }
+      // ignore set storage group concurrently
+      node = mtree.getDeviceNodeWithAutoCreating(path, sgLevel);
+      if (!(node.isStorageGroup())) {
+        logWriter.autoCreateDeviceMNode(new AutoCreateDeviceMNodePlan(node.getPartialPath()));
+      }
+      return node;
     }
-    return mNodes;
   }
 
+  /** !!!!!!Attention!!!!! must call the return node's readUnlock() if you call this method. */
+  public IMNode getDeviceNodeWithAutoCreate(PartialPath path)
+      throws MetadataException, IOException {
+    return getDeviceNodeWithAutoCreate(
+        path, config.isAutoCreateSchemaEnabled(), true, config.getDefaultStorageGroupLevel());
+  }
+
+  public void autoCreateDeviceMNode(AutoCreateDeviceMNodePlan plan) throws MetadataException {
+    mtree.getDeviceNodeWithAutoCreating(plan.getPath(), config.getDefaultStorageGroupLevel());
+  }
+  // endregion
+
+  // region Interfaces for metadata info Query
   /**
-   * Get all devices under given prefixPath.
+   * Check whether the path exists.
    *
-   * @param prefixPath a prefix of 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.
-   * @return A HashSet instance which stores devices paths with given prefixPath.
+   * @param path a full path or a prefix path
    */
-  public Set<PartialPath> getDevices(PartialPath prefixPath) throws MetadataException {
-    return mtree.getDevices(prefixPath);
+  public boolean isPathExist(PartialPath path) {
+    return mtree.isPathExist(path);
   }
 
-  public List<ShowDevicesResult> getDevices(ShowDevicesPlan plan) throws MetadataException {
-    return mtree.getDevices(plan);
+  /** Get metadata in string */
+  public String getMetadataInString() {
+    return TIME_SERIES_TREE_HEADER + mtree;
+  }
+
+  // region Interfaces for metadata count
+
+  public long getTotalSeriesNumber() {
+    return totalSeriesNumber.get();
   }
 
   /**
-   * Get all nodes from the given level
+   * To calculate the count of timeseries matching given path. The path could be a pattern of a full
+   * path, may contain wildcard.
+   */
+  public int getAllTimeseriesCount(PartialPath path) throws MetadataException {
+    return mtree.getAllTimeseriesCount(path);
+  }
+
+  /** To calculate the count of devices for given path pattern. */
+  public int getDevicesNum(PartialPath path) throws MetadataException {
+    return mtree.getDevicesNum(path);
+  }
+
+  /** To calculate the count of storage group for given path pattern. */
+  public int getStorageGroupNum(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNum(path);
+  }
+
+  /**
+   * To calculate the count of nodes in the given level for given path pattern.
+   *
+   * @param path a path pattern or a full path
+   * @param level the level should match the level of the path
+   */
+  public int getNodesCountInGivenLevel(PartialPath path, int level) throws MetadataException {
+    return mtree.getNodesCountInGivenLevel(path, level);
+  }
+
+  // endregion
+
+  // region Interfaces for level Node info Query
+  /**
+   * Get all nodes matching the given path pattern in the given level. The level of the path should
+   * match the nodeLevel. 1. The given level equals the path level with out **, e.g. give path
+   * root.*.d.* and the level should be 4. 2. The given level is greater than path level with **,
+   * e.g. give path root.** and the level could be 2 or 3.
    *
-   * @param prefixPath can be a prefix of a full path. Can not be a full path. can not have
-   *     wildcard. But, the level of the prefixPath can be smaller than the given level, e.g.,
-   *     prefixPath = root.a while the given level is 5
-   * @param nodeLevel the level can not be smaller than the level of the prefixPath
+   * @param pathPattern can be a pattern of a full path.
+   * @param nodeLevel the level should match the level of the path
    * @return A List instance which stores all node at given level
    */
-  public List<PartialPath> getNodesList(PartialPath prefixPath, int nodeLevel)
+  public List<PartialPath> getNodesListInGivenLevel(PartialPath pathPattern, int nodeLevel)
       throws MetadataException {
-    return getNodesList(prefixPath, nodeLevel, null);
+    return getNodesListInGivenLevel(pathPattern, nodeLevel, null);
+  }
+
+  public List<PartialPath> getNodesListInGivenLevel(
+      PartialPath pathPattern, int nodeLevel, StorageGroupFilter filter) throws MetadataException {
+    return mtree.getNodesListInGivenLevel(pathPattern, nodeLevel, filter);
   }
 
-  public List<PartialPath> getNodesList(
-      PartialPath prefixPath, int nodeLevel, StorageGroupFilter filter) throws MetadataException {
-    return mtree.getNodesList(prefixPath, nodeLevel, filter);
+  /**
+   * Get child node path in the next level of the given path pattern.
+   *
+   * <p>give pathPattern and the child nodes is those matching pathPattern.*
+   *
+   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
+   * return [root.sg1.d1, root.sg1.d2]
+   *
+   * @param pathPattern The given path
+   * @return All child nodes' seriesPath(s) of given seriesPath.
+   */
+  public Set<String> getChildNodePathInNextLevel(PartialPath pathPattern) throws MetadataException {
+    return mtree.getChildNodePathInNextLevel(pathPattern);
+  }
+
+  /**
+   * Get child node in the next level of the given path pattern.
+   *
+   * <p>give pathPattern and the child nodes is those matching pathPattern.*
+   *
+   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
+   * return [d1, d2] given path = root.sg.d1 return [s1,s2]
+   *
+   * @return All child nodes of given seriesPath.
+   */
+  public Set<String> getChildNodeNameInNextLevel(PartialPath pathPattern) throws MetadataException {
+    return mtree.getChildNodeNameInNextLevel(pathPattern);
+  }
+  // endregion
+
+  // region Interfaces for StorageGroup and TTL info Query
+  /**
+   * Check if the given path is storage group or not.
+   *
+   * @param path Format: root.node.(node)*
+   * @apiNote :for cluster
+   */
+  public boolean isStorageGroup(PartialPath path) {
+    return mtree.isStorageGroup(path);
+  }
+
+  /** Check whether the given path contains a storage group */
+  public boolean checkStorageGroupByPath(PartialPath path) {
+    return mtree.checkStorageGroupByPath(path);
   }
 
   /**
@@ -833,78 +976,144 @@ public class MManager {
    *
    * <p>e.g., root.sg1 is a storage group and path = root.sg1.d1, return root.sg1
    *
+   * @param path only full path, cannot be path pattern
    * @return storage group in the given path
    */
-  public PartialPath getStorageGroupPath(PartialPath path) throws StorageGroupNotSetException {
-    return mtree.getStorageGroupPath(path);
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
+    return mtree.getBelongedStorageGroup(path);
   }
 
-  /** Get all storage group paths */
-  public List<PartialPath> getAllStorageGroupPaths() {
-    return mtree.getAllStorageGroupPaths();
+  /**
+   * Get the storage group that given path pattern matches or belongs to.
+   *
+   * <p>Suppose we have (root.sg1.d1.s1, root.sg2.d2.s2), refer the following cases: 1. given path
+   * "root.sg1", ("root.sg1") will be returned. 2. given path "root.*", ("root.sg1", "root.sg2")
+   * will be returned. 3. given path "root.*.d1.s1", ("root.sg1", "root.sg2") will be returned.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @return a list contains all storage groups related to given path pattern
+   */
+  public List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    return mtree.getBelongedStorageGroups(pathPattern);
   }
 
-  public List<PartialPath> searchAllRelatedStorageGroups(PartialPath path)
+  /**
+   * Get all storage group matching given path pattern.
+   *
+   * @param pathPattern a pattern of a full path
+   * @return A ArrayList instance which stores storage group paths matching given path pattern.
+   */
+  public List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern)
       throws MetadataException {
-    return mtree.searchAllRelatedStorageGroups(path);
+    return mtree.getMatchedStorageGroups(pathPattern);
+  }
+
+  /** Get all storage group paths */
+  public List<PartialPath> getAllStorageGroupPaths() {
+    return mtree.getAllStorageGroupPaths();
   }
 
   /**
-   * Get all storage group under given prefixPath.
+   * get all storageGroups ttl
    *
-   * @param prefixPath a prefix of 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.
-   * @return A ArrayList instance which stores storage group paths with given prefixPath.
+   * @return key-> storageGroupPath, value->ttl
    */
-  public List<PartialPath> getStorageGroupPaths(PartialPath prefixPath) throws MetadataException {
-    return mtree.getStorageGroupPaths(prefixPath);
+  public Map<PartialPath, Long> getStorageGroupsTTL() {
+    Map<PartialPath, Long> storageGroupsTTL = new HashMap<>();
+    try {
+      List<PartialPath> storageGroups = this.getAllStorageGroupPaths();
+      for (PartialPath storageGroup : storageGroups) {
+        long ttl = getStorageGroupNodeByStorageGroupPath(storageGroup).getDataTTL();
+        storageGroupsTTL.put(storageGroup, ttl);
+      }
+    } catch (MetadataException e) {
+      logger.error("get storage groups ttl failed.", e);
+    }
+    return storageGroupsTTL;
   }
 
-  /** Get all storage group MNodes */
-  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
-    return mtree.getAllStorageGroupNodes();
-  }
+  // endregion
 
+  // region Interfaces for Entity/Device info Query
   /**
-   * Return all paths for given path if the path is abstract. Or return the path itself. Regular
-   * expression in this method is formed by the amalgamation of seriesPath and the character '*'.
+   * To reduce the String number in memory, use the deviceId from MManager instead of the deviceId
+   * read from disk
    *
-   * @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.
+   * @param devicePath read from disk
+   * @return deviceId
    */
-  public List<PartialPath> getAllTimeseriesPath(PartialPath prefixPath) throws MetadataException {
-    return mtree.getAllTimeseriesPath(prefixPath);
+  public String getDeviceId(PartialPath devicePath) {
+    String device = null;
+    try {
+      IMNode deviceNode = getDeviceNode(devicePath);
+      device = deviceNode.getFullPath();
+    } catch (MetadataException | NullPointerException e) {
+      // Cannot get deviceId from MManager, return the input deviceId
+    }
+    return device;
   }
 
-  /** Similar to method getAllTimeseriesPath(), but return Path with alias alias. */
-  public Pair<List<PartialPath>, Integer> getAllTimeseriesPathWithAlias(
-      PartialPath prefixPath, int limit, int offset) throws MetadataException {
-    return mtree.getAllTimeseriesPathWithAlias(prefixPath, limit, offset);
+  /**
+   * Get all devices under given prefixPath.
+   *
+   * @param prefixPath a prefix of a full path or a pattern of the prefixPath.
+   * @return A HashSet instance which stores devices paths with given prefixPath.
+   */
+  public Set<PartialPath> getDevicesByPrefix(PartialPath prefixPath) throws MetadataException {
+    return mtree.getDevices(prefixPath, true);
   }
 
-  /** To calculate the count of timeseries for given prefix path. */
-  public int getAllTimeseriesCount(PartialPath prefixPath) throws MetadataException {
-    return mtree.getAllTimeseriesCount(prefixPath);
+  /**
+   * Get all devices that one of the timeseries, matching the given timeseries path pattern, belongs
+   * to.
+   *
+   * @param timeseries a path pattern of the target timeseries
+   * @return A HashSet instance which stores devices paths.
+   */
+  public Set<PartialPath> getBelongedDevices(PartialPath timeseries) throws MetadataException {
+    return mtree.getDevicesByTimeseries(timeseries);
   }
 
-  /** To calculate the count of devices for given prefix path. */
-  public int getDevicesNum(PartialPath prefixPath) throws MetadataException {
-    return mtree.getDevicesNum(prefixPath);
+  /**
+   * Get all device paths matching the path pattern.
+   *
+   * @param pathPattern the pattern of the target devices.
+   * @return A HashSet instance which stores devices paths matching the given path pattern.
+   */
+  public Set<PartialPath> getMatchedDevices(PartialPath pathPattern) throws MetadataException {
+    return mtree.getDevices(pathPattern, false);
   }
 
-  /** To calculate the count of storage group for given prefix path. */
-  public int getStorageGroupNum(PartialPath prefixPath) throws MetadataException {
-    return mtree.getStorageGroupNum(prefixPath);
+  /**
+   * Get all device paths and according storage group paths as ShowDevicesResult.
+   *
+   * @param plan ShowDevicesPlan which contains the path pattern and restriction params.
+   * @return ShowDevicesResult.
+   */
+  public List<ShowDevicesResult> getMatchedDevices(ShowDevicesPlan plan) throws MetadataException {
+    return mtree.getDevices(plan);
   }
+  // endregion
 
+  // region Interfaces for timeseries, measurement and schema info Query
   /**
-   * To calculate the count of nodes in the given level for given prefix path.
+   * Return all paths for given path if the path is abstract. Or return the path itself. Regular
+   * expression in this method is formed by the amalgamation of seriesPath and the character '*'.
    *
-   * @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
+   * @param pathPattern can be a pattern or a full path of timeseries.
+   */
+  public List<PartialPath> getAllTimeseriesPath(PartialPath pathPattern) throws MetadataException {
+    return mtree.getAllTimeseriesPath(pathPattern);
+  }
+
+  /**
+   * Similar to method getAllTimeseriesPath(), but return Path with alias and filter the result by
+   * limit and offset.
    */
-  public int getNodesCountInGivenLevel(PartialPath prefixPath, int level) throws MetadataException {
-    return mtree.getNodesCountInGivenLevel(prefixPath, level);
+  public Pair<List<PartialPath>, Integer> getAllTimeseriesPathWithAlias(
+      PartialPath pathPattern, int limit, int offset) throws MetadataException {
+    return mtree.getAllTimeseriesPathWithAlias(pathPattern, limit, offset);
   }
 
   public List<ShowTimeSeriesResult> showTimeseries(ShowTimeSeriesPlan plan, QueryContext context)
@@ -924,13 +1133,13 @@ public class MManager {
     List<IMeasurementMNode> allMatchedNodes = tagManager.getMatchedTimeseriesInIndex(plan, context);
 
     List<ShowTimeSeriesResult> res = new LinkedList<>();
-    String[] prefixNodes = plan.getPath().getNodes();
+    PartialPath pathPattern = plan.getPath();
     int curOffset = -1;
     int count = 0;
     int limit = plan.getLimit();
     int offset = plan.getOffset();
     for (IMeasurementMNode leaf : allMatchedNodes) {
-      if (match(leaf.getPartialPath(), prefixNodes)) {
+      if (pathPattern.matchFullPath(leaf.getPartialPath())) {
         if (limit != 0 || offset != 0) {
           curOffset++;
           if (curOffset < offset || count == limit) {
@@ -945,7 +1154,7 @@ public class MManager {
               new ShowTimeSeriesResult(
                   leaf.getFullPath(),
                   leaf.getAlias(),
-                  getStorageGroupPath(leaf.getPartialPath()).getFullPath(),
+                  getBelongedStorageGroup(leaf.getPartialPath()).getFullPath(),
                   measurementSchema.getType(),
                   measurementSchema.getEncodingType(),
                   measurementSchema.getCompressor(),
@@ -963,20 +1172,6 @@ public class MManager {
     return res;
   }
 
-  /** whether the full path has the prefixNodes */
-  private boolean match(PartialPath fullPath, String[] prefixNodes) {
-    String[] nodes = fullPath.getNodes();
-    if (nodes.length < prefixNodes.length) {
-      return false;
-    }
-    for (int i = 0; i < prefixNodes.length; i++) {
-      if (!"*".equals(prefixNodes[i]) && !prefixNodes[i].equals(nodes[i])) {
-        return false;
-      }
-    }
-    return true;
-  }
-
   /**
    * Get the result of ShowTimeseriesPlan
    *
@@ -1019,6 +1214,30 @@ public class MManager {
   }
 
   /**
+   * Get series type for given seriesPath.
+   *
+   * @param fullPath full path
+   */
+  public TSDataType getSeriesType(PartialPath fullPath) throws MetadataException {
+    if (fullPath.equals(SQLConstant.TIME_PATH)) {
+      return TSDataType.INT64;
+    }
+
+    IMeasurementSchema schema = mtree.getSchema(fullPath);
+    if (schema instanceof MeasurementSchema) {
+      return schema.getType();
+    } else {
+      if (((VectorPartialPath) fullPath).getSubSensorsList().size() != 1) {
+        return TSDataType.VECTOR;
+      } else {
+        String subSensor = ((VectorPartialPath) fullPath).getSubSensor(0);
+        List<String> measurements = schema.getSubMeasurementsList();
+        return schema.getSubMeasurementsTSDataTypeList().get(measurements.indexOf(subSensor));
+      }
+    }
+  }
+
+  /**
    * get MeasurementSchema or VectorMeasurementSchema which contains the measurement
    *
    * @param device device path
@@ -1061,183 +1280,64 @@ public class MManager {
     TSDataType[] types = new TSDataType[subMeasurements.size()];
     TSEncoding[] encodings = new TSEncoding[subMeasurements.size()];
 
-    for (int i = 0; i < subMeasurements.size(); i++) {
-      int index = measurementsInLeaf.indexOf(subMeasurements.get(i));
-      types[i] = schema.getSubMeasurementsTSDataTypeList().get(index);
-      encodings[i] = schema.getSubMeasurementsTSEncodingList().get(index);
-    }
-    String[] array = new String[subMeasurements.size()];
-    for (int i = 0; i < array.length; i++) {
-      array[i] = subMeasurements.get(i);
-    }
-    return new VectorMeasurementSchema(
-        schema.getMeasurementId(), array, types, encodings, schema.getCompressor());
-  }
-
-  /**
-   * Get schema of partialPaths, in which aligned time series should only organized to one schema.
-   * BEFORE this method, all the aligned time series is NOT united. For example,
-   * VectorMeasurementSchema(root.sg.d1.vector1 [s1,s2]) will be * root.sg.d1.vector1[s1],
-   * root.sg.d1.vector1[s2]
-   *
-   * @param fullPaths full path list without uniting the sub measurement under the same aligned time
-   *     series.
-   * @return Size of partial path list could NOT equal to the input list size. For example, the
-   *     VectorMeasurementSchema (s1,s2) would be returned once; Size of integer list must equal to
-   *     the input list size. It indicates the index of elements of original list in the result list
-   */
-  public Pair<List<PartialPath>, Map<String, Integer>> getSeriesSchemas(List<PartialPath> fullPaths)
-      throws MetadataException {
-    Map<IMNode, PartialPath> nodeToPartialPath = new LinkedHashMap<>();
-    Map<String, Integer> pathIndex = new LinkedHashMap<>();
-    for (int i = 0; i < fullPaths.size(); i++) {
-      PartialPath path = fullPaths.get(i);
-      pathIndex.put(path.getExactFullPath(), i);
-      if (path.isMeasurementAliasExists()) {
-        pathIndex.put(path.getFullPathWithAlias(), i);
-      }
-
-      IMeasurementMNode node = getMeasurementMNode(path);
-      if (!nodeToPartialPath.containsKey(node)) {
-        nodeToPartialPath.put(node, path.copy());
-      } else {
-        // if nodeToPartialPath contains node
-        PartialPath existPath = nodeToPartialPath.get(node);
-        if (!existPath.equals(path)) {
-          // could be VectorPartialPath
-          ((VectorPartialPath) existPath)
-              .addSubSensor(((VectorPartialPath) path).getSubSensorsList());
-        }
-      }
-    }
-    return new Pair<>(new ArrayList<>(nodeToPartialPath.values()), pathIndex);
-  }
-
-  /**
-   * Get child node path in the next level of the given path.
-   *
-   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
-   * return [root.sg1.d1, root.sg1.d2]
-   *
-   * @param path The given path
-   * @return All child nodes' seriesPath(s) of given seriesPath.
-   */
-  public Set<String> getChildNodePathInNextLevel(PartialPath path) throws MetadataException {
-    return mtree.getChildNodePathInNextLevel(path);
-  }
-
-  /**
-   * Get child node in the next level of the given path.
-   *
-   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
-   * return [d1, d2] given path = root.sg.d1 return [s1,s2]
-   *
-   * @return All child nodes of given seriesPath.
-   */
-  public Set<String> getChildNodeInNextLevel(PartialPath path) throws MetadataException {
-    return mtree.getChildNodeInNextLevel(path);
-  }
-
-  /**
-   * Check whether the path exists.
-   *
-   * @param path a full path or a prefix path
-   */
-  public boolean isPathExist(PartialPath path) {
-    return mtree.isPathExist(path);
-  }
-
-  protected IMeasurementMNode getMeasurementMNode(PartialPath fullPath) throws MetadataException {
-    return (IMeasurementMNode) getNodeByPath(fullPath);
-  }
-
-  /** Get node by path */
-  public IMNode getNodeByPath(PartialPath path) throws MetadataException {
-    return mtree.getNodeByPath(path);
-  }
-
-  /**
-   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
-   * device], return the MNode of root.sg Get storage group node by path. If storage group is not
-   * set, StorageGroupNotSetException will be thrown
-   */
-  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
-      throws MetadataException {
-    return mtree.getStorageGroupNodeByStorageGroupPath(path);
-  }
-
-  /** Get storage group node by path. the give path don't need to be storage group path. */
-  public IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException {
-    return mtree.getStorageGroupNodeByPath(path);
+    for (int i = 0; i < subMeasurements.size(); i++) {
+      int index = measurementsInLeaf.indexOf(subMeasurements.get(i));
+      types[i] = schema.getSubMeasurementsTSDataTypeList().get(index);
+      encodings[i] = schema.getSubMeasurementsTSEncodingList().get(index);
+    }
+    String[] array = new String[subMeasurements.size()];
+    for (int i = 0; i < array.length; i++) {
+      array[i] = subMeasurements.get(i);
+    }
+    return new VectorMeasurementSchema(
+        schema.getMeasurementId(), array, types, encodings, schema.getCompressor());
   }
 
   /**
-   * get device node, if the storage group is not set, create it when autoCreateSchema is true
-   *
-   * <p>(we develop this method as we need to get the node's lock after we get the lock.writeLock())
+   * Get schema of partialPaths, in which aligned time series should only organized to one schema.
+   * BEFORE this method, all the aligned time series is NOT united. For example,
+   * VectorMeasurementSchema(root.sg.d1.vector1 [s1,s2]) will be * root.sg.d1.vector1[s1],
+   * root.sg.d1.vector1[s2]
    *
-   * @param path path
-   * @param allowCreateSg The stand-alone version can create an sg at will, but the cluster version
-   *     needs to make the Meta group aware of the creation of an SG, so an exception needs to be
-   *     thrown here
+   * @param fullPaths full path list without uniting the sub measurement under the same aligned time
+   *     series.
+   * @return Size of partial path list could NOT equal to the input list size. For example, the
+   *     VectorMeasurementSchema (s1,s2) would be returned once; Size of integer list must equal to
+   *     the input list size. It indicates the index of elements of original list in the result list
    */
-  public IMNode getDeviceNodeWithAutoCreate(
-      PartialPath path, boolean autoCreateSchema, boolean allowCreateSg, int sgLevel)
-      throws IOException, MetadataException {
-    IMNode node;
-    boolean shouldSetStorageGroup;
-    try {
-      node = mNodeCache.get(path);
-      return node;
-    } catch (CacheException e) {
-      if (!autoCreateSchema) {
-        throw new PathNotExistException(path.getFullPath());
+  public Pair<List<PartialPath>, Map<String, Integer>> getSeriesSchemas(List<PartialPath> fullPaths)
+      throws MetadataException {
+    Map<IMNode, PartialPath> nodeToPartialPath = new LinkedHashMap<>();
+    Map<String, Integer> pathIndex = new LinkedHashMap<>();
+    for (int i = 0; i < fullPaths.size(); i++) {
+      PartialPath path = fullPaths.get(i);
+      pathIndex.put(path.getExactFullPath(), i);
+      if (path.isMeasurementAliasExists()) {
+        pathIndex.put(path.getFullPathWithAlias(), i);
       }
-      shouldSetStorageGroup = e.getCause() instanceof StorageGroupNotSetException;
-    }
 
-    try {
-      if (shouldSetStorageGroup) {
-        if (allowCreateSg) {
-          PartialPath storageGroupPath = MetaUtils.getStorageGroupPathByLevel(path, sgLevel);
-          setStorageGroup(storageGroupPath);
-        } else {
-          throw new StorageGroupNotSetException(path.getFullPath());
+      IMeasurementMNode node = getMeasurementMNode(path);
+      if (!nodeToPartialPath.containsKey(node)) {
+        nodeToPartialPath.put(node, path.copy());
+      } else {
+        // if nodeToPartialPath contains node
+        PartialPath existPath = nodeToPartialPath.get(node);
+        if (!existPath.equals(path)) {
+          // could be VectorPartialPath
+          ((VectorPartialPath) existPath)
+              .addSubSensor(((VectorPartialPath) path).getSubSensorsList());
         }
       }
-      node = mtree.getDeviceNodeWithAutoCreating(path, sgLevel);
-      if (!(node.isStorageGroup())) {
-        logWriter.autoCreateDeviceMNode(new AutoCreateDeviceMNodePlan(node.getPartialPath()));
-      }
-      return node;
-    } catch (StorageGroupAlreadySetException e) {
-      if (e.isHasChild()) {
-        // if setStorageGroup failure is because of child, the deviceNode should not be created.
-        // Timeseries can't be create under a deviceNode without storageGroup.
-        throw e;
-      }
-      // ignore set storage group concurrently
-      node = mtree.getDeviceNodeWithAutoCreating(path, sgLevel);
-      if (!(node.isStorageGroup())) {
-        logWriter.autoCreateDeviceMNode(new AutoCreateDeviceMNodePlan(node.getPartialPath()));
-      }
-      return node;
     }
-  }
-
-  /** !!!!!!Attention!!!!! must call the return node's readUnlock() if you call this method. */
-  public IMNode getDeviceNodeWithAutoCreate(PartialPath path)
-      throws MetadataException, IOException {
-    return getDeviceNodeWithAutoCreate(
-        path, config.isAutoCreateSchemaEnabled(), true, config.getDefaultStorageGroupLevel());
+    return new Pair<>(new ArrayList<>(nodeToPartialPath.values()), pathIndex);
   }
 
   // attention: this path must be a device node
-  public List<IMeasurementSchema> getAllMeasurementByDevicePath(PartialPath path)
+  public List<IMeasurementSchema> getAllMeasurementByDevicePath(PartialPath devicePath)
       throws PathNotExistException {
     Set<IMeasurementSchema> res = new HashSet<>();
     try {
-      IMNode node = mNodeCache.get(path);
+      IMNode node = mNodeCache.get(devicePath);
       Template template = node.getUpperTemplate();
 
       for (IMNode child : node.getChildren().values()) {
@@ -1252,11 +1352,39 @@ public class MManager {
         res.addAll(template.getSchemaMap().values());
       }
     } catch (CacheException e) {
-      throw new PathNotExistException(path.getFullPath());
+      throw new PathNotExistException(devicePath.getFullPath());
     }
 
     return new ArrayList<>(res);
   }
+  // endregion
+  // endregion
+
+  // region Interfaces and methods for MNode query
+  /** Get node by path */
+  public IMNode getNodeByPath(PartialPath path) throws MetadataException {
+    return mtree.getNodeByPath(path);
+  }
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
+   * device], return the MNode of root.sg Get storage group node by path. If storage group is not
+   * set, StorageGroupNotSetException will be thrown
+   */
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException {
+    return mtree.getStorageGroupNodeByStorageGroupPath(path);
+  }
+
+  /** Get storage group node by path. the give path don't need to be storage group path. */
+  public IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNodeByPath(path);
+  }
+
+  /** Get all storage group MNodes */
+  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
+    return mtree.getAllStorageGroupNodes();
+  }
 
   public IMNode getDeviceNode(PartialPath path) throws MetadataException {
     IMNode node;
@@ -1268,55 +1396,48 @@ public class MManager {
     }
   }
 
-  /**
-   * To reduce the String number in memory, use the deviceId from MManager instead of the deviceId
-   * read from disk
-   *
-   * @param path read from disk
-   * @return deviceId
-   */
-  public String getDeviceId(PartialPath path) {
-    String device = null;
-    try {
-      IMNode deviceNode = getDeviceNode(path);
-      device = deviceNode.getFullPath();
-    } catch (MetadataException | NullPointerException e) {
-      // Cannot get deviceId from MManager, return the input deviceId
+  public IMeasurementMNode[] getMNodes(PartialPath deviceId, String[] measurements)
+      throws MetadataException {
+    IMeasurementMNode[] mNodes = new IMeasurementMNode[measurements.length];
+    for (int i = 0; i < mNodes.length; i++) {
+      try {
+        mNodes[i] = (IMeasurementMNode) getNodeByPath(deviceId.concatNode(measurements[i]));
+      } catch (PathNotExistException ignored) {
+        logger.warn("{} does not exist in {}", measurements[i], deviceId);
+      }
+      if (mNodes[i] == null && !IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
+        throw new MetadataException(measurements[i] + " does not exist in " + deviceId);
+      }
     }
-    return device;
-  }
-
-  /** Get metadata in string */
-  public String getMetadataInString() {
-    return TIME_SERIES_TREE_HEADER + mtree;
+    return mNodes;
   }
 
-  public void setTTL(PartialPath storageGroup, long dataTTL) throws MetadataException, IOException {
-    getStorageGroupNodeByStorageGroupPath(storageGroup).setDataTTL(dataTTL);
-    if (!isRecovering) {
-      logWriter.setTTL(storageGroup, dataTTL);
-    }
+  protected IMeasurementMNode getMeasurementMNode(PartialPath fullPath) throws MetadataException {
+    return (IMeasurementMNode) getNodeByPath(fullPath);
   }
 
   /**
-   * get all storageGroups ttl
-   *
-   * @return key-> storageGroupPath, value->ttl
+   * Invoked during insertPlan process. Get target MeasurementMNode from given EntityMNode. If the
+   * result is not null and is not MeasurementMNode, it means a timeseries with same path cannot be
+   * created thus throw PathAlreadyExistException.
    */
-  public Map<PartialPath, Long> getStorageGroupsTTL() {
-    Map<PartialPath, Long> storageGroupsTTL = new HashMap<>();
-    try {
-      List<PartialPath> storageGroups = this.getAllStorageGroupPaths();
-      for (PartialPath storageGroup : storageGroups) {
-        long ttl = getStorageGroupNodeByStorageGroupPath(storageGroup).getDataTTL();
-        storageGroupsTTL.put(storageGroup, ttl);
-      }
-    } catch (MetadataException e) {
-      logger.error("get storage groups ttl failed.", e);
+  protected IMeasurementMNode getMeasurementMNode(IMNode deviceMNode, String measurementName)
+      throws PathAlreadyExistException {
+    IMNode result = deviceMNode.getChild(measurementName);
+    if (result == null) {
+      return null;
+    }
+
+    if (result.isMeasurement()) {
+      return (IMeasurementMNode) result;
+    } else {
+      throw new PathAlreadyExistException(
+          deviceMNode.getFullPath() + PATH_SEPARATOR + measurementName);
     }
-    return storageGroupsTTL;
   }
+  // endregion
 
+  // region Interfaces for alias and tag/attribute operations
   /**
    * Check whether the given path contains a storage group change or set the new offset of a
    * timeseries
@@ -1513,39 +1634,39 @@ public class MManager {
     tagManager.renameTagOrAttributeKey(oldKey, newKey, fullPath, leafMNode);
   }
 
-  /** Check whether the given path contains a storage group */
-  boolean checkStorageGroupByPath(PartialPath path) {
-    return mtree.checkStorageGroupByPath(path);
+  /** remove the node from the tag inverted index */
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private void removeFromTagInvertedIndex(IMeasurementMNode node) throws IOException {
+    tagManager.removeFromTagInvertedIndex(node);
   }
+  // endregion
+
+  // region Interfaces only for Cluster module usage
 
   /**
-   * Get all storage groups under the given path
+   * Collect the timeseries schemas under "prefixPath".
    *
-   * @return List of String represented all storage group names
    * @apiNote :for cluster
    */
-  List<String> getStorageGroupByPath(PartialPath path) throws MetadataException {
+  public void collectSeries(PartialPath prefixPath, List<IMeasurementSchema> measurementSchemas) {
+    IMNode node;
     try {
-      return mtree.getStorageGroupByPath(path);
+      node = getNodeByPath(prefixPath);
     } catch (MetadataException e) {
-      throw new MetadataException(e);
+      return;
     }
+    collectMeasurementSchema(node, measurementSchemas);
   }
 
-  public void collectTimeseriesSchema(
-      IMNode startingNode, Collection<TimeseriesSchema> timeseriesSchemas) {
+  private void collectMeasurementSchema(
+      IMNode startingNode, Collection<IMeasurementSchema> measurementSchemas) {
     Deque<IMNode> nodeDeque = new ArrayDeque<>();
     nodeDeque.addLast(startingNode);
     while (!nodeDeque.isEmpty()) {
       IMNode node = nodeDeque.removeFirst();
       if (node.isMeasurement()) {
         IMeasurementSchema nodeSchema = ((IMeasurementMNode) node).getSchema();
-        timeseriesSchemas.add(
-            new TimeseriesSchema(
-                node.getFullPath(),
-                nodeSchema.getType(),
-                nodeSchema.getEncodingType(),
-                nodeSchema.getCompressor()));
+        measurementSchemas.add(nodeSchema);
       } else if (!node.getChildren().isEmpty()) {
         nodeDeque.addAll(node.getChildren().values());
       }
@@ -1553,62 +1674,64 @@ public class MManager {
   }
 
   public void collectTimeseriesSchema(
-      String prefixPath, Collection<TimeseriesSchema> timeseriesSchemas) throws MetadataException {
-    collectTimeseriesSchema(getNodeByPath(new PartialPath(prefixPath)), timeseriesSchemas);
+      PartialPath prefixPath, Collection<TimeseriesSchema> timeseriesSchemas) {
+    IMNode node;
+    try {
+      node = getNodeByPath(prefixPath);
+    } catch (MetadataException e) {
+      return;
+    }
+    collectTimeseriesSchema(node, timeseriesSchemas);
   }
 
-  public void collectMeasurementSchema(
-      IMNode startingNode, Collection<IMeasurementSchema> measurementSchemas) {
+  private void collectTimeseriesSchema(
+      IMNode startingNode, Collection<TimeseriesSchema> timeseriesSchemas) {
     Deque<IMNode> nodeDeque = new ArrayDeque<>();
     nodeDeque.addLast(startingNode);
     while (!nodeDeque.isEmpty()) {
       IMNode node = nodeDeque.removeFirst();
       if (node.isMeasurement()) {
         IMeasurementSchema nodeSchema = ((IMeasurementMNode) node).getSchema();
-        measurementSchemas.add(nodeSchema);
+        timeseriesSchemas.add(
+            new TimeseriesSchema(
+                node.getFullPath(),
+                nodeSchema.getType(),
+                nodeSchema.getEncodingType(),
+                nodeSchema.getCompressor()));
       } else if (!node.getChildren().isEmpty()) {
         nodeDeque.addAll(node.getChildren().values());
       }
     }
   }
 
-  /** Collect the timeseries schemas under "startingPath". */
-  public void collectSeries(PartialPath startingPath, List<IMeasurementSchema> measurementSchemas) {
-    IMNode node;
-    try {
-      node = getNodeByPath(startingPath);
-    } catch (MetadataException e) {
-      return;
-    }
-    collectMeasurementSchema(node, measurementSchemas);
-  }
-
   /**
-   * For a path, infer all storage groups it may belong to. The path can have wildcards.
+   * For a path, infer all storage groups it may belong to. The path can have wildcards. Resolve the
+   * path or path pattern into StorageGroupName-FullPath pairs that FullPath matches the given path.
    *
    * <p>Consider the path into two parts: (1) the sub path which can not contain a storage group
    * name and (2) the sub path which is substring that begin after the storage group name.
    *
    * <p>(1) Suppose the part of the path can not contain a storage group name (e.g.,
-   * "root".contains("root.sg") == false), then: If the wildcard is not at the tail, then for each
-   * wildcard, only one level will be inferred and the wildcard will be removed. If the wildcard is
-   * at the tail, then the inference will go on until the storage groups are found and the wildcard
-   * will be kept. (2) Suppose the part of the path is a substring that begin after the storage
-   * group name. (e.g., For "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part
-   * is "a.*.b.*"). For this part, keep what it is.
+   * "root".contains("root.sg") == false), then: For each one level wildcard *, only one level will
+   * be inferred and the wildcard will be removed. For each multi level wildcard **, then the
+   * inference will go on until the storage groups are found and the wildcard will be kept. (2)
+   * Suppose the part of the path is a substring that begin after the storage group name. (e.g., For
+   * "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part is "a.*.b.*"). For
+   * this part, keep what it is.
    *
    * <p>Assuming we have three SGs: root.group1, root.group2, root.area1.group3 Eg1: for input
-   * "root.*", returns ("root.group1", "root.group1.*"), ("root.group2", "root.group2.*")
-   * ("root.area1.group3", "root.area1.group3.*") Eg2: for input "root.*.s1", returns
+   * "root.**", returns ("root.group1", "root.group1.**"), ("root.group2", "root.group2.**")
+   * ("root.area1.group3", "root.area1.group3.**") Eg2: for input "root.*.s1", returns
    * ("root.group1", "root.group1.s1"), ("root.group2", "root.group2.s1")
    *
-   * <p>Eg3: for input "root.area1.*", returns ("root.area1.group3", "root.area1.group3.*")
+   * <p>Eg3: for input "root.area1.**", returns ("root.area1.group3", "root.area1.group3.**")
    *
-   * @param path can be a prefix or a full path.
+   * @param path can be a path pattern or a full path.
    * @return StorageGroupName-FullPath pairs
+   * @apiNote :for cluster
    */
-  public Map<String, String> determineStorageGroup(PartialPath path) throws IllegalPathException {
-    Map<String, String> sgPathMap = mtree.determineStorageGroup(path);
+  public Map<String, String> groupPathByStorageGroup(PartialPath path) throws MetadataException {
+    Map<String, String> sgPathMap = mtree.groupPathByStorageGroup(path);
     if (logger.isDebugEnabled()) {
       logger.debug("The storage groups of path {} are {}", path, sgPathMap.keySet());
     }
@@ -1625,6 +1748,18 @@ public class MManager {
   }
 
   /**
+   * StorageGroupFilter filters unsatisfied storage groups in metadata queries to speed up and
+   * deduplicate.
+   */
+  @FunctionalInterface
+  public interface StorageGroupFilter {
+
+    boolean satisfy(String storageGroup);
+  }
+  // endregion
+
+  // region Interfaces for lastCache operations
+  /**
    * Update the last cache value of time series of given seriesPath.
    *
    * <p>MManager will use the seriesPath to search the node first and then process the lastCache in
@@ -1817,7 +1952,9 @@ public class MManager {
           (IEntityMNode) node, originalPath, startTime, endTime);
     }
   }
+  // endregion
 
+  // region Interfaces and Implementation for InsertPlan process
   /** get schema for device. Attention!!! Only support insertPlan */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
@@ -1860,14 +1997,12 @@ public class MManager {
     for (int i = 0; i < measurementList.length; i++) {
       try {
         String measurement = measurementList[i];
-        IMNode child = getMNode(deviceMNode, plan.isAligned() ? vectorId : measurement);
-        if (child != null && child.isMeasurement()) {
-          measurementMNode = (IMeasurementMNode) child;
-        } else if (child != null && child.isStorageGroup()) {
-          throw new PathAlreadyExistException(deviceId + PATH_SEPARATOR + measurement);
-        } else if ((measurementMNode = findTemplate(deviceMNode, measurement, vectorId)) != null) {
-          // empty
-        } else {
+        measurementMNode =
+            getMeasurementMNode(deviceMNode, plan.isAligned() ? vectorId : measurement);
+        if (measurementMNode == null) {
+          measurementMNode = findTemplate(deviceMNode, measurement, vectorId);
+        }
+        if (measurementMNode == null) {
           if (!config.isAutoCreateSchemaEnabled()) {
             throw new PathNotExistException(deviceId + PATH_SEPARATOR + measurement);
           } else {
@@ -2002,10 +2137,6 @@ public class MManager {
     return dataType;
   }
 
-  public IMNode getMNode(IMNode deviceMNode, String measurementName) {
-    return deviceMNode.getChild(measurementName);
-  }
-
   private IMeasurementMNode findTemplate(IMNode deviceMNode, String measurement, String vectorId)
       throws MetadataException {
     Template curTemplate = deviceMNode.getUpperTemplate();
@@ -2061,17 +2192,9 @@ public class MManager {
         encodings,
         TSFileDescriptor.getInstance().getConfig().getCompressor());
   }
+  // endregion
 
-  /**
-   * StorageGroupFilter filters unsatisfied storage groups in metadata queries to speed up and
-   * deduplicate.
-   */
-  @FunctionalInterface
-  public interface StorageGroupFilter {
-
-    boolean satisfy(String storageGroup);
-  }
-
+  // region Interfaces and Implementation for Template operations
   public void createSchemaTemplate(CreateTemplatePlan plan) throws MetadataException {
     try {
       templateManager.createSchemaTemplate(plan);
@@ -2108,10 +2231,6 @@ public class MManager {
     }
   }
 
-  public void autoCreateDeviceMNode(AutoCreateDeviceMNodePlan plan) throws MetadataException {
-    mtree.getDeviceNodeWithAutoCreating(plan.getPath(), config.getDefaultStorageGroupLevel());
-  }
-
   private void setUsingSchemaTemplate(SetUsingSchemaTemplatePlan plan) throws MetadataException {
     try {
       setUsingSchemaTemplate(getDeviceNode(plan.getPrefixPath()));
@@ -2134,13 +2253,24 @@ public class MManager {
     }
     return entityMNode;
   }
+  // endregion
 
-  public long getTotalSeriesNumber() {
-    return totalSeriesNumber.get();
+  // region TestOnly Interfaces
+  /**
+   * Attention!!!!!, this method could only be used for Tests involving multiple mmanagers. The
+   * singleton of templateManager and tagManager will cause interference between mmanagers if one of
+   * the mmanagers invoke init method or clear method
+   */
+  @TestOnly
+  public void initForMultiMManagerTest() {
+    templateManager = TemplateManager.getNewInstanceForTest();
+    tagManager = TagManager.getNewInstanceForTest();
+    init();
   }
 
   @TestOnly
   public void flushAllMlogForTest() throws IOException {
     logWriter.close();
   }
+  // endregion
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
deleted file mode 100644
index 5852108..0000000
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ /dev/null
@@ -1,1963 +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;
-
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
-import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
-import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
-import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
-import org.apache.iotdb.db.metadata.lastCache.LastCacheManager;
-import org.apache.iotdb.db.metadata.logfile.MLogReader;
-import org.apache.iotdb.db.metadata.logfile.MLogWriter;
-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.StorageGroupMNode;
-import org.apache.iotdb.db.metadata.template.Template;
-import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
-import org.apache.iotdb.db.metadata.utils.MetaUtils;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
-import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
-import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
-import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
-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.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
-
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.file.Files;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.regex.Pattern;
-import java.util.stream.Stream;
-
-import static java.util.stream.Collectors.toList;
-import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
-import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
-
-/** The hierarchical struct of the Metadata Tree is implemented in this class. */
-public class MTree implements Serializable {
-
-  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
-  private static final long serialVersionUID = -4200394435237291964L;
-  private static final Logger logger = LoggerFactory.getLogger(MTree.class);
-  private static final String NO_CHILDNODE_MSG = " does not have the child node ";
-  private static transient ThreadLocal<Integer> limit = new ThreadLocal<>();
-  private static transient ThreadLocal<Integer> offset = new ThreadLocal<>();
-  private static transient ThreadLocal<Integer> count = new ThreadLocal<>();
-  private static transient ThreadLocal<Integer> curOffset = new ThreadLocal<>();
-  private IMNode root;
-
-  private String mtreeSnapshotPath;
-  private String mtreeSnapshotTmpPath;
-
-  MTree() {
-    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
-  }
-
-  private MTree(InternalMNode root) {
-    this.root = root;
-  }
-
-  public void init() throws IOException {
-    mtreeSnapshotPath =
-        IoTDBDescriptor.getInstance().getConfig().getSchemaDir()
-            + File.separator
-            + MetadataConstant.MTREE_SNAPSHOT;
-    mtreeSnapshotTmpPath =
-        IoTDBDescriptor.getInstance().getConfig().getSchemaDir()
-            + File.separator
-            + MetadataConstant.MTREE_SNAPSHOT_TMP;
-
-    File tmpFile = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath);
-    if (tmpFile.exists()) {
-      logger.warn("Creating MTree snapshot not successful before crashing...");
-      Files.delete(tmpFile.toPath());
-    }
-
-    File mtreeSnapshot = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
-    long time = System.currentTimeMillis();
-    if (mtreeSnapshot.exists()) {
-      this.root = deserializeFrom(mtreeSnapshot).root;
-      logger.debug(
-          "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
-    }
-  }
-
-  public void clear() {
-    root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
-  }
-
-  public void createSnapshot() throws IOException {
-    long time = System.currentTimeMillis();
-    logger.info("Start creating MTree snapshot to {}", mtreeSnapshotPath);
-    try {
-      serializeTo(mtreeSnapshotTmpPath);
-      File tmpFile = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath);
-      File snapshotFile = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
-      if (snapshotFile.exists()) {
-        Files.delete(snapshotFile.toPath());
-      }
-      if (tmpFile.renameTo(snapshotFile)) {
-        logger.info(
-            "Finish creating MTree snapshot to {}, spend {} ms.",
-            mtreeSnapshotPath,
-            System.currentTimeMillis() - time);
-      }
-    } catch (IOException e) {
-      logger.warn("Failed to create MTree snapshot to {}", mtreeSnapshotPath, e);
-      if (SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath).exists()) {
-        try {
-          Files.delete(SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath).toPath());
-        } catch (IOException e1) {
-          logger.warn("delete file {} failed: {}", mtreeSnapshotTmpPath, e1.getMessage());
-        }
-      }
-      throw e;
-    }
-  }
-
-  private static String jsonToString(JsonObject jsonObject) {
-    return GSON.toJson(jsonObject);
-  }
-
-  /** combine multiple metadata in string format */
-  @TestOnly
-  static JsonObject combineMetadataInStrings(String[] metadataStrs) {
-    JsonObject[] jsonObjects = new JsonObject[metadataStrs.length];
-    for (int i = 0; i < jsonObjects.length; i++) {
-      jsonObjects[i] = GSON.fromJson(metadataStrs[i], JsonObject.class);
-    }
-
-    JsonObject root = jsonObjects[0];
-    for (int i = 1; i < jsonObjects.length; i++) {
-      root = combineJsonObjects(root, jsonObjects[i]);
-    }
-
-    return root;
-  }
-
-  private static JsonObject combineJsonObjects(JsonObject a, JsonObject b) {
-    JsonObject res = new JsonObject();
-
-    Set<String> retainSet = new HashSet<>(a.keySet());
-    retainSet.retainAll(b.keySet());
-    Set<String> aCha = new HashSet<>(a.keySet());
-    Set<String> bCha = new HashSet<>(b.keySet());
-    aCha.removeAll(retainSet);
-    bCha.removeAll(retainSet);
-
-    for (String key : aCha) {
-      res.add(key, a.get(key));
-    }
-
-    for (String key : bCha) {
-      res.add(key, b.get(key));
-    }
-    for (String key : retainSet) {
-      JsonElement v1 = a.get(key);
-      JsonElement v2 = b.get(key);
-      if (v1 instanceof JsonObject && v2 instanceof JsonObject) {
-        res.add(key, combineJsonObjects((JsonObject) v1, (JsonObject) v2));
-      } else {
-        res.add(v1.getAsString(), v2);
-      }
-    }
-    return res;
-  }
-
-  /**
-   * Create a timeseries with a full path from root to leaf node. Before creating a timeseries, the
-   * storage group should be set first, throw exception otherwise
-   *
-   * @param path timeseries path
-   * @param dataType data type
-   * @param encoding encoding
-   * @param compressor compressor
-   * @param props props
-   * @param alias alias of measurement
-   */
-  IMeasurementMNode createTimeseries(
-      PartialPath path,
-      TSDataType dataType,
-      TSEncoding encoding,
-      CompressionType compressor,
-      Map<String, String> props,
-      String alias)
-      throws MetadataException {
-    String[] nodeNames = path.getNodes();
-    if (nodeNames.length <= 2 || !nodeNames[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-    MetaFormatUtils.checkTimeseries(path);
-    IMNode cur = root;
-    boolean hasSetStorageGroup = false;
-    Template upperTemplate = cur.getSchemaTemplate();
-    // e.g, path = root.sg.d1.s1,  create internal nodes and set cur to d1 node
-    for (int i = 1; i < nodeNames.length - 1; i++) {
-      if (cur.isMeasurement()) {
-        throw new PathAlreadyExistException(cur.getFullPath());
-      }
-      if (cur.isStorageGroup()) {
-        hasSetStorageGroup = true;
-      }
-      String childName = nodeNames[i];
-      if (!cur.hasChild(childName)) {
-        if (!hasSetStorageGroup) {
-          throw new StorageGroupNotSetException("Storage group should be created first");
-        }
-        if (cur.isUseTemplate() && upperTemplate.hasSchema(childName)) {
-          throw new PathAlreadyExistException(
-              cur.getPartialPath().concatNode(childName).getFullPath());
-        }
-        cur.addChild(childName, new InternalMNode(cur, childName));
-      }
-      cur = cur.getChild(childName);
-
-      if (cur.getSchemaTemplate() != null) {
-        upperTemplate = cur.getSchemaTemplate();
-      }
-    }
-
-    if (cur.isMeasurement()) {
-      throw new PathAlreadyExistException(cur.getFullPath());
-    }
-
-    if (upperTemplate != null && !upperTemplate.isCompatible(path)) {
-      throw new PathAlreadyExistException(
-          path.getFullPath() + " ( which is incompatible with template )");
-    }
-
-    MetaFormatUtils.checkTimeseriesProps(path.getFullPath(), props);
-
-    String leafName = nodeNames[nodeNames.length - 1];
-
-    // synchronize check and add, we need addChild and add Alias become atomic operation
-    // only write on mtree will be synchronized
-    synchronized (this) {
-      if (cur.hasChild(leafName)) {
-        throw new PathAlreadyExistException(path.getFullPath());
-      }
-
-      if (alias != null && cur.hasChild(alias)) {
-        throw new AliasAlreadyExistException(path.getFullPath(), alias);
-      }
-
-      IEntityMNode entityMNode = IEntityMNode.setToEntity(cur);
-
-      IMeasurementMNode measurementMNode =
-          new MeasurementMNode(entityMNode, leafName, alias, dataType, encoding, compressor, props);
-      entityMNode.addChild(leafName, measurementMNode);
-      // link alias to LeafMNode
-      if (alias != null) {
-        entityMNode.addAlias(alias, measurementMNode);
-      }
-      return measurementMNode;
-    }
-  }
-
-  /**
-   * Create aligned timeseries with full paths from root to one leaf node. Before creating
-   * timeseries, the * storage group 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 compressor compressor
-   */
-  void createAlignedTimeseries(
-      PartialPath devicePath,
-      List<String> measurements,
-      List<TSDataType> dataTypes,
-      List<TSEncoding> encodings,
-      CompressionType compressor)
-      throws MetadataException {
-    String[] deviceNodeNames = devicePath.getNodes();
-    if (deviceNodeNames.length <= 1 || !deviceNodeNames[0].equals(root.getName())) {
-      throw new IllegalPathException(devicePath.getFullPath());
-    }
-    MetaFormatUtils.checkTimeseries(devicePath);
-    MetaFormatUtils.checkSchemaMeasurementNames(measurements);
-    IMNode cur = root;
-    boolean hasSetStorageGroup = false;
-    // e.g, devicePath = root.sg.d1, create internal nodes and set cur to d1 node
-    for (int i = 1; i < deviceNodeNames.length - 1; i++) {
-      if (cur.isMeasurement()) {
-        throw new PathAlreadyExistException(cur.getFullPath());
-      }
-      if (cur.isStorageGroup()) {
-        hasSetStorageGroup = true;
-      }
-      String nodeName = deviceNodeNames[i];
-      if (!cur.hasChild(nodeName)) {
-        if (!hasSetStorageGroup) {
-          throw new StorageGroupNotSetException("Storage group should be created first");
-        }
-        cur.addChild(nodeName, new InternalMNode(cur, nodeName));
-      }
-      cur = cur.getChild(nodeName);
-    }
-
-    if (cur.isMeasurement()) {
-      throw new PathAlreadyExistException(cur.getFullPath());
-    }
-
-    String leafName = deviceNodeNames[deviceNodeNames.length - 1];
-
-    // synchronize check and add, we need addChild and add Alias become atomic operation
-    // only write on mtree will be synchronized
-    synchronized (this) {
-      if (cur.hasChild(leafName)) {
-        throw new PathAlreadyExistException(devicePath.getFullPath() + "." + leafName);
-      }
-
-      IEntityMNode entityMNode = IEntityMNode.setToEntity(cur);
-
-      int measurementsSize = measurements.size();
-
-      // this measurementMNode could be a leaf or not.
-      IMeasurementMNode measurementMNode =
-          new MeasurementMNode(
-              entityMNode,
-              leafName,
-              new VectorMeasurementSchema(
-                  leafName,
-                  measurements.toArray(new String[measurementsSize]),
-                  dataTypes.toArray(new TSDataType[measurementsSize]),
-                  encodings.toArray(new TSEncoding[measurementsSize]),
-                  compressor),
-              null);
-      entityMNode.addChild(leafName, measurementMNode);
-    }
-  }
-
-  /**
-   * 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, int sgLevel) throws MetadataException {
-    String[] nodeNames = deviceId.getNodes();
-    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
-      throw new IllegalPathException(deviceId.getFullPath());
-    }
-    IMNode cur = root;
-    Template upperTemplate = cur.getSchemaTemplate();
-    for (int i = 1; i < nodeNames.length; i++) {
-      if (!cur.hasChild(nodeNames[i])) {
-        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
-          throw new PathAlreadyExistException(
-              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
-        }
-        if (i == sgLevel) {
-          cur.addChild(
-              nodeNames[i],
-              new StorageGroupMNode(
-                  cur, nodeNames[i], IoTDBDescriptor.getInstance().getConfig().getDefaultTTL()));
-        } else {
-          cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
-        }
-      }
-      cur = cur.getChild(nodeNames[i]);
-      // update upper template
-      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
-    }
-
-    return cur;
-  }
-
-  /**
-   * Check whether the given path exists.
-   *
-   * @param path a full path or a prefix path
-   */
-  boolean isPathExist(PartialPath path) {
-    String[] nodeNames = path.getNodes();
-    IMNode cur = root;
-    if (!nodeNames[0].equals(root.getName())) {
-      return false;
-    }
-    Template upperTemplate = cur.getSchemaTemplate();
-    for (int i = 1; i < nodeNames.length; i++) {
-      if (!cur.hasChild(nodeNames[i])) {
-        return cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i]);
-      }
-      cur = cur.getChild(nodeNames[i]);
-      if (cur.isMeasurement()) {
-        if (i == nodeNames.length - 1) {
-          return true;
-        }
-        if (((IMeasurementMNode) cur).getSchema() instanceof VectorMeasurementSchema) {
-          return i == nodeNames.length - 2
-              && ((IMeasurementMNode) cur).getSchema().containsSubMeasurement(nodeNames[i + 1]);
-        } else {
-          return false;
-        }
-      }
-      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
-    }
-    return true;
-  }
-
-  /**
-   * Set storage group. Make sure check seriesPath before setting storage group
-   *
-   * @param path path
-   */
-  void setStorageGroup(PartialPath path) throws MetadataException {
-    String[] nodeNames = path.getNodes();
-    MetaFormatUtils.checkStorageGroup(path.getFullPath());
-    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-    IMNode cur = root;
-    Template upperTemplate = cur.getSchemaTemplate();
-    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]);
-      if (temp == null) {
-        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
-          throw new PathAlreadyExistException(
-              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
-        }
-        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
-      } else if (temp.isStorageGroup()) {
-        // before set storage group, check whether the exists or not
-        throw new StorageGroupAlreadySetException(temp.getFullPath());
-      }
-      cur = cur.getChild(nodeNames[i]);
-      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
-      i++;
-    }
-
-    // synchronize check and add, we need addChild become atomic operation
-    // only write on mtree will be synchronized
-    synchronized (this) {
-      if (cur.hasChild(nodeNames[i])) {
-        // node b has child sg
-        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
-          throw new StorageGroupAlreadySetException(path.getFullPath());
-        } else {
-          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
-        }
-      } else {
-        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
-          throw new PathAlreadyExistException(
-              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
-        }
-        IStorageGroupMNode storageGroupMNode =
-            new StorageGroupMNode(
-                cur, nodeNames[i], IoTDBDescriptor.getInstance().getConfig().getDefaultTTL());
-        cur.addChild(nodeNames[i], storageGroupMNode);
-      }
-    }
-  }
-
-  /** Delete a storage group */
-  List<IMeasurementMNode> deleteStorageGroup(PartialPath path) throws MetadataException {
-    IMNode cur = getNodeByPath(path);
-    if (!(cur.isStorageGroup())) {
-      throw new StorageGroupNotSetException(path.getFullPath());
-    }
-    // Suppose current system has root.a.b.sg1, root.a.sg2, and delete root.a.b.sg1
-    // delete the storage group node sg1
-    cur.getParent().deleteChild(cur.getName());
-
-    // collect all the LeafMNode in this storage group
-    List<IMeasurementMNode> leafMNodes = new LinkedList<>();
-    Queue<IMNode> queue = new LinkedList<>();
-    queue.add(cur);
-    while (!queue.isEmpty()) {
-      IMNode node = queue.poll();
-      for (IMNode child : node.getChildren().values()) {
-        if (child.isMeasurement()) {
-          leafMNodes.add((IMeasurementMNode) child);
-        } else {
-          queue.add(child);
-        }
-      }
-    }
-
-    cur = cur.getParent();
-    // delete node b while retain root.a.sg2
-    while (!IoTDBConstant.PATH_ROOT.equals(cur.getName()) && cur.getChildren().size() == 0) {
-      cur.getParent().deleteChild(cur.getName());
-      cur = cur.getParent();
-    }
-    return leafMNodes;
-  }
-
-  /**
-   * Check whether path is storage group or not
-   *
-   * <p>e.g., path = root.a.b.sg. if nor a and b is StorageGroupMNode and sg is a StorageGroupMNode
-   * path is a storage group
-   *
-   * @param path path
-   * @apiNote :for cluster
-   */
-  boolean isStorageGroup(PartialPath path) {
-    String[] nodeNames = path.getNodes();
-    if (nodeNames.length <= 1 || !nodeNames[0].equals(IoTDBConstant.PATH_ROOT)) {
-      return false;
-    }
-    IMNode cur = root;
-    int i = 1;
-    while (i < nodeNames.length - 1) {
-      cur = cur.getChild(nodeNames[i]);
-      if (cur == null || cur.isStorageGroup()) {
-        return false;
-      }
-      i++;
-    }
-    cur = cur.getChild(nodeNames[i]);
-    return cur != null && cur.isStorageGroup();
-  }
-
-  /**
-   * Delete path. The path should be a full path from root to leaf node
-   *
-   * @param path Format: root.node(.node)+
-   */
-  Pair<PartialPath, IMeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(PartialPath path)
-      throws MetadataException {
-    IMNode curNode = getNodeByPath(path);
-    if (!(curNode.isMeasurement())) {
-      throw new PathNotExistException(path.getFullPath());
-    }
-    String[] nodes = path.getNodes();
-    if (nodes.length == 0 || !IoTDBConstant.PATH_ROOT.equals(nodes[0])) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-
-    IMeasurementMNode deletedNode = (IMeasurementMNode) curNode;
-
-    // delete the last node of path
-    curNode.getParent().deleteChild(path.getMeasurement());
-    if (deletedNode.getAlias() != null) {
-      deletedNode.getParent().deleteAliasChild(((IMeasurementMNode) curNode).getAlias());
-    }
-    curNode = curNode.getParent();
-    // delete all empty ancestors except storage group and MeasurementMNode
-    while (!IoTDBConstant.PATH_ROOT.equals(curNode.getName())
-        && !(curNode.isMeasurement())
-        && curNode.getChildren().size() == 0) {
-      // if current storage group has no time series, return the storage group name
-      if (curNode.isStorageGroup()) {
-        return new Pair<>(curNode.getPartialPath(), deletedNode);
-      }
-      curNode.getParent().deleteChild(curNode.getName());
-      curNode = curNode.getParent();
-    }
-    return new Pair<>(null, deletedNode);
-  }
-
-  /**
-   * Get measurement schema for a given path. Path must be a complete Path from root to leaf node.
-   */
-  IMeasurementSchema getSchema(PartialPath path) throws MetadataException {
-    IMeasurementMNode node = (IMeasurementMNode) getNodeByPath(path);
-    return node.getSchema();
-  }
-
-  /**
-   * Get node by path with storage group check If storage group is not set,
-   * StorageGroupNotSetException will be thrown
-   */
-  IMNode getNodeByPathWithStorageGroupCheck(PartialPath path) throws MetadataException {
-    boolean storageGroupChecked = false;
-    String[] nodes = path.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-
-    IMNode cur = root;
-
-    for (int i = 1; i < nodes.length; i++) {
-      cur = cur.getChild(nodes[i]);
-      if (cur == null) {
-        // not find
-        if (!storageGroupChecked) {
-          throw new StorageGroupNotSetException(path.getFullPath());
-        }
-        throw new PathNotExistException(path.getFullPath());
-      }
-
-      if (cur.isStorageGroup()) {
-        storageGroupChecked = true;
-      }
-    }
-
-    if (!storageGroupChecked) {
-      throw new StorageGroupNotSetException(path.getFullPath());
-    }
-    return cur;
-  }
-
-  /**
-   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
-   * device], throw exception Get storage group node, if the give path is not a storage group, throw
-   * exception
-   */
-  IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
-      throws MetadataException {
-    IMNode node = getNodeByPath(path);
-    if (node.isStorageGroup()) {
-      return (IStorageGroupMNode) node;
-    } else {
-      throw new StorageGroupNotSetException(path.getFullPath(), true);
-    }
-  }
-
-  /**
-   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
-   * device], return the MNode of root.sg Get storage group node, the give path don't need to be
-   * storage group path.
-   */
-  IStorageGroupMNode getStorageGroupNodeByPath(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;
-    for (int i = 1; i < nodes.length; i++) {
-      cur = cur.getChild(nodes[i]);
-      if (cur == null) {
-        break;
-      }
-      if (cur.isStorageGroup()) {
-        return (IStorageGroupMNode) cur;
-      }
-    }
-    throw new StorageGroupNotSetException(path.getFullPath());
-  }
-
-  /**
-   * Get node by the path
-   *
-   * @return last node in given seriesPath
-   */
-  IMNode getNodeByPath(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;
-    Template upperTemplate = cur.getSchemaTemplate();
-
-    for (int i = 1; i < nodes.length; i++) {
-      if (cur.isMeasurement()) {
-        if (i == nodes.length - 1
-            || ((IMeasurementMNode) cur).getSchema() instanceof VectorMeasurementSchema) {
-          return cur;
-        } else {
-          throw new PathNotExistException(path.getFullPath(), true);
-        }
-      }
-      if (cur.getSchemaTemplate() != null) {
-        upperTemplate = cur.getSchemaTemplate();
-      }
-      IMNode next = cur.getChild(nodes[i]);
-      if (next == null) {
-        if (upperTemplate == null) {
-          throw new PathNotExistException(path.getFullPath(), true);
-        }
-
-        String realName = nodes[i];
-        IMeasurementSchema schema = upperTemplate.getSchemaMap().get(realName);
-        if (schema == null) {
-          throw new PathNotExistException(path.getFullPath(), true);
-        }
-        return new MeasurementMNode(cur, schema.getMeasurementId(), schema, null);
-      }
-      cur = next;
-    }
-    return cur;
-  }
-
-  /**
-   * Get all storage groups under the given path
-   *
-   * @return storage group list
-   * @apiNote :for cluster
-   */
-  List<String> getStorageGroupByPath(PartialPath path) throws MetadataException {
-    List<String> storageGroups = new ArrayList<>();
-    String[] nodes = path.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-    findStorageGroup(root, nodes, 1, "", storageGroups);
-    return storageGroups;
-  }
-
-  /**
-   * Recursively find all storage group according to a specific path
-   *
-   * @apiNote :for cluster
-   */
-  private void findStorageGroup(
-      IMNode node, String[] nodes, int idx, String parent, List<String> storageGroupNames) {
-    if (node.isStorageGroup()) {
-      storageGroupNames.add(node.getFullPath());
-      return;
-    }
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
-    if (!(PATH_WILDCARD).equals(nodeReg)) {
-      IMNode next = node.getChild(nodeReg);
-      if (next != null) {
-        findStorageGroup(
-            next, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR, storageGroupNames);
-      }
-    } else {
-      for (IMNode child : node.getChildren().values()) {
-        findStorageGroup(
-            child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR, storageGroupNames);
-      }
-    }
-  }
-
-  /**
-   * Get all storage group names
-   *
-   * @return a list contains all distinct storage groups
-   */
-  List<PartialPath> getAllStorageGroupPaths() {
-    List<PartialPath> res = new ArrayList<>();
-    Deque<IMNode> nodeStack = new ArrayDeque<>();
-    nodeStack.add(root);
-    while (!nodeStack.isEmpty()) {
-      IMNode current = nodeStack.pop();
-      if (current.isStorageGroup()) {
-        res.add(current.getPartialPath());
-      } else {
-        nodeStack.addAll(current.getChildren().values());
-      }
-    }
-    return res;
-  }
-
-  /**
-   * Get the storage group that given path belonged to or under given path All related storage
-   * groups refer two cases: 1. Storage groups with a prefix that is identical to path, e.g. given
-   * path "root.sg1", storage group "root.sg1.sg2" and "root.sg1.sg3" will be added into result
-   * list. 2. Storage group that this path belongs to, e.g. given path "root.sg1.d1", and it is in
-   * storage group "root.sg1". Then we adds "root.sg1" into result list.
-   *
-   * @return a list contains all storage groups related to given path
-   */
-  List<PartialPath> searchAllRelatedStorageGroups(PartialPath path) throws MetadataException {
-    String[] nodes = path.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-    List<PartialPath> storageGroupPaths = new ArrayList<>();
-    findStorageGroupPaths(root, nodes, 1, "", storageGroupPaths, false);
-    return storageGroupPaths;
-  }
-
-  /**
-   * Get all storage group under given path
-   *
-   * @return a list contains all storage group names under give path
-   */
-  List<PartialPath> getStorageGroupPaths(PartialPath prefixPath) throws MetadataException {
-    String[] nodes = prefixPath.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(prefixPath.getFullPath());
-    }
-    List<PartialPath> storageGroupPaths = new ArrayList<>();
-    findStorageGroupPaths(root, nodes, 1, "", storageGroupPaths, true);
-    return storageGroupPaths;
-  }
-
-  /**
-   * Traverse the MTree to match all storage group with prefix path. When trying to find storage
-   * groups via a path, we divide into two cases: 1. This path is only regarded as a prefix, in
-   * other words, this path is part of the result storage groups. 2. This path is a full path and we
-   * use this method to find its belonged storage group. When prefixOnly is set to true, storage
-   * group paths in 1 is only added into result, otherwise, both 1 and 2 are returned.
-   *
-   * @param node the current traversing node
-   * @param nodes split the prefix path with '.'
-   * @param idx the current index of array nodes
-   * @param parent current parent path
-   * @param storageGroupPaths store all matched storage group names
-   * @param prefixOnly only return storage groups that start with this prefix path
-   */
-  private void findStorageGroupPaths(
-      IMNode node,
-      String[] nodes,
-      int idx,
-      String parent,
-      List<PartialPath> storageGroupPaths,
-      boolean prefixOnly) {
-    if (node.isStorageGroup() && (!prefixOnly || idx >= nodes.length)) {
-      storageGroupPaths.add(node.getPartialPath());
-      return;
-    }
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
-    if (!(PATH_WILDCARD).equals(nodeReg)) {
-      IMNode next = node.getChild(nodeReg);
-      if (next != null) {
-        findStorageGroupPaths(
-            node.getChild(nodeReg),
-            nodes,
-            idx + 1,
-            parent + node.getName() + PATH_SEPARATOR,
-            storageGroupPaths,
-            prefixOnly);
-      }
-    } else {
-      for (IMNode child : node.getChildren().values()) {
-        findStorageGroupPaths(
-            child,
-            nodes,
-            idx + 1,
-            parent + node.getName() + PATH_SEPARATOR,
-            storageGroupPaths,
-            prefixOnly);
-      }
-    }
-  }
-
-  /** Get all storage group MNodes */
-  List<IStorageGroupMNode> getAllStorageGroupNodes() {
-    List<IStorageGroupMNode> ret = new ArrayList<>();
-    Deque<IMNode> nodeStack = new ArrayDeque<>();
-    nodeStack.add(root);
-    while (!nodeStack.isEmpty()) {
-      IMNode current = nodeStack.pop();
-      if (current.isStorageGroup()) {
-        ret.add((IStorageGroupMNode) current);
-      } else {
-        nodeStack.addAll(current.getChildren().values());
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * Get storage group path by path
-   *
-   * <p>e.g., root.sg1 is storage group, path is root.sg1.d1, return root.sg1
-   *
-   * @return storage group in the given path
-   */
-  PartialPath getStorageGroupPath(PartialPath path) throws StorageGroupNotSetException {
-    String[] nodes = path.getNodes();
-    IMNode cur = root;
-    for (int i = 1; i < nodes.length; i++) {
-      cur = cur.getChild(nodes[i]);
-      if (cur == null) {
-        throw new StorageGroupNotSetException(path.getFullPath());
-      } else if (cur.isStorageGroup()) {
-        return cur.getPartialPath();
-      }
-    }
-    throw new StorageGroupNotSetException(path.getFullPath());
-  }
-
-  /** Check whether the given path contains a storage group */
-  boolean checkStorageGroupByPath(PartialPath path) {
-    String[] nodes = path.getNodes();
-    IMNode cur = root;
-    for (int i = 1; i < nodes.length; i++) {
-      cur = cur.getChild(nodes[i]);
-      if (cur == null) {
-        return false;
-      } else if (cur.isStorageGroup()) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Get all timeseries under the given path
-   *
-   * @param prefixPath a prefix path or a full path, may contain '*'.
-   */
-  List<PartialPath> getAllTimeseriesPath(PartialPath prefixPath) throws MetadataException {
-    ShowTimeSeriesPlan plan = new ShowTimeSeriesPlan(prefixPath);
-    List<Pair<PartialPath, String[]>> res = getAllMeasurementSchema(plan);
-    List<PartialPath> paths = new ArrayList<>();
-    for (Pair<PartialPath, String[]> p : res) {
-      paths.add(p.left);
-    }
-    return paths;
-  }
-
-  /**
-   * Get all timeseries paths under the given path
-   *
-   * @param prefixPath a prefix path or a full path, may contain '*'.
-   * @return Pair.left contains all the satisfied paths Pair.right means the current offset or zero
-   *     if we don't set offset.
-   */
-  Pair<List<PartialPath>, Integer> getAllTimeseriesPathWithAlias(
-      PartialPath prefixPath, int limit, int offset) throws MetadataException {
-    PartialPath prePath = new PartialPath(prefixPath.getNodes());
-    ShowTimeSeriesPlan plan = new ShowTimeSeriesPlan(prefixPath);
-    plan.setLimit(limit);
-    plan.setOffset(offset);
-    List<Pair<PartialPath, String[]>> res = getAllMeasurementSchema(plan, false);
-    List<PartialPath> paths = new ArrayList<>();
-    for (Pair<PartialPath, String[]> p : res) {
-      if (prePath.getMeasurement().equals(p.right[0])) {
-        p.left.setMeasurementAlias(p.right[0]);
-      }
-      paths.add(p.left);
-    }
-    if (curOffset.get() == null) {
-      offset = 0;
-    } else {
-      offset = curOffset.get() + 1;
-    }
-    curOffset.remove();
-    return new Pair<>(paths, offset);
-  }
-
-  /**
-   * Get the count of timeseries under the given prefix path. if prefixPath contains '*', then not
-   * throw PathNotExistException()
-   *
-   * @param prefixPath a prefix path or a full path, may contain '*'.
-   */
-  int getAllTimeseriesCount(PartialPath prefixPath) throws MetadataException {
-    String[] nodes = prefixPath.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(prefixPath.getFullPath());
-    }
-    try {
-      return getCount(root, nodes, 1, false);
-    } catch (PathNotExistException e) {
-      throw new PathNotExistException(prefixPath.getFullPath());
-    }
-  }
-
-  /** Traverse the MTree to get the count of timeseries. */
-  private int getCount(IMNode node, String[] nodes, int idx, boolean wildcard)
-      throws PathNotExistException {
-    if (node.isMeasurement()) {
-      if (idx < nodes.length) {
-        if (((IMeasurementMNode) node).getSchema().containsSubMeasurement(nodes[idx])) {
-          return 1;
-        } else {
-          if (!wildcard) {
-            throw new PathNotExistException(node.getName() + NO_CHILDNODE_MSG + nodes[idx]);
-          } else {
-            return 0;
-          }
-        }
-      } else {
-        return ((IMeasurementMNode) node).getMeasurementCount();
-      }
-    }
-    if (idx < nodes.length) {
-      if (PATH_WILDCARD.equals(nodes[idx])) {
-        int sum = 0;
-        for (IMNode child : node.getChildren().values()) {
-          sum += getCount(child, nodes, idx + 1, true);
-        }
-        return sum;
-      } else {
-        IMNode child = node.getChild(nodes[idx]);
-        if (child == null) {
-          if (node.isUseTemplate()
-              && node.getUpperTemplate().getSchemaMap().containsKey(nodes[idx])) {
-            return 1;
-          }
-          if (!wildcard) {
-            throw new PathNotExistException(node.getName() + NO_CHILDNODE_MSG + nodes[idx]);
-          } else {
-            return 0;
-          }
-        }
-        return getCount(child, nodes, idx + 1, wildcard);
-      }
-    } else {
-      int sum = 0;
-      if (node.isUseTemplate()) {
-        sum += node.getUpperTemplate().getSchemaMap().size();
-      }
-      for (IMNode child : node.getChildren().values()) {
-        sum += getCount(child, nodes, idx + 1, wildcard);
-      }
-      return sum;
-    }
-  }
-
-  /**
-   * Get the count of devices under the given prefix path.
-   *
-   * @param prefixPath a prefix path or a full path, may contain '*'.
-   */
-  int getDevicesNum(PartialPath prefixPath) throws MetadataException {
-    String[] nodes = prefixPath.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(prefixPath.getFullPath());
-    }
-    return getDevicesCount(root, nodes, 1);
-  }
-
-  /**
-   * Get the count of storage group under the given prefix path.
-   *
-   * @param prefixPath a prefix path or a full path, may contain '*'.
-   */
-  int getStorageGroupNum(PartialPath prefixPath) throws MetadataException {
-    String[] nodes = prefixPath.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(prefixPath.getFullPath());
-    }
-    return getStorageGroupCount(root, nodes, 1, "");
-  }
-
-  /** Get the count of nodes in the given level under the given prefix path. */
-  int getNodesCountInGivenLevel(PartialPath prefixPath, int level) throws MetadataException {
-    String[] nodes = prefixPath.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(prefixPath.getFullPath());
-    }
-    IMNode node = root;
-    int i;
-    for (i = 1; i < nodes.length; i++) {
-      if (nodes[i].equals("*")) {
-        break;
-      }
-      if (node.getChild(nodes[i]) != null) {
-        node = node.getChild(nodes[i]);
-      } else {
-        throw new MetadataException(nodes[i - 1] + NO_CHILDNODE_MSG + nodes[i]);
-      }
-    }
-    return getCountInGivenLevel(node, level - (i - 1));
-  }
-
-  /** Traverse the MTree to get the count of devices. */
-  private int getDevicesCount(IMNode node, String[] nodes, int idx) {
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
-    boolean curIsDevice = node.isUseTemplate();
-    int cnt = curIsDevice ? 1 : 0;
-    if (!(PATH_WILDCARD).equals(nodeReg)) {
-      IMNode next = node.getChild(nodeReg);
-      if (next != null) {
-        if (next.isMeasurement() && idx >= nodes.length && !curIsDevice) {
-          cnt++;
-        } else {
-          cnt += getDevicesCount(node.getChild(nodeReg), nodes, idx + 1);
-        }
-      }
-    } else {
-      for (IMNode child : node.getChildren().values()) {
-        if (child.isMeasurement() && !curIsDevice && idx >= nodes.length) {
-          cnt++;
-          curIsDevice = true;
-        }
-        cnt += getDevicesCount(child, nodes, idx + 1);
-      }
-    }
-    return cnt;
-  }
-
-  /** Traverse the MTree to get the count of storage group. */
-  private int getStorageGroupCount(IMNode node, String[] nodes, int idx, String parent) {
-    int cnt = 0;
-    if (node.isStorageGroup() && idx >= nodes.length) {
-      cnt++;
-      return cnt;
-    }
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
-    if (!(PATH_WILDCARD).equals(nodeReg)) {
-      IMNode next = node.getChild(nodeReg);
-      if (next != null) {
-        cnt += getStorageGroupCount(next, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR);
-      }
-    } else {
-      for (IMNode child : node.getChildren().values()) {
-        cnt +=
-            getStorageGroupCount(child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR);
-      }
-    }
-    return cnt;
-  }
-
-  /**
-   * Traverse the MTree to get the count of timeseries in the given level.
-   *
-   * @param targetLevel Record the distance to the target level, 0 means the target level.
-   */
-  private int getCountInGivenLevel(IMNode node, int targetLevel) {
-    if (targetLevel == 0) {
-      return 1;
-    }
-    int cnt = 0;
-    for (IMNode child : node.getChildren().values()) {
-      cnt += getCountInGivenLevel(child, targetLevel - 1);
-    }
-    return cnt;
-  }
-
-  /**
-   * Get all time series schema under the given path order by insert frequency
-   *
-   * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
-   */
-  List<Pair<PartialPath, String[]>> getAllMeasurementSchemaByHeatOrder(
-      ShowTimeSeriesPlan plan, QueryContext queryContext) throws MetadataException {
-    String[] nodes = plan.getPath().getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(plan.getPath().getFullPath());
-    }
-    List<Pair<PartialPath, String[]>> allMatchedNodes = new ArrayList<>();
-
-    findPath(root, nodes, 1, allMatchedNodes, false, true, queryContext, null);
-
-    Stream<Pair<PartialPath, String[]>> sortedStream =
-        allMatchedNodes.stream()
-            .sorted(
-                Comparator.comparingLong(
-                        (Pair<PartialPath, String[]> p) -> Long.parseLong(p.right[6]))
-                    .reversed()
-                    .thenComparing((Pair<PartialPath, String[]> p) -> p.left));
-
-    // no limit
-    if (plan.getLimit() == 0) {
-      return sortedStream.collect(toList());
-    } else {
-      return sortedStream.skip(plan.getOffset()).limit(plan.getLimit()).collect(toList());
-    }
-  }
-
-  /**
-   * Get all time series schema under the given path
-   *
-   * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
-   */
-  List<Pair<PartialPath, String[]>> getAllMeasurementSchema(ShowTimeSeriesPlan plan)
-      throws MetadataException {
-    return getAllMeasurementSchema(plan, true);
-  }
-
-  List<Pair<PartialPath, String[]>> getAllMeasurementSchema(
-      ShowTimeSeriesPlan plan, boolean removeCurrentOffset) throws MetadataException {
-    List<Pair<PartialPath, String[]>> res = new LinkedList<>();
-    String[] nodes = plan.getPath().getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(plan.getPath().getFullPath());
-    }
-    limit.set(plan.getLimit());
-    offset.set(plan.getOffset());
-    curOffset.set(-1);
-    count.set(0);
-    findPath(root, nodes, 1, res, offset.get() != 0 || limit.get() != 0, false, null, null);
-    // avoid memory leaks
-    limit.remove();
-    offset.remove();
-    if (removeCurrentOffset) {
-      curOffset.remove();
-    }
-    count.remove();
-    return res;
-  }
-
-  /**
-   * Iterate through MTree to fetch metadata info of all leaf nodes under the given seriesPath
-   *
-   * @param needLast if false, lastTimeStamp in timeseriesSchemaList will be null
-   * @param timeseriesSchemaList List<timeseriesSchema> result: [name, alias, storage group,
-   *     dataType, encoding, compression, offset, lastTimeStamp]
-   */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void findPath(
-      IMNode curNode,
-      String[] nodes,
-      int childIndex,
-      List<Pair<PartialPath, String[]>> timeseriesSchemaList,
-      boolean hasLimit,
-      boolean needLast,
-      QueryContext queryContext,
-      Template upperTemplate)
-      throws MetadataException {
-    if (curNode.isMeasurement()) {
-      if ((nodes.length <= childIndex
-          || ((IMeasurementMNode) curNode).getSchema() instanceof VectorMeasurementSchema)) {
-        if (hasLimit) {
-          curOffset.set(curOffset.get() + 1);
-          if (curOffset.get() < offset.get() || count.get().intValue() == limit.get().intValue()) {
-            return;
-          }
-        }
-        IMeasurementSchema measurementSchema = ((IMeasurementMNode) curNode).getSchema();
-        if (measurementSchema instanceof MeasurementSchema) {
-          addMeasurementSchema(
-              curNode, timeseriesSchemaList, needLast, queryContext, measurementSchema, "*");
-        } else if (measurementSchema instanceof VectorMeasurementSchema) {
-          addVectorMeasurementSchema(
-              curNode,
-              timeseriesSchemaList,
-              needLast,
-              queryContext,
-              measurementSchema,
-              childIndex < nodes.length ? nodes[childIndex] : "*");
-        }
-        if (hasLimit) {
-          count.set(count.get() + 1);
-        }
-      }
-      return;
-    }
-
-    String nodeReg = MetaUtils.getNodeRegByIdx(childIndex, nodes);
-    if (curNode.getSchemaTemplate() != null) {
-      upperTemplate = curNode.getSchemaTemplate();
-    }
-
-    // we should use template when all child is measurement or this curNode has no child
-    if (!nodeReg.contains(PATH_WILDCARD)) {
-      IMNode next = curNode.getChild(nodeReg);
-      if (next != null) {
-        findPath(
-            next,
-            nodes,
-            childIndex + 1,
-            timeseriesSchemaList,
-            hasLimit,
-            needLast,
-            queryContext,
-            upperTemplate);
-      }
-    } else {
-      for (IMNode child : curNode.getChildren().values()) {
-        if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
-          continue;
-        }
-        findPath(
-            child,
-            nodes,
-            childIndex + 1,
-            timeseriesSchemaList,
-            hasLimit,
-            needLast,
-            queryContext,
-            upperTemplate);
-        if (hasLimit && count.get().intValue() == limit.get().intValue()) {
-          return;
-        }
-      }
-    }
-
-    // template part
-    if (curNode.isUseTemplate()) {
-      if (upperTemplate != null) {
-        HashSet<IMeasurementSchema> set = new HashSet<>();
-        for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-          if (set.add(schema)) {
-            if (schema instanceof MeasurementSchema) {
-              addMeasurementSchema(
-                  new MeasurementMNode(curNode, schema.getMeasurementId(), schema, null),
-                  timeseriesSchemaList,
-                  needLast,
-                  queryContext,
-                  schema,
-                  nodeReg);
-            } else if (schema instanceof VectorMeasurementSchema) {
-              VectorMeasurementSchema vectorMeasurementSchema = (VectorMeasurementSchema) schema;
-              if (Pattern.matches(
-                  nodeReg.replace("*", ".*"), vectorMeasurementSchema.getMeasurementId())) {
-                String firstNode = schema.getSubMeasurementsList().get(0);
-                addVectorMeasurementSchemaForTemplate(
-                    new MeasurementMNode(curNode, firstNode, schema, null),
-                    timeseriesSchemaList,
-                    needLast,
-                    queryContext,
-                    schema,
-                    MetaUtils.getNodeRegByIdx(childIndex + 1, nodes),
-                    vectorMeasurementSchema.getMeasurementId());
-              }
-            }
-          }
-        }
-      }
-    }
-  }
-
-  private void addMeasurementSchema(
-      IMNode node,
-      List<Pair<PartialPath, String[]>> timeseriesSchemaList,
-      boolean needLast,
-      QueryContext queryContext,
-      IMeasurementSchema measurementSchema,
-      String reg)
-      throws StorageGroupNotSetException {
-    if (Pattern.matches(reg.replace("*", ".*"), measurementSchema.getMeasurementId())) {
-      PartialPath nodePath = node.getPartialPath();
-      String[] tsRow = new String[7];
-      tsRow[0] = ((IMeasurementMNode) node).getAlias();
-      tsRow[1] = getStorageGroupPath(nodePath).getFullPath();
-      tsRow[2] = measurementSchema.getType().toString();
-      tsRow[3] = measurementSchema.getEncodingType().toString();
-      tsRow[4] = measurementSchema.getCompressor().toString();
-      tsRow[5] = String.valueOf(((IMeasurementMNode) node).getOffset());
-      tsRow[6] =
-          needLast
-              ? String.valueOf(
-                  LastCacheManager.getLastTimeStamp((IMeasurementMNode) node, queryContext))
-              : null;
-      Pair<PartialPath, String[]> temp = new Pair<>(nodePath, tsRow);
-      timeseriesSchemaList.add(temp);
-    }
-  }
-
-  private void addVectorMeasurementSchema(
-      IMNode node,
-      List<Pair<PartialPath, String[]>> timeseriesSchemaList,
-      boolean needLast,
-      QueryContext queryContext,
-      IMeasurementSchema schema,
-      String reg)
-      throws StorageGroupNotSetException, IllegalPathException {
-    List<String> measurements = schema.getSubMeasurementsList();
-    for (int i = 0; i < measurements.size(); i++) {
-      if (!Pattern.matches(reg.replace("*", ".*"), measurements.get(i))) {
-        continue;
-      }
-      PartialPath devicePath = node.getPartialPath();
-      String[] tsRow = new String[7];
-      tsRow[0] = null;
-      tsRow[1] = getStorageGroupPath(devicePath).getFullPath();
-      tsRow[2] = schema.getSubMeasurementsTSDataTypeList().get(i).toString();
-      tsRow[3] = schema.getSubMeasurementsTSEncodingList().get(i).toString();
-      tsRow[4] = schema.getCompressor().toString();
-      tsRow[5] = "-1";
-      tsRow[6] =
-          needLast
-              ? String.valueOf(
-                  LastCacheManager.getLastTimeStamp((IMeasurementMNode) node, queryContext))
-              : null;
-      Pair<PartialPath, String[]> temp =
-          new Pair<>(new VectorPartialPath(devicePath.getFullPath(), measurements.get(i)), tsRow);
-      timeseriesSchemaList.add(temp);
-    }
-  }
-
-  private void addVectorMeasurementSchemaForTemplate(
-      IMNode node,
-      List<Pair<PartialPath, String[]>> timeseriesSchemaList,
-      boolean needLast,
-      QueryContext queryContext,
-      IMeasurementSchema schema,
-      String reg,
-      String vectorId)
-      throws StorageGroupNotSetException, IllegalPathException {
-    List<String> measurements = schema.getSubMeasurementsList();
-    for (int i = 0; i < measurements.size(); i++) {
-      if (!Pattern.matches(reg.replace("*", ".*"), measurements.get(i))) {
-        continue;
-      }
-      PartialPath devicePath =
-          new PartialPath(node.getPartialPath().getDevicePath().getFullPath(), vectorId);
-      String[] tsRow = new String[7];
-      tsRow[0] = null;
-      tsRow[1] = getStorageGroupPath(devicePath).getFullPath();
-      tsRow[2] = schema.getSubMeasurementsTSDataTypeList().get(i).toString();
-      tsRow[3] = schema.getSubMeasurementsTSEncodingList().get(i).toString();
-      tsRow[4] = schema.getCompressor().toString();
-      tsRow[5] = "-1";
-      tsRow[6] =
-          needLast
-              ? String.valueOf(
-                  LastCacheManager.getLastTimeStamp((IMeasurementMNode) node, queryContext))
-              : null;
-      Pair<PartialPath, String[]> temp =
-          new Pair<>(new VectorPartialPath(devicePath.getFullPath(), measurements.get(i)), tsRow);
-      timeseriesSchemaList.add(temp);
-    }
-  }
-
-  /**
-   * Get child node path in the next level of the given path.
-   *
-   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
-   * return [root.sg1.d1, root.sg1.d2]
-   *
-   * @param path The given path
-   * @return All child nodes' seriesPath(s) of given seriesPath.
-   */
-  Set<String> getChildNodePathInNextLevel(PartialPath path) throws MetadataException {
-    String[] nodes = path.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-    Set<String> childNodePaths = new TreeSet<>();
-    findChildNodePathInNextLevel(root, nodes, 1, "", childNodePaths, nodes.length + 1);
-    return childNodePaths;
-  }
-
-  /**
-   * Traverse the MTree to match all child node path in next level
-   *
-   * @param node the current traversing node
-   * @param nodes split the prefix path with '.'
-   * @param idx the current index of array nodes
-   * @param parent store the node string having traversed
-   * @param res store all matched device names
-   * @param length expected length of path
-   */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void findChildNodePathInNextLevel(
-      IMNode node, String[] nodes, int idx, String parent, Set<String> res, int length) {
-    if (node == null) {
-      return;
-    }
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
-    if (!nodeReg.contains(PATH_WILDCARD)) {
-      if (idx == length) {
-        res.add(parent + node.getName());
-      } else {
-        findChildNodePathInNextLevel(
-            node.getChild(nodeReg),
-            nodes,
-            idx + 1,
-            parent + node.getName() + PATH_SEPARATOR,
-            res,
-            length);
-      }
-    } else {
-      if (node.getChildren().size() > 0) {
-        for (IMNode child : node.getChildren().values()) {
-          if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
-            continue;
-          }
-          if (idx == length) {
-            res.add(parent + node.getName());
-          } else {
-            findChildNodePathInNextLevel(
-                child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR, res, length);
-          }
-        }
-      } else if (idx == length) {
-        String nodeName = node.getName();
-        res.add(parent + nodeName);
-      }
-    }
-  }
-
-  /**
-   * Get child node in the next level of the given path.
-   *
-   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
-   * return [d1, d2]
-   *
-   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1.d1
-   * return [s1, s2]
-   *
-   * @param path Path
-   * @return All child nodes' seriesPath(s) of given seriesPath.
-   */
-  Set<String> getChildNodeInNextLevel(PartialPath path) throws MetadataException {
-    String[] nodes = path.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-    Set<String> childNodes = new TreeSet<>();
-    findChildNodeInNextLevel(root, nodes, 1, "", childNodes, nodes.length + 1);
-    return childNodes;
-  }
-
-  /**
-   * Traverse the MTree to match all child node path in next level
-   *
-   * @param node the current traversing node
-   * @param nodes split the prefix path with '.'
-   * @param idx the current index of array nodes
-   * @param parent store the node string having traversed
-   * @param res store all matched device names
-   * @param length expected length of path
-   */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void findChildNodeInNextLevel(
-      IMNode node, String[] nodes, int idx, String parent, Set<String> res, int length) {
-    if (node == null) {
-      return;
-    }
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
-    if (!nodeReg.contains(PATH_WILDCARD)) {
-      if (idx == length) {
-        res.add(node.getName());
-      } else {
-        findChildNodeInNextLevel(
-            node.getChild(nodeReg),
-            nodes,
-            idx + 1,
-            parent + node.getName() + PATH_SEPARATOR,
-            res,
-            length);
-      }
-    } else {
-      if (node.getChildren().size() > 0) {
-        for (IMNode child : node.getChildren().values()) {
-          if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
-            continue;
-          }
-          if (idx == length) {
-            res.add(node.getName());
-          } else {
-            findChildNodeInNextLevel(
-                child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR, res, length);
-          }
-        }
-      } else if (idx == length) {
-        String nodeName = node.getName();
-        res.add(nodeName);
-      }
-    }
-  }
-
-  /**
-   * Get all devices under give path
-   *
-   * @return a list contains all distinct devices names
-   */
-  Set<PartialPath> getDevices(PartialPath prefixPath) throws MetadataException {
-    String[] nodes = prefixPath.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(prefixPath.getFullPath());
-    }
-    Set<PartialPath> devices = new TreeSet<>();
-    findDevices(root, nodes, 1, devices, false, null);
-    return devices;
-  }
-
-  List<ShowDevicesResult> getDevices(ShowDevicesPlan plan) throws MetadataException {
-    String[] nodes = plan.getPath().getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(plan.getPath().getFullPath());
-    }
-    Set<PartialPath> devices = new TreeSet<>();
-    limit.set(plan.getLimit());
-    offset.set(plan.getOffset());
-    curOffset.set(-1);
-    count.set(0);
-    findDevices(root, nodes, 1, devices, offset.get() != 0 || limit.get() != 0, null);
-    // avoid memory leaks
-    limit.remove();
-    offset.remove();
-    curOffset.remove();
-    count.remove();
-    List<ShowDevicesResult> res = new ArrayList<>();
-    for (PartialPath device : devices) {
-      if (plan.hasSgCol()) {
-        res.add(
-            new ShowDevicesResult(device.getFullPath(), getStorageGroupPath(device).getFullPath()));
-      } else {
-        res.add(new ShowDevicesResult(device.getFullPath()));
-      }
-    }
-    return res;
-  }
-
-  /**
-   * Traverse the MTree to match all devices with prefix path.
-   *
-   * @param node the current traversing node
-   * @param nodes split the prefix path with '.'
-   * @param idx the current index of array nodes
-   * @param res store all matched device names
-   */
-  @SuppressWarnings("squid:S3776")
-  private void findDevices(
-      IMNode node,
-      String[] nodes,
-      int idx,
-      Set<PartialPath> res,
-      boolean hasLimit,
-      Template upperTemplate) {
-    upperTemplate = node.getSchemaTemplate() == null ? upperTemplate : node.getSchemaTemplate();
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
-    // the node path doesn't contains '*'
-    if (!nodeReg.contains(PATH_WILDCARD)) {
-      IMNode next = node.getChild(nodeReg);
-      if (next != null) {
-        if (next.isMeasurement()) {
-          if (idx >= nodes.length) {
-            if (hasLimit) {
-              curOffset.set(curOffset.get() + 1);
-              if (curOffset.get() < offset.get()
-                  || count.get().intValue() == limit.get().intValue()) {
-                return;
-              }
-              count.set(count.get() + 1);
-            }
-            res.add(node.getPartialPath());
-          }
-        } else {
-          findDevices(next, nodes, idx + 1, res, hasLimit, upperTemplate);
-        }
-      }
-    } else { // the node path contains '*'
-      boolean deviceAdded = false;
-      List<IMNode> children = new ArrayList<>(node.getChildren().values());
-      // template part
-      if (node.isUseTemplate() && upperTemplate != null) {
-        children.addAll(upperTemplate.getMeasurementMNode());
-      }
-
-      for (IMNode child : children) {
-        // use '.*' to replace '*' to form a regex to match
-        // if the match failed, skip it.
-        if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
-          continue;
-        }
-        if (child.isMeasurement()) {
-          if (!deviceAdded && idx >= nodes.length) {
-            if (hasLimit) {
-              curOffset.set(curOffset.get() + 1);
-              if (curOffset.get() < offset.get()
-                  || count.get().intValue() == limit.get().intValue()) {
-                return;
-              }
-              count.set(count.get() + 1);
-            }
-            res.add(node.getPartialPath());
-            deviceAdded = true;
-          }
-        } else {
-          findDevices(child, nodes, idx + 1, res, hasLimit, upperTemplate);
-        }
-      }
-    }
-  }
-
-  /** Get all paths from root to the given level. */
-  List<PartialPath> getNodesList(PartialPath path, int nodeLevel) throws MetadataException {
-    return getNodesList(path, nodeLevel, null);
-  }
-
-  /** Get all paths from root to the given level */
-  List<PartialPath> getNodesList(PartialPath path, int nodeLevel, StorageGroupFilter filter)
-      throws MetadataException {
-    String[] nodes = path.getNodes();
-    if (!nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-    List<PartialPath> res = new ArrayList<>();
-    IMNode node = root;
-    for (int i = 1; i < nodes.length; i++) {
-      if (node.getChild(nodes[i]) != null) {
-        node = node.getChild(nodes[i]);
-        if (node.isStorageGroup() && filter != null && !filter.satisfy(node.getFullPath())) {
-          return res;
-        }
-      } else {
-        throw new MetadataException(nodes[i - 1] + NO_CHILDNODE_MSG + nodes[i]);
-      }
-    }
-    findNodes(node, path, res, nodeLevel - (nodes.length - 1), filter);
-    return res;
-  }
-
-  /**
-   * Get all paths under the given level.
-   *
-   * @param targetLevel Record the distance to the target level, 0 means the target level.
-   */
-  private void findNodes(
-      IMNode node,
-      PartialPath path,
-      List<PartialPath> res,
-      int targetLevel,
-      StorageGroupFilter filter) {
-    if (node == null
-        || node.isStorageGroup() && filter != null && !filter.satisfy(node.getFullPath())) {
-      return;
-    }
-    if (targetLevel == 0) {
-      res.add(path);
-      return;
-    }
-    for (IMNode child : node.getChildren().values()) {
-      findNodes(child, path.concatNode(child.toString()), res, targetLevel - 1, filter);
-    }
-  }
-
-  public void serializeTo(String snapshotPath) throws IOException {
-    try (MLogWriter mLogWriter = new MLogWriter(snapshotPath)) {
-      root.serializeTo(mLogWriter);
-    }
-  }
-
-  public static MTree deserializeFrom(File mtreeSnapshot) {
-    try (MLogReader mLogReader = new MLogReader(mtreeSnapshot)) {
-      return new MTree(deserializeFromReader(mLogReader));
-    } catch (IOException e) {
-      logger.warn("Failed to deserialize from {}. Use a new MTree.", mtreeSnapshot.getPath());
-      return new MTree();
-    } finally {
-      limit = new ThreadLocal<>();
-      offset = new ThreadLocal<>();
-      count = new ThreadLocal<>();
-      curOffset = new ThreadLocal<>();
-    }
-  }
-
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private static InternalMNode deserializeFromReader(MLogReader mLogReader) {
-    Deque<IMNode> nodeStack = new ArrayDeque<>();
-    IMNode node = null;
-    while (mLogReader.hasNext()) {
-      PhysicalPlan plan = null;
-      try {
-        plan = mLogReader.next();
-        if (plan == null) {
-          continue;
-        }
-        int childrenSize = 0;
-        if (plan instanceof StorageGroupMNodePlan) {
-          node = StorageGroupMNode.deserializeFrom((StorageGroupMNodePlan) plan);
-          childrenSize = ((StorageGroupMNodePlan) plan).getChildSize();
-        } else if (plan instanceof MeasurementMNodePlan) {
-          node = MeasurementMNode.deserializeFrom((MeasurementMNodePlan) plan);
-          childrenSize = ((MeasurementMNodePlan) plan).getChildSize();
-        } else if (plan instanceof MNodePlan) {
-          node = InternalMNode.deserializeFrom((MNodePlan) plan);
-          childrenSize = ((MNodePlan) plan).getChildSize();
-        }
-
-        if (childrenSize != 0) {
-          ConcurrentHashMap<String, IMNode> childrenMap = new ConcurrentHashMap<>();
-          for (int i = 0; i < childrenSize; i++) {
-            IMNode child = nodeStack.removeFirst();
-            childrenMap.put(child.getName(), child);
-            if (child.isMeasurement()) {
-              if (!node.isEntity()) {
-                node = IEntityMNode.setToEntity(node);
-              }
-              String alias = ((IMeasurementMNode) child).getAlias();
-              if (alias != null) {
-                ((IEntityMNode) node).addAlias(alias, (IMeasurementMNode) child);
-              }
-            }
-            child.setParent(node);
-          }
-          node.setChildren(childrenMap);
-        }
-        nodeStack.push(node);
-      } catch (Exception e) {
-        logger.error(
-            "Can not operate cmd {} for err:", plan == null ? "" : plan.getOperatorType(), e);
-      }
-    }
-    if (!IoTDBConstant.PATH_ROOT.equals(node.getName())) {
-      logger.error("Snapshot file corrupted!");
-      //      throw new MetadataException("Snapshot file corrupted!");
-    }
-
-    return (InternalMNode) node;
-  }
-
-  @Override
-  public String toString() {
-    JsonObject jsonObject = new JsonObject();
-    jsonObject.add(root.getName(), mNodeToJSON(root, null));
-    return jsonToString(jsonObject);
-  }
-
-  private JsonObject mNodeToJSON(IMNode node, String storageGroupName) {
-    JsonObject jsonObject = new JsonObject();
-    if (node.getChildren().size() > 0) {
-      if (node.isStorageGroup()) {
-        storageGroupName = node.getFullPath();
-      }
-      for (IMNode child : node.getChildren().values()) {
-        jsonObject.add(child.getName(), mNodeToJSON(child, storageGroupName));
-      }
-    } else if (node.isMeasurement()) {
-      IMeasurementMNode leafMNode = (IMeasurementMNode) node;
-      jsonObject.add("DataType", GSON.toJsonTree(leafMNode.getSchema().getType()));
-      jsonObject.add("Encoding", GSON.toJsonTree(leafMNode.getSchema().getEncodingType()));
-      jsonObject.add("Compressor", GSON.toJsonTree(leafMNode.getSchema().getCompressor()));
-      if (leafMNode.getSchema().getProps() != null) {
-        jsonObject.addProperty("args", leafMNode.getSchema().getProps().toString());
-      }
-      jsonObject.addProperty("StorageGroup", storageGroupName);
-    }
-    return jsonObject;
-  }
-
-  Map<String, String> determineStorageGroup(PartialPath path) throws IllegalPathException {
-    Map<String, String> paths = new HashMap<>();
-    String[] nodes = path.getNodes();
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path.getFullPath());
-    }
-
-    Deque<IMNode> nodeStack = new ArrayDeque<>();
-    Deque<Integer> depthStack = new ArrayDeque<>();
-    if (!root.getChildren().isEmpty()) {
-      nodeStack.push(root);
-      depthStack.push(0);
-    }
-
-    while (!nodeStack.isEmpty()) {
-      IMNode node = nodeStack.removeFirst();
-      int depth = depthStack.removeFirst();
-
-      determineStorageGroup(depth + 1, nodes, node, paths, nodeStack, depthStack);
-    }
-    return paths;
-  }
-
-  /**
-   * Try determining the storage group using the children of a mNode. If one child is a storage
-   * group node, put a storageGroupName-fullPath pair into paths. Otherwise put the children that
-   * match the path into the queue and discard other children.
-   */
-  private void determineStorageGroup(
-      int depth,
-      String[] nodes,
-      IMNode node,
-      Map<String, String> paths,
-      Deque<IMNode> nodeStack,
-      Deque<Integer> depthStack) {
-    String currNode = depth >= nodes.length ? PATH_WILDCARD : nodes[depth];
-    for (Entry<String, IMNode> entry : node.getChildren().entrySet()) {
-      if (!currNode.equals(PATH_WILDCARD) && !currNode.equals(entry.getKey())) {
-        continue;
-      }
-      // this child is desired
-      IMNode child = entry.getValue();
-      if (child.isStorageGroup()) {
-        // we have found one storage group, record it
-        String sgName = child.getFullPath();
-        // concat the remaining path with the storage group name
-        StringBuilder pathWithKnownSG = new StringBuilder(sgName);
-        for (int i = depth + 1; i < nodes.length; i++) {
-          pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(nodes[i]);
-        }
-        if (depth >= nodes.length - 1 && currNode.equals(PATH_WILDCARD)) {
-          // the we find the sg at the last node and the last node is a wildcard (find "root
-          // .group1", for "root.*"), also append the wildcard (to make "root.group1.*")
-          pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(PATH_WILDCARD);
-        }
-        paths.put(sgName, pathWithKnownSG.toString());
-      } else if (!child.getChildren().isEmpty()) {
-        // push it back so we can traver its children later
-        nodeStack.push(child);
-        depthStack.push(depth);
-      }
-    }
-  }
-
-  IEntityMNode setToEntity(IMNode node) {
-    // synchronize check and replace, we need replaceChild become atomic operation
-    // only write on mtree will be synchronized
-    synchronized (this) {
-      return IEntityMNode.setToEntity(node);
-    }
-  }
-
-  /**
-   * check whether there is template on given path and the subTree has template return true,
-   * otherwise false
-   */
-  void checkTemplateOnPath(PartialPath path) throws MetadataException {
-    String[] nodeNames = path.getNodes();
-    IMNode cur = root;
-    if (!nodeNames[0].equals(root.getName())) {
-      return;
-    }
-    if (cur.getSchemaTemplate() != null) {
-      throw new MetadataException("Template already exists on " + cur.getFullPath());
-    }
-    for (int i = 1; i < nodeNames.length; i++) {
-      if (cur.isMeasurement()) {
-        return;
-      }
-      if (!cur.hasChild(nodeNames[i])) {
-        return;
-      }
-      cur = cur.getChild(nodeNames[i]);
-      if (cur.getSchemaTemplate() != null) {
-        throw new MetadataException("Template already exists on " + cur.getFullPath());
-      }
-    }
-
-    checkTemplateOnSubtree(cur);
-  }
-
-  // traverse  all the  descendant of the given path node
-  private void checkTemplateOnSubtree(IMNode node) throws MetadataException {
-    if (node.isMeasurement()) {
-      return;
-    }
-    for (IMNode child : node.getChildren().values()) {
-      if (child.isMeasurement()) {
-        continue;
-      }
-      if (child.getSchemaTemplate() != null) {
-        throw new MetadataException("Template already exists on " + child.getFullPath());
-      }
-      checkTemplateOnSubtree(child);
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
index 524b10f..04f6abc 100755
--- a/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
@@ -32,6 +32,10 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
 
 /**
  * A prefix path, suffix path or fullPath generated from SQL. Usually used in the IoTDB server
@@ -136,24 +140,49 @@ public class PartialPath extends Path implements Comparable<Path> {
   }
 
   /**
-   * Test if this PartialPath matches a full path. rPath is supposed to be a full timeseries path
-   * without wildcards. e.g. "root.sg.device.*" matches path "root.sg.device.s1" whereas it does not
-   * match "root.sg.device" and "root.sg.vehicle.s1"
+   * Test if this PartialPath matches a full path. This partialPath acts as a full path pattern.
+   * rPath is supposed to be a full timeseries path without wildcards. e.g. "root.sg.device.*"
+   * matches path "root.sg.device.s1" whereas it does not match "root.sg.device" and
+   * "root.sg.vehicle.s1"
    *
    * @param rPath a plain full path of a timeseries
    * @return true if a successful match, otherwise return false
    */
   public boolean matchFullPath(PartialPath rPath) {
-    String[] rNodes = rPath.getNodes();
-    if (rNodes.length < nodes.length) {
+    return matchFullPath(rPath.getNodes(), 0, 0, false);
+  }
+
+  private boolean matchFullPath(
+      String[] pathNodes, int pathIndex, int patternIndex, boolean multiLevelWild) {
+    if (pathIndex == pathNodes.length && patternIndex == nodes.length) {
+      return true;
+    } else if (patternIndex == nodes.length && multiLevelWild) {
+      return matchFullPath(pathNodes, pathIndex + 1, patternIndex, true);
+    } else if (pathIndex >= pathNodes.length || patternIndex >= nodes.length) {
       return false;
     }
-    for (int i = 0; i < nodes.length; i++) {
-      if (!nodes[i].equals(IoTDBConstant.PATH_WILDCARD) && !nodes[i].equals(rNodes[i])) {
-        return false;
+
+    String pathNode = pathNodes[pathIndex];
+    String patternNode = nodes[patternIndex];
+    boolean isMatch = false;
+    if (patternNode.equals(MULTI_LEVEL_PATH_WILDCARD)) {
+      isMatch = matchFullPath(pathNodes, pathIndex + 1, patternIndex + 1, true);
+    } else {
+      if (patternNode.contains(ONE_LEVEL_PATH_WILDCARD)) {
+        if (Pattern.matches(patternNode.replace("*", ".*"), pathNode)) {
+          isMatch = matchFullPath(pathNodes, pathIndex + 1, patternIndex + 1, false);
+        }
+      } else {
+        if (patternNode.equals(pathNode)) {
+          isMatch = matchFullPath(pathNodes, pathIndex + 1, patternIndex + 1, false);
+        }
+      }
+
+      if (!isMatch && multiLevelWild) {
+        isMatch = matchFullPath(pathNodes, pathIndex + 1, patternIndex, true);
       }
     }
-    return true;
+    return isMatch;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
index e6c5660..432ed4e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
 import org.apache.iotdb.db.metadata.template.Template;
@@ -56,8 +55,6 @@ public interface IMNode extends Serializable {
 
   void replaceChild(String oldChildName, IMNode newChildNode);
 
-  IMNode getChildOfAlignedTimeseries(String name) throws MetadataException;
-
   Map<String, IMNode> getChildren();
 
   void setChildren(Map<String, IMNode> children);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
index fabfda6..4bdbfe0 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/InternalMNode.java
@@ -18,17 +18,12 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
 import org.apache.iotdb.db.metadata.template.Template;
-import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
@@ -177,25 +172,6 @@ public class InternalMNode extends MNode {
   }
 
   @Override
-  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
-    IMNode node = null;
-    // for aligned timeseries
-    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
-    for (String measurement : measurementList) {
-      IMNode nodeOfMeasurement = getChild(measurement);
-      if (node == null) {
-        node = nodeOfMeasurement;
-      } else {
-        if (node != nodeOfMeasurement) {
-          throw new AlignedTimeseriesException(
-              "Cannot get node of children in different aligned timeseries", name);
-        }
-      }
-    }
-    return node;
-  }
-
-  @Override
   public Map<String, IMNode> getChildren() {
     if (children == null) {
       return Collections.emptyMap();
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
index 7ea67fd..05f59bf 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
@@ -19,7 +19,6 @@
 package org.apache.iotdb.db.metadata.mnode;
 
 import org.apache.iotdb.db.engine.trigger.executor.TriggerExecutor;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
 import org.apache.iotdb.db.metadata.lastCache.container.LastCacheContainer;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
@@ -250,11 +249,6 @@ public class MeasurementMNode extends MNode implements IMeasurementMNode {
   public void replaceChild(String oldChildName, IMNode newChildNode) {}
 
   @Override
-  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
-    return null;
-  }
-
-  @Override
   public Map<String, IMNode> getChildren() {
     return Collections.emptyMap();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
new file mode 100644
index 0000000..62b3e16
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
@@ -0,0 +1,1280 @@
+/*
+ * 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.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+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.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.PathGrouperByStorageGroup;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.EntityPathCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MeasurementPathCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MeasurementSchemaCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupPathCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.TSEntityPathCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.EntityCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MeasurementCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+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.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.file.Files;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+/**
+ * The hierarchical struct of the Metadata Tree is implemented in this class.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>MTree initialization, clear and serialization
+ *   <li>Timeseries operation, including create and delete
+ *   <li>Entity/Device operation
+ *   <li>StorageGroup Operation, including set and delete
+ *   <li>Interfaces and Implementation for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for Storage Group info Query
+ *         <li>Interfaces for Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *         <li>Interfaces for Level Node info Query
+ *         <li>Interfaces and Implementation for metadata count
+ *       </ol>
+ *   <li>Interfaces and Implementation for MNode Query
+ *   <li>Interfaces and Implementation for Template check
+ *   <li>TestOnly Interface
+ * </ol>
+ */
+public class MTree implements Serializable {
+
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+  private static final long serialVersionUID = -4200394435237291964L;
+  private static final Logger logger = LoggerFactory.getLogger(MTree.class);
+  private IMNode root;
+
+  private String mtreeSnapshotPath;
+  private String mtreeSnapshotTmpPath;
+
+  // region MTree initialization, clear and serialization
+  public MTree() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  private MTree(InternalMNode root) {
+    this.root = root;
+  }
+
+  public void init() throws IOException {
+    mtreeSnapshotPath =
+        IoTDBDescriptor.getInstance().getConfig().getSchemaDir()
+            + File.separator
+            + MetadataConstant.MTREE_SNAPSHOT;
+    mtreeSnapshotTmpPath =
+        IoTDBDescriptor.getInstance().getConfig().getSchemaDir()
+            + File.separator
+            + MetadataConstant.MTREE_SNAPSHOT_TMP;
+
+    File tmpFile = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath);
+    if (tmpFile.exists()) {
+      logger.warn("Creating MTree snapshot not successful before crashing...");
+      Files.delete(tmpFile.toPath());
+    }
+
+    File mtreeSnapshot = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
+    long time = System.currentTimeMillis();
+    if (mtreeSnapshot.exists()) {
+      this.root = deserializeFrom(mtreeSnapshot).root;
+      logger.debug(
+          "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+    }
+  }
+
+  public void clear() {
+    root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void createSnapshot() throws IOException {
+    long time = System.currentTimeMillis();
+    logger.info("Start creating MTree snapshot to {}", mtreeSnapshotPath);
+    try {
+      serializeTo(mtreeSnapshotTmpPath);
+      File tmpFile = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath);
+      File snapshotFile = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
+      if (snapshotFile.exists()) {
+        Files.delete(snapshotFile.toPath());
+      }
+      if (tmpFile.renameTo(snapshotFile)) {
+        logger.info(
+            "Finish creating MTree snapshot to {}, spend {} ms.",
+            mtreeSnapshotPath,
+            System.currentTimeMillis() - time);
+      }
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot to {}", mtreeSnapshotPath, e);
+      if (SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath).exists()) {
+        try {
+          Files.delete(SystemFileFactory.INSTANCE.getFile(mtreeSnapshotTmpPath).toPath());
+        } catch (IOException e1) {
+          logger.warn("delete file {} failed: {}", mtreeSnapshotTmpPath, e1.getMessage());
+        }
+      }
+      throw e;
+    }
+  }
+
+  private static String jsonToString(JsonObject jsonObject) {
+    return GSON.toJson(jsonObject);
+  }
+
+  public void serializeTo(String snapshotPath) throws IOException {
+    try (MLogWriter mLogWriter = new MLogWriter(snapshotPath)) {
+      root.serializeTo(mLogWriter);
+    }
+  }
+
+  public static MTree deserializeFrom(File mtreeSnapshot) {
+    try (MLogReader mLogReader = new MLogReader(mtreeSnapshot)) {
+      return new MTree(deserializeFromReader(mLogReader));
+    } catch (IOException e) {
+      logger.warn("Failed to deserialize from {}. Use a new MTree.", mtreeSnapshot.getPath());
+      return new MTree();
+    }
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private static InternalMNode deserializeFromReader(MLogReader mLogReader) {
+    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    IMNode node = null;
+    while (mLogReader.hasNext()) {
+      PhysicalPlan plan = null;
+      try {
+        plan = mLogReader.next();
+        if (plan == null) {
+          continue;
+        }
+        int childrenSize = 0;
+        if (plan instanceof StorageGroupMNodePlan) {
+          node = StorageGroupMNode.deserializeFrom((StorageGroupMNodePlan) plan);
+          childrenSize = ((StorageGroupMNodePlan) plan).getChildSize();
+        } else if (plan instanceof MeasurementMNodePlan) {
+          node = MeasurementMNode.deserializeFrom((MeasurementMNodePlan) plan);
+          childrenSize = ((MeasurementMNodePlan) plan).getChildSize();
+        } else if (plan instanceof MNodePlan) {
+          node = InternalMNode.deserializeFrom((MNodePlan) plan);
+          childrenSize = ((MNodePlan) plan).getChildSize();
+        }
+
+        if (childrenSize != 0) {
+          ConcurrentHashMap<String, IMNode> childrenMap = new ConcurrentHashMap<>();
+          for (int i = 0; i < childrenSize; i++) {
+            IMNode child = nodeStack.removeFirst();
+            childrenMap.put(child.getName(), child);
+            if (child.isMeasurement()) {
+              if (!node.isEntity()) {
+                node = IEntityMNode.setToEntity(node);
+              }
+              String alias = ((IMeasurementMNode) child).getAlias();
+              if (alias != null) {
+                ((IEntityMNode) node).addAlias(alias, (IMeasurementMNode) child);
+              }
+            }
+            child.setParent(node);
+          }
+          node.setChildren(childrenMap);
+        }
+        nodeStack.push(node);
+      } catch (Exception e) {
+        logger.error(
+            "Can not operate cmd {} for err:", plan == null ? "" : plan.getOperatorType(), e);
+      }
+    }
+    if (!IoTDBConstant.PATH_ROOT.equals(node.getName())) {
+      logger.error("Snapshot file corrupted!");
+      //      throw new MetadataException("Snapshot file corrupted!");
+    }
+
+    return (InternalMNode) node;
+  }
+
+  @Override
+  public String toString() {
+    JsonObject jsonObject = new JsonObject();
+    jsonObject.add(root.getName(), mNodeToJSON(root, null));
+    return jsonToString(jsonObject);
+  }
+
+  private JsonObject mNodeToJSON(IMNode node, String storageGroupName) {
+    JsonObject jsonObject = new JsonObject();
+    if (node.getChildren().size() > 0) {
+      if (node.isStorageGroup()) {
+        storageGroupName = node.getFullPath();
+      }
+      for (IMNode child : node.getChildren().values()) {
+        jsonObject.add(child.getName(), mNodeToJSON(child, storageGroupName));
+      }
+    } else if (node.isMeasurement()) {
+      IMeasurementMNode leafMNode = (IMeasurementMNode) node;
+      jsonObject.add("DataType", GSON.toJsonTree(leafMNode.getSchema().getType()));
+      jsonObject.add("Encoding", GSON.toJsonTree(leafMNode.getSchema().getEncodingType()));
+      jsonObject.add("Compressor", GSON.toJsonTree(leafMNode.getSchema().getCompressor()));
+      if (leafMNode.getSchema().getProps() != null) {
+        jsonObject.addProperty("args", leafMNode.getSchema().getProps().toString());
+      }
+      jsonObject.addProperty("StorageGroup", storageGroupName);
+    }
+    return jsonObject;
+  }
+  // endregion
+
+  // region Timeseries operation, including create and delete
+  /**
+   * Create a timeseries with a full path from root to leaf node. Before creating a timeseries, the
+   * storage group should be set first, throw exception otherwise
+   *
+   * @param path timeseries path
+   * @param dataType data type
+   * @param encoding encoding
+   * @param compressor compressor
+   * @param props props
+   * @param alias alias of measurement
+   */
+  public IMeasurementMNode createTimeseries(
+      PartialPath path,
+      TSDataType dataType,
+      TSEncoding encoding,
+      CompressionType compressor,
+      Map<String, String> props,
+      String alias)
+      throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    if (nodeNames.length <= 2 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    MetaFormatUtils.checkTimeseries(path);
+    IMNode cur = root;
+    boolean hasSetStorageGroup = false;
+    Template upperTemplate = cur.getSchemaTemplate();
+    // e.g, path = root.sg.d1.s1,  create internal nodes and set cur to d1 node
+    for (int i = 1; i < nodeNames.length - 1; i++) {
+      if (cur.isMeasurement()) {
+        throw new PathAlreadyExistException(cur.getFullPath());
+      }
+      if (cur.isStorageGroup()) {
+        hasSetStorageGroup = true;
+      }
+      String childName = nodeNames[i];
+      if (!cur.hasChild(childName)) {
+        if (!hasSetStorageGroup) {
+          throw new StorageGroupNotSetException("Storage group should be created first");
+        }
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(childName)) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(childName).getFullPath());
+        }
+        cur.addChild(childName, new InternalMNode(cur, childName));
+      }
+      cur = cur.getChild(childName);
+
+      if (cur.getSchemaTemplate() != null) {
+        upperTemplate = cur.getSchemaTemplate();
+      }
+    }
+
+    if (cur.isMeasurement()) {
+      throw new PathAlreadyExistException(cur.getFullPath());
+    }
+
+    if (upperTemplate != null && !upperTemplate.isCompatible(path)) {
+      throw new PathAlreadyExistException(
+          path.getFullPath() + " ( which is incompatible with template )");
+    }
+
+    MetaFormatUtils.checkTimeseriesProps(path.getFullPath(), props);
+
+    String leafName = nodeNames[nodeNames.length - 1];
+
+    // synchronize check and add, we need addChild and add Alias become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(leafName)) {
+        throw new PathAlreadyExistException(path.getFullPath());
+      }
+
+      if (alias != null && cur.hasChild(alias)) {
+        throw new AliasAlreadyExistException(path.getFullPath(), alias);
+      }
+
+      IEntityMNode entityMNode = IEntityMNode.setToEntity(cur);
+
+      IMeasurementMNode measurementMNode =
+          new MeasurementMNode(entityMNode, leafName, alias, dataType, encoding, compressor, props);
+      entityMNode.addChild(leafName, measurementMNode);
+      // link alias to LeafMNode
+      if (alias != null) {
+        entityMNode.addAlias(alias, measurementMNode);
+      }
+      return measurementMNode;
+    }
+  }
+
+  /**
+   * Create aligned timeseries with full paths from root to one leaf node. Before creating
+   * timeseries, the * storage group 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 compressor compressor
+   */
+  public void createAlignedTimeseries(
+      PartialPath devicePath,
+      List<String> measurements,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings,
+      CompressionType compressor)
+      throws MetadataException {
+    String[] deviceNodeNames = devicePath.getNodes();
+    if (deviceNodeNames.length <= 1 || !deviceNodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(devicePath.getFullPath());
+    }
+    MetaFormatUtils.checkTimeseries(devicePath);
+    MetaFormatUtils.checkSchemaMeasurementNames(measurements);
+    IMNode cur = root;
+    boolean hasSetStorageGroup = false;
+    // e.g, devicePath = root.sg.d1, create internal nodes and set cur to d1 node
+    for (int i = 1; i < deviceNodeNames.length - 1; i++) {
+      if (cur.isMeasurement()) {
+        throw new PathAlreadyExistException(cur.getFullPath());
+      }
+      if (cur.isStorageGroup()) {
+        hasSetStorageGroup = true;
+      }
+      String nodeName = deviceNodeNames[i];
+      if (!cur.hasChild(nodeName)) {
+        if (!hasSetStorageGroup) {
+          throw new StorageGroupNotSetException("Storage group should be created first");
+        }
+        cur.addChild(nodeName, new InternalMNode(cur, nodeName));
+      }
+      cur = cur.getChild(nodeName);
+    }
+
+    if (cur.isMeasurement()) {
+      throw new PathAlreadyExistException(cur.getFullPath());
+    }
+
+    String leafName = deviceNodeNames[deviceNodeNames.length - 1];
+
+    // synchronize check and add, we need addChild and add Alias become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(leafName)) {
+        throw new PathAlreadyExistException(devicePath.getFullPath() + "." + leafName);
+      }
+
+      IEntityMNode entityMNode = IEntityMNode.setToEntity(cur);
+
+      int measurementsSize = measurements.size();
+
+      // this measurementMNode could be a leaf or not.
+      IMeasurementMNode measurementMNode =
+          new MeasurementMNode(
+              entityMNode,
+              leafName,
+              new VectorMeasurementSchema(
+                  leafName,
+                  measurements.toArray(new String[measurementsSize]),
+                  dataTypes.toArray(new TSDataType[measurementsSize]),
+                  encodings.toArray(new TSEncoding[measurementsSize]),
+                  compressor),
+              null);
+      entityMNode.addChild(leafName, measurementMNode);
+    }
+  }
+
+  /**
+   * Delete path. The path should be a full path from root to leaf node
+   *
+   * @param path Format: root.node(.node)+
+   */
+  public Pair<PartialPath, IMeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(
+      PartialPath path) throws MetadataException {
+    IMNode curNode = getNodeByPath(path);
+    if (!(curNode.isMeasurement())) {
+      throw new PathNotExistException(path.getFullPath());
+    }
+    String[] nodes = path.getNodes();
+    if (nodes.length == 0 || !IoTDBConstant.PATH_ROOT.equals(nodes[0])) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+
+    IMeasurementMNode deletedNode = (IMeasurementMNode) curNode;
+
+    // delete the last node of path
+    curNode.getParent().deleteChild(path.getMeasurement());
+    if (deletedNode.getAlias() != null) {
+      deletedNode.getParent().deleteAliasChild(((IMeasurementMNode) curNode).getAlias());
+    }
+    curNode = curNode.getParent();
+    // delete all empty ancestors except storage group and MeasurementMNode
+    while (!IoTDBConstant.PATH_ROOT.equals(curNode.getName())
+        && !(curNode.isUseTemplate())
+        && curNode.getChildren().size() == 0) {
+      // if current storage group has no time series, return the storage group name
+      if (curNode.isStorageGroup()) {
+        return new Pair<>(curNode.getPartialPath(), deletedNode);
+      }
+      curNode.getParent().deleteChild(curNode.getName());
+      curNode = curNode.getParent();
+    }
+    return new Pair<>(null, deletedNode);
+  }
+  // endregion
+
+  // region Entity/Device operation
+  // including device auto creation and transform from InternalMNode to EntityMNode
+  /**
+   * 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
+   */
+  public IMNode getDeviceNodeWithAutoCreating(PartialPath deviceId, int sgLevel)
+      throws MetadataException {
+    String[] nodeNames = deviceId.getNodes();
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(deviceId.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    for (int i = 1; i < nodeNames.length; i++) {
+      if (!cur.hasChild(nodeNames[i])) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        if (i == sgLevel) {
+          cur.addChild(
+              nodeNames[i],
+              new StorageGroupMNode(
+                  cur, nodeNames[i], IoTDBDescriptor.getInstance().getConfig().getDefaultTTL()));
+        } else {
+          cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+        }
+      }
+      cur = cur.getChild(nodeNames[i]);
+      // update upper template
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+    }
+
+    return cur;
+  }
+
+  public IEntityMNode setToEntity(IMNode node) {
+    // synchronize check and replace, we need replaceChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      return IEntityMNode.setToEntity(node);
+    }
+  }
+  // endregion
+
+  // region StorageGroup Operation, including set and delete
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    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]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
+        }
+      } else {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        IStorageGroupMNode storageGroupMNode =
+            new StorageGroupMNode(
+                cur, nodeNames[i], IoTDBDescriptor.getInstance().getConfig().getDefaultTTL());
+        cur.addChild(nodeNames[i], storageGroupMNode);
+      }
+    }
+  }
+
+  /** Delete a storage group */
+  public List<IMeasurementMNode> deleteStorageGroup(PartialPath path) throws MetadataException {
+    IMNode cur = getNodeByPath(path);
+    if (!(cur.isStorageGroup())) {
+      throw new StorageGroupNotSetException(path.getFullPath());
+    }
+    // Suppose current system has root.a.b.sg1, root.a.sg2, and delete root.a.b.sg1
+    // delete the storage group node sg1
+    cur.getParent().deleteChild(cur.getName());
+
+    // collect all the LeafMNode in this storage group
+    List<IMeasurementMNode> leafMNodes = new LinkedList<>();
+    Queue<IMNode> queue = new LinkedList<>();
+    queue.add(cur);
+    while (!queue.isEmpty()) {
+      IMNode node = queue.poll();
+      for (IMNode child : node.getChildren().values()) {
+        if (child.isMeasurement()) {
+          leafMNodes.add((IMeasurementMNode) child);
+        } else {
+          queue.add(child);
+        }
+      }
+    }
+
+    cur = cur.getParent();
+    // delete node b while retain root.a.sg2
+    while (!IoTDBConstant.PATH_ROOT.equals(cur.getName()) && cur.getChildren().size() == 0) {
+      cur.getParent().deleteChild(cur.getName());
+      cur = cur.getParent();
+    }
+    return leafMNodes;
+  }
+  // endregion
+
+  // region Interfaces and Implementation for metadata info Query
+  /**
+   * Check whether the given path exists.
+   *
+   * @param path a full path or a prefix path
+   */
+  public boolean isPathExist(PartialPath path) {
+    String[] nodeNames = path.getNodes();
+    IMNode cur = root;
+    if (!nodeNames[0].equals(root.getName())) {
+      return false;
+    }
+    Template upperTemplate = cur.getSchemaTemplate();
+    for (int i = 1; i < nodeNames.length; i++) {
+      if (!cur.hasChild(nodeNames[i])) {
+        return cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i]);
+      }
+      cur = cur.getChild(nodeNames[i]);
+      if (cur.isMeasurement()) {
+        if (i == nodeNames.length - 1) {
+          return true;
+        }
+        if (((IMeasurementMNode) cur).getSchema() instanceof VectorMeasurementSchema) {
+          return i == nodeNames.length - 2
+              && ((IMeasurementMNode) cur).getSchema().containsSubMeasurement(nodeNames[i + 1]);
+        } else {
+          return false;
+        }
+      }
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+    }
+    return true;
+  }
+
+  // region Interfaces for Storage Group info Query
+  /**
+   * Check whether path is storage group or not
+   *
+   * <p>e.g., path = root.a.b.sg. if nor a and b is StorageGroupMNode and sg is a StorageGroupMNode
+   * path is a storage group
+   *
+   * @param path path
+   * @apiNote :for cluster
+   */
+  public boolean isStorageGroup(PartialPath path) {
+    String[] nodeNames = path.getNodes();
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(IoTDBConstant.PATH_ROOT)) {
+      return false;
+    }
+    IMNode cur = root;
+    int i = 1;
+    while (i < nodeNames.length - 1) {
+      cur = cur.getChild(nodeNames[i]);
+      if (cur == null || cur.isStorageGroup()) {
+        return false;
+      }
+      i++;
+    }
+    cur = cur.getChild(nodeNames[i]);
+    return cur != null && cur.isStorageGroup();
+  }
+
+  /** Check whether the given path contains a storage group */
+  public boolean checkStorageGroupByPath(PartialPath path) {
+    String[] nodes = path.getNodes();
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        return false;
+      } else if (cur.isStorageGroup()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Get storage group path by path
+   *
+   * <p>e.g., root.sg1 is storage group, path is root.sg1.d1, return root.sg1
+   *
+   * @return storage group in the given path
+   */
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
+    String[] nodes = path.getNodes();
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        throw new StorageGroupNotSetException(path.getFullPath());
+      } else if (cur.isStorageGroup()) {
+        return cur.getPartialPath();
+      }
+    }
+    throw new StorageGroupNotSetException(path.getFullPath());
+  }
+
+  /**
+   * Get the storage group that given path pattern matches or belongs to.
+   *
+   * <p>Suppose we have (root.sg1.d1.s1, root.sg2.d2.s2), refer the following cases: 1. given path
+   * "root.sg1", ("root.sg1") will be returned. 2. given path "root.*", ("root.sg1", "root.sg2")
+   * will be returned. 3. given path "root.*.d1.s1", ("root.sg1", "root.sg2") will be returned.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @return a list contains all storage groups related to given path
+   */
+  public List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    StorageGroupPathCollector collector = new StorageGroupPathCollector(root, pathPattern);
+    collector.setCollectInternal(true);
+    collector.traverse();
+    return collector.getResult();
+  }
+
+  /**
+   * Get all storage group that the given path pattern matches.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @return a list contains all storage group names under given path pattern
+   */
+  public List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    StorageGroupPathCollector collector = new StorageGroupPathCollector(root, pathPattern);
+    collector.setCollectInternal(false);
+    collector.traverse();
+    return collector.getResult();
+  }
+
+  /**
+   * Get all storage group names
+   *
+   * @return a list contains all distinct storage groups
+   */
+  public List<PartialPath> getAllStorageGroupPaths() {
+    List<PartialPath> res = new ArrayList<>();
+    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    nodeStack.add(root);
+    while (!nodeStack.isEmpty()) {
+      IMNode current = nodeStack.pop();
+      if (current.isStorageGroup()) {
+        res.add(current.getPartialPath());
+      } else {
+        nodeStack.addAll(current.getChildren().values());
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Resolve the path or path pattern into StorageGroupName-FullPath pairs. Try determining the
+   * storage group using the children of a mNode. If one child is a storage group node, put a
+   * storageGroupName-fullPath pair into paths.
+   */
+  public Map<String, String> groupPathByStorageGroup(PartialPath path) throws MetadataException {
+    PathGrouperByStorageGroup resolver = new PathGrouperByStorageGroup(root, path);
+    resolver.traverse();
+    return resolver.getResult();
+  }
+  // endregion
+
+  // region Interfaces for Device info Query
+  /**
+   * Get all devices matching the given path pattern. If isPrefixMatch, then the devices under the
+   * paths matching given path pattern will be collected too.
+   *
+   * @return a list contains all distinct devices names
+   */
+  public Set<PartialPath> getDevices(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    EntityPathCollector collector = new EntityPathCollector(root, pathPattern);
+    collector.setPrefixMatch(isPrefixMatch);
+    collector.traverse();
+    return collector.getResult();
+  }
+
+  public List<ShowDevicesResult> getDevices(ShowDevicesPlan plan) throws MetadataException {
+    EntityPathCollector collector =
+        new EntityPathCollector(root, plan.getPath(), plan.getLimit(), plan.getOffset());
+    collector.traverse();
+    Set<PartialPath> devices = collector.getResult();
+    List<ShowDevicesResult> res = new ArrayList<>();
+    for (PartialPath device : devices) {
+      if (plan.hasSgCol()) {
+        res.add(
+            new ShowDevicesResult(
+                device.getFullPath(), getBelongedStorageGroup(device).getFullPath()));
+      } else {
+        res.add(new ShowDevicesResult(device.getFullPath()));
+      }
+    }
+    return res;
+  }
+
+  public Set<PartialPath> getDevicesByTimeseries(PartialPath timeseries) throws MetadataException {
+    TSEntityPathCollector collector = new TSEntityPathCollector(root, timeseries);
+    collector.traverse();
+    return collector.getResult();
+  }
+  // endregion
+
+  // region Interfaces for timeseries, measurement and schema info Query
+  /**
+   * Get measurement schema for a given path. Path must be a complete Path from root to leaf node.
+   */
+  public IMeasurementSchema getSchema(PartialPath fullPath) throws MetadataException {
+    IMeasurementMNode node = (IMeasurementMNode) getNodeByPath(fullPath);
+    return node.getSchema();
+  }
+
+  /**
+   * Get all timeseries matching the given path pattern
+   *
+   * @param pathPattern a path pattern or a full path, may contain wildcard.
+   */
+  public List<PartialPath> getAllTimeseriesPath(PartialPath pathPattern) throws MetadataException {
+    return getAllTimeseriesPathWithAlias(pathPattern, 0, 0).left;
+  }
+
+  /**
+   * Get all timeseries paths matching the given path pattern
+   *
+   * @param pathPattern a path pattern or a full path, may contain wildcard
+   * @return Pair.left contains all the satisfied paths Pair.right means the current offset or zero
+   *     if we don't set offset.
+   */
+  public Pair<List<PartialPath>, Integer> getAllTimeseriesPathWithAlias(
+      PartialPath pathPattern, int limit, int offset) throws MetadataException {
+    MeasurementPathCollector collector =
+        new MeasurementPathCollector(root, pathPattern, limit, offset);
+    collector.traverse();
+    offset = collector.getCurOffset() + 1;
+    List<PartialPath> res = collector.getResult();
+    return new Pair<>(res, offset);
+  }
+
+  /**
+   * Get all time series schema matching the given path pattern order by insert frequency
+   *
+   * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
+   */
+  public List<Pair<PartialPath, String[]>> getAllMeasurementSchemaByHeatOrder(
+      ShowTimeSeriesPlan plan, QueryContext queryContext) throws MetadataException {
+    MeasurementSchemaCollector collector = new MeasurementSchemaCollector(root, plan.getPath());
+    collector.setQueryContext(queryContext);
+    collector.setNeedLast(true);
+    collector.traverse();
+    List<Pair<PartialPath, String[]>> allMatchedNodes = collector.getResult();
+
+    Stream<Pair<PartialPath, String[]>> sortedStream =
+        allMatchedNodes.stream()
+            .sorted(
+                Comparator.comparingLong(
+                        (Pair<PartialPath, String[]> p) -> Long.parseLong(p.right[6]))
+                    .reversed()
+                    .thenComparing((Pair<PartialPath, String[]> p) -> p.left));
+
+    // no limit
+    if (plan.getLimit() == 0) {
+      return sortedStream.collect(toList());
+    } else {
+      return sortedStream.skip(plan.getOffset()).limit(plan.getLimit()).collect(toList());
+    }
+  }
+
+  /**
+   * Get all time series schema matching the given path pattern
+   *
+   * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
+   */
+  public List<Pair<PartialPath, String[]>> getAllMeasurementSchema(ShowTimeSeriesPlan plan)
+      throws MetadataException {
+    MeasurementSchemaCollector collector =
+        new MeasurementSchemaCollector(root, plan.getPath(), plan.getLimit(), plan.getOffset());
+    collector.traverse();
+    return collector.getResult();
+  }
+  // endregion
+
+  // region Interfaces for Level Node info Query
+  /**
+   * Get child node path in the next level of the given path pattern.
+   *
+   * <p>give pathPattern and the child nodes is those matching pathPattern.*.
+   *
+   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
+   * return [root.sg1.d1, root.sg1.d2]
+   *
+   * @param pathPattern The given path
+   * @return All child nodes' seriesPath(s) of given seriesPath.
+   */
+  public Set<String> getChildNodePathInNextLevel(PartialPath pathPattern) throws MetadataException {
+    try {
+      MNodeCollector<Set<String>> collector =
+          new MNodeCollector<Set<String>>(root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD)) {
+            @Override
+            protected void transferToResult(IMNode node) {
+              resultSet.add(node.getFullPath());
+            }
+          };
+      collector.setResultSet(new TreeSet<>());
+      collector.traverse();
+      return collector.getResult();
+    } catch (IllegalPathException e) {
+      throw new IllegalPathException(pathPattern.getFullPath());
+    }
+  }
+
+  /**
+   * Get child node in the next level of the given path.
+   *
+   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
+   * return [d1, d2]
+   *
+   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1.d1
+   * return [s1, s2]
+   *
+   * @param pathPattern Path
+   * @return All child nodes' seriesPath(s) of given seriesPath.
+   */
+  public Set<String> getChildNodeNameInNextLevel(PartialPath pathPattern) throws MetadataException {
+    try {
+      MNodeCollector<Set<String>> collector =
+          new MNodeCollector<Set<String>>(root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD)) {
+            @Override
+            protected void transferToResult(IMNode node) {
+              resultSet.add(node.getName());
+            }
+          };
+      collector.setResultSet(new TreeSet<>());
+      collector.traverse();
+      return collector.getResult();
+    } catch (IllegalPathException e) {
+      throw new IllegalPathException(pathPattern.getFullPath());
+    }
+  }
+
+  /** Get all paths from root to the given level */
+  public List<PartialPath> getNodesListInGivenLevel(
+      PartialPath pathPattern, int nodeLevel, StorageGroupFilter filter) throws MetadataException {
+    MNodeCollector<List<PartialPath>> collector =
+        new MNodeCollector<List<PartialPath>>(root, pathPattern) {
+          @Override
+          protected void transferToResult(IMNode node) {
+            resultSet.add(node.getPartialPath());
+          }
+        };
+    collector.setResultSet(new LinkedList<>());
+    collector.setTargetLevel(nodeLevel);
+    collector.setStorageGroupFilter(filter);
+    collector.traverse();
+    return collector.getResult();
+  }
+  // endregion
+
+  // region Interfaces and Implementation for metadata count
+  /**
+   * Get the count of timeseries matching the given path.
+   *
+   * @param path a path pattern or a full path, may contain wildcard
+   */
+  public int getAllTimeseriesCount(PartialPath path) throws MetadataException {
+    CounterTraverser counter = new MeasurementCounter(root, path);
+    counter.traverse();
+    return counter.getCount();
+  }
+
+  /**
+   * Get the count of devices matching the given path.
+   *
+   * @param path a path pattern or a full path, may contain wildcard
+   */
+  public int getDevicesNum(PartialPath path) throws MetadataException {
+    CounterTraverser counter = new EntityCounter(root, path);
+    counter.traverse();
+    return counter.getCount();
+  }
+
+  /**
+   * Get the count of storage group matching the given path.
+   *
+   * @param path a path pattern or a full path, may contain wildcard.
+   */
+  public int getStorageGroupNum(PartialPath path) throws MetadataException {
+    CounterTraverser counter = new StorageGroupCounter(root, path);
+    counter.traverse();
+    return counter.getCount();
+  }
+
+  /** Get the count of nodes in the given level matching the given path. */
+  public int getNodesCountInGivenLevel(PartialPath path, int level) throws MetadataException {
+    MNodeLevelCounter counter = new MNodeLevelCounter(root, path, level);
+    counter.traverse();
+    return counter.getCount();
+  }
+  // endregion
+
+  // endregion
+
+  // region Interfaces and Implementation for MNode Query
+  /**
+   * Get node by the path
+   *
+   * @return last node in given seriesPath
+   */
+  public IMNode getNodeByPath(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;
+    Template upperTemplate = cur.getSchemaTemplate();
+
+    for (int i = 1; i < nodes.length; i++) {
+      if (cur.isMeasurement()) {
+        if (i == nodes.length - 1
+            || ((IMeasurementMNode) cur).getSchema() instanceof VectorMeasurementSchema) {
+          return cur;
+        } else {
+          throw new PathNotExistException(path.getFullPath(), true);
+        }
+      }
+      if (cur.getSchemaTemplate() != null) {
+        upperTemplate = cur.getSchemaTemplate();
+      }
+      IMNode next = cur.getChild(nodes[i]);
+      if (next == null) {
+        if (upperTemplate == null) {
+          throw new PathNotExistException(path.getFullPath(), true);
+        }
+
+        String realName = nodes[i];
+        IMeasurementSchema schema = upperTemplate.getSchemaMap().get(realName);
+        if (schema == null) {
+          throw new PathNotExistException(path.getFullPath(), true);
+        }
+        return new MeasurementMNode(cur, schema.getMeasurementId(), schema, null);
+      }
+      cur = next;
+    }
+    return cur;
+  }
+
+  /**
+   * Get node by path with storage group check If storage group is not set,
+   * StorageGroupNotSetException will be thrown
+   */
+  public IMNode getNodeByPathWithStorageGroupCheck(PartialPath path) throws MetadataException {
+    boolean storageGroupChecked = false;
+    String[] nodes = path.getNodes();
+    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+
+    IMNode cur = root;
+
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        // not find
+        if (!storageGroupChecked) {
+          throw new StorageGroupNotSetException(path.getFullPath());
+        }
+        throw new PathNotExistException(path.getFullPath());
+      }
+
+      if (cur.isStorageGroup()) {
+        storageGroupChecked = true;
+      }
+    }
+
+    if (!storageGroupChecked) {
+      throw new StorageGroupNotSetException(path.getFullPath());
+    }
+    return cur;
+  }
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
+   * device], throw exception Get storage group node, if the give path is not a storage group, throw
+   * exception
+   */
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException {
+    IMNode node = getNodeByPath(path);
+    if (node.isStorageGroup()) {
+      return (IStorageGroupMNode) node;
+    } else {
+      throw new StorageGroupNotSetException(path.getFullPath(), true);
+    }
+  }
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
+   * device], return the MNode of root.sg Get storage group node, the give path don't need to be
+   * storage group path.
+   */
+  public IStorageGroupMNode getStorageGroupNodeByPath(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;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        break;
+      }
+      if (cur.isStorageGroup()) {
+        return (IStorageGroupMNode) cur;
+      }
+    }
+    throw new StorageGroupNotSetException(path.getFullPath());
+  }
+
+  /** Get all storage group MNodes */
+  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
+    List<IStorageGroupMNode> ret = new ArrayList<>();
+    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    nodeStack.add(root);
+    while (!nodeStack.isEmpty()) {
+      IMNode current = nodeStack.pop();
+      if (current.isStorageGroup()) {
+        ret.add((IStorageGroupMNode) current);
+      } else {
+        nodeStack.addAll(current.getChildren().values());
+      }
+    }
+    return ret;
+  }
+  // endregion
+
+  // region Interfaces and Implementation for Template check
+  /**
+   * check whether there is template on given path and the subTree has template return true,
+   * otherwise false
+   */
+  public void checkTemplateOnPath(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    IMNode cur = root;
+    if (!nodeNames[0].equals(root.getName())) {
+      return;
+    }
+    if (cur.getSchemaTemplate() != null) {
+      throw new MetadataException("Template already exists on " + cur.getFullPath());
+    }
+    for (int i = 1; i < nodeNames.length; i++) {
+      if (cur.isMeasurement()) {
+        return;
+      }
+      if (!cur.hasChild(nodeNames[i])) {
+        return;
+      }
+      cur = cur.getChild(nodeNames[i]);
+      if (cur.getSchemaTemplate() != null) {
+        throw new MetadataException("Template already exists on " + cur.getFullPath());
+      }
+    }
+
+    checkTemplateOnSubtree(cur);
+  }
+
+  // traverse  all the  descendant of the given path node
+  private void checkTemplateOnSubtree(IMNode node) throws MetadataException {
+    if (node.isMeasurement()) {
+      return;
+    }
+    for (IMNode child : node.getChildren().values()) {
+      if (child.isMeasurement()) {
+        continue;
+      }
+      if (child.getSchemaTemplate() != null) {
+        throw new MetadataException("Template already exists on " + child.getFullPath());
+      }
+      checkTemplateOnSubtree(child);
+    }
+  }
+  // endregion
+
+  // region TestOnly Interface
+  /** combine multiple metadata in string format */
+  @TestOnly
+  public static JsonObject combineMetadataInStrings(String[] metadataStrs) {
+    JsonObject[] jsonObjects = new JsonObject[metadataStrs.length];
+    for (int i = 0; i < jsonObjects.length; i++) {
+      jsonObjects[i] = GSON.fromJson(metadataStrs[i], JsonObject.class);
+    }
+
+    JsonObject root = jsonObjects[0];
+    for (int i = 1; i < jsonObjects.length; i++) {
+      root = combineJsonObjects(root, jsonObjects[i]);
+    }
+
+    return root;
+  }
+
+  private static JsonObject combineJsonObjects(JsonObject a, JsonObject b) {
+    JsonObject res = new JsonObject();
+
+    Set<String> retainSet = new HashSet<>(a.keySet());
+    retainSet.retainAll(b.keySet());
+    Set<String> aCha = new HashSet<>(a.keySet());
+    Set<String> bCha = new HashSet<>(b.keySet());
+    aCha.removeAll(retainSet);
+    bCha.removeAll(retainSet);
+
+    for (String key : aCha) {
+      res.add(key, a.get(key));
+    }
+
+    for (String key : bCha) {
+      res.add(key, b.get(key));
+    }
+    for (String key : retainSet) {
+      JsonElement v1 = a.get(key);
+      JsonElement v2 = b.get(key);
+      if (v1 instanceof JsonObject && v2 instanceof JsonObject) {
+        res.add(key, combineJsonObjects((JsonObject) v1, (JsonObject) v2));
+      } else {
+        res.add(v1.getAsString(), v2);
+      }
+    }
+    return res;
+  }
+  // endregion
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/PathGrouperByStorageGroup.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/PathGrouperByStorageGroup.java
new file mode 100644
index 0000000..917d1da
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/PathGrouperByStorageGroup.java
@@ -0,0 +1,103 @@
+/*
+ * 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.traverser;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+/**
+ * This class implements the storage group resolution function as following description.
+ *
+ * <p>For a path, infer all storage groups it may belong to. The path can have wildcards. Resolve
+ * the path or path pattern into StorageGroupName-FullPath pairs that FullPath matches the given
+ * path.
+ *
+ * <p>Consider the path into two parts: (1) the sub path which can not contain a storage group name
+ * and (2) the sub path which is substring that begin after the storage group name.
+ *
+ * <p>(1) Suppose the part of the path can not contain a storage group name (e.g.,
+ * "root".contains("root.sg") == false), then: For each one level wildcard *, only one level will be
+ * inferred and the wildcard will be removed. For each multi level wildcard **, then the inference
+ * will go on until the storage groups are found and the wildcard will be kept. (2) Suppose the part
+ * of the path is a substring that begin after the storage group name. (e.g., For
+ * "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part is "a.*.b.*"). For this
+ * part, keep what it is.
+ *
+ * <p>Assuming we have three SGs: root.group1, root.group2, root.area1.group3 Eg1: for input
+ * "root.**", returns ("root.group1", "root.group1.**"), ("root.group2", "root.group2.**")
+ * ("root.area1.group3", "root.area1.group3.**") Eg2: for input "root.*.s1", returns ("root.group1",
+ * "root.group1.s1"), ("root.group2", "root.group2.s1")
+ *
+ * <p>Eg3: for input "root.area1.**", returns ("root.area1.group3", "root.area1.group3.**")
+ *
+ * <p>ResultSet: StorageGroupName-FullPath pairs
+ */
+public class PathGrouperByStorageGroup extends Traverser {
+
+  private Map<String, String> resultSet = new HashMap<>();
+
+  public PathGrouperByStorageGroup(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    if (node.isStorageGroup()) {
+      transferToResult(node, idx);
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (node.isStorageGroup()) {
+      transferToResult(node, idx);
+      return true;
+    }
+    return false;
+  }
+
+  protected void transferToResult(IMNode node, int idx) {
+    // we have found one storage group, record it
+    String sgName = node.getFullPath();
+    // concat the remaining path with the storage group name
+    StringBuilder pathWithKnownSG = new StringBuilder(sgName);
+    for (int i = idx + 1; i < nodes.length; i++) {
+      pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(nodes[i]);
+    }
+    if (idx >= nodes.length - 1 && nodes[nodes.length - 1].equals(MULTI_LEVEL_PATH_WILDCARD)) {
+      // the we find the sg match the last node and the last node is a wildcard (find "root
+      // .group1", for "root.**"), also append the wildcard (to make "root.group1.**")
+      pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(MULTI_LEVEL_PATH_WILDCARD);
+    }
+    resultSet.put(sgName, pathWithKnownSG.toString());
+  }
+
+  public Map<String, String> getResult() {
+    return resultSet;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
new file mode 100644
index 0000000..b5fd71e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
@@ -0,0 +1,235 @@
+/*
+ * 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.traverser;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+/**
+ * This class defines the main traversal framework and declares some methods for result process
+ * extension. This class could be extended to implement concrete tasks. Currently, the tasks are
+ * classified into two type: 1. counter: to count the node num or measurement num that matches the
+ * path pattern 2. collector: to collect customized results of the matched node or measurement
+ */
+public abstract class Traverser {
+
+  protected IMNode startNode;
+  protected String[] nodes;
+
+  // if isMeasurementTraverser, measurement in template should be processed
+  protected boolean isMeasurementTraverser = false;
+
+  // default false means fullPath pattern match
+  protected boolean isPrefixMatch = false;
+
+  public Traverser(IMNode startNode, PartialPath path) throws MetadataException {
+    String[] nodes = path.getNodes();
+    if (nodes.length == 0 || !nodes[0].equals(startNode.getName())) {
+      throw new IllegalPathException(
+          path.getFullPath(), path.getFullPath() + " doesn't start with " + startNode.getName());
+    }
+    this.startNode = startNode;
+    this.nodes = nodes;
+  }
+
+  /**
+   * The interface to start the traversal. The node process should be defined before traversal by
+   * overriding or implement concerned methods.
+   */
+  public void traverse() throws MetadataException {
+    traverse(startNode, 0, 0);
+  }
+
+  /**
+   * The recursive method for MTree traversal. If the node matches nodes[idx], then do some
+   * operation and traverse the children with nodes[idx+1].
+   *
+   * @param node current node that match the targetName in given path
+   * @param idx the index of targetName in given path
+   * @param level the level of current node in MTree
+   * @throws MetadataException some result process may throw MetadataException
+   */
+  protected void traverse(IMNode node, int idx, int level) throws MetadataException {
+
+    if (processMatchedMNode(node, idx, level)) {
+      return;
+    }
+
+    if (idx >= nodes.length - 1) {
+      if (nodes[nodes.length - 1].equals(MULTI_LEVEL_PATH_WILDCARD) || isPrefixMatch) {
+        processMultiLevelWildcard(node, idx, level);
+      }
+      return;
+    }
+
+    if (node.isMeasurement()) {
+      return;
+    }
+
+    String targetName = nodes[idx + 1];
+    if (MULTI_LEVEL_PATH_WILDCARD.equals(targetName)) {
+      processMultiLevelWildcard(node, idx, level);
+    } else if (targetName.contains(ONE_LEVEL_PATH_WILDCARD)) {
+      processOneLevelWildcard(node, idx, level);
+    } else {
+      processNameMatch(node, idx, level);
+    }
+  }
+
+  /**
+   * process curNode that matches the targetName during traversal. there are two cases: 1. internal
+   * match: root.sg internal match root.sg.**(pattern) 2. full match: root.sg.d full match
+   * root.sg.**(pattern) Both of them are default abstract and should be implemented according
+   * concrete tasks.
+   *
+   * @return whether this branch of recursive traversal should stop; if true, stop
+   */
+  private boolean processMatchedMNode(IMNode node, int idx, int level) throws MetadataException {
+    if (idx < nodes.length - 1) {
+      return processInternalMatchedMNode(node, idx, level);
+    } else {
+      return processFullMatchedMNode(node, idx, level);
+    }
+  }
+
+  /**
+   * internal match: root.sg internal match root.sg.**(pattern)
+   *
+   * @return whether this branch of recursive traversal should stop; if true, stop
+   */
+  protected abstract boolean processInternalMatchedMNode(IMNode node, int idx, int level)
+      throws MetadataException;
+
+  /**
+   * full match: root.sg.d full match root.sg.**(pattern)
+   *
+   * @return whether this branch of recursive traversal should stop; if true, stop
+   */
+  protected abstract boolean processFullMatchedMNode(IMNode node, int idx, int level)
+      throws MetadataException;
+
+  protected void processMultiLevelWildcard(IMNode node, int idx, int level)
+      throws MetadataException {
+    for (IMNode child : node.getChildren().values()) {
+      traverse(child, idx + 1, level + 1);
+    }
+
+    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+      return;
+    }
+
+    Template upperTemplate = node.getUpperTemplate();
+    for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
+      traverse(
+          new MeasurementMNode(node, schema.getMeasurementId(), schema, null), idx + 1, level + 1);
+    }
+  }
+
+  protected void processOneLevelWildcard(IMNode node, int idx, int level) throws MetadataException {
+    boolean multiLevelWildcard = nodes[idx].equals(MULTI_LEVEL_PATH_WILDCARD);
+    String targetNameRegex = nodes[idx + 1].replace("*", ".*");
+    for (IMNode child : node.getChildren().values()) {
+      if (child.isMeasurement()) {
+        String alias = ((IMeasurementMNode) child).getAlias();
+        if (!Pattern.matches(targetNameRegex, child.getName())
+            && !(alias != null && Pattern.matches(targetNameRegex, alias))) {
+          continue;
+        }
+      } else {
+        if (!Pattern.matches(targetNameRegex, child.getName())) {
+          continue;
+        }
+      }
+      traverse(child, idx + 1, level + 1);
+    }
+    if (multiLevelWildcard) {
+      for (IMNode child : node.getChildren().values()) {
+        traverse(child, idx, level + 1);
+      }
+    }
+
+    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+      return;
+    }
+
+    Template upperTemplate = node.getUpperTemplate();
+    for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
+      if (!Pattern.matches(targetNameRegex, schema.getMeasurementId())) {
+        continue;
+      }
+      traverse(
+          new MeasurementMNode(node, schema.getMeasurementId(), schema, null), idx + 1, level + 1);
+    }
+    if (multiLevelWildcard) {
+      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
+        traverse(
+            new MeasurementMNode(node, schema.getMeasurementId(), schema, null), idx, level + 1);
+      }
+    }
+  }
+
+  protected void processNameMatch(IMNode node, int idx, int level) throws MetadataException {
+    boolean multiLevelWildcard = nodes[idx].equals(MULTI_LEVEL_PATH_WILDCARD);
+    String targetName = nodes[idx + 1];
+    IMNode next = node.getChild(targetName);
+    if (next != null) {
+      traverse(next, idx + 1, level + 1);
+    }
+    if (multiLevelWildcard) {
+      for (IMNode child : node.getChildren().values()) {
+        traverse(child, idx, level + 1);
+      }
+    }
+
+    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+      return;
+    }
+
+    Template upperTemplate = node.getUpperTemplate();
+    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
+    if (targetSchema != null) {
+      traverse(
+          new MeasurementMNode(node, targetSchema.getMeasurementId(), targetSchema, null),
+          idx + 1,
+          level + 1);
+    }
+
+    if (multiLevelWildcard) {
+      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
+        traverse(
+            new MeasurementMNode(node, schema.getMeasurementId(), schema, null), idx, level + 1);
+      }
+    }
+  }
+
+  public void setPrefixMatch(boolean isPrefixMatch) {
+    this.isPrefixMatch = isPrefixMatch;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/CollectorTraverser.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/CollectorTraverser.java
new file mode 100644
index 0000000..bdf2890
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/CollectorTraverser.java
@@ -0,0 +1,93 @@
+/*
+ * 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.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.Traverser;
+
+// This class defines the generic resultSet as traversal result and add more restrictions on MTree
+// traversal.
+public abstract class CollectorTraverser<T> extends Traverser {
+
+  // used for implement slimit and offset function in DDL
+  protected int limit;
+  protected int offset;
+
+  protected boolean hasLimit = false;
+  protected int count = 0;
+  protected int curOffset = -1;
+
+  protected T resultSet;
+
+  public CollectorTraverser(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+  }
+
+  public CollectorTraverser(IMNode startNode, PartialPath path, int limit, int offset)
+      throws MetadataException {
+    super(startNode, path);
+    this.limit = limit;
+    this.offset = offset;
+    if (limit != 0 || offset != 0) {
+      hasLimit = true;
+    }
+  }
+
+  /** extends traversal with limit restriction */
+  @Override
+  protected void traverse(IMNode node, int idx, int level) throws MetadataException {
+    if (hasLimit && count == limit) {
+      return;
+    }
+    super.traverse(node, idx, level);
+  }
+
+  /**
+   * After invoke traverse(), this method could be invoked to get result
+   *
+   * @return the traversal result
+   */
+  public T getResult() {
+    return resultSet;
+  }
+
+  public void setResultSet(T resultSet) {
+    this.resultSet = resultSet;
+  }
+
+  public int getCurOffset() {
+    return curOffset;
+  }
+
+  public void setLimit(int limit) {
+    this.limit = limit;
+    if (limit != 0) {
+      hasLimit = true;
+    }
+  }
+
+  public void setOffset(int offset) {
+    this.offset = offset;
+    if (offset != 0) {
+      hasLimit = true;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/EntityPathCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/EntityPathCollector.java
new file mode 100644
index 0000000..1c3a8ee
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/EntityPathCollector.java
@@ -0,0 +1,65 @@
+/*
+ * 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.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+// This class implements the EntityMNode path collection function.
+// Compared with TSEntityPathCollector, this class only process entities that full match the path
+// pattern.
+public class EntityPathCollector extends CollectorTraverser<Set<PartialPath>> {
+
+  public EntityPathCollector(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+    this.resultSet = new TreeSet<>();
+  }
+
+  public EntityPathCollector(IMNode startNode, PartialPath path, int limit, int offset)
+      throws MetadataException {
+    super(startNode, path, limit, offset);
+    this.resultSet = new TreeSet<>();
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    return false;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (node.isEntity()) {
+      if (hasLimit) {
+        curOffset += 1;
+        if (curOffset < offset) {
+          return true;
+        }
+      }
+      resultSet.add(node.getPartialPath());
+      if (hasLimit) {
+        count += 1;
+      }
+    }
+    return false;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MNodeCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MNodeCollector.java
new file mode 100644
index 0000000..dcce8bd
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MNodeCollector.java
@@ -0,0 +1,76 @@
+/*
+ * 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.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+
+// This class defines any node in MTree as potential target node.
+public abstract class MNodeCollector<T> extends CollectorTraverser<T> {
+
+  // traverse for specific storage group
+  protected StorageGroupFilter storageGroupFilter = null;
+
+  // level query option
+  protected int targetLevel;
+
+  public MNodeCollector(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+  }
+
+  @Override
+  protected void traverse(IMNode node, int idx, int level) throws MetadataException {
+    if (storageGroupFilter != null
+        && node.isStorageGroup()
+        && !storageGroupFilter.satisfy(node.getFullPath())) {
+      return;
+    }
+    super.traverse(node, idx, level);
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    return false;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (targetLevel > 0) {
+      if (level == targetLevel) {
+        transferToResult(node);
+        return true;
+      }
+    } else {
+      transferToResult(node);
+    }
+    return false;
+  }
+
+  protected abstract void transferToResult(IMNode node);
+
+  public void setStorageGroupFilter(StorageGroupFilter storageGroupFilter) {
+    this.storageGroupFilter = storageGroupFilter;
+  }
+
+  public void setTargetLevel(int targetLevel) {
+    this.targetLevel = targetLevel;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
new file mode 100644
index 0000000..af476e2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
@@ -0,0 +1,134 @@
+/*
+ * 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.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This class defines MeasurementMNode as target node and defines the measurement process framework.
+public abstract class MeasurementCollector<T> extends CollectorTraverser<T> {
+
+  public MeasurementCollector(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+    isMeasurementTraverser = true;
+  }
+
+  public MeasurementCollector(IMNode startNode, PartialPath path, int limit, int offset)
+      throws MetadataException {
+    super(startNode, path, limit, offset);
+    isMeasurementTraverser = true;
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level)
+      throws MetadataException {
+    if (!node.isMeasurement() || idx != nodes.length - 2) {
+      return false;
+    }
+    IMeasurementSchema schema = ((IMeasurementMNode) node).getSchema();
+    if (schema instanceof VectorMeasurementSchema) {
+      List<String> measurements = schema.getSubMeasurementsList();
+      String targetNameRegex = nodes[idx + 1].replace("*", ".*");
+      for (int i = 0; i < measurements.size(); i++) {
+        if (!Pattern.matches(targetNameRegex, measurements.get(i))) {
+          continue;
+        }
+        if (hasLimit) {
+          curOffset += 1;
+          if (curOffset < offset) {
+            break;
+          }
+        }
+        collectVectorMeasurement((IMeasurementMNode) node, i);
+        if (hasLimit) {
+          count += 1;
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level)
+      throws MetadataException {
+    if (!node.isMeasurement()) {
+      return false;
+    }
+    IMeasurementSchema schema = ((IMeasurementMNode) node).getSchema();
+    if (schema instanceof MeasurementSchema) {
+      if (hasLimit) {
+        curOffset += 1;
+        if (curOffset < offset) {
+          return true;
+        }
+      }
+      collectUnaryMeasurement((IMeasurementMNode) node);
+      if (hasLimit) {
+        count += 1;
+      }
+    } else if (schema instanceof VectorMeasurementSchema) {
+      if (idx >= nodes.length - 1
+          && !nodes[nodes.length - 1].equals(MULTI_LEVEL_PATH_WILDCARD)
+          && !isPrefixMatch) {
+        return true;
+      }
+      // only when idx > nodes.length or nodes ends with ** or isPrefixMatch
+      List<String> measurements = schema.getSubMeasurementsList();
+      for (int i = 0; i < measurements.size(); i++) {
+        if (hasLimit) {
+          curOffset += 1;
+          if (curOffset < offset) {
+            return true;
+          }
+        }
+        collectVectorMeasurement((IMeasurementMNode) node, i);
+        if (hasLimit) {
+          count += 1;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * collect the information of unary measurement
+   *
+   * @param node MeasurementMNode holding unary the measurement schema
+   */
+  protected abstract void collectUnaryMeasurement(IMeasurementMNode node) throws MetadataException;
+
+  /**
+   * collect the information of target sub measurement of vector measurement
+   *
+   * @param node MeasurementMNode holding the vector measurement schema
+   * @param index the index of target sub measurement
+   */
+  protected abstract void collectVectorMeasurement(IMeasurementMNode node, int index)
+      throws MetadataException;
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementPathCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementPathCollector.java
new file mode 100644
index 0000000..434ac6d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementPathCollector.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.mtree.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+
+import java.util.LinkedList;
+import java.util.List;
+
+// This class implements the measurement path collection function.
+public class MeasurementPathCollector extends MeasurementCollector<List<PartialPath>> {
+
+  public MeasurementPathCollector(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+  }
+
+  public MeasurementPathCollector(IMNode startNode, PartialPath path, int limit, int offset)
+      throws MetadataException {
+    super(startNode, path, limit, offset);
+    this.resultSet = new LinkedList<>();
+  }
+
+  @Override
+  protected void collectUnaryMeasurement(IMeasurementMNode node) throws MetadataException {
+    PartialPath path = node.getPartialPath();
+    if (nodes[nodes.length - 1].equals(node.getAlias())) {
+      // only when user query with alias, the alias in path will be set
+      path.setMeasurementAlias(node.getAlias());
+    }
+    resultSet.add(path);
+  }
+
+  @Override
+  protected void collectVectorMeasurement(IMeasurementMNode node, int index)
+      throws MetadataException {
+    resultSet.add(
+        new VectorPartialPath(
+            node.getFullPath(), node.getSchema().getSubMeasurementsList().get(index)));
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementSchemaCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementSchemaCollector.java
new file mode 100644
index 0000000..db8d5ea
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementSchemaCollector.java
@@ -0,0 +1,114 @@
+/*
+ * 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.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.apache.iotdb.db.metadata.lastCache.LastCacheManager.getLastTimeStamp;
+
+// This class implements the measurement collection function.
+public class MeasurementSchemaCollector
+    extends MeasurementCollector<List<Pair<PartialPath, String[]>>> {
+
+  // whether show timeseries with last value
+  protected boolean needLast = false;
+  // queryContext helps get last value
+  protected QueryContext queryContext;
+
+  public MeasurementSchemaCollector(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+    this.resultSet = new LinkedList<>();
+  }
+
+  public MeasurementSchemaCollector(IMNode startNode, PartialPath path, int limit, int offset)
+      throws MetadataException {
+    super(startNode, path, limit, offset);
+    this.resultSet = new LinkedList<>();
+  }
+
+  public void setNeedLast(boolean needLast) {
+    this.needLast = needLast;
+  }
+
+  public void setQueryContext(QueryContext queryContext) {
+    this.queryContext = queryContext;
+  }
+
+  @Override
+  protected void collectUnaryMeasurement(IMeasurementMNode node) throws MetadataException {
+    IMeasurementSchema measurementSchema = node.getSchema();
+    String[] tsRow = new String[7];
+    tsRow[0] = node.getAlias();
+    tsRow[1] = getStorageGroupPath(node).getFullPath();
+    tsRow[2] = measurementSchema.getType().toString();
+    tsRow[3] = measurementSchema.getEncodingType().toString();
+    tsRow[4] = measurementSchema.getCompressor().toString();
+    tsRow[5] = String.valueOf(node.getOffset());
+    tsRow[6] = needLast ? String.valueOf(getLastTimeStamp(node, queryContext)) : null;
+    Pair<PartialPath, String[]> temp = new Pair<>(node.getPartialPath(), tsRow);
+    resultSet.add(temp);
+  }
+
+  @Override
+  protected void collectVectorMeasurement(IMeasurementMNode node, int index)
+      throws MetadataException {
+    IMeasurementSchema schema = node.getSchema();
+    List<String> measurements = schema.getSubMeasurementsList();
+    String[] tsRow = new String[7];
+    tsRow[0] = null;
+    tsRow[1] = getStorageGroupPath(node).getFullPath();
+    tsRow[2] = schema.getSubMeasurementsTSDataTypeList().get(index).toString();
+    tsRow[3] = schema.getSubMeasurementsTSEncodingList().get(index).toString();
+    tsRow[4] = schema.getCompressor().toString();
+    tsRow[5] = "-1";
+    tsRow[6] = needLast ? String.valueOf(getLastTimeStamp(node, queryContext)) : null;
+    Pair<PartialPath, String[]> temp =
+        new Pair<>(new VectorPartialPath(node.getFullPath(), measurements.get(index)), tsRow);
+    resultSet.add(temp);
+  }
+
+  private PartialPath getStorageGroupPath(IMeasurementMNode node)
+      throws StorageGroupNotSetException {
+    if (node == null) {
+      return null;
+    }
+    IMNode temp = node;
+    while (temp != null) {
+      if (temp.isStorageGroup()) {
+        break;
+      }
+      temp = temp.getParent();
+    }
+    if (temp == null) {
+      throw new StorageGroupNotSetException(node.getFullPath());
+    }
+    return temp.getPartialPath();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/StorageGroupPathCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/StorageGroupPathCollector.java
new file mode 100644
index 0000000..f9eef82
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/StorageGroupPathCollector.java
@@ -0,0 +1,65 @@
+/*
+ * 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.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+
+import java.util.LinkedList;
+import java.util.List;
+
+// This class implements storage group path collection function.
+public class StorageGroupPathCollector extends CollectorTraverser<List<PartialPath>> {
+
+  protected boolean collectInternal = false;
+
+  public StorageGroupPathCollector(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+    this.resultSet = new LinkedList<>();
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    if (node.isStorageGroup()) {
+      if (collectInternal) {
+        transferToResult(node);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (node.isStorageGroup()) {
+      transferToResult(node);
+      return true;
+    }
+    return false;
+  }
+
+  private void transferToResult(IMNode node) {
+    resultSet.add(node.getPartialPath());
+  }
+
+  public void setCollectInternal(boolean collectInternal) {
+    this.collectInternal = collectInternal;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/TSEntityPathCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/TSEntityPathCollector.java
new file mode 100644
index 0000000..a336d11
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/TSEntityPathCollector.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.mtree.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This class implements related entity paths collection for given timeseries path pattern.
+// All the entities, that one of the timeseries matching the path pattern belongs to, will be
+// collected.
+public class TSEntityPathCollector extends CollectorTraverser<Set<PartialPath>> {
+
+  public TSEntityPathCollector(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+    this.resultSet = new TreeSet<>();
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    if (!node.isMeasurement() || idx != nodes.length - 2) {
+      return false;
+    }
+    IMeasurementSchema schema = ((IMeasurementMNode) node).getSchema();
+    if (schema instanceof VectorMeasurementSchema) {
+      List<String> measurements = schema.getSubMeasurementsList();
+      String regex = nodes[idx + 1].replace("*", ".*");
+      for (String measurement : measurements) {
+        if (!Pattern.matches(regex, measurement)) {
+          resultSet.add(node.getParent().getPartialPath());
+          break;
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (!node.isMeasurement()) {
+      return false;
+    }
+    IMeasurementSchema schema = ((IMeasurementMNode) node).getSchema();
+    if (schema instanceof MeasurementSchema) {
+      resultSet.add(node.getParent().getPartialPath());
+    } else if (schema instanceof VectorMeasurementSchema) {
+      if (idx >= nodes.length - 1
+          && !nodes[nodes.length - 1].equals(MULTI_LEVEL_PATH_WILDCARD)
+          && !isPrefixMatch) {
+        return true;
+      }
+      // only when idx > nodes.length or nodes ends with ** or isPrefixMatch
+      resultSet.add(node.getParent().getPartialPath());
+    }
+    return true;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/CounterTraverser.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/CounterTraverser.java
new file mode 100644
index 0000000..30ac8d7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/CounterTraverser.java
@@ -0,0 +1,38 @@
+/*
+ * 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.traverser.counter;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.Traverser;
+
+// This class define the count as traversal result.
+public abstract class CounterTraverser extends Traverser {
+
+  protected int count;
+
+  public CounterTraverser(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+  }
+
+  public int getCount() {
+    return count;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/EntityCounter.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/EntityCounter.java
new file mode 100644
index 0000000..13dacbc
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/EntityCounter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.traverser.counter;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+
+// This class implements the entity count function.
+public class EntityCounter extends CounterTraverser {
+
+  public EntityCounter(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    return false;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (node.isEntity()) {
+      count++;
+    }
+    return false;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MNodeLevelCounter.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MNodeLevelCounter.java
new file mode 100644
index 0000000..8309dc9
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MNodeLevelCounter.java
@@ -0,0 +1,51 @@
+/*
+ * 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.traverser.counter;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+
+// This node implements node count function.
+public class MNodeLevelCounter extends CounterTraverser {
+
+  // level query option
+  protected int targetLevel;
+
+  public MNodeLevelCounter(IMNode startNode, PartialPath path, int targetLevel)
+      throws MetadataException {
+    super(startNode, path);
+    this.targetLevel = targetLevel;
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    return false;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (level == targetLevel) {
+      count++;
+      return true;
+    } else {
+      return false;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MeasurementCounter.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MeasurementCounter.java
new file mode 100644
index 0000000..9d304d1
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MeasurementCounter.java
@@ -0,0 +1,79 @@
+/*
+ * 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.traverser.counter;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This method implements the measurement count function.
+// TODO distinguish timeseries count and measurement count, an aligned timeseries stands for one
+// timeseries but several measurement
+public class MeasurementCounter extends CounterTraverser {
+
+  public MeasurementCounter(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+    isMeasurementTraverser = true;
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    if (!node.isMeasurement() || idx != nodes.length - 2) {
+      return false;
+    }
+    IMeasurementSchema schema = ((IMeasurementMNode) node).getSchema();
+    if (schema instanceof VectorMeasurementSchema) {
+      List<String> measurements = schema.getSubMeasurementsList();
+      String regex = nodes[idx + 1].replace("*", ".*");
+      for (String measurement : measurements) {
+        if (Pattern.matches(regex, measurement)) {
+          count++;
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (!node.isMeasurement()) {
+      return false;
+    }
+    IMeasurementSchema schema = ((IMeasurementMNode) node).getSchema();
+    if (schema instanceof MeasurementSchema) {
+      count++;
+    } else if (schema instanceof VectorMeasurementSchema) {
+      if (idx >= nodes.length - 1 && !nodes[nodes.length - 1].equals(MULTI_LEVEL_PATH_WILDCARD)) {
+        return true;
+      }
+      // only when idx > nodes.length or nodes ends with **
+      count += ((IMeasurementMNode) node).getMeasurementCount();
+    }
+    return true;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/StorageGroupCounter.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/StorageGroupCounter.java
new file mode 100644
index 0000000..3313995
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/StorageGroupCounter.java
@@ -0,0 +1,46 @@
+/*
+ * 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.traverser.counter;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+
+// This class implements storage group count function.
+public class StorageGroupCounter extends CounterTraverser {
+
+  public StorageGroupCounter(IMNode startNode, PartialPath path) throws MetadataException {
+    super(startNode, path);
+  }
+
+  @Override
+  protected boolean processInternalMatchedMNode(IMNode node, int idx, int level) {
+    return node.isStorageGroup();
+  }
+
+  @Override
+  protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
+    if (node.isStorageGroup()) {
+      count++;
+      return true;
+    } else {
+      return false;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
index 7ee8ee1..27a37c6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
@@ -32,14 +32,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
 
 public class MetaUtils {
 
   private MetaUtils() {}
 
   public static String getNodeRegByIdx(int idx, String[] nodes) {
-    return idx >= nodes.length ? PATH_WILDCARD : nodes[idx];
+    return idx >= nodes.length ? ONE_LEVEL_PATH_WILDCARD : nodes[idx];
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 34b4c5d..9572839 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -34,7 +34,7 @@ public class SQLConstant {
     // forbidding instantiation
   }
 
-  private static final String[] SINGLE_ROOT_ARRAY = {"root"};
+  private static final String[] SINGLE_ROOT_ARRAY = {"root", "**"};
   private static final String[] SINGLE_TIME_ARRAY = {"time"};
   public static final PartialPath TIME_PATH = new PartialPath(SINGLE_TIME_ARRAY);
   public static final String ALIGNBY_DEVICE_COLUMN_NAME = "Device";
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 c01942e..a7c7634 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
@@ -727,7 +727,7 @@ public class PlanExecutor implements IPlanExecutor {
 
   protected List<PartialPath> getNodesList(PartialPath schemaPattern, int level)
       throws MetadataException {
-    return IoTDB.metaManager.getNodesList(schemaPattern, level);
+    return IoTDB.metaManager.getNodesListInGivenLevel(schemaPattern, level);
   }
 
   private QueryDataSet processCountTimeSeries(CountPlan countPlan) throws MetadataException {
@@ -740,10 +740,6 @@ public class PlanExecutor implements IPlanExecutor {
     return new ShowDevicesDataSet(showDevicesPlan);
   }
 
-  protected Set<PartialPath> getDevices(PartialPath path) throws MetadataException {
-    return IoTDB.metaManager.getDevices(path);
-  }
-
   private QueryDataSet processShowChildPaths(ShowChildPathsPlan showChildPathsPlan)
       throws MetadataException {
     Set<String> childPathsList = getPathNextChildren(showChildPathsPlan.getPath());
@@ -784,11 +780,11 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   protected Set<String> getNodeNextChildren(PartialPath path) throws MetadataException {
-    return IoTDB.metaManager.getChildNodeInNextLevel(path);
+    return IoTDB.metaManager.getChildNodeNameInNextLevel(path);
   }
 
   protected List<PartialPath> getStorageGroupNames(PartialPath path) throws MetadataException {
-    return IoTDB.metaManager.getStorageGroupPaths(path);
+    return IoTDB.metaManager.getMatchedStorageGroups(path);
   }
 
   private QueryDataSet processShowStorageGroup(ShowStorageGroupPlan showStorageGroupPlan)
@@ -1276,7 +1272,7 @@ public class PlanExecutor implements IPlanExecutor {
   private void operateTTL(SetTTLPlan plan) throws QueryProcessException {
     try {
       List<PartialPath> storageGroupPaths =
-          IoTDB.metaManager.getStorageGroupPaths(plan.getStorageGroup());
+          IoTDB.metaManager.getMatchedStorageGroups(plan.getStorageGroup());
       for (PartialPath storagePath : storageGroupPaths) {
         IoTDB.metaManager.setTTL(storagePath, plan.getDataTTL());
         StorageEngine.getInstance().setTTL(storagePath, plan.getDataTTL());
@@ -1730,7 +1726,7 @@ public class PlanExecutor implements IPlanExecutor {
     try {
       for (PartialPath storageGroupPath : deleteStorageGroupPlan.getPaths()) {
         List<PartialPath> allRelatedStorageGroupPath =
-            IoTDB.metaManager.getStorageGroupPaths(storageGroupPath);
+            IoTDB.metaManager.getMatchedStorageGroups(storageGroupPath);
         if (allRelatedStorageGroupPath.isEmpty()) {
           throw new PathNotExistException(storageGroupPath.getFullPath(), true);
         }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java
index 352f5ab..bf78ca9 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/QueryOperator.java
@@ -479,7 +479,7 @@ public class QueryOperator extends Operator {
   }
 
   protected Set<PartialPath> getMatchedDevices(PartialPath path) throws MetadataException {
-    return IoTDB.metaManager.getDevices(path);
+    return IoTDB.metaManager.getMatchedDevices(path);
   }
 
   protected List<PartialPath> getMatchedTimeseries(PartialPath path) throws MetadataException {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/ShowDevicesDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/ShowDevicesDataSet.java
index 0c14f31..1462708 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/ShowDevicesDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/ShowDevicesDataSet.java
@@ -61,7 +61,8 @@ public class ShowDevicesDataSet extends ShowDataSet {
 
   @Override
   public List<RowRecord> getQueryDataSet() throws MetadataException {
-    List<ShowDevicesResult> devicesList = IoTDB.metaManager.getDevices((ShowDevicesPlan) plan);
+    List<ShowDevicesResult> devicesList =
+        IoTDB.metaManager.getMatchedDevices((ShowDevicesPlan) plan);
     List<RowRecord> records = new ArrayList<>();
     for (ShowDevicesResult result : devicesList) {
       RowRecord record = new RowRecord(0);
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index bca5ac8..3ec0d8e 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -931,7 +931,7 @@ public class TSServiceImpl implements TSIService.Iface {
         for (int i = 0; i < resultColumns.size(); ++i) {
           if (isJdbcQuery) {
             String sgName =
-                IoTDB.metaManager.getStorageGroupPath(plan.getPaths().get(i)).getFullPath();
+                IoTDB.metaManager.getBelongedStorageGroup(plan.getPaths().get(i)).getFullPath();
             respSgColumns.add(sgName);
             if (resultColumns.get(i).getAlias() == null) {
               respColumns.add(
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/virtualsg/DeviceMappingViewer.java b/server/src/main/java/org/apache/iotdb/db/tools/virtualsg/DeviceMappingViewer.java
index d4e4581..27908b0 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/virtualsg/DeviceMappingViewer.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/virtualsg/DeviceMappingViewer.java
@@ -43,7 +43,7 @@ public class DeviceMappingViewer {
     MManager mManager = MManager.getInstance();
     mManager.init();
 
-    Set<PartialPath> partialPathSet = mManager.getDevices(new PartialPath("root.*"));
+    Set<PartialPath> partialPathSet = mManager.getMatchedDevices(new PartialPath("root.**"));
 
     if (partialPathSet.isEmpty() && args.length == 1) {
       System.out.println("no mlog in given system schema dir: " + args[0] + " please have a check");
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
index e5baeab..50a94cf 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
@@ -145,7 +145,9 @@ public class FilePathUtils {
       if (k <= pathLevel) {
         transformedPath.append(TsFileConstant.PATH_SEPARATOR).append(tmpPath[k]);
       } else {
-        transformedPath.append(TsFileConstant.PATH_SEPARATOR).append(IoTDBConstant.PATH_WILDCARD);
+        transformedPath
+            .append(TsFileConstant.PATH_SEPARATOR)
+            .append(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD);
       }
     }
     transformedPath.append(TsFileConstant.PATH_SEPARATOR).append(tmpPath[tmpPath.length - 1]);
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java b/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
index 96af80a..2128521 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
@@ -141,7 +141,7 @@ public class LogReplayer {
   private void replayDelete(DeletePlan deletePlan) throws IOException, MetadataException {
     List<PartialPath> paths = deletePlan.getPaths();
     for (PartialPath path : paths) {
-      for (PartialPath device : IoTDB.metaManager.getDevices(path.getDevicePath())) {
+      for (PartialPath device : IoTDB.metaManager.getBelongedDevices(path)) {
         recoverMemTable.delete(
             path, device, deletePlan.getDeleteStartTime(), deletePlan.getDeleteEndTime());
       }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAddSubDeviceIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAddSubDeviceIT.java
deleted file mode 100644
index 67b0da8..0000000
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAddSubDeviceIT.java
+++ /dev/null
@@ -1,249 +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.integration;
-
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.jdbc.Config;
-
-import org.junit.*;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.Statement;
-import java.sql.Types;
-
-import static org.junit.Assert.fail;
-
-/**
- * Test if measurement is also a sub device. This test will be ignored since nested measurement has
- * been removed from v0.13
- */
-@Ignore // nested measurement has been forbidden
-public class IoTDBAddSubDeviceIT {
-
-  private static String[] sqls =
-      new String[] {
-        "CREATE TIMESERIES root.sg1.d1.s1 with datatype=INT32,encoding=RLE",
-        "CREATE TIMESERIES root.sg1.d1.s1.s1 with datatype=INT32,encoding=RLE",
-        "CREATE TIMESERIES root.sg1.d1.s1.s2 with datatype=INT32,encoding=RLE"
-      };
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    EnvironmentUtils.closeStatMonitor();
-    EnvironmentUtils.envSetUp();
-
-    insertData();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    EnvironmentUtils.cleanEnv();
-  }
-
-  private static void insertData() throws ClassNotFoundException {
-    Class.forName(Config.JDBC_DRIVER_NAME);
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-
-      for (String sql : sqls) {
-        statement.execute(sql);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void showDevicesWithSg() throws ClassNotFoundException {
-    String[] retArray =
-        new String[] {
-          "root.sg1.d1,root.sg1,", "root.sg1.d1.s1,root.sg1,",
-        };
-
-    Class.forName(Config.JDBC_DRIVER_NAME);
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("SHOW DEVICES WITH STORAGE GROUP");
-      Assert.assertTrue(hasResultSet);
-
-      try (ResultSet resultSet = statement.getResultSet()) {
-        ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
-        StringBuilder header = new StringBuilder();
-        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-          header.append(resultSetMetaData.getColumnName(i)).append(",");
-        }
-        Assert.assertEquals("devices,storage group,", header.toString());
-        Assert.assertEquals(Types.VARCHAR, resultSetMetaData.getColumnType(1));
-
-        int cnt = 0;
-        while (resultSet.next()) {
-          StringBuilder builder = new StringBuilder();
-          for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-            builder.append(resultSet.getString(i)).append(",");
-          }
-          Assert.assertEquals(retArray[cnt], builder.toString());
-          cnt++;
-        }
-        Assert.assertEquals(retArray.length, cnt);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  @Test
-  public void showDevices() throws ClassNotFoundException {
-    String[] retArray =
-        new String[] {
-          "root.sg1.d1,", "root.sg1.d1.s1,",
-        };
-
-    Class.forName(Config.JDBC_DRIVER_NAME);
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("SHOW DEVICES");
-      Assert.assertTrue(hasResultSet);
-
-      try (ResultSet resultSet = statement.getResultSet()) {
-        ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
-        StringBuilder header = new StringBuilder();
-        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-          header.append(resultSetMetaData.getColumnName(i)).append(",");
-        }
-        Assert.assertEquals("devices,", header.toString());
-        Assert.assertEquals(Types.VARCHAR, resultSetMetaData.getColumnType(1));
-
-        int cnt = 0;
-        while (resultSet.next()) {
-          StringBuilder builder = new StringBuilder();
-          for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-            builder.append(resultSet.getString(i)).append(",");
-          }
-          Assert.assertEquals(retArray[cnt], builder.toString());
-          cnt++;
-        }
-        Assert.assertEquals(retArray.length, cnt);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  @Test
-  public void showTimeseries() throws ClassNotFoundException {
-    String[] retArray =
-        new String[] {
-          "root.sg1.d1.s1,null,root.sg1,INT32,RLE,SNAPPY,null,null,",
-          "root.sg1.d1.s1.s1,null,root.sg1,INT32,RLE,SNAPPY,null,null,",
-          "root.sg1.d1.s1.s2,null,root.sg1,INT32,RLE,SNAPPY,null,null,",
-        };
-
-    Class.forName(Config.JDBC_DRIVER_NAME);
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("SHOW TIMESERIES");
-      Assert.assertTrue(hasResultSet);
-
-      try (ResultSet resultSet = statement.getResultSet()) {
-        ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
-        StringBuilder header = new StringBuilder();
-        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-          header.append(resultSetMetaData.getColumnName(i)).append(",");
-        }
-        Assert.assertEquals(
-            "timeseries,alias,storage group,dataType,encoding,compression,tags,attributes,",
-            header.toString());
-        Assert.assertEquals(Types.VARCHAR, resultSetMetaData.getColumnType(1));
-
-        int cnt = 0;
-        while (resultSet.next()) {
-          StringBuilder builder = new StringBuilder();
-          for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-            builder.append(resultSet.getString(i)).append(",");
-          }
-          Assert.assertEquals(retArray[cnt], builder.toString());
-          cnt++;
-        }
-        Assert.assertEquals(retArray.length, cnt);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  @Test
-  public void insertDataInAllSeries() throws ClassNotFoundException {
-    String[] retArray =
-        new String[] {
-          "0,1,2,3,",
-        };
-
-    Class.forName(Config.JDBC_DRIVER_NAME);
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-
-      statement.execute("INSERT INTO root.sg1.d1 (timestamp, s1) VALUES(0, 1)");
-      statement.execute("INSERT INTO root.sg1.d1.s1 (timestamp, s1) VALUES(0, 2)");
-      statement.execute("INSERT INTO root.sg1.d1.s1 (timestamp, s2) VALUES(0, 3)");
-
-      boolean hasResultSet = statement.execute("SELECT * FROM root.sg1.d1");
-      Assert.assertTrue(hasResultSet);
-
-      try (ResultSet resultSet = statement.getResultSet()) {
-        ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
-        StringBuilder header = new StringBuilder();
-        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-          header.append(resultSetMetaData.getColumnName(i)).append(",");
-        }
-        Assert.assertEquals(
-            "Time,root.sg1.d1.s1,root.sg1.d1.s1.s1,root.sg1.d1.s1.s2,", header.toString());
-
-        int cnt = 0;
-        while (resultSet.next()) {
-          StringBuilder builder = new StringBuilder();
-          for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
-            builder.append(resultSet.getString(i)).append(",");
-          }
-          Assert.assertEquals(retArray[cnt], builder.toString());
-          cnt++;
-        }
-        Assert.assertEquals(retArray.length, cnt);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java
index c049a7e..2333032 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java
@@ -162,7 +162,7 @@ public class IoTDBAlignByDeviceIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select * from root.vehicle align by device");
+      boolean hasResultSet = statement.execute("select * from root.vehicle.** align by device");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -520,7 +520,7 @@ public class IoTDBAlignByDeviceIT {
         Statement statement = connection.createStatement()) {
       boolean hasResultSet =
           statement.execute(
-              "select count(*) from root.vehicle GROUP BY ([2,50),20ms) align by device");
+              "select count(*) from root.vehicle.** GROUP BY ([2,50),20ms) align by device");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -625,7 +625,7 @@ public class IoTDBAlignByDeviceIT {
         Statement statement = connection.createStatement()) {
       boolean hasResultSet =
           statement.execute(
-              "select * from root.vehicle where time = 3 Fill(int32[previous, 5ms]) align by device");
+              "select * from root.vehicle.* where time = 3 Fill(int32[previous, 5ms]) align by device");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -844,7 +844,8 @@ public class IoTDBAlignByDeviceIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select *, '11' from root.vehicle align by device");
+      boolean hasResultSet =
+          statement.execute("select *, '11' from root.vehicle.** align by device");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java
index 59305bd..b952e6a 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java
@@ -27,7 +27,6 @@ import org.apache.iotdb.jdbc.IoTDBSQLException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.sql.Connection;
@@ -37,7 +36,6 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -157,54 +155,6 @@ public class IoTDBAutoCreateSchemaIT {
   }
 
   /**
-   * insert data when the time series that is a prefix path of an existing time series hasn't been
-   * created
-   */
-  @Ignore // nested measurement has been forbidden
-  @Test
-  public void testInsertAutoCreate1() throws Exception {
-    String[] timeSeriesArray = {"root.sg1.a.a", "root.sg1.a", "root.sg1.a.a.a"};
-
-    for (String timeSeries : timeSeriesArray) {
-      statement.execute(
-          String.format("INSERT INTO %s(timestamp, a) values(123, \"aabb\")", timeSeries));
-    }
-
-    // ensure that insert data in cache is right.
-    insertAutoCreate1Tool();
-
-    statement.close();
-    connection.close();
-    EnvironmentUtils.stopDaemon();
-    setUp();
-
-    // ensure that insert data in cache is right after recovering.
-    insertAutoCreate1Tool();
-  }
-
-  private void insertAutoCreate1Tool() throws SQLException {
-    boolean hasResult = statement.execute("select * from root.sg1");
-    Assert.assertTrue(hasResult);
-
-    Set<String> strSet = new HashSet<>();
-    String[] valueList = {};
-    try (ResultSet resultSet = statement.getResultSet()) {
-      while (resultSet.next()) {
-        valueList =
-            new String[] {
-              resultSet.getString("root.sg1.a.a"),
-              resultSet.getString("root.sg1.a.a.a"),
-              resultSet.getString("root.sg1.a.a.a.a")
-            };
-        strSet = new HashSet<>(Arrays.asList(valueList));
-      }
-    }
-    Assert.assertEquals(3, valueList.length);
-    Assert.assertEquals(1, strSet.size());
-    Assert.assertTrue(strSet.contains("aabb"));
-  }
-
-  /**
    * test if automatically creating a time series will cause the storage group with same name to
    * disappear
    */
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java
index 66f6b75..65e0f78 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java
@@ -118,7 +118,7 @@ public class IoTDBCloseIT {
           "1,1,1", "2,2,2", "3,3,3", "4,4,4", "5,5,0", "6,6,1", "7,7,2", "8,8,3", "9,9,4"
         };
 
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java
index 383f8d8..eb59ede 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java
@@ -106,7 +106,7 @@ public class IoTDBCompleteIT {
       "CREATE TIMESERIES root.vehicle.d4.s8 WITH DATATYPE=INT32,ENCODING=RLE,MAX_POINT_NUMBER=100",
       "CREATE TIMESERIES root.vehicle.d5.s9 WITH DATATYPE=FLOAT,ENCODING=PLAIN,compressor=SNAPPY,MAX_POINT_NUMBER=10",
       "CREATE TIMESERIES root.vehicle.d6.s10 WITH DATATYPE=DOUBLE,ENCODING=RLE,compressor=UNCOMPRESSED,MAX_POINT_NUMBER=10",
-      "DELETE TIMESERIES root.vehicle.d0.*",
+      "DELETE TIMESERIES root.vehicle.d0.**",
       "SHOW TIMESERIES",
       "===  Timeseries Tree  ===\n"
           + "\n"
@@ -170,7 +170,7 @@ public class IoTDBCompleteIT {
           + "\t\t}\n"
           + "\t}\n"
           + "}",
-      "DELETE TIMESERIES root.vehicle.*",
+      "DELETE TIMESERIES root.vehicle.**",
       "SHOW TIMESERIES",
       "===  Timeseries Tree  ===\n"
           + "\n"
@@ -193,7 +193,7 @@ public class IoTDBCompleteIT {
       "INSERT INTO root.vehicle.d0(timestamp,s0) values(2000-01-01T08:00:00+08:00,105)",
       "SELECT * FROM root.vehicle.d0",
       "1,101,null,\n" + "2,102,202,\n" + "946684800000,105,null,\n" + "NOW(),104,null,\n",
-      "DELETE TIMESERIES root.vehicle.*"
+      "DELETE TIMESERIES root.vehicle.**"
     };
     executeSQL(sqlS);
   }
@@ -208,7 +208,7 @@ public class IoTDBCompleteIT {
       "INSERT INTO root.vehicle.d0(timestamp,s0) values(2000-01-01T08:00:00+08:00,105)",
       "SELECT * FROM root.vehicle.d0",
       "1,101,null,\n" + "2,102,202,\n" + "946684800000,105,null,\n" + "NOW(),104,null,\n",
-      "DELETE TIMESERIES root.vehicle.*",
+      "DELETE TIMESERIES root.vehicle.**",
       "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32,ENCODING=RLE",
       "INSERT INTO root.vehicle.d0(timestamp,s0) values(1,1)",
       "INSERT INTO root.vehicle.d0(timestamp,s0) values(2,1)",
@@ -243,15 +243,15 @@ public class IoTDBCompleteIT {
       "INSERT INTO root.vehicle.d1(timestamp,s1) values(1,1)",
       "INSERT INTO root.vehicle.d0(timestamp,s0) values(5,5)",
       "INSERT INTO root.vehicle.d1(timestamp,s1) values(5,5)",
-      "SELECT * FROM root.vehicle",
+      "SELECT * FROM root.vehicle.*",
       "1,1,1,\n" + "5,5,5,\n",
       "DELETE FROM root.vehicle.d0.s0,root.vehicle.d1.s1 WHERE time < 3",
-      "SELECT * FROM root.vehicle",
+      "SELECT * FROM root.vehicle.*",
       "5,5,5,\n",
-      "DELETE FROM root.vehicle.* WHERE time < 7",
-      "SELECT * FROM root.vehicle",
+      "DELETE FROM root.vehicle.** WHERE time < 7",
+      "SELECT * FROM root.vehicle.*",
       "",
-      "DELETE TIMESERIES root.vehicle.*"
+      "DELETE TIMESERIES root.vehicle.**"
     };
     executeSQL(sqlS);
   }
@@ -284,7 +284,7 @@ public class IoTDBCompleteIT {
           + "8,108,\n"
           + "9,109,\n"
           + "10,110,\n",
-      "DELETE TIMESERIES root.vehicle.*"
+      "DELETE TIMESERIES root.vehicle.**"
     };
     executeSQL(sqlS);
   }
@@ -332,7 +332,7 @@ public class IoTDBCompleteIT {
       "DELETE FROM root.vehicle.d0.s0 WHERE time <= NOW()",
       "SELECT * FROM root.vehicle.d0",
       "",
-      "DELETE TIMESERIES root.vehicle.*"
+      "DELETE TIMESERIES root.vehicle.**"
     };
     executeSQL(sqlS);
   }
@@ -387,7 +387,7 @@ public class IoTDBCompleteIT {
       "",
       "SELECT MIN_VALUE(s0) FROM root.vehicle.d0 WHERE time < 5",
       "",
-      "DELETE TIMESERIES root.vehicle.*"
+      "DELETE TIMESERIES root.vehicle.**"
     };
     executeSQL(sqlS);
   }
@@ -422,7 +422,7 @@ public class IoTDBCompleteIT {
       "1,1,101,\n" + "2,3,104,\n" + "5,1,105,\n",
       "SELECT MIN_VALUE(s0), MAX_TIME(s1) FROM root.vehicle.d0 WHERE s1 > 102 and time < 9 GROUP BY(3ms,1,[1,4],[6,9])",
       "1,108,3,\n" + "4,105,6,\n" + "7,103,8,\n",
-      "DELETE TIMESERIES root.vehicle.*"
+      "DELETE TIMESERIES root.vehicle.**"
     };
     executeSQL(sqlS);
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java
index f1c7775..28e80e5 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java
@@ -242,7 +242,7 @@ public class IoTDBDaemonIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select * from root");
+      boolean hasResultSet = statement.execute("select * from root.**");
       Assert.assertTrue(hasResultSet);
 
       int cnt;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteStorageGroupIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteStorageGroupIT.java
index 13c0e84..313de8e 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteStorageGroupIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteStorageGroupIT.java
@@ -154,7 +154,7 @@ public class IoTDBDeleteStorageGroupIT {
       statement.execute("SET STORAGE GROUP TO root.ln4.wf01.wt02");
       statement.execute("SET STORAGE GROUP TO root.ln4.wf02.wt03");
       statement.execute("SET STORAGE GROUP TO root.ln4.wf02.wt04");
-      statement.execute("DELETE STORAGE GROUP root.*");
+      statement.execute("DELETE STORAGE GROUP root.**");
       boolean hasResult = statement.execute("SHOW STORAGE GROUP");
       assertTrue(hasResult);
       List<String> result = new ArrayList<>();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java
index db52ec2..ac19070 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java
@@ -117,7 +117,7 @@ public class IoTDBDeleteTimeseriesIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResult = statement.execute("SELECT * FROM root");
+      boolean hasResult = statement.execute("SELECT * FROM root.**");
       Assert.assertTrue(hasResult);
     }
   }
@@ -175,7 +175,7 @@ public class IoTDBDeleteTimeseriesIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResult = statement.execute("SELECT * FROM root");
+      boolean hasResult = statement.execute("SELECT * FROM root.**");
       Assert.assertTrue(hasResult);
     }
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
index 0c04f0b..52fbbc9 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
@@ -166,7 +166,7 @@ public class IoTDBDeletionIT {
       statement.execute(
           "DELETE FROM root.vehicle.d0.s1,root.vehicle.d0.s2,root.vehicle.d0.s3"
               + " WHERE time <= 350");
-      statement.execute("DELETE FROM root.vehicle.d0 WHERE time <= 150");
+      statement.execute("DELETE FROM root.vehicle.d0.** WHERE time <= 150");
 
       try (ResultSet set = statement.executeQuery("SELECT * FROM root.vehicle.d0")) {
         int cnt = 0;
@@ -204,7 +204,7 @@ public class IoTDBDeletionIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
       statement.execute("merge");
-      statement.execute("DELETE FROM root.vehicle.d0 WHERE time <= 15000");
+      statement.execute("DELETE FROM root.vehicle.d0.** WHERE time <= 15000");
 
       // before merge completes
       try (ResultSet set = statement.executeQuery("SELECT * FROM root.vehicle.d0")) {
@@ -276,7 +276,7 @@ public class IoTDBDeletionIT {
         assertEquals(150, cnt);
       }
 
-      statement.execute("DELETE FROM root.vehicle.d0 WHERE time > 50 and time <= 250");
+      statement.execute("DELETE FROM root.vehicle.d0.** WHERE time > 50 and time <= 250");
       try (ResultSet set = statement.executeQuery("SELECT * FROM root.vehicle.d0")) {
         int cnt = 0;
         while (set.next()) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDisableAlignIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDisableAlignIT.java
index 60edeca..c06b53b 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDisableAlignIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDisableAlignIT.java
@@ -146,7 +146,7 @@ public class IoTDBDisableAlignIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select * from root.vehicle disable align");
+      boolean hasResultSet = statement.execute("select * from root.vehicle.** disable align");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -325,7 +325,7 @@ public class IoTDBDisableAlignIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
       boolean hasResultSet =
-          statement.execute("select * from root.vehicle.* slimit 2 soffset 1 disable align");
+          statement.execute("select * from root.vehicle.** slimit 2 soffset 1 disable align");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -352,7 +352,7 @@ public class IoTDBDisableAlignIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
       statement.execute(
-          "select * from root.vehicle where time = 3 Fill(int32[previous, 5ms]) disable align");
+          "select * from root.vehicle.** where time = 3 Fill(int32[previous, 5ms]) disable align");
       fail("No exception thrown.");
     } catch (Exception e) {
       Assert.assertTrue(e.getMessage().contains("doesn't support disable align clause."));
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBExecuteBatchIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBExecuteBatchIT.java
index 82ce8bb..e96efc8 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBExecuteBatchIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBExecuteBatchIT.java
@@ -57,7 +57,7 @@ public class IoTDBExecuteBatchIT {
           "insert into root.ln.wf01.wt01(timestamp,temperature) values(1509465600000,1.2)");
       statement.addBatch(
           "insert into root.ln.wf01.wt01(timestamp,temperature) values(1509465600001,2.3)");
-      statement.addBatch("delete timeseries root.ln.wf01.wt01");
+      statement.addBatch("delete timeseries root.ln.wf01.wt01.**");
       statement.addBatch(
           "insert into root.ln.wf01.wt01(timestamp,temperature) values(1509465600002,3.4)");
       statement.executeBatch();
@@ -89,7 +89,7 @@ public class IoTDBExecuteBatchIT {
           "insert into root.ln.wf01.wt01(timestamp,temperature) values(1509465600000,1.2)");
       statement.addBatch(
           "insert into root.ln.wf01.wt01(timestamp,temperature) values(1509465600001,2.3)");
-      statement.addBatch("delete timeseries root.ln.wf01.wt01");
+      statement.addBatch("delete timeseries root.ln.wf01.wt01.**");
       statement.addBatch(
           "create timeseries root.turbine.d1.s1(s1) with datatype=boolean, encoding=plain , compression=snappy tags(tag1=v1, tag2=v2) attributes(attr1=v3, attr2=v4)");
       statement.addBatch(
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFloatPrecisionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFloatPrecisionIT.java
index 933f230e..a966ce7 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFloatPrecisionIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFloatPrecisionIT.java
@@ -108,7 +108,7 @@ public class IoTDBFloatPrecisionIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select * from root");
+      boolean hasResultSet = statement.execute("select * from root.**");
       Assert.assertTrue(hasResultSet);
       int cnt;
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -139,7 +139,7 @@ public class IoTDBFloatPrecisionIT {
       }
 
       statement.execute("flush");
-      try (ResultSet resultSet = statement.executeQuery("select * from root")) {
+      try (ResultSet resultSet = statement.executeQuery("select * from root.**")) {
         cnt = 0;
         while (resultSet.next()) {
           assertEquals(TIMESTAMP + "", resultSet.getString(TIMESTAMP_STR));
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java
index 94a46ac..9996acd 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java
@@ -98,7 +98,7 @@ public class IoTDBFlushQueryMergeIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("SELECT * FROM root");
+      boolean hasResultSet = statement.execute("SELECT * FROM root.**");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBInsertWithQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBInsertWithQueryIT.java
index afa6d6b..5002d6a 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBInsertWithQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBInsertWithQueryIT.java
@@ -473,7 +473,7 @@ public class IoTDBInsertWithQueryIT {
 
   // test count
   private void selectAndCount(int res) throws ClassNotFoundException {
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
@@ -503,7 +503,7 @@ public class IoTDBInsertWithQueryIT {
 
   // test order
   private void select() throws ClassNotFoundException {
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
index a0fbae6..891d5d5 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
@@ -238,10 +238,10 @@ public class IoTDBLargeDataIT {
     }
   }
 
-  // "select * from root.vehicle" : test select wild data
+  // "select * from root.vehicle.**" : test select wild data
   @Test
   public void selectAllTest() throws ClassNotFoundException {
-    String selectSql = "select * from root.vehicle";
+    String selectSql = "select * from root.vehicle.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
index e30e923..6c1c12d 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
@@ -164,7 +164,7 @@ public class IoTDBLastIT {
             DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
 
-      boolean hasResultSet = statement.execute("select last * from root.* order by time desc");
+      boolean hasResultSet = statement.execute("select last * from root.** order by time desc");
       assertTrue(hasResultSet);
       ResultSet resultSet = statement.getResultSet();
       int cnt = 0;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java
index 5e559dd..2a752a5 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java
@@ -367,7 +367,7 @@ public class IoTDBLoadExternalTsfileIT {
         Statement statement = connection.createStatement()) {
 
       // check query result
-      boolean hasResultSet = statement.execute("SELECT * FROM root");
+      boolean hasResultSet = statement.execute("SELECT * FROM root.**");
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
         int cnt = 0;
@@ -503,7 +503,7 @@ public class IoTDBLoadExternalTsfileIT {
           new File(tmpDir, new PartialPath("root.test") + File.separator + "0").listFiles().length);
 
       // check query result
-      hasResultSet = statement.execute("SELECT  * FROM root");
+      hasResultSet = statement.execute("SELECT * FROM root.**");
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
         int cnt = 0;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
index 6f96df7..d6424d9 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
@@ -401,7 +401,7 @@ public class IoTDBMetadataFetchIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      String[] sqls = new String[] {"COUNT TIMESERIES root.ln", "COUNT TIMESERIES"};
+      String[] sqls = new String[] {"COUNT TIMESERIES root.ln.**", "COUNT TIMESERIES"};
       String[] standards = new String[] {"2,\n", "2,\n"};
       for (int n = 0; n < sqls.length; n++) {
         String sql = sqls[n];
@@ -438,7 +438,9 @@ public class IoTDBMetadataFetchIT {
         Statement statement = connection.createStatement()) {
       String[] sqls =
           new String[] {
-            "COUNT DEVICES root.ln", "COUNT DEVICES", "COUNT DEVICES root.ln.wf01.wt01.temperature"
+            "COUNT DEVICES root.ln.**",
+            "COUNT DEVICES",
+            "COUNT DEVICES root.ln.wf01.wt01.temperature"
           };
       String[] standards = new String[] {"1,\n", "1,\n", "0,\n"};
       for (int n = 0; n < sqls.length; n++) {
@@ -476,7 +478,7 @@ public class IoTDBMetadataFetchIT {
         Statement statement = connection.createStatement()) {
       String[] sqls =
           new String[] {
-            "count storage group root.ln",
+            "count storage group root.ln.**",
             "count storage group",
             "count storage group root.ln.wf01.wt01.status"
           };
@@ -547,7 +549,7 @@ public class IoTDBMetadataFetchIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      String[] sqls = new String[] {"COUNT NODES root level=1"};
+      String[] sqls = new String[] {"COUNT NODES root.** level=1"};
       String[] standards = new String[] {"3,\n"};
       for (int n = 0; n < sqls.length; n++) {
         String sql = sqls[n];
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java
index 5ffac74..aa1a5c5 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java
@@ -238,10 +238,9 @@ public class IoTDBMultiDeviceIT {
     }
   }
 
-  // "select * from root.vehicle" : test select wild data
   @Test
   public void selectAllTest() throws ClassNotFoundException {
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
@@ -269,10 +268,9 @@ public class IoTDBMultiDeviceIT {
     }
   }
 
-  // "select * from root.vehicle" : test select wild data
   @Test
   public void selectAfterDeleteTest() throws ClassNotFoundException {
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
@@ -280,10 +278,10 @@ public class IoTDBMultiDeviceIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
 
-      statement.execute("DELETE FROM root.fans.* WHERE time <= 1000");
-      statement.execute("DELETE FROM root.car.* WHERE time <= 1000");
-      statement.execute("DELETE FROM root.fans.* WHERE time >= 200500 and time < 201000");
-      statement.execute("DELETE FROM root.car.* WHERE time >= 200500 and time < 201000");
+      statement.execute("DELETE FROM root.fans.** WHERE time <= 1000");
+      statement.execute("DELETE FROM root.car.** WHERE time <= 1000");
+      statement.execute("DELETE FROM root.fans.** WHERE time >= 200500 and time < 201000");
+      statement.execute("DELETE FROM root.car.** WHERE time >= 200500 and time < 201000");
 
       boolean hasResultSet = statement.execute(selectSql);
       Assert.assertTrue(hasResultSet);
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
index b74d43c..ff0f737 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
@@ -259,10 +259,9 @@ public class IoTDBMultiSeriesIT {
     }
   }
 
-  // "select * from root.vehicle" : test select wild data
   @Test
   public void selectAllTest() throws ClassNotFoundException {
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
@@ -290,10 +289,10 @@ public class IoTDBMultiSeriesIT {
     }
   }
 
-  // "select * from root.vehicle" : test select wild data
+  // "select * from root.vehicle.**" : test select wild data
   @Test
   public void selectAllFromVehicleTest() throws ClassNotFoundException {
-    String selectSql = "select * from root.vehicle";
+    String selectSql = "select * from root.vehicle.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
index 633a0ee..db7959f 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
@@ -117,7 +117,6 @@ public class IoTDBMultiStatementsIT {
     }
   }
 
-  // "select * from root.vehicle" : test select wild data
   @Test
   public void selectAllTest() throws ClassNotFoundException {
     String[] retArray =
@@ -125,7 +124,7 @@ public class IoTDBMultiStatementsIT {
           "1,1,1", "2,2,2", "3,3,3", "4,4,4", "5,5,0", "6,6,1", "7,7,2", "8,8,3", "9,9,4"
         };
 
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java
index 217028f..ad2d9ca 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java
@@ -164,7 +164,7 @@ public class IoTDBQueryDemoIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select * from root where time>10");
+      boolean hasResultSet = statement.execute("select * from root.** where time>10");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -227,7 +227,8 @@ public class IoTDBQueryDemoIT {
       // test 1: fetchSize < limitNumber
       statement.setFetchSize(4);
       Assert.assertEquals(4, statement.getFetchSize());
-      boolean hasResultSet = statement.execute("select * from root where time>10 limit 5 offset 3");
+      boolean hasResultSet =
+          statement.execute("select * from root.** where time>10 limit 5 offset 3");
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
         ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
@@ -267,7 +268,7 @@ public class IoTDBQueryDemoIT {
       // test 1: fetchSize > limitNumber
       statement.setFetchSize(10000);
       Assert.assertEquals(10000, statement.getFetchSize());
-      hasResultSet = statement.execute("select * from root where time>10 limit 5 offset 3");
+      hasResultSet = statement.execute("select * from root.** where time>10 limit 5 offset 3");
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
         ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
@@ -331,7 +332,7 @@ public class IoTDBQueryDemoIT {
       Assert.assertEquals(4, statement.getFetchSize());
       boolean hasResultSet =
           statement.execute(
-              "select * from root where time in (1509465780000, 1509465840000, 1509465900000, 1509465960000, 1509466020000)");
+              "select * from root.** where time in (1509465780000, 1509465840000, 1509465900000, 1509465960000, 1509466020000)");
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
         ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
@@ -378,7 +379,7 @@ public class IoTDBQueryDemoIT {
           };
       hasResultSet =
           statement.execute(
-              "select * from root where time not in (1509465780000, 1509465840000, 1509465900000, 1509465960000, 1509466020000)");
+              "select * from root.** where time not in (1509465780000, 1509465840000, 1509465900000, 1509465960000, 1509466020000)");
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
         ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
@@ -423,7 +424,7 @@ public class IoTDBQueryDemoIT {
           };
       hasResultSet =
           statement.execute(
-              "select * from root where ln.wf01.wt01.temperature in (20.18, 20.71, 22.58)");
+              "select * from root.** where root.ln.wf01.wt01.temperature in (20.18, 20.71, 22.58)");
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
         ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryMemoryControlIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryMemoryControlIT.java
index d8cf402..c352441 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryMemoryControlIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryMemoryControlIT.java
@@ -94,7 +94,7 @@ public class IoTDBQueryMemoryControlIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
       try {
-        statement.execute("select * from root");
+        statement.execute("select * from root.**");
       } catch (SQLException e) {
         assertTrue(e.getMessage().contains("Too many paths in one query!"));
       }
@@ -116,7 +116,7 @@ public class IoTDBQueryMemoryControlIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      statement.execute("select * from root slimit 10");
+      statement.execute("select * from root.** slimit 10");
       statement.execute("select count(*) from root slimit 10");
     } catch (SQLException e) {
       e.printStackTrace();
@@ -131,7 +131,7 @@ public class IoTDBQueryMemoryControlIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
       try {
-        statement.execute("select * from root slimit 11");
+        statement.execute("select * from root.** slimit 11");
       } catch (SQLException e) {
         assertTrue(e.getMessage().contains("Too many paths in one query!"));
       }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
index 94ef0d1..8a2f750 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
@@ -308,7 +308,7 @@ public class IoTDBRestartIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
 
-      boolean hasResultSet = statement.execute("select * from root");
+      boolean hasResultSet = statement.execute("select * from root.**");
       assertTrue(hasResultSet);
       ResultSet resultSet = statement.getResultSet();
       int cnt = 0;
@@ -341,7 +341,7 @@ public class IoTDBRestartIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
 
-      boolean hasResultSet = statement.execute("select * from root");
+      boolean hasResultSet = statement.execute("select * from root.**");
       assertTrue(hasResultSet);
       ResultSet resultSet = statement.getResultSet();
       int cnt = 0;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRpcCompressionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRpcCompressionIT.java
index 9f03a85..9559e2b 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRpcCompressionIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRpcCompressionIT.java
@@ -126,7 +126,7 @@ public class IoTDBRpcCompressionIT {
     statement.execute("flush");
     statement.execute("insert into root.demo.d1(time,s3) values(1,1)");
     statement.execute("flush");
-    try (ResultSet set = statement.executeQuery("SELECT * FROM root")) {
+    try (ResultSet set = statement.executeQuery("SELECT * FROM root.**")) {
       int cnt = 0;
       while (set.next()) {
         cnt++;
@@ -136,7 +136,7 @@ public class IoTDBRpcCompressionIT {
     }
     Thread.sleep(1000);
     // before merge completes
-    try (ResultSet set = statement.executeQuery("SELECT * FROM root")) {
+    try (ResultSet set = statement.executeQuery("SELECT * FROM root.**")) {
       int cnt = 0;
       while (set.next()) {
         cnt++;
@@ -146,6 +146,6 @@ public class IoTDBRpcCompressionIT {
     }
 
     // after merge completes
-    statement.execute("DELETE FROM root.demo.d1");
+    statement.execute("DELETE FROM root.demo.d1.**");
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSameMeasurementsDifferentTypesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSameMeasurementsDifferentTypesIT.java
index a5fd296..7e3062c 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSameMeasurementsDifferentTypesIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSameMeasurementsDifferentTypesIT.java
@@ -110,7 +110,6 @@ public class IoTDBSameMeasurementsDifferentTypesIT {
     }
   }
 
-  // "select * from root.vehicle" : test select wild data
   @Test
   public void selectAllTest() throws ClassNotFoundException {
     String[] retArray =
@@ -118,7 +117,7 @@ public class IoTDBSameMeasurementsDifferentTypesIT {
           "1,1,1", "2,2,2", "3,3,3", "4,4,4", "5,5,0", "6,6,1", "7,7,2", "8,8,3", "9,9,4"
         };
 
-    String selectSql = "select * from root";
+    String selectSql = "select * from root.**";
 
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSensorUpdateIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSensorUpdateIT.java
index 3d751f4..3f7930a 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSensorUpdateIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSensorUpdateIT.java
@@ -61,7 +61,7 @@ public class IoTDBSensorUpdateIT {
       statement.execute("flush");
       statement.execute("insert into root.demo.d1(time,s3) values(1,1)");
       statement.execute("flush");
-      try (ResultSet set = statement.executeQuery("SELECT * FROM root")) {
+      try (ResultSet set = statement.executeQuery("SELECT * FROM root.**")) {
         int cnt = 0;
         while (set.next()) {
           cnt++;
@@ -72,7 +72,7 @@ public class IoTDBSensorUpdateIT {
       statement.execute("merge");
       Thread.sleep(1000);
       // before merge completes
-      try (ResultSet set = statement.executeQuery("SELECT * FROM root")) {
+      try (ResultSet set = statement.executeQuery("SELECT * FROM root.**")) {
         int cnt = 0;
         while (set.next()) {
           cnt++;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
index 76daf0d..ea2fda0 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
@@ -265,7 +265,7 @@ public class IoTDBSequenceDataQueryIT {
   @Test
   public void readWithValueFilterTest()
       throws IOException, StorageEngineException, QueryProcessException, IllegalPathException {
-    // select * from root where root.vehicle.d0.s0 >=14
+    // select * from root.** where root.vehicle.d0.s0 >=14
     QueryRouter queryRouter = new QueryRouter();
     List<PartialPath> pathList = new ArrayList<>();
     List<TSDataType> dataTypes = new ArrayList<>();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSetSystemReadOnlyWritableIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSetSystemReadOnlyWritableIT.java
index a5145d6..a84c0ff 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSetSystemReadOnlyWritableIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSetSystemReadOnlyWritableIT.java
@@ -202,7 +202,7 @@ public class IoTDBSetSystemReadOnlyWritableIT {
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select * from root where time>10");
+      boolean hasResultSet = statement.execute("select * from root.** where time>10");
       Assert.assertTrue(hasResultSet);
 
       try (ResultSet resultSet = statement.getResultSet()) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
index f6402fb..dffe7eb 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
@@ -183,7 +183,7 @@ public class IoTDBSimpleQueryIT {
       String[] results = {"root.turbine.d1.s1", "root.turbine.d1.s2"};
 
       int count = 0;
-      try (ResultSet resultSet = statement.executeQuery("select last * from root")) {
+      try (ResultSet resultSet = statement.executeQuery("select last ** from root")) {
         while (resultSet.next()) {
           String path = resultSet.getString("timeseries");
           assertEquals(results[count], path);
@@ -278,7 +278,7 @@ public class IoTDBSimpleQueryIT {
       statement.execute(sql);
       statement.execute("flush");
 
-      ResultSet resultSet = statement.executeQuery("select * from root");
+      ResultSet resultSet = statement.executeQuery("select * from root.**");
       int count = 0;
 
       String[] timestamps = {"1", "7", "15", "16", "17", "18"};
@@ -325,7 +325,8 @@ public class IoTDBSimpleQueryIT {
       statement.execute("flush");
 
       for (int i = 1; i < originalValues.length; i++) {
-        String sql = "select * from root where time = " + i + " fill(int32 [linear, 20ms, 20ms])";
+        String sql =
+            "select * from root.** where time = " + i + " fill(int32 [linear, 20ms, 20ms])";
         ResultSet resultSet = statement.executeQuery(sql);
 
         while (resultSet.next()) {
@@ -376,7 +377,7 @@ public class IoTDBSimpleQueryIT {
       statement.execute(sql);
       statement.execute("flush");
 
-      ResultSet resultSet = statement.executeQuery("select * from root");
+      ResultSet resultSet = statement.executeQuery("select * from root.**");
       int count = 0;
 
       // will not store time = 16 since time distance to last stored time 15 is within compMinTime
@@ -413,7 +414,7 @@ public class IoTDBSimpleQueryIT {
       }
       statement.execute("flush");
 
-      ResultSet resultSet = statement.executeQuery("select * from root");
+      ResultSet resultSet = statement.executeQuery("select * from root.**");
       int count = 0;
 
       String[] timestamps = {"1", "21", "41", "49"};
@@ -595,7 +596,7 @@ public class IoTDBSimpleQueryIT {
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
 
-      ResultSet resultSet = statement.executeQuery("select * from root");
+      ResultSet resultSet = statement.executeQuery("select * from root.**");
       // has an empty time column
       Assert.assertEquals(1, resultSet.getMetaData().getColumnCount());
       try {
@@ -619,7 +620,7 @@ public class IoTDBSimpleQueryIT {
           fail();
         }
 
-        resultSet = statement.executeQuery("select * from root align by device");
+        resultSet = statement.executeQuery("select * from root.** align by device");
         // has time and device columns
         Assert.assertEquals(2, resultSet.getMetaData().getColumnCount());
         while (resultSet.next()) {
@@ -675,7 +676,8 @@ public class IoTDBSimpleQueryIT {
           };
 
       int cur = 0;
-      try (ResultSet resultSet = statement.executeQuery("select * from root order by time desc")) {
+      try (ResultSet resultSet =
+          statement.executeQuery("select * from root.** order by time desc")) {
         while (resultSet.next()) {
           String ans =
               resultSet.getString("Time")
@@ -1085,7 +1087,7 @@ public class IoTDBSimpleQueryIT {
 
       long count = 0;
 
-      try (ResultSet resultSet = statement.executeQuery("select * from root")) {
+      try (ResultSet resultSet = statement.executeQuery("select * from root.**")) {
         while (resultSet.next()) {
           count++;
         }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
index 3364318..62184c6 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
@@ -367,7 +367,7 @@ public class IoTDBTagIT {
               + "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)");
 
       boolean hasResult =
-          statement.execute("show timeseries root.turbine.d1 where tag1=v1 limit 2 offset 1");
+          statement.execute("show timeseries root.turbine.d1.** where tag1=v1 limit 2 offset 1");
       assertTrue(hasResult);
       int count = 0;
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -735,7 +735,7 @@ public class IoTDBTagIT {
       }
 
       // with *
-      boolean hasResult = statement.execute("show timeseries root.turbine.* where unit=f");
+      boolean hasResult = statement.execute("show timeseries root.turbine.** where unit=f");
       assertTrue(hasResult);
       int count = 0;
       Set<String> res = new HashSet<>();
@@ -766,7 +766,7 @@ public class IoTDBTagIT {
       assertEquals(ret.size(), count);
 
       // no *
-      hasResult = statement.execute("show timeseries root.turbine where unit=f");
+      hasResult = statement.execute("show timeseries root.turbine.** where unit=f");
       assertTrue(hasResult);
       count = 0;
       res.clear();
@@ -967,7 +967,8 @@ public class IoTDBTagIT {
       assertEquals(ret, res);
       assertEquals(ret.size(), count);
 
-      hasResult = statement.execute("show timeseries root.ln where description contains 'test1'");
+      hasResult =
+          statement.execute("show timeseries root.ln.** where description contains 'test1'");
       assertTrue(hasResult);
       count = 0;
       res.clear();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java
index c6f6a36..cbda20e 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java
@@ -94,7 +94,7 @@ public class IoTDBTimeZoneIT {
    * root.timezone(timestamp,tz1) values(2018-1-1T13:00:09+08:00,12) insert into
    * root.timezone(timestamp,tz1) values(2018-1-1T12:00:10+07:00,13)
    *
-   * <p>select * from root
+   * <p>select * from root.**
    */
   @Test
   public void timezoneTest() throws ClassNotFoundException, SQLException, TException {
@@ -127,7 +127,7 @@ public class IoTDBTimeZoneIT {
       statement.execute(String.format(insertSQLTemplate, "2018-1-1T13:00:09+08:00", "12"));
       statement.execute(String.format(insertSQLTemplate, "2018-1-1T12:00:10+07:00", "13"));
 
-      boolean hasResultSet = statement.execute("select * from root");
+      boolean hasResultSet = statement.execute("select * from root.**");
       Assert.assertTrue(hasResultSet);
 
       ResultSet resultSet = statement.getResultSet();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
index 96688f1..b27ca68 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
@@ -220,7 +220,7 @@ public class IoTDBTtlIT {
         Statement statement = connection.createStatement()) {
       statement.execute("SET STORAGE GROUP TO root.group1");
       statement.execute("SET STORAGE GROUP TO root.group2.sgroup1");
-      statement.execute("SET TTL TO root.group2 10000");
+      statement.execute("SET TTL TO root.group2.** 10000");
       String result = doQuery(statement, "SHOW ALL TTL");
       assertTrue(
           result.equals("root.group1,null\n" + "root.group2.sgroup1,10000\n")
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java
index d2082d0..5120595 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java
@@ -802,7 +802,7 @@ public class IoTDBAggregationSmallDataIT {
     try (Connection connection =
             DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("SELECT * FROM root");
+      boolean hasResultSet = statement.execute("SELECT * FROM root.**");
       if (hasResultSet) {
         try (ResultSet resultSet = statement.getResultSet()) {
           int cnt = 0;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/auth/IoTDBAuthorizationIT.java b/server/src/test/java/org/apache/iotdb/db/integration/auth/IoTDBAuthorizationIT.java
index 33c0208..291c7f5 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/auth/IoTDBAuthorizationIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/auth/IoTDBAuthorizationIT.java
@@ -618,7 +618,7 @@ public class IoTDBAuthorizationIT {
         userStmt.execute("CREATE TIMESERIES root.a.c WITH DATATYPE=INT32,ENCODING=PLAIN");
         userStmt.execute("INSERT INTO root.a(timestamp,b,c) VALUES (1,100,1000)");
         // userStmt.execute("DELETE FROM root.a.b WHERE TIME <= 1000000000");
-        userStmt.execute("SELECT * FROM root");
+        userStmt.execute("SELECT * FROM root.**");
         userStmt.getResultSet().close();
 
         adminStmt.execute("REVOKE ROLE admin PRIVILEGES 'DELETE_TIMESERIES' on root");
@@ -632,7 +632,7 @@ public class IoTDBAuthorizationIT {
 
         adminStmt.execute("GRANT USER tempuser PRIVILEGES 'READ_TIMESERIES' on root");
         adminStmt.execute("REVOKE admin FROM tempuser");
-        userStmt.execute("SELECT * FROM root");
+        userStmt.execute("SELECT * FROM root.**");
         userStmt.getResultSet().close();
         caught = false;
         try {
@@ -1135,7 +1135,7 @@ public class IoTDBAuthorizationIT {
               e.getMessage());
         }
       }
-      ResultSet resultSet = adminStmt.executeQuery("select * from root");
+      ResultSet resultSet = adminStmt.executeQuery("select * from root.**");
       String[] expected = new String[] {"1, 1.0", "1, null", "3, null", "3, 1.0"};
       List<String> expectedList = new ArrayList<>();
       Collections.addAll(expectedList, expected);
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 b31cbcc..9b6e04e 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
@@ -149,19 +149,19 @@ public class MManagerAdvancedTest {
       // test filename by seriesPath
       assertEquals(
           new PartialPath("root.vehicle.d0"),
-          mmanager.getStorageGroupPath(new PartialPath("root.vehicle.d0.s1")));
+          mmanager.getBelongedStorageGroup(new PartialPath("root.vehicle.d0.s1")));
       List<PartialPath> pathList =
-          mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d1.*"));
+          mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d1.**"));
       assertEquals(6, pathList.size());
-      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d0"));
+      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d0.**"));
       assertEquals(6, pathList.size());
-      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d*"));
+      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d*.**"));
       assertEquals(12, pathList.size());
-      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.ve*.*"));
+      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.ve*.**"));
       assertEquals(12, pathList.size());
       pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle*.d*.s1"));
       assertEquals(2, pathList.size());
-      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d2"));
+      pathList = mmanager.getAllTimeseriesPath(new PartialPath("root.vehicle.d2.**"));
       assertEquals(0, pathList.size());
     } catch (MetadataException e) {
       e.printStackTrace();
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 a121683..52e0c82 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
@@ -295,7 +295,7 @@ public class MManagerBasicTest {
     }
 
     try {
-      manager.deleteTimeseries(new PartialPath("root.laptop.d1.vector"));
+      manager.deleteTimeseries(new PartialPath("root.laptop.d1.vector.*"));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -384,23 +384,20 @@ public class MManagerBasicTest {
           CompressionType.GZIP,
           null);
 
-      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root")), 6);
-      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop")), 6);
-      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.*")), 6);
-      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.*.*")), 5);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.**")), 6);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.**")), 6);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.*")), 1);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.*.*")), 4);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.*.**")), 5);
       assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.*.*.t1")), 1);
       assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.*.s1")), 2);
-      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1")), 3);
-      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1.*")), 3);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1.**")), 3);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1.*")), 2);
       assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d2.s1")), 1);
-      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d2")), 2);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d2.**")), 2);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop")), 0);
+      assertEquals(manager.getAllTimeseriesCount(new PartialPath("root.laptop.d3.s1")), 0);
 
-      try {
-        manager.getAllTimeseriesCount(new PartialPath("root.laptop.d3.s1"));
-        fail("Expected exception");
-      } catch (MetadataException e) {
-        assertEquals("Path [root.laptop.d3.s1] does not exist", e.getMessage());
-      }
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -468,7 +465,7 @@ public class MManagerBasicTest {
       // prefix with *
       assertEquals(
           devices,
-          manager.getDevices(new PartialPath("root.*")).stream()
+          manager.getMatchedDevices(new PartialPath("root.**")).stream()
               .map(PartialPath::getFullPath)
               .collect(Collectors.toSet()));
 
@@ -481,7 +478,7 @@ public class MManagerBasicTest {
       // prefix with *
       assertEquals(
           devices,
-          manager.getDevices(new PartialPath("root.*")).stream()
+          manager.getMatchedDevices(new PartialPath("root.**")).stream()
               .map(PartialPath::getFullPath)
               .collect(Collectors.toSet()));
 
@@ -495,7 +492,7 @@ public class MManagerBasicTest {
       // prefix with *
       assertEquals(
           devices,
-          recoverManager.getDevices(new PartialPath("root.*")).stream()
+          recoverManager.getMatchedDevices(new PartialPath("root.**")).stream()
               .map(PartialPath::getFullPath)
               .collect(Collectors.toSet()));
 
@@ -526,14 +523,15 @@ public class MManagerBasicTest {
           CompressionType.GZIP,
           null);
 
-      List<String> list = new ArrayList<>();
+      List<PartialPath> list = new ArrayList<>();
 
-      list.add("root.laptop.d1");
-      assertEquals(list, manager.getStorageGroupByPath(new PartialPath("root.laptop.d1.s1")));
-      assertEquals(list, manager.getStorageGroupByPath(new PartialPath("root.laptop.d1")));
-      list.add("root.laptop.d2");
-      assertEquals(list, manager.getStorageGroupByPath(new PartialPath("root.laptop")));
-      assertEquals(list, manager.getStorageGroupByPath(new PartialPath("root")));
+      list.add(new PartialPath("root.laptop.d1"));
+      assertEquals(list, manager.getBelongedStorageGroups(new PartialPath("root.laptop.d1.s1")));
+      assertEquals(list, manager.getBelongedStorageGroups(new PartialPath("root.laptop.d1")));
+
+      list.add(new PartialPath("root.laptop.d2"));
+      assertEquals(list, manager.getBelongedStorageGroups(new PartialPath("root.laptop.**")));
+      assertEquals(list, manager.getBelongedStorageGroups(new PartialPath("root.**")));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -546,24 +544,36 @@ public class MManagerBasicTest {
 
     try {
       assertTrue(manager.getAllTimeseriesPath(new PartialPath("root")).isEmpty());
-      assertTrue(manager.getStorageGroupByPath(new PartialPath("root.vehicle")).isEmpty());
-      assertTrue(manager.getStorageGroupByPath(new PartialPath("root.vehicle.device")).isEmpty());
+      assertTrue(manager.getBelongedStorageGroups(new PartialPath("root")).isEmpty());
+      assertTrue(manager.getBelongedStorageGroups(new PartialPath("root.vehicle")).isEmpty());
+      assertTrue(
+          manager.getBelongedStorageGroups(new PartialPath("root.vehicle.device")).isEmpty());
       assertTrue(
-          manager.getStorageGroupByPath(new PartialPath("root.vehicle.device.sensor")).isEmpty());
+          manager
+              .getBelongedStorageGroups(new PartialPath("root.vehicle.device.sensor"))
+              .isEmpty());
 
       manager.setStorageGroup(new PartialPath("root.vehicle"));
-      assertFalse(manager.getStorageGroupByPath(new PartialPath("root.vehicle")).isEmpty());
-      assertFalse(manager.getStorageGroupByPath(new PartialPath("root.vehicle.device")).isEmpty());
+      assertFalse(manager.getBelongedStorageGroups(new PartialPath("root.vehicle")).isEmpty());
       assertFalse(
-          manager.getStorageGroupByPath(new PartialPath("root.vehicle.device.sensor")).isEmpty());
-      assertTrue(manager.getStorageGroupByPath(new PartialPath("root.vehicle1")).isEmpty());
-      assertTrue(manager.getStorageGroupByPath(new PartialPath("root.vehicle1.device")).isEmpty());
+          manager.getBelongedStorageGroups(new PartialPath("root.vehicle.device")).isEmpty());
+      assertFalse(
+          manager
+              .getBelongedStorageGroups(new PartialPath("root.vehicle.device.sensor"))
+              .isEmpty());
+      assertTrue(manager.getBelongedStorageGroups(new PartialPath("root.vehicle1")).isEmpty());
+      assertTrue(
+          manager.getBelongedStorageGroups(new PartialPath("root.vehicle1.device")).isEmpty());
 
       manager.setStorageGroup(new PartialPath("root.vehicle1.device"));
-      assertTrue(manager.getStorageGroupByPath(new PartialPath("root.vehicle1.device1")).isEmpty());
-      assertTrue(manager.getStorageGroupByPath(new PartialPath("root.vehicle1.device2")).isEmpty());
-      assertTrue(manager.getStorageGroupByPath(new PartialPath("root.vehicle1.device3")).isEmpty());
-      assertFalse(manager.getStorageGroupByPath(new PartialPath("root.vehicle1.device")).isEmpty());
+      assertTrue(
+          manager.getBelongedStorageGroups(new PartialPath("root.vehicle1.device1")).isEmpty());
+      assertTrue(
+          manager.getBelongedStorageGroups(new PartialPath("root.vehicle1.device2")).isEmpty());
+      assertTrue(
+          manager.getBelongedStorageGroups(new PartialPath("root.vehicle1.device3")).isEmpty());
+      assertFalse(
+          manager.getBelongedStorageGroups(new PartialPath("root.vehicle1.device")).isEmpty());
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -597,9 +607,10 @@ public class MManagerBasicTest {
       Set<String> nodes2 = new HashSet<>(Arrays.asList("laptop"));
       Set<String> nodes3 = new HashSet<>(Arrays.asList("d1", "d2"));
       Set<String> nexLevelNodes1 =
-          manager.getChildNodeInNextLevel(new PartialPath("root.laptop.d1"));
-      Set<String> nexLevelNodes2 = manager.getChildNodeInNextLevel(new PartialPath("root"));
-      Set<String> nexLevelNodes3 = manager.getChildNodeInNextLevel(new PartialPath("root.laptop"));
+          manager.getChildNodeNameInNextLevel(new PartialPath("root.laptop.d1"));
+      Set<String> nexLevelNodes2 = manager.getChildNodeNameInNextLevel(new PartialPath("root"));
+      Set<String> nexLevelNodes3 =
+          manager.getChildNodeNameInNextLevel(new PartialPath("root.laptop"));
       // usual condition
       assertEquals(nodes, nexLevelNodes1);
       assertEquals(nodes2, nexLevelNodes2);
@@ -720,7 +731,7 @@ public class MManagerBasicTest {
       // usual condition
       assertEquals(
           devices,
-          manager.getDevices(new PartialPath("root.laptop")).stream()
+          manager.getMatchedDevices(new PartialPath("root.laptop.**")).stream()
               .map(PartialPath::getFullPath)
               .collect(Collectors.toSet()));
       manager.setStorageGroup(new PartialPath("root.vehicle"));
@@ -734,7 +745,7 @@ public class MManagerBasicTest {
       // prefix with *
       assertEquals(
           devices,
-          manager.getDevices(new PartialPath("root.*")).stream()
+          manager.getMatchedDevices(new PartialPath("root.**")).stream()
               .map(PartialPath::getFullPath)
               .collect(Collectors.toSet()));
     } catch (MetadataException e) {
@@ -1175,7 +1186,7 @@ public class MManagerBasicTest {
       // show timeseries root.laptop.d1.vector
       showTimeSeriesPlan =
           new ShowTimeSeriesPlan(
-              new PartialPath("root.laptop.d1.vector"), false, null, null, 0, 0, false);
+              new PartialPath("root.laptop.d1.vector.*"), false, null, null, 0, 0, false);
       result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
       assertEquals(3, result.size());
       for (int i = 0; i < result.size(); i++) {
@@ -1280,7 +1291,7 @@ public class MManagerBasicTest {
 
       // show timeseries root.laptop.d1.(s1,s2,s3)
       showTimeSeriesPlan =
-          new ShowTimeSeriesPlan(new PartialPath("root"), false, null, null, 0, 0, false);
+          new ShowTimeSeriesPlan(new PartialPath("root.**"), false, null, null, 0, 0, false);
       result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
       assertEquals(4, result.size());
       Set<String> set = new HashSet<>();
@@ -1365,13 +1376,13 @@ public class MManagerBasicTest {
       manager.setSchemaTemplate(setSchemaTemplatePlan);
       manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.computer.d1")));
 
-      Assert.assertEquals(2, manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1")));
+      Assert.assertEquals(2, manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1.**")));
       Assert.assertEquals(1, manager.getAllTimeseriesCount(new PartialPath("root.laptop.d1.s1")));
       Assert.assertEquals(1, manager.getAllTimeseriesCount(new PartialPath("root.computer.d1.s1")));
       Assert.assertEquals(1, manager.getAllTimeseriesCount(new PartialPath("root.computer.d1.s2")));
-      Assert.assertEquals(3, manager.getAllTimeseriesCount(new PartialPath("root.computer.d1")));
-      Assert.assertEquals(3, manager.getAllTimeseriesCount(new PartialPath("root.computer")));
-      Assert.assertEquals(5, manager.getAllTimeseriesCount(new PartialPath("root")));
+      Assert.assertEquals(3, manager.getAllTimeseriesCount(new PartialPath("root.computer.d1.**")));
+      Assert.assertEquals(3, manager.getAllTimeseriesCount(new PartialPath("root.computer.**")));
+      Assert.assertEquals(5, manager.getAllTimeseriesCount(new PartialPath("root.**")));
 
     } catch (MetadataException e) {
       e.printStackTrace();
@@ -1429,7 +1440,8 @@ public class MManagerBasicTest {
 
       Assert.assertEquals(1, manager.getDevicesNum(new PartialPath("root.laptop.d1")));
       Assert.assertEquals(1, manager.getDevicesNum(new PartialPath("root.laptop.d2")));
-      Assert.assertEquals(2, manager.getDevicesNum(new PartialPath("root.laptop")));
+      Assert.assertEquals(2, manager.getDevicesNum(new PartialPath("root.laptop.*")));
+      Assert.assertEquals(2, manager.getDevicesNum(new PartialPath("root.laptop.**")));
 
     } catch (MetadataException e) {
       e.printStackTrace();
@@ -1549,7 +1561,7 @@ public class MManagerBasicTest {
 
       // call getSeriesSchemasAndReadLockDevice
       IMNode node = manager.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-      assertEquals(3, manager.getAllTimeseriesCount(node.getPartialPath()));
+      assertEquals(3, manager.getAllTimeseriesCount(node.getPartialPath().concatNode("**")));
       assertEquals(1, node.getMeasurementMNodeCount());
       assertNull(insertRowPlan.getMeasurementMNodes()[0]);
       assertNull(insertRowPlan.getMeasurementMNodes()[1]);
@@ -1868,4 +1880,33 @@ public class MManagerBasicTest {
       Assert.assertFalse(manager.isPathExist(new PartialPath("root.a.d")));
     }
   }
+
+  @Test
+  public void testTimeseriesDeletionWithEntityUsingTemplate() throws MetadataException {
+    MManager manager = IoTDB.metaManager;
+    manager.setStorageGroup(new PartialPath("root.sg"));
+
+    CreateTemplatePlan plan = getCreateTemplatePlan("s1");
+    manager.createSchemaTemplate(plan);
+    SetSchemaTemplatePlan setPlan = new SetSchemaTemplatePlan("template1", "root.sg.d1");
+    manager.setSchemaTemplate(setPlan);
+    manager.createTimeseries(
+        new PartialPath("root.sg.d1.s2"),
+        TSDataType.valueOf("INT32"),
+        TSEncoding.valueOf("RLE"),
+        compressionType,
+        Collections.emptyMap());
+    manager.setUsingSchemaTemplate(manager.getDeviceNode(new PartialPath("root.sg.d1")));
+    manager.deleteTimeseries(new PartialPath("root.sg.d1.s2"));
+    assertTrue(manager.isPathExist(new PartialPath("root.sg.d1")));
+
+    manager.createTimeseries(
+        new PartialPath("root.sg.d2.s2"),
+        TSDataType.valueOf("INT32"),
+        TSEncoding.valueOf("RLE"),
+        compressionType,
+        Collections.emptyMap());
+    manager.deleteTimeseries(new PartialPath("root.sg.d2.s2"));
+    assertFalse(manager.isPathExist(new PartialPath("root.sg.d2")));
+  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
index 2070c69..f65a0a0 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
@@ -22,13 +22,16 @@ import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTree;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 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.utils.Pair;
 
 import org.junit.After;
 import org.junit.Assert;
@@ -293,6 +296,12 @@ public class MTreeTest {
       assertEquals(2, result2.size());
       assertEquals("root.a.d0.temperature", result2.get(0).getFullPathWithAlias());
       assertEquals("root.a.d1.temperature", result2.get(1).getFullPathWithAlias());
+
+      Pair<List<PartialPath>, Integer> result3 =
+          root.getAllTimeseriesPathWithAlias(new PartialPath("root.a.**"), 2, 0);
+      assertEquals(2, result3.left.size());
+      assertEquals(2, result3.right.intValue());
+
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -431,15 +440,15 @@ public class MTreeTest {
           null);
 
       // getChildNodeByPath
-      Set<String> result1 = root.getChildNodeInNextLevel(new PartialPath("root.a.d0"));
-      Set<String> result2 = root.getChildNodeInNextLevel(new PartialPath("root.a"));
-      Set<String> result3 = root.getChildNodeInNextLevel(new PartialPath("root"));
+      Set<String> result1 = root.getChildNodeNameInNextLevel(new PartialPath("root.a.d0"));
+      Set<String> result2 = root.getChildNodeNameInNextLevel(new PartialPath("root.a"));
+      Set<String> result3 = root.getChildNodeNameInNextLevel(new PartialPath("root"));
       assertEquals(result1, new HashSet<>(Arrays.asList("s0", "s1")));
       assertEquals(result2, new HashSet<>(Arrays.asList("d0", "d5")));
       assertEquals(result3, new HashSet<>(Arrays.asList("a")));
 
       // if child node is nll   will return  null HashSet
-      Set<String> result5 = root.getChildNodeInNextLevel(new PartialPath("root.a.d5"));
+      Set<String> result5 = root.getChildNodeNameInNextLevel(new PartialPath("root.a.d5"));
       assertEquals(result5, new HashSet<>(Arrays.asList()));
     } catch (MetadataException e1) {
       e1.printStackTrace();
@@ -456,12 +465,12 @@ public class MTreeTest {
       assertTrue(root.checkStorageGroupByPath(new PartialPath("root.laptop.d1")));
       assertEquals(
           "root.laptop.d1",
-          root.getStorageGroupPath(new PartialPath("root.laptop.d1")).getFullPath());
+          root.getBelongedStorageGroup(new PartialPath("root.laptop.d1")).getFullPath());
       assertFalse(root.isPathExist(new PartialPath("root.laptop.d1.s1")));
       assertTrue(root.checkStorageGroupByPath(new PartialPath("root.laptop.d1.s1")));
       assertEquals(
... 776 lines suppressed ...