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

[iotdb] branch master updated: [IOTDB-5342][IOTDB-5359][IOTDB-5360] Some optimization in AbstractTreeVisitor (#8721)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7c9395de11 [IOTDB-5342][IOTDB-5359][IOTDB-5360] Some optimization in AbstractTreeVisitor (#8721)
7c9395de11 is described below

commit 7c9395de1127f469a7a4d8b6b61b63cfc5d71b28
Author: Chen YZ <43...@users.noreply.github.com>
AuthorDate: Wed Jan 4 22:13:26 2023 +0800

    [IOTDB-5342][IOTDB-5359][IOTDB-5360] Some optimization in AbstractTreeVisitor (#8721)
---
 .../commons/schema/tree/AbstractTreeVisitor.java   | 112 ++++++++++++-------
 .../tree/AbstractTreeVisitorWithLimitOffset.java   | 101 -----------------
 .../mpp/common/schematree/ClusterSchemaTree.java   |  22 ++--
 .../visitor/SchemaTreeDeviceVisitor.java           |  15 ++-
 .../visitor/SchemaTreeMeasurementVisitor.java      |  18 ++--
 .../schematree/visitor/SchemaTreeVisitor.java      |  22 ++--
 ...eVisitor.java => SchemaTreeVisitorFactory.java} |  43 +++-----
 .../SchemaTreeVisitorWithLimitOffsetWrapper.java   | 120 +++++++++++++++++++++
 .../common/schematree/ClusterSchemaTreeTest.java   | 110 +++++++++++++------
 .../ClusterSchemaTreeTestWithRelease.java}         |  42 +++-----
 .../MockSchemaTreeMeasurementVisitor.java          | 105 ++++++++++++++++++
 11 files changed, 452 insertions(+), 258 deletions(-)

diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java
index 2a92d8126c..b752728af7 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java
@@ -90,7 +90,13 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
   private Throwable throwable;
 
   // cached result variables
