You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2020/08/11 11:25:54 UTC

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #1496: Support double quotation in Path and optimize the Path String usage

jt2594838 commented on a change in pull request #1496:
URL: https://github.com/apache/incubator-iotdb/pull/1496#discussion_r468446405



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
##########
@@ -270,31 +283,28 @@ public ServiceType getID() {
     return ServiceType.STORAGE_ENGINE_SERVICE;
   }
 
-  public StorageGroupProcessor getProcessor(String path) throws StorageEngineException {
-    String storageGroupName;
+  public StorageGroupProcessor getProcessor(List<String> detachedPath) throws StorageEngineException {
+    StorageGroupMNode storageGroupNode;
     try {
-      storageGroupName = IoTDB.metaManager.getStorageGroupName(path);
+      storageGroupNode = IoTDB.metaManager.getBelongedStorageGroupMNode(detachedPath);
       StorageGroupProcessor processor;
-      processor = processorMap.get(storageGroupName);
+      processor = processorMap.get(storageGroupNode);
       if (processor == null) {
         // if finish recover
         if (isAllSgReady.get()) {
-          storageGroupName = storageGroupName.intern();
-          synchronized (storageGroupName) {
-            processor = processorMap.get(storageGroupName);
+          synchronized (storageGroupNode) {
+            processor = processorMap.get(storageGroupNode);
             if (processor == null) {
               logger.info("construct a processor instance, the storage group is {}, Thread is {}",
-                storageGroupName, Thread.currentThread().getId());
-              processor = new StorageGroupProcessor(systemDir, storageGroupName, fileFlushPolicy);
-              StorageGroupMNode storageGroup = IoTDB.metaManager
-                .getStorageGroupNode(storageGroupName);
-              processor.setDataTTL(storageGroup.getDataTTL());
-              processorMap.put(storageGroupName, processor);
+                  storageGroupNode.getFullPath(), Thread.currentThread().getId());
+              processor = new StorageGroupProcessor(systemDir, storageGroupNode.getFullPath(), fileFlushPolicy);
+              processor.setDataTTL((storageGroupNode).getDataTTL());

Review comment:
       Unnecessary parenthesis.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -472,19 +476,19 @@ public String deleteTimeseries(String prefixPath) throws MetadataException {
       mNodeCache.clear();
     }
     try {
-      List<String> allTimeseries = mtree.getAllTimeseriesName(prefixPath);
+      List<MeasurementMNode> allMeasurementMNodes = mtree.getAllMeasurementMNodes(detachedPath);
       // Monitor storage group seriesPath is not allowed to be deleted
-      allTimeseries.removeIf(p -> p.startsWith(MonitorConstants.STAT_STORAGE_GROUP_PREFIX));
+      allMeasurementMNodes.removeIf(p -> p.getFullPath().contains(MonitorConstants.STATS));

Review comment:
       'contains' is a bit risky, is there any chance that the user creates paths like "root.root.stats" and such paths are removed here?

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1334,16 +1337,17 @@ public void forceCloseAllWorkingTsFileProcessors() throws TsFileProcessorExcepti
   }
 
   // TODO need a read lock, please consider the concurrency with flush manager threads.
-  public QueryDataSource query(String deviceId, String measurementId, QueryContext context,
+  public QueryDataSource query(List<String> deviceNodes, String measurementId, QueryContext context,
       QueryFileManager filePathsManager, Filter timeFilter) throws QueryProcessException {
     insertLock.readLock().lock();
     mergeLock.readLock().lock();
     try {
       List<TsFileResource> seqResources = getFileResourceListForQuery(sequenceFileTreeSet,
-          upgradeSeqFileList, deviceId, measurementId, context, timeFilter, true);
+          upgradeSeqFileList, deviceNodes, measurementId, context, timeFilter, true);
       List<TsFileResource> unseqResources = getFileResourceListForQuery(unSequenceFileList,
-          upgradeUnseqFileList, deviceId, measurementId, context, timeFilter, false);
-      QueryDataSource dataSource = new QueryDataSource(new Path(deviceId, measurementId),
+          upgradeUnseqFileList, deviceNodes, measurementId, context, timeFilter, false);
+      deviceNodes.add(measurementId); // device nodes is full path nodes now

Review comment:
       Just to make sure, the `devicesNodes` is a parameter but is modified in the method, so I hope the caller will not still use it as `deviceNodes` any more.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -643,17 +755,18 @@ private int getCountInGivenLevel(MNode node, int targetLevel) {
    *
    * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
    */
-  List<String[]> getAllMeasurementSchemaByHeatOrder(ShowTimeSeriesPlan plan,
-      QueryContext queryContext) throws MetadataException {
-    String[] nodes = MetaUtils.getNodeNames(plan.getPath().getFullPath());
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
+
+  List<String[]> getAllMeasurementSchemaByHeatOrder(ShowTimeSeriesPlan plan, QueryContext queryContext)
+      throws MetadataException {
+    String[] detachedPath = plan.getPath().getDetachedPath().toArray(new String[0]);

Review comment:
       Better to focus on one instead of unnecessary copies and converts.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
##########
@@ -446,11 +456,13 @@ public void delete(String deviceId, String measurementId, long startTime, long e
   public QueryDataSource query(SingleSeriesExpression seriesExpression, QueryContext context,
       QueryFileManager filePathsManager)
       throws StorageEngineException, QueryProcessException {
-    String deviceId = seriesExpression.getSeriesPath().getDevice();
-    String measurementId = seriesExpression.getSeriesPath().getMeasurement();
-    StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
+    List<String> pathNodes = seriesExpression.getSeriesPath().getDetachedPath();
+    String measurementId = pathNodes.get(pathNodes.size() - 1);
+    List<String> deviceNodes = new ArrayList<>(pathNodes);
+    deviceNodes.remove(deviceNodes.size() - 1);

Review comment:
       How about just using `subList()`?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -330,34 +324,38 @@ public void operation(String cmd) throws IOException, MetadataException {
           tagMap = tagLogFile.readTag(config.getTagAttributeTotalSize(), offset);
         }
 
-        CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(new Path(args[1]),
+        CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(new Path(MetaUtils.splitPathToDetachedPath(args[1])),
             TSDataType.deserialize(Short.parseShort(args[2])),
             TSEncoding.deserialize(Short.parseShort(args[3])),
             CompressionType.deserialize(Short.parseShort(args[4])), props, tagMap, null, alias);
 
         createTimeseries(plan, offset);
         break;
       case MetadataOperationType.DELETE_TIMESERIES:
-        String failedTimeseries = deleteTimeseries(args[1]);
+        String failedTimeseries = deleteTimeseries(MetaUtils.splitPathToDetachedPath(args[1]));
         if (!failedTimeseries.isEmpty()) {
           throw new DeleteFailedException(failedTimeseries);
         }
         break;
       case MetadataOperationType.SET_STORAGE_GROUP:
-        setStorageGroup(args[1]);
+        setStorageGroup(MetaUtils.splitPathToDetachedPath(args[1]));
         break;
       case MetadataOperationType.DELETE_STORAGE_GROUP:
         List<String> storageGroups = new ArrayList<>(Arrays.asList(args).subList(1, args.length));

Review comment:
       `new ArrayList<>` seems unnecessary.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -906,13 +907,14 @@ private void tryToUpdateBatchInsertLastCache(InsertTabletPlan plan, Long latestF
       }
       if (tmpMeasurementNode != null) {
         // just for performance, because in single node version, we do not need the full path of measurement
-        // so, we want to avoid concat the device and measurement string in single node version
-        IoTDB.metaManager.updateLastCache(node.getFullPath(),
-            plan.composeLastTimeValuePair(i), true, latestFlushedTime, tmpMeasurementNode);
+        // so, we want just pass a null
+        IoTDB.metaManager.updateLastCache(null,
+          plan.composeLastTimeValuePair(i), true, latestFlushedTime, tmpMeasurementNode);
       } else {
-        IoTDB.metaManager
-            .updateLastCache(node.getFullPath() + IoTDBConstant.PATH_SEPARATOR + measurementList[i],
-                plan.composeLastTimeValuePair(i), true, latestFlushedTime, tmpMeasurementNode);
+        List<String> fullPath = new ArrayList<>(plan.getDetachedDevice());
+        fullPath.add(measurementList[i]);
+        IoTDB.metaManager.updateLastCache( fullPath,
+          plan.composeLastTimeValuePair(i), true, latestFlushedTime, tmpMeasurementNode);

Review comment:
       Maybe this `fullPath` can be declared outside the loop and reused within the loop, so you do not have to copy it again and again.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
##########
@@ -499,15 +511,15 @@ public void mergeAll(boolean fullMerge) throws StorageEngineException {
    * delete all data files (both memory data and file on disk) in a storage group. It is used when
    * there is no timeseries (which are all deleted) in this storage group)
    */
-  public void deleteAllDataFilesInOneStorageGroup(String storageGroupName) {
-    if (processorMap.containsKey(storageGroupName)) {
-      syncDeleteDataFiles(storageGroupName);
+  public void deleteAllDataFilesInOneStorageGroup(MNode mNode) {
+    if (processorMap.containsKey(mNode)) {
+      syncDeleteDataFiles(mNode);
     }
   }
 
-  private void syncDeleteDataFiles(String storageGroupName) {
-    logger.info("Force to delete the data in storage group processor {}", storageGroupName);
-    StorageGroupProcessor processor = processorMap.get(storageGroupName);
+  private void syncDeleteDataFiles(MNode mNode) {
+    logger.info("Force to delete the data in storage group processor {}", mNode.getFullPath());

Review comment:
       Better wrap it with `if (logger.isInfoEnabled())`.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -279,130 +278,170 @@ void setStorageGroup(String path) throws MetadataException {
    * <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
+   * @param nodeNames path
    * @apiNote :for cluster
    */
-  boolean isStorageGroup(String path) {
-    String[] nodeNames = MetaUtils.getNodeNames(path);
-    if (nodeNames.length <= 1 || !nodeNames[0].equals(IoTDBConstant.PATH_ROOT)) {
+  boolean isStorageGroup(List<String> nodeNames) {
+    if (nodeNames.size() <= 1 || !nodeNames.get(0).equals(IoTDBConstant.PATH_ROOT)) {
       return false;
     }
     MNode cur = root;
     int i = 1;
-    while (i < nodeNames.length - 1) {
-      cur = cur.getChild(nodeNames[i]);
+    while (i < nodeNames.size() - 1) {
+      cur = cur.getChild(nodeNames.get(i));
       if (cur == null || cur instanceof StorageGroupMNode) {
         return false;
       }
       i++;
     }
-    cur = cur.getChild(nodeNames[i]);
+    cur = cur.getChild(nodeNames.get(i));
     return cur instanceof StorageGroupMNode;
   }
 
   /**
    * Delete path. The path should be a full path from root to leaf node
    *
-   * @param path Format: root.node(.node)+
+   * @param measurementMNode measurementMNode
    */
-  Pair<String, MeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(String path)
+  Pair<String, MeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(MeasurementMNode measurementMNode)
       throws MetadataException {
-    MNode curNode = getNodeByPath(path);
-    if (!(curNode instanceof MeasurementMNode)) {
-      throw new PathNotExistException(path);
-    }
-    String[] nodes = MetaUtils.getNodeNames(path);
-    if (nodes.length == 0 || !IoTDBConstant.PATH_ROOT.equals(nodes[0])) {
-      throw new IllegalPathException(path);
-    }
+    
     // delete the last node of path
-    curNode.getParent().deleteChild(curNode.getName());
-    MeasurementMNode deletedNode = (MeasurementMNode) curNode;
-    if (deletedNode.getAlias() != null) {
-      curNode.getParent().deleteAliasChild(((MeasurementMNode) curNode).getAlias());
+    measurementMNode.getParent().deleteChild(measurementMNode.getName());
+    if (measurementMNode.getAlias() != null) {
+      measurementMNode.getParent().deleteAliasChild(measurementMNode.getAlias());
     }
-    curNode = curNode.getParent();
+    MNode curNode = measurementMNode.getParent();
     // delete all empty ancestors except storage group
     while (!IoTDBConstant.PATH_ROOT.equals(curNode.getName())
         && curNode.getChildren().size() == 0) {
       // if current storage group has no time series, return the storage group name
       if (curNode instanceof StorageGroupMNode) {
-        return new Pair<>(curNode.getFullPath(), deletedNode);
+        return new Pair<>(curNode.getFullPath(), measurementMNode);
       }
       curNode.getParent().deleteChild(curNode.getName());
       curNode = curNode.getParent();
     }
-    return new Pair<>(null, deletedNode);
+    return new Pair<>(null, measurementMNode);
   }
 
   /**
    * Get measurement schema for a given path. Path must be a complete Path from root to leaf node.
    */
-  MeasurementSchema getSchema(String path) throws MetadataException {
-    MeasurementMNode node = (MeasurementMNode) getNodeByPath(path);
+  MeasurementSchema getSchema(List<String> detachedPath) throws MetadataException {
+    MeasurementMNode node = (MeasurementMNode) getMNodeByDetachedPath(detachedPath);
     return node.getSchema();
   }
 
+  /**
+   * Get measurement schema for a given measurementMNode.
+   */
+  MeasurementSchema getSchemaByMNode(MeasurementMNode measurementMNode) {
+    return measurementMNode.getSchema();
+  }
+
+  List<String> getDetachedPathByMNode(MNode mNode) {
+    List<String> detachedPath = new ArrayList<>();
+    detachedPath.add(mNode.getName());
+    while (mNode.getParent() != null) {
+      mNode = mNode.getParent();
+      detachedPath.add(0, mNode.getName());

Review comment:
       Try using `Collections.reverse`, which results in a  complexity of O(n), instead of O(n^2).

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java
##########
@@ -18,61 +18,84 @@
  */
 package org.apache.iotdb.db.metadata;
 
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 
 public class MetaUtils {
 
   public static final String PATH_SEPARATOR = "\\.";
 
   private MetaUtils() {
+    throw new IllegalStateException("Utility class");
+  }
 
+  static String getNodeNameRegByIdx(int idx, String[] nodes) {
+    return idx >= nodes.length ? PATH_WILDCARD : nodes[idx];
   }
 
-  public static String[] getNodeNames(String path) {
-    String[] nodeNames;
-    if (path.contains("\"") || path.contains("'")) {
-      // e.g., root.sg.d1."s1.int"  ->  root.sg.d1, s1.int
-      String[] measurementDeviceNode = path.trim().replace("'", "\"").split("\"");
-      // s1.int
-      String measurement = measurementDeviceNode[1];
-      // root.sg.d1 -> root, sg, d1
-      String[] deviceNodeName = measurementDeviceNode[0].split(PATH_SEPARATOR);
-      int nodeNumber = deviceNodeName.length + 1;
-      nodeNames = new String[nodeNumber];
-      System.arraycopy(deviceNodeName, 0, nodeNames, 0, nodeNumber - 1);
-      // nodeNames = [root, sg, d1, s1.int]
-      nodeNames[nodeNumber - 1] = measurement;
-    } else {
-      nodeNames = path.split(PATH_SEPARATOR);
-    }
-    return nodeNames;
+  @TestOnly
+  public static List<String> splitPathByDot(String path) {
+    List<String> detachedDevicePath = new ArrayList<>();
+    Collections.addAll(detachedDevicePath, path.split(PATH_SEPARATOR));
+    return detachedDevicePath;
   }
 
-  static String getNodeRegByIdx(int idx, String[] nodes) {
-    return idx >= nodes.length ? PATH_WILDCARD : nodes[idx];
+  public static List<String> splitPathToDetachedPath(String path) throws IllegalPathException {
+    List<String> nodes = new ArrayList<>();
+    int startIndex = 0;
+    for (int i = 0; i < path.length(); i++) {
+      if (path.charAt(i) == IoTDBConstant.PATH_SEPARATOR) {
+        nodes.add(path.substring(startIndex, i));
+        startIndex = i + 1;
+      } else if (path.charAt(i) == '"') {
+        int endIndex = path.indexOf('"', i + 1);
+        if (endIndex != -1 && (endIndex == path.length() - 1 || path.charAt(endIndex + 1) == '.')) {
+          nodes.add(path.substring(startIndex, endIndex + 1));
+          i = endIndex + 1;
+          startIndex = endIndex + 2;
+        } else {
+          throw new IllegalPathException("Illegal path: " + path);
+        }
+      } else if (path.charAt(i) == '\'') {
+        throw new IllegalPathException("Illegal path with single quote: " + path);
+      }
+    }
+    if (startIndex <= path.length() - 1) {
+      nodes.add(path.substring(startIndex));
+    }
+    return nodes;
   }
 
   /**
    * Get storage group name when creating schema automatically is enable
    *
-   * e.g., path = root.a.b.c and level = 1, return root.a
+   * e.g., nodes = [root, a, b, c] and level = 1, return [root, a]
    *
-   * @param path path
+   * @param detachedPath nodeNames
    * @param level level
    */
-  public static String getStorageGroupNameByLevel(String path, int level) throws MetadataException {
-    String[] nodeNames = MetaUtils.getNodeNames(path);
-    if (nodeNames.length <= level || !nodeNames[0].equals(IoTDBConstant.PATH_ROOT)) {
-      throw new IllegalPathException(path);
-    }
-    StringBuilder storageGroupName = new StringBuilder(nodeNames[0]);
-    for (int i = 1; i <= level; i++) {
-      storageGroupName.append(IoTDBConstant.PATH_SEPARATOR).append(nodeNames[i]);
+  public static List<String> getDetachedStorageGroupByLevel(List<String> detachedPath, int level) throws MetadataException {
+    if (detachedPath.size() <= level || !detachedPath.get(0).equals(IoTDBConstant.PATH_ROOT)) {
+      throw new IllegalPathException(concatDetachedPathByDot(detachedPath));
     }
-    return storageGroupName.toString();
+    return new ArrayList<>(detachedPath.subList(0, level+1));

Review comment:
       Is it necessary to use a new List? (Is the result modifiable?)

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -413,13 +452,12 @@ MNode getNodeByPath(String path) throws MetadataException {
    * @return storage group list
    * @apiNote :for cluster
    */
-  List<String> getStorageGroupByPath(String path) throws MetadataException {
+  List<String> getStorageGroupByDetachedPath(List<String> detachedPath) throws MetadataException {
     List<String> storageGroups = new ArrayList<>();
-    String[] nodes = MetaUtils.getNodeNames(path);
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path);
+    if (detachedPath.isEmpty() || !detachedPath.get(0).equals(root.getName())) {
+      throw new IllegalPathException(MetaUtils.concatDetachedPathByDot(detachedPath));
     }
-    findStorageGroup(root, nodes, 1, "", storageGroups);
+    findStorageGroup(root, detachedPath.toArray(new String[0]), 1, "", storageGroups);

Review comment:
       I think it may be more efficient to modify the method signature instead of converting to an array.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -497,28 +535,63 @@ private void findStorageGroup(
    *
    * @return storage group in the given path
    */
-  String getStorageGroupName(String path) throws StorageGroupNotSetException {
-    String[] nodes = MetaUtils.getNodeNames(path);
+  String getStorageGroup(List<String> detachedPath) throws StorageGroupNotSetException {
     MNode cur = root;
-    for (int i = 1; i < nodes.length; i++) {
-      cur = cur.getChild(nodes[i]);
+    for (int i = 1; i < detachedPath.size(); i++) {
+      cur = cur.getChild(detachedPath.get(i));
       if (cur instanceof StorageGroupMNode) {
         return cur.getFullPath();
       } else if (cur == null) {
-        throw new StorageGroupNotSetException(path);
+        throw new StorageGroupNotSetException(MetaUtils.concatDetachedPathByDot(detachedPath));
       }
     }
-    throw new StorageGroupNotSetException(path);
+    throw new StorageGroupNotSetException(MetaUtils.concatDetachedPathByDot(detachedPath));
+  }
+
+  /**
+   * Get storage group mNode by mNode
+   *
+   * @return storage group mNode of the given mNode
+   */
+  StorageGroupMNode getStorageGroupMNodeByMNode(MNode mNode) {
+    while (!(mNode instanceof StorageGroupMNode)) {
+      mNode = mNode.getParent();

Review comment:
       This is risky, what if `mNode` has no parent?

##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -1559,10 +1621,25 @@ protected TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         : RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR);
   }
 
-  private TSStatus checkPathValidity(String path) {
-    if (!PATH_PATTERN.matcher(path).matches()) {
-      logger.warn("Illegal path: {}", path);
-      return RpcUtils.getStatus(TSStatusCode.PATH_ILLEGAL, path + " path is illegal");
+  private TSStatus checkPathValidityWithNodes(List<String> nodes) {
+    if (nodes == null) {
+      return RpcUtils.getStatus(TSStatusCode.PATH_ILLEGAL, "path is illegal");
+    }
+    int index = 0;
+    if (nodes.get(index).equals(IoTDBConstant.PATH_ROOT)) {
+      for (index = 1; index < nodes.size(); index++) {
+        if (!NODE_NAME_PATTERN.matcher(nodes.get(index)).matches()) {
+          // support double quote on measurement only now
+          if (index == nodes.size() - 1 && NODE_NAME_WITH_QUOTATION_PATTERN.matcher(nodes.get(index)).matches()) {
+            continue;
+          }

Review comment:
       "// support double quote on measurement only now" is not consistent with your PR description.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -158,13 +157,12 @@ protected MManager() {
     isRecovering = true;
 
     int cacheSize = config.getmManagerCacheSize();
-    mNodeCache = new RandomDeleteCache<String, MNode>(cacheSize) {
-
+    mNodeCache = new RandomDeleteCache<String, MNode>(cacheSize){
       @Override
-      public MNode loadObjectByKey(String key) throws CacheException {
+      public MNode loadObjectByKey(String key, List<String> keyNodes) throws CacheException {

Review comment:
       This `key` seems useless.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -279,130 +278,170 @@ void setStorageGroup(String path) throws MetadataException {
    * <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
+   * @param nodeNames path
    * @apiNote :for cluster
    */
-  boolean isStorageGroup(String path) {
-    String[] nodeNames = MetaUtils.getNodeNames(path);
-    if (nodeNames.length <= 1 || !nodeNames[0].equals(IoTDBConstant.PATH_ROOT)) {
+  boolean isStorageGroup(List<String> nodeNames) {
+    if (nodeNames.size() <= 1 || !nodeNames.get(0).equals(IoTDBConstant.PATH_ROOT)) {
       return false;
     }
     MNode cur = root;
     int i = 1;
-    while (i < nodeNames.length - 1) {
-      cur = cur.getChild(nodeNames[i]);
+    while (i < nodeNames.size() - 1) {
+      cur = cur.getChild(nodeNames.get(i));
       if (cur == null || cur instanceof StorageGroupMNode) {
         return false;
       }
       i++;
     }
-    cur = cur.getChild(nodeNames[i]);
+    cur = cur.getChild(nodeNames.get(i));
     return cur instanceof StorageGroupMNode;
   }
 
   /**
    * Delete path. The path should be a full path from root to leaf node
    *
-   * @param path Format: root.node(.node)+
+   * @param measurementMNode measurementMNode
    */
-  Pair<String, MeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(String path)
+  Pair<String, MeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(MeasurementMNode measurementMNode)
       throws MetadataException {
-    MNode curNode = getNodeByPath(path);
-    if (!(curNode instanceof MeasurementMNode)) {
-      throw new PathNotExistException(path);
-    }
-    String[] nodes = MetaUtils.getNodeNames(path);
-    if (nodes.length == 0 || !IoTDBConstant.PATH_ROOT.equals(nodes[0])) {
-      throw new IllegalPathException(path);
-    }
+    
     // delete the last node of path
-    curNode.getParent().deleteChild(curNode.getName());
-    MeasurementMNode deletedNode = (MeasurementMNode) curNode;
-    if (deletedNode.getAlias() != null) {
-      curNode.getParent().deleteAliasChild(((MeasurementMNode) curNode).getAlias());
+    measurementMNode.getParent().deleteChild(measurementMNode.getName());
+    if (measurementMNode.getAlias() != null) {
+      measurementMNode.getParent().deleteAliasChild(measurementMNode.getAlias());
     }
-    curNode = curNode.getParent();
+    MNode curNode = measurementMNode.getParent();
     // delete all empty ancestors except storage group
     while (!IoTDBConstant.PATH_ROOT.equals(curNode.getName())
         && curNode.getChildren().size() == 0) {
       // if current storage group has no time series, return the storage group name
       if (curNode instanceof StorageGroupMNode) {
-        return new Pair<>(curNode.getFullPath(), deletedNode);
+        return new Pair<>(curNode.getFullPath(), measurementMNode);
       }
       curNode.getParent().deleteChild(curNode.getName());
       curNode = curNode.getParent();
     }
-    return new Pair<>(null, deletedNode);
+    return new Pair<>(null, measurementMNode);
   }
 
   /**
    * Get measurement schema for a given path. Path must be a complete Path from root to leaf node.
    */
-  MeasurementSchema getSchema(String path) throws MetadataException {
-    MeasurementMNode node = (MeasurementMNode) getNodeByPath(path);
+  MeasurementSchema getSchema(List<String> detachedPath) throws MetadataException {
+    MeasurementMNode node = (MeasurementMNode) getMNodeByDetachedPath(detachedPath);
     return node.getSchema();
   }
 
+  /**
+   * Get measurement schema for a given measurementMNode.
+   */
+  MeasurementSchema getSchemaByMNode(MeasurementMNode measurementMNode) {
+    return measurementMNode.getSchema();
+  }
+
+  List<String> getDetachedPathByMNode(MNode mNode) {
+    List<String> detachedPath = new ArrayList<>();
+    detachedPath.add(mNode.getName());
+    while (mNode.getParent() != null) {
+      mNode = mNode.getParent();
+      detachedPath.add(0, mNode.getName());
+    }
+    return detachedPath;
+  }
+
+
   /**
    * Get node by path with storage group check If storage group is not set,
    * StorageGroupNotSetException will be thrown
    */
-  MNode getNodeByPathWithStorageGroupCheck(String path) throws MetadataException {
+  MNode getMNodeByDetachedPathWithStorageGroupCheck(List<String> detachedPath) throws MetadataException {
     boolean storageGroupChecked = false;
-    String[] nodes = MetaUtils.getNodeNames(path);
-    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
-      throw new IllegalPathException(path);
+    if (detachedPath.isEmpty() || !detachedPath.get(0).equals(root.getName())) {
+      throw new IllegalPathException(MetaUtils.concatDetachedPathByDot(detachedPath));
     }
 
     MNode cur = root;
-    for (int i = 1; i < nodes.length; i++) {
-      if (!cur.hasChild(nodes[i])) {
+    for (int i = 1; i < detachedPath.size(); i++) {
+      if (!cur.hasChild(detachedPath.get(i))) {
         if (!storageGroupChecked) {
-          throw new StorageGroupNotSetException(path);
+          throw new StorageGroupNotSetException(MetaUtils.concatDetachedPathByDot(detachedPath));
         }
-        throw new PathNotExistException(path);
+        throw new PathNotExistException(MetaUtils.concatDetachedPathByDot(detachedPath));
       }
-      cur = cur.getChild(nodes[i]);
+      cur = cur.getChild(detachedPath.get(i));
 
       if (cur instanceof StorageGroupMNode) {
         storageGroupChecked = true;
       }
     }
 
     if (!storageGroupChecked) {
-      throw new StorageGroupNotSetException(path);
+      throw new StorageGroupNotSetException(MetaUtils.concatDetachedPathByDot(detachedPath));
     }
     return cur;
   }
 
   /**
-   * Get storage group node, if the give path is not a storage group, throw exception
+   * Get storage group node, the give path don't need to be storage group path.
    */
-  StorageGroupMNode getStorageGroupNode(String path) throws MetadataException {
-    MNode node = getNodeByPath(path);
-    if (node instanceof StorageGroupMNode) {
-      return (StorageGroupMNode) node;
-    } else {
-      throw new StorageGroupNotSetException(path);
+  StorageGroupMNode getStorageGroupMNode(List<String> detachedPath) throws MetadataException {
+    if (detachedPath.isEmpty() || !detachedPath.get(0).equals(root.getName())) {
+      throw new IllegalPathException(MetaUtils.concatDetachedPathByDot(detachedPath));
+    }
+    MNode cur = root;
+    for (int i = 1; i < detachedPath.size(); i++) {
+      cur = cur.getChild(detachedPath.get(i));
+      if (cur instanceof StorageGroupMNode) {
+        return (StorageGroupMNode) cur;
+      }
     }
+    throw new StorageGroupNotSetException(MetaUtils.concatDetachedPathByDot(detachedPath));
+  }
+
+  StorageGroupMNode getgetStorageGroupMNodeByMNode(MNode mNode) throws StorageGroupNotSetException {

Review comment:
       `getget`?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
##########
@@ -203,7 +205,7 @@ RootOperator getLogicalPlan() {
   public void enterCountTimeseries(CountTimeseriesContext ctx) {
     super.enterCountTimeseries(ctx);
     PrefixPathContext pathContext = ctx.prefixPath();
-    Path path = (pathContext != null ? parsePrefixPath(pathContext) : new Path(SQLConstant.ROOT));
+    Path path = (pathContext != null ? parsePrefixPath(pathContext) : new Path(new ArrayList<>(Collections.singletonList(SQLConstant.ROOT))));

Review comment:
       `new ArrayList` or `singletonList`, one of them must be unnecessary.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -835,26 +838,24 @@ public void update(Path path, long startTime, long endTime, String value) {
   }
 
   @Override
-  public void delete(Path path, long startTime, long endTime) throws QueryProcessException {
-    String deviceId = path.getDevice();
-    String measurementId = path.getMeasurement();
+  public void delete(List<String> path, long startTime, long endTime) throws QueryProcessException {
+    String measurementId = path.get(path.size()-1);
     try {
-      if (!mManager.isPathExist(path.getFullPath())) {
+      if (!mManager.isPathExist(path)) {
         throw new QueryProcessException(
-            String.format("Time series %s does not exist.", path.getFullPath()));
+            String.format("Time series %s does not exist.", MetaUtils.concatDetachedPathByDot(path)));
       }
-      mManager.getStorageGroupName(path.getFullPath());
-      StorageEngine.getInstance().delete(deviceId, measurementId, startTime, endTime);
+      mManager.getStorageGroup(path);
+      path.remove(path.size()-1);
+      StorageEngine.getInstance().delete(path, measurementId, startTime, endTime);

Review comment:
       What is `mManager.getStorageGroup(path);` for?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -697,78 +840,246 @@ private int getCountInGivenLevel(MNode node, int targetLevel) {
   }
 
   /**
-   * Iterate through MTree to fetch metadata info of all leaf nodes under the given seriesPath
+   * Get all time series schema and path name detachedPath under the given detachedPath
+   *
+   * <p>result: [[root, node], [alias, storage group, dataType, encoding, compression, offset]]
+   */
+  private List<List<String>> getAllMeasurementSchemaAndDetachedPathByDetachedPath(ShowTimeSeriesPlan plan) throws MetadataException {
+    List<List<String>> res;
+    List<String> detachedPath = plan.getPath().getDetachedPath();
+    if (detachedPath.isEmpty() || !detachedPath.get(0).equals(root.getName())) {
+      throw new IllegalPathException(plan.getPath().getFullPath());
+    }
+    limit.set(plan.getLimit());
+    offset.set(plan.getOffset());
+    curOffset.set(-1);
+    count.set(0);
+    if (offset.get() != 0 || limit.get() != 0) {
+      res = new LinkedList<>();
+      findDetachedPathAndSchema(root, detachedPath.toArray(new String[0]), 1, res, true, false, null);
+    } else {
+      res = new LinkedList<>();
+      findDetachedPathAndSchema(root, detachedPath.toArray(new String[0]), 1, res, false, false, null);
+    }
+    // avoid memory leaks
+    limit.remove();
+    offset.remove();
+    curOffset.remove();
+    count.remove();
+    return res;
+  }
+
+  /**
+   * Get all time series under the given detachedPath
+   *
+   * <p>result: [[root, node], [root, node]]
+   */
+  private List<MeasurementMNode> getAllMeasurementMNodes(ShowTimeSeriesPlan plan) throws MetadataException {
+    List<MeasurementMNode> res;
+    List<String> detachedPath = plan.getPath().getDetachedPath();
+    if (detachedPath.isEmpty() || !detachedPath.get(0).equals(root.getName())) {
+      throw new IllegalPathException(plan.getPath().getFullPath());
+    }
+    limit.set(plan.getLimit());
+    offset.set(plan.getOffset());
+    curOffset.set(-1);
+    count.set(0);
+    if (offset.get() != 0 || limit.get() != 0) {
+      res = new ArrayList<>(2);
+      findMeasurementMNode(root, detachedPath.toArray(new String[0]), 1, res, true);
+    } else {
+      res = new ArrayList<>(2);
+      findMeasurementMNode(root, detachedPath.toArray(new String[0]), 1, res, false);
+    }
+    // avoid memory leaks
+    limit.remove();
+    offset.remove();
+    curOffset.remove();
+    count.remove();
+    return res;
+  }
+
+  /**
+   * Iterate through MTree to fetch metadata info of all leaf detachedPath 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]
    */
-  private void findPath(MNode node, String[] nodes, int idx, List<String[]> timeseriesSchemaList,
+  private void findPath(MNode mNode, String[] detachedPath, int idx, List<String[]> timeseriesSchemaList,
       boolean hasLimit, boolean needLast, QueryContext queryContext) throws MetadataException {
-    if (node instanceof MeasurementMNode && nodes.length <= idx) {
+    if (mNode instanceof MeasurementMNode && detachedPath.length <= idx) {
       if (hasLimit) {
         curOffset.set(curOffset.get() + 1);
         if (curOffset.get() < offset.get() || count.get().intValue() == limit.get().intValue()) {
           return;
         }
       }
       String nodeName;
-      if (node.getName().contains(TsFileConstant.PATH_SEPARATOR)) {
-        nodeName = "\"" + node + "\"";
+      if (mNode.getName().contains(TsFileConstant.PATH_SEPARATOR)) {
+        nodeName = "\"" + mNode + "\"";
       } else {
-        nodeName = node.getName();
+        nodeName = mNode.getName();
+      }
+      String nodePath = mNode.getParent().getFullPath() + TsFileConstant.PATH_SEPARATOR + nodeName;
+      detachedPath[detachedPath.length - 1] = nodeName;
+      List<String> fullPath = new ArrayList<>(Arrays.asList(detachedPath));
+      MNode temp = mNode;
+      while (temp.getParent() != null) {
+        temp = temp.getParent();
+        fullPath.add(0, temp.getName());
       }

Review comment:
       `new ArrayList<>` is unnecessary. By the way, how about using lists from the beginning?
   ![image](https://user-images.githubusercontent.com/23610645/89888308-5c038000-dc02-11ea-89e9-47e034cb9451.png)
   

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -697,78 +840,246 @@ private int getCountInGivenLevel(MNode node, int targetLevel) {
   }
 
   /**
-   * Iterate through MTree to fetch metadata info of all leaf nodes under the given seriesPath
+   * Get all time series schema and path name detachedPath under the given detachedPath
+   *
+   * <p>result: [[root, node], [alias, storage group, dataType, encoding, compression, offset]]
+   */
+  private List<List<String>> getAllMeasurementSchemaAndDetachedPathByDetachedPath(ShowTimeSeriesPlan plan) throws MetadataException {
+    List<List<String>> res;
+    List<String> detachedPath = plan.getPath().getDetachedPath();
+    if (detachedPath.isEmpty() || !detachedPath.get(0).equals(root.getName())) {
+      throw new IllegalPathException(plan.getPath().getFullPath());
+    }
+    limit.set(plan.getLimit());
+    offset.set(plan.getOffset());
+    curOffset.set(-1);
+    count.set(0);
+    if (offset.get() != 0 || limit.get() != 0) {
+      res = new LinkedList<>();
+      findDetachedPathAndSchema(root, detachedPath.toArray(new String[0]), 1, res, true, false, null);
+    } else {
+      res = new LinkedList<>();
+      findDetachedPathAndSchema(root, detachedPath.toArray(new String[0]), 1, res, false, false, null);
+    }
+    // avoid memory leaks
+    limit.remove();
+    offset.remove();
+    curOffset.remove();
+    count.remove();
+    return res;
+  }
+
+  /**
+   * Get all time series under the given detachedPath
+   *
+   * <p>result: [[root, node], [root, node]]
+   */
+  private List<MeasurementMNode> getAllMeasurementMNodes(ShowTimeSeriesPlan plan) throws MetadataException {
+    List<MeasurementMNode> res;
+    List<String> detachedPath = plan.getPath().getDetachedPath();
+    if (detachedPath.isEmpty() || !detachedPath.get(0).equals(root.getName())) {
+      throw new IllegalPathException(plan.getPath().getFullPath());
+    }
+    limit.set(plan.getLimit());
+    offset.set(plan.getOffset());
+    curOffset.set(-1);
+    count.set(0);
+    if (offset.get() != 0 || limit.get() != 0) {
+      res = new ArrayList<>(2);
+      findMeasurementMNode(root, detachedPath.toArray(new String[0]), 1, res, true);
+    } else {
+      res = new ArrayList<>(2);
+      findMeasurementMNode(root, detachedPath.toArray(new String[0]), 1, res, false);
+    }
+    // avoid memory leaks
+    limit.remove();
+    offset.remove();
+    curOffset.remove();
+    count.remove();
+    return res;
+  }
+
+  /**
+   * Iterate through MTree to fetch metadata info of all leaf detachedPath 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]
    */
-  private void findPath(MNode node, String[] nodes, int idx, List<String[]> timeseriesSchemaList,
+  private void findPath(MNode mNode, String[] detachedPath, int idx, List<String[]> timeseriesSchemaList,
       boolean hasLimit, boolean needLast, QueryContext queryContext) throws MetadataException {
-    if (node instanceof MeasurementMNode && nodes.length <= idx) {
+    if (mNode instanceof MeasurementMNode && detachedPath.length <= idx) {
       if (hasLimit) {
         curOffset.set(curOffset.get() + 1);
         if (curOffset.get() < offset.get() || count.get().intValue() == limit.get().intValue()) {
           return;
         }
       }
       String nodeName;
-      if (node.getName().contains(TsFileConstant.PATH_SEPARATOR)) {
-        nodeName = "\"" + node + "\"";
+      if (mNode.getName().contains(TsFileConstant.PATH_SEPARATOR)) {
+        nodeName = "\"" + mNode + "\"";
       } else {
-        nodeName = node.getName();
+        nodeName = mNode.getName();
+      }
+      String nodePath = mNode.getParent().getFullPath() + TsFileConstant.PATH_SEPARATOR + nodeName;
+      detachedPath[detachedPath.length - 1] = nodeName;
+      List<String> fullPath = new ArrayList<>(Arrays.asList(detachedPath));
+      MNode temp = mNode;
+      while (temp.getParent() != null) {
+        temp = temp.getParent();
+        fullPath.add(0, temp.getName());
       }
-      String nodePath = node.getParent().getFullPath() + TsFileConstant.PATH_SEPARATOR + nodeName;
       String[] tsRow = new String[8];
       tsRow[0] = nodePath;
-      tsRow[1] = ((MeasurementMNode) node).getAlias();
-      MeasurementSchema measurementSchema = ((MeasurementMNode) node).getSchema();
-      tsRow[2] = getStorageGroupName(nodePath);
+      tsRow[1] = ((MeasurementMNode) mNode).getAlias();
+      MeasurementSchema measurementSchema = ((MeasurementMNode) mNode).getSchema();
+      tsRow[2] = getStorageGroup(fullPath);
       tsRow[3] = measurementSchema.getType().toString();
       tsRow[4] = measurementSchema.getEncodingType().toString();
       tsRow[5] = measurementSchema.getCompressor().toString();
-      tsRow[6] = String.valueOf(((MeasurementMNode) node).getOffset());
+      tsRow[6] = String.valueOf(((MeasurementMNode) mNode).getOffset());
       tsRow[7] =
-          needLast ? String.valueOf(getLastTimeStamp((MeasurementMNode) node, queryContext)) : null;
+          needLast ? String.valueOf(getLastTimeStamp((MeasurementMNode) mNode, queryContext)) : null;
       timeseriesSchemaList.add(tsRow);
 
       if (hasLimit) {
         count.set(count.get() + 1);
       }
     }
-    String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
+    String nodeReg = MetaUtils.getNodeNameRegByIdx(idx, detachedPath);
     if (!nodeReg.contains(PATH_WILDCARD)) {
-      if (node.hasChild(nodeReg)) {
-        findPath(node.getChild(nodeReg), nodes, idx + 1, timeseriesSchemaList, hasLimit, needLast,
+      if (mNode.hasChild(nodeReg)) {
+        findPath(mNode.getChild(nodeReg), detachedPath, idx + 1, timeseriesSchemaList, hasLimit, needLast,
             queryContext);
       }
     } else {
-      for (MNode child : node.getChildren().values()) {
+      for (MNode child : mNode.getChildren().values()) {
         if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
           continue;
         }
-        findPath(child, nodes, idx + 1, timeseriesSchemaList, hasLimit, needLast, queryContext);
-        if (hasLimit) {
-          if (count.get().intValue() == limit.get().intValue()) {
-            return;
-          }
+        findPath(child, detachedPath, idx + 1, timeseriesSchemaList, hasLimit, needLast, queryContext);
+        if (hasLimit && count.get().intValue() == limit.get().intValue()) {
+          return;
+        }
+      }
+    }
+  }
+
+
+  /**
+   * Iterate through MTree to fetch metadata info of all leaf detachedPath under the given seriesPath
+   *
+   * @param measurementMNodes List<MeasurementMNode>
+   */
+  private void findMeasurementMNode(MNode mNode, String[] detachedPath, int idx,
+      List<MeasurementMNode> measurementMNodes,
+      boolean hasLimit) {
+    if (mNode instanceof MeasurementMNode && detachedPath.length <= idx) {
+      if (hasLimit) {
+        curOffset.set(curOffset.get() + 1);
+        if (curOffset.get() < offset.get() || count.get().intValue() == limit.get().intValue()) {
+          return;
+        }
+      }
+      measurementMNodes.add((MeasurementMNode) mNode);
+      if (hasLimit) {
+        count.set(count.get() + 1);
+      }
+    }
+    String nodeNameReg = MetaUtils.getNodeNameRegByIdx(idx, detachedPath);
+    if (!nodeNameReg.contains(PATH_WILDCARD)) {
+      if (mNode.hasChild(nodeNameReg)) {
+        findMeasurementMNode(mNode.getChild(nodeNameReg), detachedPath, idx + 1, measurementMNodes, hasLimit);
+      }
+    } else {
+      for (MNode child : mNode.getChildren().values()) {
+        if (!Pattern.matches(nodeNameReg.replace("*", ".*"), child.getName())) {
+          continue;
+        }
+        findMeasurementMNode(child, detachedPath, idx + 1, measurementMNodes, hasLimit);
+        if (hasLimit && count.get().intValue() == limit.get().intValue()) {
+          return;
+        }
+      }
+    }
+  }
+
+  /**
+   * Iterate through MTree to fetch metadata info of all leaf detachedPath under the given seriesPath
+   *
+   * @param timeseriesSchemaList List<timeseriesSchema> result: [[root,node,node], [alias, storage group,
+   *                             dataType, encoding, compression, offset, lastTimeStamp]]
+   */
+  private void findDetachedPathAndSchema(MNode mNode, String[] detachedPath, int idx,
+      List<List<String>> timeseriesSchemaList,
+      boolean hasLimit, boolean needLast, QueryContext queryContext) throws StorageGroupNotSetException {
+    if (mNode instanceof MeasurementMNode && detachedPath.length <= idx) {
+      if (hasLimit) {
+        curOffset.set(curOffset.get() + 1);
+        if (curOffset.get() < offset.get() || count.get().intValue() == limit.get().intValue()) {
+          return;
+        }
+      }
+      List<String> detachedFullPath = new ArrayList<>();
+      detachedFullPath.add(0, mNode.getName());
+      MNode tempNode = mNode;
+      while(!tempNode.getName().equals(IoTDBConstant.PATH_ROOT)) {
+        tempNode = tempNode.getParent();
+        detachedFullPath.add(0, tempNode.getName());
+      }
+      timeseriesSchemaList.add(detachedFullPath);
+      List<String> others = new ArrayList<>(7);
+      others.add(((MeasurementMNode) mNode).getAlias());
+      MeasurementSchema measurementSchema = ((MeasurementMNode) mNode).getSchema();
+      others.add(getStorageGroup(detachedFullPath));
+      others.add(measurementSchema.getType().toString());
+      others.add(measurementSchema.getEncodingType().toString());
+      others.add(measurementSchema.getCompressor().toString());
+      others.add(String.valueOf(((MeasurementMNode) mNode).getOffset()));
+      String last =  needLast ? String.valueOf(getLastTimeStamp((MeasurementMNode) mNode, queryContext)) : null;
+      others.add(last);
+      timeseriesSchemaList.add(others);
+      if (hasLimit) {
+        count.set(count.get() + 1);
+      }
+    }
+    String nodeNameReg = MetaUtils.getNodeNameRegByIdx(idx, detachedPath);
+    if (!nodeNameReg.contains(PATH_WILDCARD)) {
+      if (mNode.hasChild(nodeNameReg)) {
+        findDetachedPathAndSchema(mNode.getChild(nodeNameReg), detachedPath, idx + 1, timeseriesSchemaList, hasLimit, needLast, queryContext);
+      }
+    } else {
+      for (MNode child : mNode.getChildren().values()) {
+        if (!Pattern.matches(nodeNameReg.replace("*", ".*"), child.getName())) {
+          continue;
+        }
+        findDetachedPathAndSchema(child, detachedPath, idx + 1, timeseriesSchemaList, hasLimit, needLast, queryContext);
+        if (hasLimit && count.get().intValue() == limit.get().intValue()) {
+          return;
         }
       }
     }
   }
 
-  static long getLastTimeStamp(MeasurementMNode node, QueryContext queryContext) {
-    TimeValuePair last = node.getCachedLast();
+  static long getLastTimeStamp(MeasurementMNode mNode, QueryContext queryContext) {
+    TimeValuePair last = mNode.getCachedLast();
     if (last != null) {
-      return node.getCachedLast().getTimestamp();
+      return mNode.getCachedLast().getTimestamp();
     } else {
       try {
-        last = calculateLastPairForOneSeriesLocally(new Path(node.getFullPath()),
-            node.getSchema().getType(), queryContext, Collections.emptySet());
+        MNode temp = mNode;
+        List<String> detachedPath = new ArrayList<>();
+        detachedPath.add(temp.getName());
+        while (temp.getParent() != null) {
+          temp = temp.getParent();
+          detachedPath.add(0, temp.getName());
+        }

Review comment:
       This block is used frequently, how about making it a method in MNode and possibly caching the detachedPath of an MNode?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org