You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by xi...@apache.org on 2022/05/10 09:21:33 UTC

[iotdb] branch master updated: [IOTDB-3127] Fix tree traverse algorithm (#5842)

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

xingtanzjr 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 f0baa7dbfb [IOTDB-3127] Fix tree traverse algorithm (#5842)
f0baa7dbfb is described below

commit f0baa7dbfbec29212335ab69f48c6284fca1f27c
Author: Marcos_Zyk <38...@users.noreply.github.com>
AuthorDate: Tue May 10 17:21:27 2022 +0800

    [IOTDB-3127] Fix tree traverse algorithm (#5842)
---
 .../commons/schema}/tree/AbstractTreeVisitor.java  | 290 ++++++++++++++++-----
 .../tree/AbstractTreeVisitorWithLimitOffset.java   |   7 +-
 .../iotdb/commons/schema}/tree/ITreeNode.java      |   2 +-
 .../db/mpp/common/schematree/node/SchemaNode.java  |   2 +-
 .../visitor/SchemaTreeDeviceVisitor.java           |   7 +-
 .../visitor/SchemaTreeMeasurementVisitor.java      |  11 +-
 .../schematree/visitor/SchemaTreeVisitor.java      |   6 +-
 .../db/mpp/common/schematree/SchemaTreeTest.java   |  17 ++
 8 files changed, 262 insertions(+), 80 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/tree/AbstractTreeVisitor.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java
similarity index 51%
rename from server/src/main/java/org/apache/iotdb/db/metadata/tree/AbstractTreeVisitor.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java
index 8685d6ad81..dfb0c7acfe 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/tree/AbstractTreeVisitor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.metadata.tree;
+package org.apache.iotdb.commons.schema.tree;
 
 import org.apache.iotdb.commons.path.PartialPath;
 
@@ -61,14 +61,20 @@ import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCAR
  */
 public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Iterator<R> {
 
+  // command parameters
   protected final N root;
   protected final String[] nodes;
   protected final boolean isPrefixMatch;
 
+  // run time variables
   protected final Deque<VisitorStackEntry<N>> visitorStack = new ArrayDeque<>();
-  protected final Deque<N> ancestorStack = new ArrayDeque<>();
+  protected final Deque<AncestorStackEntry<N>> ancestorStack = new ArrayDeque<>();
+  protected boolean shouldVisitSubtree;
 
+  // result variables
   protected N nextMatchedNode;
+  protected int patternIndexOfMatchedNode;
+  protected int lastMultiLevelWildcardIndexOfMatchedNode;
 
   protected AbstractTreeVisitor(N root, PartialPath pathPattern, boolean isPrefixMatch) {
     this.root = root;
@@ -81,7 +87,8 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
 
   /**
    * Optimize the given path pattern. Currently, the node name used for one level match will be
-   * transformed into a regex.
+   * transformed into a regex. e.g. given pathPattern {"root", "sg", "d*", "s"} and the
+   * optimizedPathPattern is {"root", "sg", "d.*", "s"}.
    */
   private String[] optimizePathPattern(PartialPath pathPattern) {
     String[] rawNodes = pathPattern.getNodes();
@@ -112,7 +119,24 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
     if (!hasNext()) {
       throw new NoSuchElementException();
     }
+    return consumeNextMatchedNode();
+  }
+
+  private R consumeNextMatchedNode() {
     R result = generateResult();
+
+    // after the node be consumed, the subTree should be considered.
+    if (patternIndexOfMatchedNode == nodes.length) {
+      pushChildrenWhilePrefixMatch(
+          nextMatchedNode, patternIndexOfMatchedNode, lastMultiLevelWildcardIndexOfMatchedNode);
+    } else if (patternIndexOfMatchedNode == nodes.length - 1) {
+      pushChildrenWhileTail(
+          nextMatchedNode, patternIndexOfMatchedNode, lastMultiLevelWildcardIndexOfMatchedNode);
+    } else {
+      pushChildrenWhileInternal(
+          nextMatchedNode, patternIndexOfMatchedNode, lastMultiLevelWildcardIndexOfMatchedNode);
+    }
+
     nextMatchedNode = null;
     return result;
   }
@@ -156,66 +180,45 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
 
       // only prefixMatch
       if (patternIndex == nodes.length) {
-        if (processFullMatchedNode(node)) {
-          return;
-        }
 
-        if (!isLeafNode(node)) {
-          pushAllChildren(node, patternIndex, lastMultiLevelWildcardIndex);
-        }
+        shouldVisitSubtree = processFullMatchedNode(node) && isInternalNode(node);
 
         if (nextMatchedNode != null) {
+          saveNextMatchedNodeContext(patternIndex, lastMultiLevelWildcardIndex);
           return;
         }
 
+        if (shouldVisitSubtree) {
+          pushChildrenWhilePrefixMatch(node, patternIndex, lastMultiLevelWildcardIndex);
+        }
+
         continue;
       }
 
       if (checkIsMatch(patternIndex, node)) {
         if (patternIndex == nodes.length - 1) {
-          if (processFullMatchedNode(node)) {
+          shouldVisitSubtree = processFullMatchedNode(node) && isInternalNode(node);
+
+          if (nextMatchedNode != null) {
+            saveNextMatchedNodeContext(patternIndex, lastMultiLevelWildcardIndex);
             return;
           }
 
-          if (!isLeafNode(node)) {
-            if (nodes[patternIndex].equals(MULTI_LEVEL_PATH_WILDCARD)) {
-              pushAllChildren(node, patternIndex, patternIndex);
-            } else if (lastMultiLevelWildcardIndex != -1) {
-              pushAllChildren(
-                  node,
-                  findLastMatch(node, patternIndex, lastMultiLevelWildcardIndex) + 1,
-                  lastMultiLevelWildcardIndex);
-            } else if (isPrefixMatch) {
-              pushAllChildren(node, patternIndex + 1, lastMultiLevelWildcardIndex);
-            }
+          if (shouldVisitSubtree) {
+            pushChildrenWhileTail(node, patternIndex, lastMultiLevelWildcardIndex);
           }
+        } else {
+          shouldVisitSubtree = processInternalMatchedNode(node) && isInternalNode(node);
 
           if (nextMatchedNode != null) {
+            saveNextMatchedNodeContext(patternIndex, lastMultiLevelWildcardIndex);
             return;
           }
 
-          continue;
-        }
-
-        if (processInternalMatchedNode(node)) {
-          return;
-        }
-
-        if (!isLeafNode(node)) {
-          if (nodes[patternIndex + 1].equals(MULTI_LEVEL_PATH_WILDCARD)) {
-            pushAllChildren(node, patternIndex + 1, patternIndex + 1);
-          } else {
-            if (lastMultiLevelWildcardIndex > -1) {
-              pushAllChildren(node, patternIndex + 1, lastMultiLevelWildcardIndex);
-            } else if (nodes[patternIndex + 1].contains(ONE_LEVEL_PATH_WILDCARD)) {
-              pushAllChildren(node, patternIndex + 1, lastMultiLevelWildcardIndex);
-            } else {
-              pushSingleChild(
-                  node, nodes[patternIndex + 1], patternIndex + 1, lastMultiLevelWildcardIndex);
-            }
+          if (shouldVisitSubtree) {
+            pushChildrenWhileInternal(node, patternIndex, lastMultiLevelWildcardIndex);
           }
         }
-
       } else {
         if (lastMultiLevelWildcardIndex == -1) {
           continue;
@@ -223,37 +226,72 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
 
         int lastMatchIndex = findLastMatch(node, patternIndex, lastMultiLevelWildcardIndex);
 
-        if (processInternalMatchedNode(node)) {
+        shouldVisitSubtree = processInternalMatchedNode(node) && isInternalNode(node);
+
+        if (nextMatchedNode != null) {
+          saveNextMatchedNodeContext(lastMatchIndex, lastMultiLevelWildcardIndex);
           return;
         }
 
-        if (!isLeafNode(node)) {
-          pushAllChildren(node, lastMatchIndex + 1, lastMultiLevelWildcardIndex);
+        if (shouldVisitSubtree) {
+          pushChildrenWhileInternal(node, lastMatchIndex, lastMultiLevelWildcardIndex);
         }
       }
-
-      if (nextMatchedNode != null) {
-        return;
-      }
     }
   }
 
+  /**
+   * The context, mainly the matching info, of nextedMatchedNode should be saved. When the
+   * nextedMatchedNode is consumed, the saved info will be used to process its subtree.
+   */
+  private void saveNextMatchedNodeContext(
+      int patternIndexOfMatchedNode, int lastMultiLevelWildcardIndexOfMatchedNode) {
+    this.patternIndexOfMatchedNode = patternIndexOfMatchedNode;
+    this.lastMultiLevelWildcardIndexOfMatchedNode = lastMultiLevelWildcardIndexOfMatchedNode;
+  }
+
+  /**
+   * When current node cannot match the pattern node in nodes[patternIndex] and there is ** before
+   * current pattern node, the pattern nodes before current pattern node should be checked. For
+   * example, given path root.sg.d.s and path pattern root.**.s. A status, root.sg.d not match
+   * root.**.s, may be reached during traversing process, then it should be checked and found that
+   * root.sg.d could match root.**, after which the process could continue and find root.sg.d.s
+   * matches root.**.s.
+   */
   private int findLastMatch(N node, int patternIndex, int lastMultiLevelWildcardIndex) {
     for (int i = patternIndex - 1; i > lastMultiLevelWildcardIndex; i--) {
       if (!checkIsMatch(i, node)) {
         continue;
       }
 
-      Iterator<N> ancestors = ancestorStack.iterator();
+      Iterator<AncestorStackEntry<N>> ancestors = ancestorStack.iterator();
       boolean allMatch = true;
+      AncestorStackEntry<N> ancestor;
       for (int j = i - 1; j > lastMultiLevelWildcardIndex; j--) {
-        if (!checkIsMatch(j, ancestors.next())) {
+        ancestor = ancestors.next();
+        if (ancestor.isMatched(j)) {
+          break;
+        }
+
+        if (ancestor.hasBeenChecked(j) || !checkIsMatch(j, ancestor.node)) {
+          ancestors = ancestorStack.iterator();
+          for (int k = i - 1; k >= j; k--) {
+            ancestors.next().setNotMatched(k);
+          }
           allMatch = false;
           break;
         }
       }
 
       if (allMatch) {
+        ancestors = ancestorStack.iterator();
+        for (int k = i - 1; k > lastMultiLevelWildcardIndex; k--) {
+          ancestor = ancestors.next();
+          if (ancestor.isMatched(k)) {
+            break;
+          }
+          ancestor.setMatched(k);
+        }
         return i;
       }
     }
@@ -277,11 +315,78 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
     }
   }
 
-  protected void pushSingleChild(
-      N parent, String childName, int patternIndex, int lastMultiLevelWildcardIndex) {
-    N child = getChild(parent, childName);
+  /**
+   * This method is invoked to decide how to push children of given node. Invoked only when
+   * patternIndex == nodes.length.
+   *
+   * @param node the current processed node
+   * @param patternIndex the patternIndex for the given node
+   * @param lastMultiLevelWildcardIndex the lastMultiLevelWildcardIndex of the given node
+   */
+  private void pushChildrenWhilePrefixMatch(
+      N node, int patternIndex, int lastMultiLevelWildcardIndex) {
+    pushAllChildren(node, patternIndex, lastMultiLevelWildcardIndex);
+  }
+
+  /**
+   * This method is invoked to decide how to push children of given node. Invoked only when
+   * patternIndex == nodes.length - 1.
+   *
+   * @param node the current processed node
+   * @param patternIndex the patternIndex for the given node
+   * @param lastMultiLevelWildcardIndex the lastMultiLevelWildcardIndex of the given node
+   */
+  private void pushChildrenWhileTail(N node, int patternIndex, int lastMultiLevelWildcardIndex) {
+    if (nodes[patternIndex].equals(MULTI_LEVEL_PATH_WILDCARD)) {
+      pushAllChildren(node, patternIndex, patternIndex);
+    } else if (lastMultiLevelWildcardIndex != -1) {
+      pushAllChildren(
+          node,
+          findLastMatch(node, patternIndex, lastMultiLevelWildcardIndex) + 1,
+          lastMultiLevelWildcardIndex);
+    } else if (isPrefixMatch) {
+      pushAllChildren(node, patternIndex + 1, lastMultiLevelWildcardIndex);
+    }
+  }
+
+  /**
+   * This method is invoked to decide how to push children of given node. Invoked only when
+   * patternIndex < nodes.length - 1.
+   *
+   * @param node the current processed node
+   * @param patternIndex the patternIndex for the given node
+   * @param lastMultiLevelWildcardIndex the lastMultiLevelWildcardIndex of the given node
+   */
+  private void pushChildrenWhileInternal(
+      N node, int patternIndex, int lastMultiLevelWildcardIndex) {
+    if (nodes[patternIndex + 1].equals(MULTI_LEVEL_PATH_WILDCARD)) {
+      pushAllChildren(node, patternIndex + 1, patternIndex + 1);
+    } else {
+      if (lastMultiLevelWildcardIndex > -1) {
+        pushAllChildren(node, patternIndex + 1, lastMultiLevelWildcardIndex);
+      } else if (nodes[patternIndex + 1].contains(ONE_LEVEL_PATH_WILDCARD)) {
+        pushAllChildren(node, patternIndex + 1, lastMultiLevelWildcardIndex);
+      } else {
+        pushSingleChild(node, patternIndex + 1, lastMultiLevelWildcardIndex);
+      }
+    }
+  }
+
+  /**
+   * Push child for name match case.
+   *
+   * @param parent the parent node of target children
+   * @param patternIndex the patternIndex to match children
+   * @param lastMultiLevelWildcardIndex the lastMultiLevelWildcardIndex of child
+   */
+  protected void pushSingleChild(N parent, int patternIndex, int lastMultiLevelWildcardIndex) {
+    N child = getChild(parent, nodes[patternIndex]);
     if (child != null) {
-      ancestorStack.push(parent);
+      ancestorStack.push(
+          new AncestorStackEntry<>(
+              parent,
+              visitorStack.peek().patternIndex,
+              visitorStack.peek().lastMultiLevelWildcardIndex));
       visitorStack.push(
           new VisitorStackEntry<>(
               Collections.singletonList(child).iterator(),
@@ -291,8 +396,25 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
     }
   }
 
+  /**
+   * Push children for the following cases:
+   *
+   * <ol>
+   *   <li>the pattern to match children is **, multiLevelWildcard
+   *   <li>the pattern to match children contains *, oneLevelWildcard
+   *   <li>there's ** before the patternIndex for children
+   * </ol>
+   *
+   * @param parent the parent node of target children
+   * @param patternIndex the patternIndex to match children
+   * @param lastMultiLevelWildcardIndex the lastMultiLevelWildcardIndex of child
+   */
   protected void pushAllChildren(N parent, int patternIndex, int lastMultiLevelWildcardIndex) {
-    ancestorStack.push(parent);
+    ancestorStack.push(
+        new AncestorStackEntry<>(
+            parent,
+            visitorStack.peek().patternIndex,
+            visitorStack.peek().lastMultiLevelWildcardIndex));
     visitorStack.push(
         new VisitorStackEntry<>(
             getChildrenIterator(parent),
@@ -321,16 +443,19 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
 
   protected String[] generateFullPathNodes(N node) {
     List<String> nodeNames = new ArrayList<>();
-    Iterator<N> iterator = ancestorStack.descendingIterator();
+    Iterator<AncestorStackEntry<N>> iterator = ancestorStack.descendingIterator();
     while (iterator.hasNext()) {
-      nodeNames.add(iterator.next().getName());
+      nodeNames.add(iterator.next().node.getName());
     }
     nodeNames.add(node.getName());
     return nodeNames.toArray(new String[0]);
   }
 
-  // Check whether the given node is a leaf node of this tree.
-  protected abstract boolean isLeafNode(N node);
+  /**
+   * Check whether the given node is an internal node of this tree. Return true if the given node is
+   * an internal node. Return false if the given node is a leaf node.
+   */
+  protected abstract boolean isInternalNode(N node);
 
   // Get a child with the given childName.
   protected abstract N getChild(N parent, String childName);
@@ -343,9 +468,9 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
    * internal match root.sg.**(pattern). This method should be implemented according to concrete
    * tasks.
    *
-   * <p>If return true, the traversing process won't check the subtree with the given node as root,
-   * and the result will be return immediately. If return false, the traversing process will keep
-   * traversing the subtree.
+   * <p>Return whether the subtree of given node should be processed. If return true, the traversing
+   * process will keep traversing the subtree. If return false, the traversing process will skip the
+   * subtree of given node.
    */
   protected abstract boolean processInternalMatchedNode(N node);
 
@@ -353,9 +478,9 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
    * Full-match means the node matches the last node name of the given path pattern. root.sg.d full
    * match root.sg.**(pattern) This method should be implemented according to concrete tasks.
    *
-   * <p>If return true, the traversing process won't check the subtree with the given node as root,
-   * and the result will be return immediately. f return false, the traversing process will keep
-   * traversing the subtree.
+   * <p>Return whether the subtree of given node should be processed. If return true, the traversing
+   * process will keep traversing the subtree. If return false, the traversing process will skip the
+   * subtree of given node.
    */
   protected abstract boolean processFullMatchedNode(N node);
 
@@ -377,4 +502,39 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
       this.lastMultiLevelWildcardIndex = lastMultiLevelWildcardIndex;
     }
   }
+
+  protected static class AncestorStackEntry<N> {
+    private final N node;
+    private final int matchedIndex;
+    /** Record the check result to reduce repeating check. */
+    private final byte[] matchStatus;
+
+    AncestorStackEntry(N node, int matchedIndex, int lastMultiLevelWildcardIndex) {
+      this.node = node;
+      this.matchedIndex = matchedIndex;
+      matchStatus = new byte[matchedIndex - lastMultiLevelWildcardIndex + 1];
+      matchStatus[0] = 1;
+      matchStatus[matchStatus.length - 1] = 1;
+    }
+
+    public N getNode() {
+      return node;
+    }
+
+    boolean hasBeenChecked(int index) {
+      return matchStatus[matchedIndex - index] != 0;
+    }
+
+    boolean isMatched(int index) {
+      return matchStatus[matchedIndex - index] == 1;
+    }
+
+    void setMatched(int index) {
+      matchStatus[matchedIndex - index] = 1;
+    }
+
+    void setNotMatched(int index) {
+      matchStatus[matchedIndex - index] = -1;
+    }
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/tree/AbstractTreeVisitorWithLimitOffset.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitorWithLimitOffset.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/metadata/tree/AbstractTreeVisitorWithLimitOffset.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitorWithLimitOffset.java
index fd6165c548..122231eb6f 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/tree/AbstractTreeVisitorWithLimitOffset.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitorWithLimitOffset.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.metadata.tree;
+package org.apache.iotdb.commons.schema.tree;
 
 import org.apache.iotdb.commons.path.PartialPath;
 
@@ -98,9 +98,4 @@ public abstract class AbstractTreeVisitorWithLimitOffset<N extends ITreeNode, R>
   public int getNextOffset() {
     return curOffset + 1;
   }
-
-  @Override
-  protected boolean processInternalMatchedNode(N node) {
-    return false;
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/tree/ITreeNode.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/ITreeNode.java
similarity index 94%
rename from server/src/main/java/org/apache/iotdb/db/metadata/tree/ITreeNode.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/ITreeNode.java
index f7300d282d..900e943fbd 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/tree/ITreeNode.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/ITreeNode.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.metadata.tree;
+package org.apache.iotdb.commons.schema.tree;
 
 public interface ITreeNode {
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/node/SchemaNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/node/SchemaNode.java
index 069cbe02c8..99e76b0a62 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/node/SchemaNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/node/SchemaNode.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.mpp.common.schematree.node;
 
-import org.apache.iotdb.db.metadata.tree.ITreeNode;
+import org.apache.iotdb.commons.schema.tree.ITreeNode;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeDeviceVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeDeviceVisitor.java
index 2a492fea18..6379a7ada3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeDeviceVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeDeviceVisitor.java
@@ -34,12 +34,17 @@ public class SchemaTreeDeviceVisitor extends SchemaTreeVisitor<DeviceSchemaInfo>
     super(root, pathPattern, 0, 0, isPrefixMatch);
   }
 
+  @Override
+  protected boolean processInternalMatchedNode(SchemaNode node) {
+    return true;
+  }
+
   @Override
   protected boolean processFullMatchedNode(SchemaNode node) {
     if (node.isEntity()) {
       nextMatchedNode = node;
     }
-    return false;
+    return true;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeMeasurementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeMeasurementVisitor.java
index ac9e7ec974..f86a22ccb3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeMeasurementVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeMeasurementVisitor.java
@@ -54,13 +54,18 @@ public class SchemaTreeMeasurementVisitor extends SchemaTreeVisitor<MeasurementP
     return targetName.equals(node.getName());
   }
 
+  @Override
+  protected boolean processInternalMatchedNode(SchemaNode node) {
+    return true;
+  }
+
   @Override
   protected boolean processFullMatchedNode(SchemaNode node) {
     if (node.isMeasurement()) {
       nextMatchedNode = node;
-      return true;
+      return false;
     }
-    return false;
+    return true;
   }
 
   @Override
@@ -69,7 +74,7 @@ public class SchemaTreeMeasurementVisitor extends SchemaTreeVisitor<MeasurementP
         new MeasurementPath(
             generateFullPathNodes(nextMatchedNode),
             nextMatchedNode.getAsMeasurementNode().getSchema());
-    result.setUnderAlignedEntity(ancestorStack.peek().getAsEntityNode().isAligned());
+    result.setUnderAlignedEntity(ancestorStack.peek().getNode().getAsEntityNode().isAligned());
     String alias = nextMatchedNode.getAsMeasurementNode().getAlias();
     if (nodes[nodes.length - 1].equals(alias)) {
       result.setMeasurementAlias(alias);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java
index 9fa102d9a3..848cacc213 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.mpp.common.schematree.visitor;
 
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.metadata.tree.AbstractTreeVisitorWithLimitOffset;
+import org.apache.iotdb.commons.schema.tree.AbstractTreeVisitorWithLimitOffset;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
 
 import java.util.ArrayList;
@@ -44,8 +44,8 @@ public abstract class SchemaTreeVisitor<R>
   }
 
   @Override
-  protected boolean isLeafNode(SchemaNode node) {
-    return node.isMeasurement();
+  protected boolean isInternalNode(SchemaNode node) {
+    return !node.isMeasurement();
   }
 
   @Override
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java
index 4bad672aa4..23a5446c91 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/SchemaTreeTest.java
@@ -36,6 +36,7 @@ import org.mockito.internal.util.collections.Sets;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.List;
 import java.util.stream.Collectors;
 
@@ -435,10 +436,26 @@ public class SchemaTreeTest {
     Assert.assertEquals(2, deviceSchemaInfo.getMeasurements(Sets.newSet("*")).size());
 
     deviceSchemaInfoList = schemaTree.getMatchedDevices(new PartialPath("root.sg.*"), false);
+    deviceSchemaInfoList.sort(Comparator.comparing(DeviceSchemaInfo::getDevicePath));
     Assert.assertEquals(2, deviceSchemaInfoList.size());
+    Assert.assertEquals(new PartialPath("root.sg.d1"), deviceSchemaInfoList.get(0).getDevicePath());
+    Assert.assertEquals(new PartialPath("root.sg.d2"), deviceSchemaInfoList.get(1).getDevicePath());
 
     deviceSchemaInfoList = schemaTree.getMatchedDevices(new PartialPath("root.sg.**"), false);
+    deviceSchemaInfoList.sort(Comparator.comparing(DeviceSchemaInfo::getDevicePath));
     Assert.assertEquals(3, deviceSchemaInfoList.size());
+    Assert.assertEquals(new PartialPath("root.sg.d1"), deviceSchemaInfoList.get(0).getDevicePath());
+    Assert.assertEquals(new PartialPath("root.sg.d2"), deviceSchemaInfoList.get(1).getDevicePath());
+    Assert.assertEquals(
+        new PartialPath("root.sg.d2.a"), deviceSchemaInfoList.get(2).getDevicePath());
+
+    deviceSchemaInfoList = schemaTree.getMatchedDevices(new PartialPath("root.**"), false);
+    deviceSchemaInfoList.sort(Comparator.comparing(DeviceSchemaInfo::getDevicePath));
+    Assert.assertEquals(3, deviceSchemaInfoList.size());
+    Assert.assertEquals(new PartialPath("root.sg.d1"), deviceSchemaInfoList.get(0).getDevicePath());
+    Assert.assertEquals(new PartialPath("root.sg.d2"), deviceSchemaInfoList.get(1).getDevicePath());
+    Assert.assertEquals(
+        new PartialPath("root.sg.d2.a"), deviceSchemaInfoList.get(2).getDevicePath());
   }
 
   @Test