-  protected N nextMatchedNode;
+  private N nextMatchedNode;
+
+  // only used for wrapper
+  protected AbstractTreeVisitor() {
+    root = null;
+    patternFA = null;
+  }
 
   protected AbstractTreeVisitor(N root, PartialPath pathPattern, boolean isPrefixMatch) {
     this.root = root;
@@ -139,6 +145,7 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
   }
 
   public void reset() {
+    close();
     visitorStack.clear();
     ancestorStack.clear();
     nextMatchedNode = null;
@@ -148,6 +155,10 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
 
   @Override
   public void close() {
+    if (nextMatchedNode != null && !shouldVisitSubtree) {
+      // release nextMatchedNode
+      releaseNode(nextMatchedNode);
+    }
     while (!visitorStack.isEmpty()) {
       popStack();
     }
@@ -170,15 +181,18 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
     if (!hasNext()) {
       throw new NoSuchElementException();
     }
-    R result = generateResult();
+    R result = generateResult(nextMatchedNode);
+    if (!shouldVisitSubtree) {
+      // release nextMatchedNode
+      releaseNode(nextMatchedNode);
+    }
     nextMatchedNode = null;
     return result;
   }
 
-  protected void getNext() throws Exception {
+  private void getNext() {
     nextMatchedNode = null;
     VisitorStackEntry stackEntry;
-    N node;
     Iterator<N> iterator;
     while (!visitorStack.isEmpty()) {
       stackEntry = visitorStack.peek();
@@ -189,17 +203,28 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
         continue;
       }
 
-      node = iterator.next();
+      N nextTempNode = iterator.next();
 
       if (currentStateMatchInfo.hasFinalState()) {
-        shouldVisitSubtree = processFullMatchedNode(node);
+        if (acceptFullMatchedNode(nextTempNode)) {
+          nextMatchedNode = nextTempNode;
+        }
+        shouldVisitSubtree = shouldVisitSubtreeOfFullMatchedNode(nextTempNode);
       } else {
-        shouldVisitSubtree = processInternalMatchedNode(node);
+        if (acceptInternalMatchedNode(nextTempNode)) {
+          nextMatchedNode = nextTempNode;
+        }
+        shouldVisitSubtree = shouldVisitSubtreeOfInternalMatchedNode(nextTempNode);
       }
 
       if (shouldVisitSubtree) {
-        pushChildren(node);
+        pushChildren(nextTempNode);
+        // After adding nextTempNode into ancestorStack, nextTempNode will be released finally.
+      } else if (nextMatchedNode != nextTempNode) {
+        // Else if nextTempNode is not accepted, it needs to be released.
+        releaseNode(nextTempNode);
       }
+      // Otherwise, it will be released when invoking next()
 
       if (nextMatchedNode != null) {
         return;
@@ -209,7 +234,9 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
 
   private void pushChildren(N parent) {
     visitorStack.push(
-        new VisitorStackEntry(createChildrenIterator(parent), visitorStack.peek().level + 1));
+        new VisitorStackEntry(
+            createChildrenIterator(parent),
+            visitorStack.isEmpty() ? 1 : visitorStack.peek().level + 1));
     ancestorStack.add(new AncestorStackEntry(parent, currentStateMatchInfo));
   }
 
@@ -239,7 +266,8 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
     // The ancestor pop operation with level check supports the children of one node pushed by
     // batch.
     if (!visitorStack.isEmpty() && visitorStack.peek().level < ancestorStack.size()) {
-      ancestorStack.remove(ancestorStack.size() - 1);
+      AncestorStackEntry ancestorStackEntry = ancestorStack.remove(ancestorStack.size() - 1);
+      releaseNode(ancestorStackEntry.node);
       if (ancestorStack.size() <= firstAncestorOfTraceback) {
         firstAncestorOfTraceback = -1;
       }
@@ -268,17 +296,22 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
     }
   }
 
+  protected void setFailure(Throwable e) {
+    this.throwable = e;
+  }
+
   // Get a child with the given childName.
   protected abstract N getChild(N parent, String childName) throws Exception;
 
   // Get an iterator of all children.
   protected abstract Iterator<N> getChildrenIterator(N parent) throws Exception;
 
-  // Release a child with the given childName.
-  protected void releaseChild(N child) {}
+  // Release a child node.
+  protected void releaseNode(N node) {}
 
-  // Release an iterator of all children.
-  protected void releaseChildrenIterator(Iterator<N> childrenIterator) {}
+  // Release an iterator. It is not necessary to deal with all the elements in the iterator.
+  // Only the elements that have been fetched but not returned by next() need to be released.
+  protected void releaseNodeIterator(Iterator<N> nodeIterator) {}
 
   /**
    * Internal-match means the node matches an internal node name of the given path pattern. root.sg
@@ -289,7 +322,7 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
    * 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) throws Exception;
+  protected abstract boolean shouldVisitSubtreeOfInternalMatchedNode(N node);
 
   /**
    * Full-match means the node matches the last node name of the given path pattern. root.sg.d full
@@ -299,21 +332,23 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
    * 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) throws Exception;
+  protected abstract boolean shouldVisitSubtreeOfFullMatchedNode(N node);
 
-  protected void setFailure(Throwable e) {
-    this.throwable = e;
-  }
+  /** Only accepted nodes will be considered for hasNext() and next() */
+  protected abstract boolean acceptInternalMatchedNode(N node);
+
+  /** Only accepted nodes will be considered for hasNext() and next() */
+  protected abstract boolean acceptFullMatchedNode(N node);
 
   /** The method used for generating the result based on the matched node. */
-  protected abstract R generateResult();
+  protected abstract R generateResult(N nextMatchedNode);
 
   private class VisitorStackEntry {
 
     // children iterator
     private final AbstractChildrenIterator iterator;
 
-    // level of children taken from iterator
+    // level of children taken from iterator, start from 1
     private final int level;
 
     VisitorStackEntry(AbstractChildrenIterator iterator, int level) {
@@ -368,7 +403,11 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
 
     protected abstract void getNext() throws Exception;
 
-    protected abstract void close();
+    protected void close() {
+      if (nextMatchedChild != null) {
+        releaseNode(nextMatchedChild);
+      }
+    }
   }
 
   // the child can be got directly with the precise value of transition, there's no traceback
@@ -377,8 +416,6 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
     private final IFAState sourceState;
     private final Iterator<IFATransition> transitionIterator;
 
-    private N child;
-
     private PreciseMatchChildrenIterator(N parent, IFAState sourceState) {
       this.parent = parent;
       this.sourceState = sourceState;
@@ -390,7 +427,7 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
       IFATransition transition;
       while (transitionIterator.hasNext()) {
         transition = transitionIterator.next();
-        child = getChild(parent, transition.getAcceptEvent());
+        N child = getChild(parent, transition.getAcceptEvent());
         if (child == null) {
           continue;
         }
@@ -400,11 +437,6 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
         return;
       }
     }
-
-    @Override
-    protected void close() {
-      releaseChild(child);
-    }
   }
 
   // only one fuzzy transition which may match batch children, need to iterate and check all
@@ -436,6 +468,7 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
       while (childrenIterator.hasNext()) {
         child = childrenIterator.next();
         if (tryGetNextState(child, sourceState, transition) == null) {
+          releaseNode(child);
           continue;
         }
         saveResult(child, stateMatchInfo);
@@ -445,7 +478,8 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
 
     @Override
     protected void close() {
-      releaseChildrenIterator(childrenIterator);
+      super.close();
+      releaseNodeIterator(childrenIterator);
     }
   }
 
@@ -493,6 +527,7 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
             }
           }
           if (matchedState == null) {
+            releaseNode(child);
             continue;
           }
         }
@@ -514,7 +549,8 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
 
     @Override
     protected void close() {
-      releaseChildrenIterator(iterator);
+      super.close();
+      releaseNodeIterator(iterator);
     }
   }
 
@@ -562,6 +598,7 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
         if (stateMatchInfo.getMatchedStateSize() == 0) {
           traceback(child, stateMatchInfo, sourceStateMatchInfo.getMatchedStateSize() - 1);
           if (stateMatchInfo.getMatchedStateSize() == 0) {
+            releaseNode(child);
             continue;
           }
         }
@@ -571,11 +608,6 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
       }
     }
 
-    @Override
-    protected void close() {
-      releaseChildrenIterator(iterator);
-    }
-
     /**
      * Try to get next matched state from sourceState and add it into currentStateMatchInfo
      *
@@ -700,6 +732,12 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R>
         }
       }
     }
+
+    @Override
+    protected void close() {
+      super.close();
+      releaseNodeIterator(iterator);
+    }
   }
 
   // the match process of FA graph is a dfs on FA Graph
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitorWithLimitOffset.java b/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitorWithLimitOffset.java
deleted file mode 100644
index 5208043ab8..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitorWithLimitOffset.java
+++ /dev/null
@@ -1,101 +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.commons.schema.tree;
-
-import org.apache.iotdb.commons.path.PartialPath;
-
-/**
- * This class defines a dfs-based traversing algorithm with limit and offset based on
- * AbstractTreeVisitor.
- *
- * <p>This class takes two extra parameters as input:
- *
- * <ol>
- *   <li>int limit: the max count of the results collected by one traversing process.
- *   <li>int offset: the index of first matched node to be collected.
- * </ol>
- */
-public abstract class AbstractTreeVisitorWithLimitOffset<N extends ITreeNode, R>
-    extends AbstractTreeVisitor<N, R> {
-
-  protected final int limit;
-  protected final int offset;
-  protected final boolean hasLimit;
-
-  protected int count = 0;
-  protected int curOffset = -1;
-
-  protected AbstractTreeVisitorWithLimitOffset(
-      N root, PartialPath pathPattern, int limit, int offset, boolean isPrefixMatch) {
-    super(root, pathPattern, isPrefixMatch);
-    this.limit = limit;
-    this.offset = offset;
-    hasLimit = limit != 0;
-  }
-
-  @Override
-  public boolean hasNext() {
-    if (hasLimit) {
-      return count < limit && super.hasNext();
-    }
-
-    return super.hasNext();
-  }
-
-  @Override
-  protected void getNext() throws Exception {
-    if (hasLimit) {
-      if (curOffset < offset) {
-        while (curOffset < offset) {
-          super.getNext();
-          curOffset += 1;
-          if (nextMatchedNode == null) {
-            return;
-          }
-        }
-      } else {
-        super.getNext();
-        curOffset += 1;
-      }
-    } else {
-      super.getNext();
-    }
-  }
-
-  @Override
-  public R next() {
-    R result = super.next();
-    if (hasLimit) {
-      count++;
-    }
-    return result;
-  }
-
-  @Override
-  public void reset() {
-    super.reset();
-    count = 0;
-    curOffset = -1;
-  }
-
-  public int getNextOffset() {
-    return curOffset + 1;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java
index b51247fcda..4b547c28ec 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTree.java
@@ -29,7 +29,8 @@ import org.apache.iotdb.db.mpp.common.schematree.node.SchemaInternalNode;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaMeasurementNode;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
 import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeDeviceVisitor;
-import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeMeasurementVisitor;
+import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeVisitorFactory;
+import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeVisitorWithLimitOffsetWrapper;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -73,20 +74,21 @@ public class ClusterSchemaTree implements ISchemaTree {
   @Override
   public Pair<List<MeasurementPath>, Integer> searchMeasurementPaths(
       PartialPath pathPattern, int slimit, int soffset, boolean isPrefixMatch) {
-    SchemaTreeMeasurementVisitor visitor =
-        new SchemaTreeMeasurementVisitor(root, pathPattern, slimit, soffset, isPrefixMatch);
+    SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath> visitor =
+        SchemaTreeVisitorFactory.createSchemaTreeMeasurementVisitor(
+            root, pathPattern, isPrefixMatch, slimit, soffset);
     return new Pair<>(visitor.getAllResult(), visitor.getNextOffset());
   }
 
   @Override
   public Pair<List<MeasurementPath>, Integer> searchMeasurementPaths(PartialPath pathPattern) {
-    SchemaTreeMeasurementVisitor visitor =
-        new SchemaTreeMeasurementVisitor(
+    SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath> visitor =
+        SchemaTreeVisitorFactory.createSchemaTreeMeasurementVisitor(
             root,
             pathPattern,
+            false,
             IoTDBDescriptor.getInstance().getConfig().getMaxQueryDeduplicatedPathNum() + 1,
-            0,
-            false);
+            0);
     return new Pair<>(visitor.getAllResult(), visitor.getNextOffset());
   }
 
@@ -102,13 +104,15 @@ public class ClusterSchemaTree implements ISchemaTree {
    */
   @Override
   public List<DeviceSchemaInfo> getMatchedDevices(PartialPath pathPattern, boolean isPrefixMatch) {
-    SchemaTreeDeviceVisitor visitor = new SchemaTreeDeviceVisitor(root, pathPattern, isPrefixMatch);
+    SchemaTreeDeviceVisitor visitor =
+        SchemaTreeVisitorFactory.createSchemaTreeDeviceVisitor(root, pathPattern, isPrefixMatch);
     return visitor.getAllResult();
   }
 
   @Override
   public List<DeviceSchemaInfo> getMatchedDevices(PartialPath pathPattern) {
-    SchemaTreeDeviceVisitor visitor = new SchemaTreeDeviceVisitor(root, pathPattern, false);
+    SchemaTreeDeviceVisitor visitor =
+        SchemaTreeVisitorFactory.createSchemaTreeDeviceVisitor(root, pathPattern, false);
     return visitor.getAllResult();
   }
 
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 792fc83273..59072f43c0 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
@@ -32,24 +32,21 @@ import java.util.List;
 public class SchemaTreeDeviceVisitor extends SchemaTreeVisitor<DeviceSchemaInfo> {
 
   public SchemaTreeDeviceVisitor(SchemaNode root, PartialPath pathPattern, boolean isPrefixMatch) {
-    super(root, pathPattern, 0, 0, isPrefixMatch);
+    super(root, pathPattern, isPrefixMatch);
   }
 
   @Override
-  protected boolean processInternalMatchedNode(SchemaNode node) {
-    return true;
+  protected boolean acceptInternalMatchedNode(SchemaNode node) {
+    return false;
   }
 
   @Override
-  protected boolean processFullMatchedNode(SchemaNode node) {
-    if (node.isEntity()) {
-      nextMatchedNode = node;
-    }
-    return true;
+  protected boolean acceptFullMatchedNode(SchemaNode node) {
+    return node.isEntity();
   }
 
   @Override
-  protected DeviceSchemaInfo generateResult() {
+  protected DeviceSchemaInfo generateResult(SchemaNode nextMatchedNode) {
     PartialPath path = new PartialPath(generateFullPathNodes());
     List<MeasurementSchemaInfo> measurementSchemaInfoList = new ArrayList<>();
     Iterator<SchemaNode> iterator = getChildrenIterator(nextMatchedNode);
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 e524349231..9dd71548b3 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
@@ -32,8 +32,8 @@ public class SchemaTreeMeasurementVisitor extends SchemaTreeVisitor<MeasurementP
   private final String tailNode;
 
   public SchemaTreeMeasurementVisitor(
-      SchemaNode root, PartialPath pathPattern, int slimit, int soffset, boolean isPrefixMatch) {
-    super(root, pathPattern, slimit, soffset, isPrefixMatch);
+      SchemaNode root, PartialPath pathPattern, boolean isPrefixMatch) {
+    super(root, pathPattern, isPrefixMatch);
     tailNode = pathPattern.getTailNode();
   }
 
@@ -98,21 +98,17 @@ public class SchemaTreeMeasurementVisitor extends SchemaTreeVisitor<MeasurementP
   }
 
   @Override
-  protected boolean processInternalMatchedNode(SchemaNode node) {
-    return true;
+  protected boolean acceptInternalMatchedNode(SchemaNode node) {
+    return false;
   }
 
   @Override
-  protected boolean processFullMatchedNode(SchemaNode node) {
-    if (node.isMeasurement()) {
-      nextMatchedNode = node;
-      return false;
-    }
-    return true;
+  protected boolean acceptFullMatchedNode(SchemaNode node) {
+    return node.isMeasurement();
   }
 
   @Override
-  protected MeasurementPath generateResult() {
+  protected MeasurementPath generateResult(SchemaNode nextMatchedNode) {
     MeasurementPath result =
         new MeasurementPath(
             generateFullPathNodes(), nextMatchedNode.getAsMeasurementNode().getSchema());
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 4e497913e1..3e2812af75 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,19 +20,19 @@
 package org.apache.iotdb.db.mpp.common.schematree.visitor;
 
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.schema.tree.AbstractTreeVisitorWithLimitOffset;
+import org.apache.iotdb.commons.schema.tree.AbstractTreeVisitor;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-public abstract class SchemaTreeVisitor<R>
-    extends AbstractTreeVisitorWithLimitOffset<SchemaNode, R> {
+public abstract class SchemaTreeVisitor<R> extends AbstractTreeVisitor<SchemaNode, R> {
 
-  public SchemaTreeVisitor(
-      SchemaNode root, PartialPath pathPattern, int limit, int offset, boolean isPrefixMatch) {
-    super(root, pathPattern, limit, offset, isPrefixMatch);
+  protected SchemaTreeVisitor() {}
+
+  public SchemaTreeVisitor(SchemaNode root, PartialPath pathPattern, boolean isPrefixMatch) {
+    super(root, pathPattern, isPrefixMatch);
   }
 
   public List<R> getAllResult() {
@@ -43,6 +43,16 @@ public abstract class SchemaTreeVisitor<R>
     return result;
   }
 
+  @Override
+  protected boolean shouldVisitSubtreeOfInternalMatchedNode(SchemaNode node) {
+    return !node.isMeasurement();
+  }
+
+  @Override
+  protected boolean shouldVisitSubtreeOfFullMatchedNode(SchemaNode node) {
+    return !node.isMeasurement();
+  }
+
   @Override
   protected SchemaNode getChild(SchemaNode parent, String childName) {
     return parent.getChild(childName);
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/SchemaTreeVisitorFactory.java
similarity index 51%
copy from server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitorFactory.java
index 4e497913e1..2c7e7365aa 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/SchemaTreeVisitorFactory.java
@@ -16,40 +16,31 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.iotdb.db.mpp.common.schematree.visitor;
 
+import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.schema.tree.AbstractTreeVisitorWithLimitOffset;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-public abstract class SchemaTreeVisitor<R>
-    extends AbstractTreeVisitorWithLimitOffset<SchemaNode, R> {
-
-  public SchemaTreeVisitor(
-      SchemaNode root, PartialPath pathPattern, int limit, int offset, boolean isPrefixMatch) {
-    super(root, pathPattern, limit, offset, isPrefixMatch);
-  }
-
-  public List<R> getAllResult() {
-    List<R> result = new ArrayList<>();
-    while (hasNext()) {
-      result.add(next());
-    }
-    return result;
+public class SchemaTreeVisitorFactory {
+  public static SchemaTreeDeviceVisitor createSchemaTreeDeviceVisitor(
+      SchemaNode root, PartialPath pathPattern, boolean isPrefixMatch) {
+    return new SchemaTreeDeviceVisitor(root, pathPattern, isPrefixMatch);
   }
 
-  @Override
-  protected SchemaNode getChild(SchemaNode parent, String childName) {
-    return parent.getChild(childName);
+  public static SchemaTreeMeasurementVisitor createSchemaTreeMeasurementVisitor(
+      SchemaNode root, PartialPath pathPattern, boolean isPrefixMatch) {
+    return new SchemaTreeMeasurementVisitor(root, pathPattern, isPrefixMatch);
   }
 
-  @Override
-  protected Iterator<SchemaNode> getChildrenIterator(SchemaNode parent) {
-    return parent.getChildrenIterator();
+  public static SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath>
+      createSchemaTreeMeasurementVisitor(
+          SchemaNode root,
+          PartialPath pathPattern,
+          boolean isPrefixMatch,
+          int slimit,
+          int soffset) {
+    return new SchemaTreeVisitorWithLimitOffsetWrapper<>(
+        new SchemaTreeMeasurementVisitor(root, pathPattern, isPrefixMatch), slimit, soffset);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitorWithLimitOffsetWrapper.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitorWithLimitOffsetWrapper.java
new file mode 100644
index 0000000000..b9ab5a3fbc
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitorWithLimitOffsetWrapper.java
@@ -0,0 +1,120 @@
+/*
+ * 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.mpp.common.schematree.visitor;
+
+import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+public class SchemaTreeVisitorWithLimitOffsetWrapper<R> extends SchemaTreeVisitor<R> {
+  private final SchemaTreeVisitor<R> visitor;
+  private final int limit;
+  private final int offset;
+  private final boolean hasLimit;
+
+  private int count = 0;
+  int curOffset = 0;
+
+  public SchemaTreeVisitorWithLimitOffsetWrapper(
+      SchemaTreeVisitor<R> visitor, int limit, int offset) {
+    this.visitor = visitor;
+    this.limit = limit;
+    this.offset = offset;
+    hasLimit = limit > 0 || offset > 0;
+
+    if (hasLimit) {
+      while (curOffset < offset && visitor.hasNext()) {
+        visitor.next();
+        curOffset++;
+      }
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (hasLimit) {
+      return count < limit && visitor.hasNext();
+    } else {
+      return visitor.hasNext();
+    }
+  }
+
+  @Override
+  public R next() {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
+    R result = visitor.next();
+    if (hasLimit) {
+      count++;
+    }
+    return result;
+  }
+
+  @Override
+  public void close() {
+    visitor.close();
+  }
+
+  @Override
+  public List<R> getAllResult() {
+    List<R> result = new ArrayList<>();
+    while (hasNext()) {
+      result.add(next());
+    }
+    return result;
+  }
+
+  @Override
+  protected boolean acceptInternalMatchedNode(SchemaNode node) {
+    // do nothing
+    return false;
+  }
+
+  @Override
+  protected boolean acceptFullMatchedNode(SchemaNode node) {
+    // do nothing
+    return false;
+  }
+
+  @Override
+  protected R generateResult(SchemaNode nextMatchedNode) {
+    // do nothing
+    return null;
+  }
+
+  @Override
+  public void reset() {
+    visitor.reset();
+    count = 0;
+    curOffset = 0;
+    if (hasLimit) {
+      while (curOffset < offset && visitor.hasNext()) {
+        visitor.next();
+        curOffset++;
+      }
+    }
+  }
+
+  public int getNextOffset() {
+    return curOffset + count;
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTreeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTreeTest.java
index dd3583080b..161857733a 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTreeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTreeTest.java
@@ -25,7 +25,8 @@ import org.apache.iotdb.db.mpp.common.schematree.node.SchemaEntityNode;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaInternalNode;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaMeasurementNode;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
-import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeMeasurementVisitor;
+import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeVisitorFactory;
+import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeVisitorWithLimitOffsetWrapper;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -74,8 +75,8 @@ public class ClusterSchemaTreeTest {
     int round = 20;
     for (int i = 0; i < round; i++) {
       for (int j = 0; j < 1000; j++) {
-        SchemaTreeMeasurementVisitor visitor =
-            new SchemaTreeMeasurementVisitor(root, path, 0, 0, false);
+        SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath> visitor =
+            createSchemaTreeVisitorWithLimitOffsetWrapper(root, path, 0, 0, false);
 
         long calStartTime = System.currentTimeMillis();
         List<MeasurementPath> res = visitor.getAllResult();
@@ -101,8 +102,9 @@ public class ClusterSchemaTreeTest {
   public void testMultiWildcard() throws IllegalPathException {
     SchemaNode root = generateSchemaTreeWithInternalRepeatedName();
 
-    SchemaTreeMeasurementVisitor visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.**.s"), 0, 0, false);
+    SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath> visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.**.s"), 0, 0, false);
     checkVisitorResult(
         visitor,
         4,
@@ -110,7 +112,9 @@ public class ClusterSchemaTreeTest {
         null,
         new boolean[] {false, false, false, false});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.*.**.s"), 0, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.*.**.s"), 0, 0, false);
     checkVisitorResult(
         visitor,
         4,
@@ -119,7 +123,8 @@ public class ClusterSchemaTreeTest {
         new boolean[] {false, false, false, false});
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.a.**.s"), 0, 0, false);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.a.**.s"), 0, 0, false);
     checkVisitorResult(
         visitor,
         3,
@@ -128,7 +133,8 @@ public class ClusterSchemaTreeTest {
         new boolean[] {false, false, false});
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.a.**.*.s"), 0, 0, false);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.a.**.*.s"), 0, 0, false);
     checkVisitorResult(
         visitor,
         2,
@@ -137,7 +143,8 @@ public class ClusterSchemaTreeTest {
         new boolean[] {false, false, false});
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.a.**.a.*.s"), 0, 0, false);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.a.**.a.*.s"), 0, 0, false);
     checkVisitorResult(
         visitor,
         2,
@@ -145,7 +152,9 @@ public class ClusterSchemaTreeTest {
         null,
         new boolean[] {false, false, false});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.c.s1"), 0, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.c.s1"), 0, 0, false);
     checkVisitorResult(
         visitor,
         2,
@@ -154,30 +163,38 @@ public class ClusterSchemaTreeTest {
         new boolean[] {false, false});
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.c.d.c.s1"), 0, 0, false);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.c.d.c.s1"), 0, 0, false);
     checkVisitorResult(visitor, 1, new String[] {"root.c.c.c.d.c.s1"}, null, new boolean[] {false});
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.d.**.c.s1"), 0, 0, false);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.d.**.c.s1"), 0, 0, false);
     checkVisitorResult(
         visitor, 1, new String[] {"root.c.c.c.d.c.c.s1"}, null, new boolean[] {false});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.d.*.*"), 0, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.d.*.*"), 0, 0, false);
     checkVisitorResult(visitor, 1, new String[] {"root.c.c.c.d.c.s1"}, null, new boolean[] {false});
   }
 
   private void testSchemaTree(SchemaNode root) throws Exception {
 
-    SchemaTreeMeasurementVisitor visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.d2.a.s1"), 0, 0, false);
+    SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath> visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.d2.a.s1"), 0, 0, false);
     checkVisitorResult(visitor, 1, new String[] {"root.sg.d2.a.s1"}, null, new boolean[] {true});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.*.s2"), 0, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.*.s2"), 0, 0, false);
     checkVisitorResult(
         visitor, 2, new String[] {"root.sg.d1.s2", "root.sg.d2.s2"}, new String[] {"", ""}, null);
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.*.status"), 0, 0, false);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.*.status"), 0, 0, false);
     checkVisitorResult(
         visitor,
         2,
@@ -186,7 +203,8 @@ public class ClusterSchemaTreeTest {
         null);
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.d2.*.*"), 0, 0, false);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.d2.*.*"), 0, 0, false);
     checkVisitorResult(
         visitor,
         2,
@@ -194,7 +212,9 @@ public class ClusterSchemaTreeTest {
         new String[] {"", ""},
         new boolean[] {true, true});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.d1"), 0, 0, true);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.d1"), 0, 0, true);
     checkVisitorResult(
         visitor,
         2,
@@ -202,7 +222,9 @@ public class ClusterSchemaTreeTest {
         new String[] {"", ""},
         new boolean[] {false, false});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.*.a"), 0, 0, true);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.*.a"), 0, 0, true);
     checkVisitorResult(
         visitor,
         2,
@@ -211,7 +233,9 @@ public class ClusterSchemaTreeTest {
         new boolean[] {true, true},
         new int[] {0, 0});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.*.*"), 2, 2, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.*.*"), 2, 2, false);
     checkVisitorResult(
         visitor,
         2,
@@ -220,7 +244,9 @@ public class ClusterSchemaTreeTest {
         new boolean[] {false, false},
         new int[] {3, 4});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.*"), 2, 3, true);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.*"), 2, 3, true);
     checkVisitorResult(
         visitor,
         2,
@@ -229,7 +255,9 @@ public class ClusterSchemaTreeTest {
         new boolean[] {true, false},
         new int[] {4, 5});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.d1.**"), 0, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.d1.**"), 0, 0, false);
     checkVisitorResult(
         visitor,
         2,
@@ -237,7 +265,9 @@ public class ClusterSchemaTreeTest {
         new String[] {"", ""},
         new boolean[] {false, false});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.d2.**"), 3, 1, true);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.d2.**"), 3, 1, true);
     checkVisitorResult(
         visitor,
         3,
@@ -247,7 +277,8 @@ public class ClusterSchemaTreeTest {
         new int[] {2, 3, 4});
 
     visitor =
-        new SchemaTreeMeasurementVisitor(root, new PartialPath("root.sg.**.status"), 2, 1, true);
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.sg.**.status"), 2, 1, true);
     checkVisitorResult(
         visitor,
         2,
@@ -256,7 +287,9 @@ public class ClusterSchemaTreeTest {
         new boolean[] {true, false},
         new int[] {2, 3});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.*"), 10, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.*"), 10, 0, false);
     checkVisitorResult(
         visitor,
         6,
@@ -272,7 +305,9 @@ public class ClusterSchemaTreeTest {
         new boolean[] {false, false, true, true, false, false},
         new int[] {1, 2, 3, 4, 5, 6});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.**.*.**"), 10, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.**.*.**"), 10, 0, false);
     checkVisitorResult(
         visitor,
         6,
@@ -288,7 +323,9 @@ public class ClusterSchemaTreeTest {
         new boolean[] {false, false, true, true, false, false},
         new int[] {1, 2, 3, 4, 5, 6});
 
-    visitor = new SchemaTreeMeasurementVisitor(root, new PartialPath("root.*.**.**"), 10, 0, false);
+    visitor =
+        createSchemaTreeVisitorWithLimitOffsetWrapper(
+            root, new PartialPath("root.*.**.**"), 10, 0, false);
     checkVisitorResult(
         visitor,
         6,
@@ -430,7 +467,7 @@ public class ClusterSchemaTreeTest {
   }
 
   private void checkVisitorResult(
-      SchemaTreeMeasurementVisitor visitor,
+      SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath> visitor,
       int expectedNum,
       String[] expectedPath,
       String[] expectedAlias,
@@ -452,10 +489,11 @@ public class ClusterSchemaTreeTest {
         Assert.assertEquals(expectedAligned[i], result.get(i).isUnderAlignedEntity());
       }
     }
+    visitor.close();
   }
 
   private void checkVisitorResult(
-      SchemaTreeMeasurementVisitor visitor,
+      SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath> visitor,
       int expectedNum,
       String[] expectedPath,
       String[] expectedAlias,
@@ -475,6 +513,7 @@ public class ClusterSchemaTreeTest {
       i++;
     }
     Assert.assertEquals(expectedNum, i);
+    visitor.close();
   }
 
   @Test
@@ -678,4 +717,15 @@ public class ClusterSchemaTreeTest {
             .searchDeviceSchemaInfo(new PartialPath("root.sg.d1"), Collections.singletonList("s1"))
             .isAligned());
   }
+
+  protected SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath>
+      createSchemaTreeVisitorWithLimitOffsetWrapper(
+          SchemaNode root,
+          PartialPath pathPattern,
+          int slimit,
+          int soffset,
+          boolean isPrefixMatch) {
+    return SchemaTreeVisitorFactory.createSchemaTreeMeasurementVisitor(
+        root, pathPattern, isPrefixMatch, slimit, soffset);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTreeTestWithRelease.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java
copy to server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTreeTestWithRelease.java
index 4e497913e1..fc2db8c881 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/schematree/visitor/SchemaTreeVisitor.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/ClusterSchemaTreeTestWithRelease.java
@@ -16,40 +16,24 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.mpp.common.schematree;
 
-package org.apache.iotdb.db.mpp.common.schematree.visitor;
-
+import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.schema.tree.AbstractTreeVisitorWithLimitOffset;
 import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
+import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeVisitorWithLimitOffsetWrapper;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-public abstract class SchemaTreeVisitor<R>
-    extends AbstractTreeVisitorWithLimitOffset<SchemaNode, R> {
-
-  public SchemaTreeVisitor(
-      SchemaNode root, PartialPath pathPattern, int limit, int offset, boolean isPrefixMatch) {
-    super(root, pathPattern, limit, offset, isPrefixMatch);
-  }
-
-  public List<R> getAllResult() {
-    List<R> result = new ArrayList<>();
-    while (hasNext()) {
-      result.add(next());
-    }
-    return result;
-  }
-
-  @Override
-  protected SchemaNode getChild(SchemaNode parent, String childName) {
-    return parent.getChild(childName);
-  }
+public class ClusterSchemaTreeTestWithRelease extends ClusterSchemaTreeTest {
 
   @Override
-  protected Iterator<SchemaNode> getChildrenIterator(SchemaNode parent) {
-    return parent.getChildrenIterator();
+  protected SchemaTreeVisitorWithLimitOffsetWrapper<MeasurementPath>
+      createSchemaTreeVisitorWithLimitOffsetWrapper(
+          SchemaNode root,
+          PartialPath pathPattern,
+          int slimit,
+          int soffset,
+          boolean isPrefixMatch) {
+    return new SchemaTreeVisitorWithLimitOffsetWrapper<>(
+        new MockSchemaTreeMeasurementVisitor(root, pathPattern, isPrefixMatch), slimit, soffset);
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/MockSchemaTreeMeasurementVisitor.java b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/MockSchemaTreeMeasurementVisitor.java
new file mode 100644
index 0000000000..142e4a0878
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/common/schematree/MockSchemaTreeMeasurementVisitor.java
@@ -0,0 +1,105 @@
+/*
+ * 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.mpp.common.schematree;
+
+import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.mpp.common.schematree.node.SchemaNode;
+import org.apache.iotdb.db.mpp.common.schematree.visitor.SchemaTreeMeasurementVisitor;
+
+import org.junit.Assert;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+public class MockSchemaTreeMeasurementVisitor extends SchemaTreeMeasurementVisitor {
+  Map<SchemaNode, Integer> map = new HashMap<>();
+
+  @Override
+  protected MeasurementPath generateResult(SchemaNode nextMatchedNode) {
+    Assert.assertTrue(map.get(nextMatchedNode) > 0);
+    return super.generateResult(nextMatchedNode);
+  }
+
+  public MockSchemaTreeMeasurementVisitor(
+      SchemaNode root, PartialPath pathPattern, boolean isPrefixMatch) {
+    super(root, pathPattern, isPrefixMatch);
+  }
+
+  @Override
+  protected Iterator<SchemaNode> getChildrenIterator(SchemaNode parent) {
+    return new CountIterator(super.getChildrenIterator(parent));
+  }
+
+  @Override
+  protected SchemaNode getChild(SchemaNode parent, String childName) {
+    SchemaNode node = super.getChild(parent, childName);
+    if (node != null) {
+      if (map.containsKey(node)) {
+        map.put(node, map.get(node) + 1);
+      } else {
+        map.put(node, 1);
+      }
+    }
+    return node;
+  }
+
+  @Override
+  protected void releaseNode(SchemaNode child) {
+    map.computeIfPresent(child, (node, cnt) -> cnt - 1);
+  }
+
+  @Override
+  protected void releaseNodeIterator(Iterator<SchemaNode> nodeIterator) {
+    super.releaseNodeIterator(nodeIterator);
+  }
+
+  @Override
+  public void close() {
+    super.close();
+    for (int cnt : map.values()) {
+      Assert.assertEquals(0, cnt);
+    }
+  }
+
+  private class CountIterator implements Iterator<SchemaNode> {
+    Iterator<SchemaNode> iterator;
+
+    private CountIterator(Iterator<SchemaNode> iterator) {
+      this.iterator = iterator;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    @Override
+    public SchemaNode next() {
+      SchemaNode node = iterator.next();
+      if (map.containsKey(node)) {
+        map.put(node, map.get(node) + 1);
+      } else {
+        map.put(node, 1);
+      }
+      return node;
+    }
+  }
+}