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 2022/12/28 09:41:53 UTC

[iotdb] branch master updated: [IOTDB-5106] Parse PathPattern to Deterministic Finite Automate (#8607)

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 217e3d7747 [IOTDB-5106] Parse PathPattern to Deterministic Finite Automate (#8607)
217e3d7747 is described below

commit 217e3d7747bc6df22b9fb727a0881d3f90220c66
Author: Chen YZ <43...@users.noreply.github.com>
AuthorDate: Wed Dec 28 17:41:48 2022 +0800

    [IOTDB-5106] Parse PathPattern to Deterministic Finite Automate (#8607)
    
    [IOTDB-5106] Parse PathPattern to Deterministic Finite Automate (#8607)
---
 node-commons/pom.xml                               |   4 +
 .../apache/iotdb/commons/path/fa/FAFactory.java    |  55 +++++++
 .../iotdb/commons/path/fa/IFATransition.java       |   4 +-
 .../apache/iotdb/commons/path/fa/IPatternFA.java   |  58 +++++++
 .../apache/iotdb/commons/path/fa/dfa/DFAState.java |  66 ++++++++
 .../iotdb/commons/path/fa/dfa/PatternDFA.java      | 152 ++++++++++++++++++
 .../iotdb/commons/path/fa/dfa/graph/Closure.java   |  66 ++++++++
 .../iotdb/commons/path/fa/dfa/graph/DFAGraph.java  | 173 +++++++++++++++++++++
 .../iotdb/commons/path/fa/dfa/graph/NFAGraph.java  | 126 +++++++++++++++
 .../fa/dfa/transition/AbstractDFATransition.java   |  54 +++++++
 .../transition/DFAPreciseTransition.java}          |  32 ++--
 .../fa/dfa/transition/DFAWildcardTransition.java   |  56 +++++++
 .../iotdb/commons/path/fa/{ => nfa}/SimpleNFA.java |  12 +-
 .../commons/schema/tree/AbstractTreeVisitor.java   |  32 +++-
 .../apache/iotdb/commons/path/PatternDFATest.java  | 153 ++++++++++++++++++
 .../common/schematree/ClusterSchemaTreeTest.java   |  86 ++++++++++
 server/src/test/resources/logback-test.xml         |   1 +
 17 files changed, 1110 insertions(+), 20 deletions(-)

diff --git a/node-commons/pom.xml b/node-commons/pom.xml
index 9402be2894..4914097dd0 100644
--- a/node-commons/pom.xml
+++ b/node-commons/pom.xml
@@ -148,6 +148,10 @@
             <artifactId>commons-codec</artifactId>
             <version>1.15</version>
         </dependency>
+        <dependency>
+            <groupId>com.github.ben-manes.caffeine</groupId>
+            <artifactId>caffeine</artifactId>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/FAFactory.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/FAFactory.java
new file mode 100644
index 0000000000..c2b50414c3
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/FAFactory.java
@@ -0,0 +1,55 @@
+/*
+ * 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.path.fa;
+
+import org.apache.iotdb.commons.path.fa.dfa.PatternDFA;
+import org.apache.iotdb.commons.path.fa.nfa.SimpleNFA;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+
+public class FAFactory {
+
+  private final LoadingCache<IPatternFA.Builder, PatternDFA> dfaCache;
+  private static final int DFA_CACHE_SIZE = 20;
+
+  public IPatternFA constructDFA(IPatternFA.Builder builder) {
+    return dfaCache.get(builder);
+  }
+
+  public IPatternFA constructNFA(IPatternFA.Builder builder) {
+    return new SimpleNFA(builder.getPathPattern(), builder.isPrefixMatch());
+  }
+
+  private FAFactory() {
+    dfaCache =
+        Caffeine.newBuilder()
+            .maximumSize(DFA_CACHE_SIZE)
+            .build(builder -> new PatternDFA(builder.getPathPattern(), builder.isPrefixMatch()));
+  }
+
+  public static FAFactory getInstance() {
+    return FAFactoryHolder.INSTANCE;
+  }
+
+  /** singleton pattern. */
+  private static class FAFactoryHolder {
+    private static final FAFactory INSTANCE = new FAFactory();
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IFATransition.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IFATransition.java
index 143c864ec6..2663cb79f8 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IFATransition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IFATransition.java
@@ -23,7 +23,7 @@ package org.apache.iotdb.commons.path.fa;
 public interface IFATransition {
 
   /** @return the value of this transition, which is used to match the events */
-  String getValue();
+  String getAcceptEvent();
 
   /**
    * @param event event happened on one of the source state of this transition and is trying to find
@@ -31,4 +31,6 @@ public interface IFATransition {
    * @return whether this transition can match the event
    */
   boolean isMatch(String event);
+
+  int getIndex();
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IPatternFA.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IPatternFA.java
index cd5c114ab7..825531d536 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IPatternFA.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IPatternFA.java
@@ -19,8 +19,11 @@
 
 package org.apache.iotdb.commons.path.fa;
 
+import org.apache.iotdb.commons.path.PartialPath;
+
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * This interface defines the behaviour of a FA(Finite Automation), generated from a path pattern or
@@ -72,4 +75,59 @@ public interface IPatternFA {
    * @return the state identified by given index
    */
   IFAState getState(int index);
+
+  /**
+   * Determines if there is overlap between the state transfer events of this FA. If it returns
+   * true, then there may be overlap. If it returns false, there must be no overlap.
+   *
+   * @return may transition overlap
+   */
+  boolean mayTransitionOverlap();
+
+  final class Builder {
+    private PartialPath pathPattern;
+    private boolean isPrefixMatch = false;
+
+    public Builder() {}
+
+    public Builder pattern(PartialPath pattern) {
+      this.pathPattern = pattern;
+      return this;
+    }
+
+    public Builder isPrefixMatch(boolean isPrefixMatch) {
+      this.isPrefixMatch = isPrefixMatch;
+      return this;
+    }
+
+    public PartialPath getPathPattern() {
+      return pathPattern;
+    }
+
+    public boolean isPrefixMatch() {
+      return isPrefixMatch;
+    }
+
+    public IPatternFA buildNFA() {
+      return FAFactory.getInstance().constructNFA(this);
+    }
+
+    public IPatternFA buildDFA() {
+      return FAFactory.getInstance().constructDFA(this);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      Builder builder = (Builder) o;
+      return isPrefixMatch == builder.isPrefixMatch
+          && Objects.equals(pathPattern, builder.pathPattern);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(pathPattern, isPrefixMatch);
+    }
+  }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/DFAState.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/DFAState.java
new file mode 100644
index 0000000000..578df8eaff
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/DFAState.java
@@ -0,0 +1,66 @@
+/*
+ * 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.path.fa.dfa;
+
+import org.apache.iotdb.commons.path.fa.IFAState;
+
+import java.util.Objects;
+
+public class DFAState implements IFAState {
+  private final int index;
+  private final boolean isFinal;
+
+  public DFAState(int state) {
+    this.index = state;
+    this.isFinal = false;
+  }
+
+  public DFAState(int state, boolean isFinal) {
+    this.index = state;
+    this.isFinal = isFinal;
+  }
+
+  @Override
+  public int getIndex() {
+    return index;
+  }
+
+  @Override
+  public boolean isInitial() {
+    return index == 0;
+  }
+
+  @Override
+  public boolean isFinal() {
+    return isFinal;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    DFAState state = (DFAState) o;
+    return index == state.index && isFinal == state.isFinal;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(index, isFinal);
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/PatternDFA.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/PatternDFA.java
new file mode 100644
index 0000000000..c0f909d84c
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/PatternDFA.java
@@ -0,0 +1,152 @@
+/*
+ * 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.path.fa.dfa;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.fa.IFAState;
+import org.apache.iotdb.commons.path.fa.IFATransition;
+import org.apache.iotdb.commons.path.fa.IPatternFA;
+import org.apache.iotdb.commons.path.fa.dfa.graph.DFAGraph;
+import org.apache.iotdb.commons.path.fa.dfa.graph.NFAGraph;
+import org.apache.iotdb.commons.path.fa.dfa.transition.DFAPreciseTransition;
+import org.apache.iotdb.commons.path.fa.dfa.transition.DFAWildcardTransition;
+import org.apache.iotdb.commons.utils.TestOnly;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class PatternDFA implements IPatternFA {
+
+  private final List<IFATransition> preciseMatchTransitionList = new ArrayList<>();
+  private final List<IFATransition> batchMatchTransitionList = new ArrayList<>();
+  // Map<AcceptEvent, IFATransition>
+  private final Map<String, IFATransition> transitionMap = new HashMap<>();
+  private final DFAGraph dfaGraph;
+
+  // cached
+  private final Map<String, IFATransition>[] preciseMatchTransitionCached;
+  private final List<IFATransition>[] batchMatchTransitionCached;
+
+  public PatternDFA(PartialPath pathPattern, boolean isPrefix) {
+    // 1. build transition
+    boolean wildcard = false;
+    AtomicInteger transitionIndex = new AtomicInteger();
+    for (String node : pathPattern.getNodes()) {
+      if (IoTDBConstant.ONE_LEVEL_PATH_WILDCARD.equals(node)
+          || IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD.equals(node)) {
+        wildcard = true;
+      } else {
+        transitionMap.computeIfAbsent(
+            node,
+            i -> {
+              IFATransition transition =
+                  new DFAPreciseTransition(transitionIndex.getAndIncrement(), node);
+              preciseMatchTransitionList.add(transition);
+              return transition;
+            });
+      }
+    }
+    if (wildcard || isPrefix) {
+      IFATransition transition =
+          new DFAWildcardTransition(
+              transitionIndex.getAndIncrement(), new ArrayList<>(transitionMap.keySet()));
+      transitionMap.put(transition.getAcceptEvent(), transition);
+      batchMatchTransitionList.add(transition);
+    }
+
+    // 2. build NFA
+    NFAGraph nfaGraph = new NFAGraph(pathPattern, isPrefix, transitionMap);
+
+    // 3. NFA to DFA
+    dfaGraph = new DFAGraph(nfaGraph, transitionMap.values());
+    preciseMatchTransitionCached = new HashMap[dfaGraph.getStateSize()];
+    batchMatchTransitionCached = new List[dfaGraph.getStateSize()];
+  }
+
+  @Override
+  public Map<String, IFATransition> getPreciseMatchTransition(IFAState state) {
+    if (preciseMatchTransitionCached[state.getIndex()] == null) {
+      preciseMatchTransitionCached[state.getIndex()] =
+          dfaGraph.getPreciseMatchTransition(state, preciseMatchTransitionList);
+    }
+    return preciseMatchTransitionCached[state.getIndex()];
+  }
+
+  @Override
+  public Iterator<IFATransition> getPreciseMatchTransitionIterator(IFAState state) {
+    if (preciseMatchTransitionCached[state.getIndex()] == null) {
+      preciseMatchTransitionCached[state.getIndex()] =
+          dfaGraph.getPreciseMatchTransition(state, preciseMatchTransitionList);
+    }
+    return preciseMatchTransitionCached[state.getIndex()].values().iterator();
+  }
+
+  @Override
+  public Iterator<IFATransition> getFuzzyMatchTransitionIterator(IFAState state) {
+    if (batchMatchTransitionCached[state.getIndex()] == null) {
+      batchMatchTransitionCached[state.getIndex()] =
+          dfaGraph.getTransition(state, batchMatchTransitionList);
+    }
+    return batchMatchTransitionCached[state.getIndex()].iterator();
+  }
+
+  @Override
+  public int getFuzzyMatchTransitionSize(IFAState state) {
+    if (batchMatchTransitionCached[state.getIndex()] == null) {
+      batchMatchTransitionCached[state.getIndex()] =
+          dfaGraph.getTransition(state, batchMatchTransitionList);
+    }
+    return batchMatchTransitionCached[state.getIndex()].size();
+  }
+
+  @Override
+  public IFAState getNextState(IFAState currentState, IFATransition transition) {
+    return dfaGraph.getNextState(currentState, transition);
+  }
+
+  @Override
+  public IFAState getInitialState() {
+    return dfaGraph.getInitialState();
+  }
+
+  @Override
+  public int getStateSize() {
+    return dfaGraph.getStateSize();
+  }
+
+  @Override
+  public IFAState getState(int index) {
+    return dfaGraph.getState(index);
+  }
+
+  @Override
+  public boolean mayTransitionOverlap() {
+    return false;
+  }
+
+  @TestOnly
+  public List<IFATransition> getTransition(IFAState state) {
+    return dfaGraph.getTransition(state, transitionMap.values());
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/Closure.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/Closure.java
new file mode 100644
index 0000000000..2be239d2ff
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/Closure.java
@@ -0,0 +1,66 @@
+/*
+ * 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.path.fa.dfa.graph;
+
+import org.apache.iotdb.commons.path.fa.IFAState;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class Closure {
+  private final byte[] bitmap;
+  private final List<IFAState> states = new ArrayList<>();
+
+  public Closure(int capacity) {
+    this.bitmap = new byte[capacity];
+  }
+
+  public void addState(IFAState state) {
+    if (bitmap[state.getIndex()] == 0) {
+      bitmap[state.getIndex()] = 1;
+      states.add(state);
+    }
+  }
+
+  public List<IFAState> getStates() {
+    return states;
+  }
+
+  public boolean isFinal() {
+    boolean res = false;
+    for (IFAState state : states) {
+      res |= state.isFinal();
+    }
+    return res;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    Closure closure = (Closure) o;
+    return Arrays.equals(bitmap, closure.bitmap);
+  }
+
+  @Override
+  public int hashCode() {
+    return Arrays.hashCode(bitmap);
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/DFAGraph.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/DFAGraph.java
new file mode 100644
index 0000000000..98da539b38
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/DFAGraph.java
@@ -0,0 +1,173 @@
+/*
+ * 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.path.fa.dfa.graph;
+
+import org.apache.iotdb.commons.path.fa.IFAState;
+import org.apache.iotdb.commons.path.fa.IFATransition;
+import org.apache.iotdb.commons.path.fa.dfa.DFAState;
+import org.apache.iotdb.commons.utils.TestOnly;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+/**
+ * DFA graph for given path pattern. State 0 is initial state. Run PatternDFATest#printFASketch for
+ * more detail.
+ */
+public class DFAGraph {
+  private final List<IFAState> dfaStateList = new ArrayList<>();
+  private final List<IFAState>[] dfaTransitionTable;
+
+  public DFAGraph(NFAGraph nfaGraph, Collection<IFATransition> transitions) {
+    dfaTransitionTable = new List[transitions.size()];
+    int closureSize = nfaGraph.getStateSize();
+    // init start state
+    int index = 0;
+    // Map NFAStateClosure to DFASate
+    Map<Closure, DFAState> closureStateMap = new HashMap<>();
+    for (IFATransition transition : transitions) {
+      dfaTransitionTable[transition.getIndex()] = new ArrayList<>();
+      dfaTransitionTable[transition.getIndex()].add(null);
+    }
+    DFAState curState = new DFAState(index);
+    dfaStateList.add(curState);
+    Closure curClosure = new Closure(closureSize);
+    curClosure.addState(curState);
+    closureStateMap.put(curClosure, curState);
+    Stack<Closure> closureStack = new Stack<>();
+    closureStack.push(curClosure);
+
+    // construct DFA using subset construction
+    while (!closureStack.isEmpty()) {
+      curClosure = closureStack.pop();
+      for (IFATransition transition : transitions) {
+        Closure nextClosure = new Closure(closureSize);
+        boolean isEmpty = getNextClosure(nfaGraph, curClosure, nextClosure, transition);
+        if (!isEmpty) {
+          if (closureStateMap.containsKey(nextClosure)) {
+            // closure already exist
+            dfaTransitionTable[transition.getIndex()].set(
+                closureStateMap.get(curClosure).getIndex(), closureStateMap.get(nextClosure));
+          } else {
+            // new closure
+            DFAState newState = new DFAState(++index, nextClosure.isFinal());
+            dfaStateList.add(newState);
+            closureStateMap.put(nextClosure, newState);
+            for (List<IFAState> column : dfaTransitionTable) {
+              column.add(null);
+            }
+            dfaTransitionTable[transition.getIndex()].set(
+                closureStateMap.get(curClosure).getIndex(), newState);
+            closureStack.push(nextClosure);
+          }
+        }
+      }
+    }
+  }
+
+  @TestOnly
+  public void print(Map<String, IFATransition> transitionMap) {
+    System.out.println();
+    System.out.println();
+    System.out.println("DFA:");
+    System.out.println(
+        "==================================================================================================");
+    StringBuilder stringBuilder = new StringBuilder();
+    stringBuilder.append(String.format("|%-15s|", "State"));
+    for (IFATransition transfer : transitionMap.values()) {
+      stringBuilder.append(String.format("%-15s|", transfer.toString()));
+    }
+    stringBuilder.append(String.format("%-15s|", "Final"));
+    System.out.println(stringBuilder);
+    for (int i = 0; i < dfaStateList.size(); i++) {
+      stringBuilder = new StringBuilder();
+      stringBuilder.append(String.format("|%-15d|", i));
+      for (IFATransition transition : transitionMap.values()) {
+        IFAState tmp = dfaTransitionTable[transition.getIndex()].get(i);
+        stringBuilder.append(String.format("%-15s|", tmp == null ? "" : tmp.getIndex()));
+      }
+      stringBuilder.append(String.format("%-15s|", dfaStateList.get(i).isFinal()));
+      System.out.println(stringBuilder);
+    }
+  }
+
+  /**
+   * get next closure
+   *
+   * @param nfaGraph
+   * @param curClosure
+   * @param transfer
+   * @return if empty. return true
+   */
+  private boolean getNextClosure(
+      NFAGraph nfaGraph, Closure curClosure, Closure nextClosure, IFATransition transfer) {
+    boolean isEmpty = true;
+    for (IFAState state : curClosure.getStates()) {
+      if (state != null) {
+        for (IFAState nextState : nfaGraph.getTransitions(transfer, state)) {
+          nextClosure.addState(nextState);
+          isEmpty = false;
+        }
+      }
+    }
+    return isEmpty;
+  }
+
+  public Map<String, IFATransition> getPreciseMatchTransition(
+      IFAState state, Collection<IFATransition> transitions) {
+    Map<String, IFATransition> res = new HashMap<>();
+    for (IFATransition transition : transitions) {
+      if (dfaTransitionTable[transition.getIndex()].get(state.getIndex()) != null) {
+        res.put(transition.getAcceptEvent(), transition);
+      }
+    }
+    return res;
+  }
+
+  public List<IFATransition> getTransition(
+      IFAState state, Collection<IFATransition> transitionList) {
+    List<IFATransition> res = new ArrayList<>();
+    for (IFATransition transition : transitionList) {
+      if (dfaTransitionTable[transition.getIndex()].get(state.getIndex()) != null) {
+        res.add(transition);
+      }
+    }
+    return res;
+  }
+
+  public IFAState getNextState(IFAState currentState, IFATransition transition) {
+    return dfaTransitionTable[transition.getIndex()].get(currentState.getIndex());
+  }
+
+  public IFAState getInitialState() {
+    return dfaStateList.get(0);
+  }
+
+  public int getStateSize() {
+    return dfaStateList.size();
+  }
+
+  public IFAState getState(int index) {
+    return dfaStateList.get(index);
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/NFAGraph.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/NFAGraph.java
new file mode 100644
index 0000000000..47020e6b6e
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/graph/NFAGraph.java
@@ -0,0 +1,126 @@
+/*
+ * 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.path.fa.dfa.graph;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.fa.IFAState;
+import org.apache.iotdb.commons.path.fa.IFATransition;
+import org.apache.iotdb.commons.path.fa.dfa.DFAState;
+import org.apache.iotdb.commons.utils.TestOnly;
+
+import org.apache.commons.lang.StringUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * NFA graph for given path pattern. State 0 is initial state. Run PatternDFATest#printFASketch for
+ * more detail.
+ */
+public class NFAGraph {
+  private final List<IFAState> nfaStateList = new ArrayList<>();
+
+  // [transitionIndex][stateIndex]List<IFAState>
+  private final List<IFAState>[][] nfaTransitionTable;
+
+  public NFAGraph(
+      PartialPath pathPattern, boolean isPrefix, Map<String, IFATransition> transitionMap) {
+    nfaTransitionTable = new List[transitionMap.size()][pathPattern.getNodeLength() + 1];
+    // init start state, curNodeIndex=0
+    int curStateIndex = 0;
+    nfaStateList.add(new DFAState(curStateIndex));
+    for (int i = 0; i < transitionMap.size(); i++) {
+      nfaTransitionTable[i][0] = new ArrayList<>();
+    }
+    // traverse pathPattern and construct NFA
+    for (int i = 0; i < pathPattern.getNodeLength(); i++) {
+      String node = pathPattern.getNodes()[i];
+      // if it is tail node, transit to final state
+      DFAState state =
+          i == pathPattern.getNodeLength() - 1
+              ? new DFAState(++curStateIndex, true)
+              : new DFAState(++curStateIndex);
+      nfaStateList.add(state);
+      for (int j = 0; j < transitionMap.size(); j++) {
+        nfaTransitionTable[j][curStateIndex] = new ArrayList<>();
+      }
+      // construct transition
+      if (IoTDBConstant.ONE_LEVEL_PATH_WILDCARD.equals(node)) {
+        for (IFATransition transition : transitionMap.values()) {
+          nfaTransitionTable[transition.getIndex()][curStateIndex - 1].add(state);
+        }
+      } else if (IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD.equals(node)) {
+        for (IFATransition transition : transitionMap.values()) {
+          nfaTransitionTable[transition.getIndex()][curStateIndex - 1].add(state);
+          nfaTransitionTable[transition.getIndex()][curStateIndex].add(state);
+        }
+      } else {
+        nfaTransitionTable[transitionMap.get(node).getIndex()][curStateIndex - 1].add(state);
+      }
+      if (isPrefix && i == pathPattern.getNodeLength() - 1) {
+        for (IFATransition transition : transitionMap.values()) {
+          nfaTransitionTable[transition.getIndex()][curStateIndex].add(state);
+        }
+      }
+    }
+  }
+
+  @TestOnly
+  public void print(Map<String, IFATransition> transitionMap) {
+    System.out.println();
+    System.out.println();
+    System.out.println("NFA:");
+    System.out.println(
+        "==================================================================================================");
+    StringBuilder stringBuilder = new StringBuilder();
+    stringBuilder.append(String.format("|%-15s|", "State"));
+    for (IFATransition transfer : transitionMap.values()) {
+      stringBuilder.append(String.format("%-15s|", transfer.toString()));
+    }
+    stringBuilder.append(String.format("%-15s|", "Final"));
+    System.out.println(stringBuilder);
+    for (int i = 0; i < nfaStateList.size(); i++) {
+      stringBuilder = new StringBuilder();
+      stringBuilder.append(String.format("|%-15d|", i));
+      for (IFATransition transition : transitionMap.values()) {
+        stringBuilder.append(
+            String.format(
+                "%-15s|",
+                StringUtils.join(
+                    nfaTransitionTable[transition.getIndex()][i].stream()
+                        .map(IFAState::getIndex)
+                        .collect(Collectors.toList()),
+                    ",")));
+      }
+      stringBuilder.append(String.format("%-15s|", nfaStateList.get(i).isFinal()));
+      System.out.println(stringBuilder);
+    }
+  }
+
+  public List<IFAState> getTransitions(IFATransition transition, IFAState state) {
+    return nfaTransitionTable[transition.getIndex()][state.getIndex()];
+  }
+
+  public int getStateSize() {
+    return nfaStateList.size();
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/AbstractDFATransition.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/AbstractDFATransition.java
new file mode 100644
index 0000000000..5118f8079c
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/AbstractDFATransition.java
@@ -0,0 +1,54 @@
+/*
+ * 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.path.fa.dfa.transition;
+
+import org.apache.iotdb.commons.path.fa.IFATransition;
+
+import java.util.Objects;
+
+abstract class AbstractDFATransition implements IFATransition {
+  /**
+   * A transition does not change after it has been created externally, the id is used as a unique
+   * identifier. It is necessary to ensure that the other properties of a transition with a
+   * different id are different.
+   */
+  private final int id; // only used for hash
+
+  public AbstractDFATransition(int index) {
+    this.id = index;
+  }
+
+  @Override
+  public int getIndex() {
+    return id;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    AbstractDFATransition that = (AbstractDFATransition) o;
+    return Objects.equals(id, that.id);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id);
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IFATransition.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/DFAPreciseTransition.java
similarity index 61%
copy from node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IFATransition.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/DFAPreciseTransition.java
index 143c864ec6..c50ac01227 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/IFATransition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/DFAPreciseTransition.java
@@ -16,19 +16,29 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.commons.path.fa.dfa.transition;
 
-package org.apache.iotdb.commons.path.fa;
+public class DFAPreciseTransition extends AbstractDFATransition {
 
-/** This interface defines the behaviour of a FA(Finite Automation)'s transition. */
-public interface IFATransition {
+  private final String acceptEvent;
 
-  /** @return the value of this transition, which is used to match the events */
-  String getValue();
+  public DFAPreciseTransition(int index, String acceptEvent) {
+    super(index);
+    this.acceptEvent = acceptEvent;
+  }
 
-  /**
-   * @param event event happened on one of the source state of this transition and is trying to find
-   *     the next state
-   * @return whether this transition can match the event
-   */
-  boolean isMatch(String event);
+  @Override
+  public String toString() {
+    return acceptEvent;
+  }
+
+  @Override
+  public String getAcceptEvent() {
+    return acceptEvent;
+  }
+
+  @Override
+  public boolean isMatch(String event) {
+    return acceptEvent.equals(event);
+  }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/DFAWildcardTransition.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/DFAWildcardTransition.java
new file mode 100644
index 0000000000..62b43b953e
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/dfa/transition/DFAWildcardTransition.java
@@ -0,0 +1,56 @@
+/*
+ * 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.path.fa.dfa.transition;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.List;
+
+public class DFAWildcardTransition extends AbstractDFATransition {
+  private final List<String> rejectEventList;
+
+  public DFAWildcardTransition(int index, List<String> rejectEventList) {
+    super(index);
+    this.rejectEventList = rejectEventList;
+  }
+
+  @Override
+  public String toString() {
+    if (rejectEventList == null || rejectEventList.isEmpty()) {
+      return IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+    } else {
+      return IoTDBConstant.ONE_LEVEL_PATH_WILDCARD
+          + "/("
+          + StringUtils.join(rejectEventList, ",")
+          + ")";
+    }
+  }
+
+  @Override
+  public String getAcceptEvent() {
+    return IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+  }
+
+  @Override
+  public boolean isMatch(String event) {
+    return !rejectEventList.contains(event);
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/SimpleNFA.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/nfa/SimpleNFA.java
similarity index 97%
rename from node-commons/src/main/java/org/apache/iotdb/commons/path/fa/SimpleNFA.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/path/fa/nfa/SimpleNFA.java
index 5eb3381101..07638db799 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/SimpleNFA.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/nfa/SimpleNFA.java
@@ -17,9 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.path.fa;
+package org.apache.iotdb.commons.path.fa.nfa;
 
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.fa.IFAState;
+import org.apache.iotdb.commons.path.fa.IFATransition;
+import org.apache.iotdb.commons.path.fa.IPatternFA;
 
 import java.util.Collections;
 import java.util.Iterator;
@@ -106,6 +109,11 @@ public class SimpleNFA implements IPatternFA {
     return patternNodes[index];
   }
 
+  @Override
+  public boolean mayTransitionOverlap() {
+    return true;
+  }
+
   private SinglePathPatternNode getNextNode(SinglePathPatternNode currentNode) {
     if (currentNode.patternIndex == rawNodes.length) {
       return currentNode;
@@ -158,7 +166,7 @@ public class SimpleNFA implements IPatternFA {
     }
 
     @Override
-    public String getValue() {
+    public String getAcceptEvent() {
       return rawNodes[patternIndex];
     }
 
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 efe829cec1..63e3daaeff 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
@@ -19,11 +19,11 @@
 
 package org.apache.iotdb.commons.schema.tree;
 
+import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.fa.IFAState;
 import org.apache.iotdb.commons.path.fa.IFATransition;
 import org.apache.iotdb.commons.path.fa.IPatternFA;
-import org.apache.iotdb.commons.path.fa.SimpleNFA;
 import org.apache.iotdb.commons.path.fa.match.IStateMatchInfo;
 import org.apache.iotdb.commons.path.fa.match.StateMultiMatchInfo;
 import org.apache.iotdb.commons.path.fa.match.StateSingleMatchInfo;
@@ -92,7 +92,23 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
   protected AbstractTreeVisitor(N root, PartialPath pathPattern, boolean isPrefixMatch) {
     this.root = root;
 
-    this.patternFA = new SimpleNFA(pathPattern, isPrefixMatch);
+    boolean usingDFA = false;
+    // Use DFA if there are ** and no regex node in pathPattern
+    for (String pathNode : pathPattern.getNodes()) {
+      if (IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD.equals(pathNode)) {
+        // ** node
+        usingDFA = true;
+      } else if (pathNode.length() > 1
+          && pathNode.contains(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)) {
+        // regex node
+        usingDFA = false;
+        break;
+      }
+    }
+    this.patternFA =
+        usingDFA
+            ? new IPatternFA.Builder().pattern(pathPattern).isPrefixMatch(isPrefixMatch).buildDFA()
+            : new IPatternFA.Builder().pattern(pathPattern).isPrefixMatch(isPrefixMatch).buildNFA();
 
     initStack();
   }
@@ -334,7 +350,7 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
       IFATransition transition;
       while (transitionIterator.hasNext()) {
         transition = transitionIterator.next();
-        child = getChild(parent, transition.getValue());
+        child = getChild(parent, transition.getAcceptEvent());
         if (child == null) {
           continue;
         }
@@ -426,9 +442,13 @@ public abstract class AbstractTreeVisitor<N extends ITreeNode, R> implements Ite
           }
         }
 
-        if (transitionIterator.hasNext()) {
-          stateMatchInfo = new StateMultiMatchInfo(patternFA, matchedState, transitionIterator);
-          firstAncestorOfTraceback = ancestorStack.size();
+        if (patternFA.mayTransitionOverlap()) {
+          if (transitionIterator.hasNext()) {
+            stateMatchInfo = new StateMultiMatchInfo(patternFA, matchedState, transitionIterator);
+            firstAncestorOfTraceback = ancestorStack.size();
+          } else {
+            stateMatchInfo = new StateSingleMatchInfo(patternFA, matchedState);
+          }
         } else {
           stateMatchInfo = new StateSingleMatchInfo(patternFA, matchedState);
         }
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/path/PatternDFATest.java b/node-commons/src/test/java/org/apache/iotdb/commons/path/PatternDFATest.java
new file mode 100644
index 0000000000..f0ee0579e9
--- /dev/null
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/path/PatternDFATest.java
@@ -0,0 +1,153 @@
+/*
+ * 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.path;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.fa.IFAState;
+import org.apache.iotdb.commons.path.fa.IFATransition;
+import org.apache.iotdb.commons.path.fa.IPatternFA;
+import org.apache.iotdb.commons.path.fa.dfa.PatternDFA;
+import org.apache.iotdb.commons.path.fa.dfa.graph.DFAGraph;
+import org.apache.iotdb.commons.path.fa.dfa.graph.NFAGraph;
+import org.apache.iotdb.commons.path.fa.dfa.transition.DFAPreciseTransition;
+import org.apache.iotdb.commons.path.fa.dfa.transition.DFAWildcardTransition;
+
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class PatternDFATest {
+
+  @Test
+  @Ignore
+  public void printFASketch() throws IllegalPathException {
+    // Map<AcceptEvent, IFATransition>
+    Map<String, IFATransition> transitionMap = new HashMap<>();
+    PartialPath pathPattern = new PartialPath("root.**.d.s1");
+    // 1. build transition
+    boolean wildcard = false;
+    AtomicInteger transitionIndex = new AtomicInteger();
+    for (String node : pathPattern.getNodes()) {
+      if (IoTDBConstant.ONE_LEVEL_PATH_WILDCARD.equals(node)
+          || IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD.equals(node)) {
+        wildcard = true;
+      } else {
+        transitionMap.computeIfAbsent(
+            node, i -> new DFAPreciseTransition(transitionIndex.getAndIncrement(), node));
+      }
+    }
+    if (wildcard) {
+      IFATransition transition =
+          new DFAWildcardTransition(
+              transitionIndex.getAndIncrement(), new ArrayList<>(transitionMap.keySet()));
+      transitionMap.put(transition.getAcceptEvent(), transition);
+    }
+    // 2. build NFA
+    NFAGraph nfaGraph = new NFAGraph(pathPattern, false, transitionMap);
+    nfaGraph.print(transitionMap);
+    // 3. NFA to DFA
+    DFAGraph dfaGraph = new DFAGraph(nfaGraph, transitionMap.values());
+    dfaGraph.print(transitionMap);
+  }
+
+  @Test
+  public void testMatchFullPath() throws IllegalPathException {
+    PartialPath p1 = new PartialPath("root.sg1.d1.*");
+
+    Assert.assertTrue(p1.matchFullPath(new PartialPath("root.sg1.d1.s2")));
+    Assert.assertTrue(checkMatchUsingDFA(p1, new PartialPath("root.sg1.d1.s2")));
+    Assert.assertFalse(p1.matchFullPath(new PartialPath("root.sg1.d1")));
+    Assert.assertFalse(checkMatchUsingDFA(p1, new PartialPath("root.sg1.d1")));
+    Assert.assertFalse(p1.matchFullPath(new PartialPath("root.sg2.d1.*")));
+    Assert.assertFalse(checkMatchUsingDFA(p1, new PartialPath("root.sg2.d1.*")));
+    Assert.assertFalse(p1.matchFullPath(new PartialPath("", false)));
+    Assert.assertFalse(checkMatchUsingDFA(p1, new PartialPath("", false)));
+
+    PartialPath path = new PartialPath("root.sg1.d1.s1");
+    String[] patterns1 = {
+      "root.sg1.d1.s1",
+      "root.sg1.*.s1",
+      "root.*.d1.*",
+      "root.*.*.*",
+      "root.**",
+      "root.**.s1",
+      "root.sg1.**",
+    };
+    for (String pattern : patterns1) {
+      Assert.assertTrue(new PartialPath(pattern).matchFullPath(path));
+      Assert.assertTrue(checkMatchUsingDFA(new PartialPath(pattern), path));
+    }
+
+    String[] patterns2 = {
+      "root2.sg1.d1.s1",
+      "root.sg1.*.s2",
+      "root.*.d2.s1",
+      "root.*",
+      "root.*.*",
+      "root2.**",
+      "root.**.s2",
+      "root.**.d1",
+      "root.sg2.**",
+    };
+    for (String pattern : patterns2) {
+      Assert.assertFalse(new PartialPath(pattern).matchFullPath(path));
+      Assert.assertFalse(checkMatchUsingDFA(new PartialPath(pattern), path));
+    }
+  }
+
+  private boolean checkMatchUsingDFA(PartialPath pattern, PartialPath fullPath) {
+    PatternDFA patternDFA =
+        (PatternDFA) new IPatternFA.Builder().pattern(pattern).isPrefixMatch(false).buildDFA();
+    IFAState curState = patternDFA.getInitialState();
+    for (String node : fullPath.getNodes()) {
+      Iterator<IFATransition> preciseMatchTransitionIterator =
+          patternDFA.getPreciseMatchTransitionIterator(curState);
+      Iterator<IFATransition> batchMatchTransitionIterator =
+          patternDFA.getFuzzyMatchTransitionIterator(curState);
+      List<IFATransition> transitionList = patternDFA.getTransition(curState);
+      while (preciseMatchTransitionIterator.hasNext()) {
+        IFATransition transition = preciseMatchTransitionIterator.next();
+        Assert.assertTrue(transitionList.contains(transition));
+      }
+      while (batchMatchTransitionIterator.hasNext()) {
+        IFATransition transition = batchMatchTransitionIterator.next();
+        Assert.assertTrue(transitionList.contains(transition));
+      }
+
+      if (transitionList.isEmpty()) {
+        return false;
+      }
+      for (IFATransition transition : transitionList) {
+        if (transition.isMatch(node)) {
+          curState = patternDFA.getNextState(curState, transition);
+          break;
+        }
+      }
+    }
+    return curState.isFinal();
+  }
+}
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 000bdc10de..dd3583080b 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
@@ -31,8 +31,11 @@ import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.mockito.internal.util.collections.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -43,6 +46,48 @@ import java.util.List;
 import java.util.stream.Collectors;
 
 public class ClusterSchemaTreeTest {
+  private static final Logger logger = LoggerFactory.getLogger(ClusterSchemaTreeTest.class);
+
+  @Test
+  @Ignore
+  public void testPerformanceOnSimpleTree() throws IllegalPathException {
+    long startTime = System.currentTimeMillis();
+    int round = 20;
+    for (int i = 0; i < round; i++) {
+      for (int j = 0; j < 10000; j++) {
+        testMultiWildcard();
+      }
+    }
+    long endTime = System.currentTimeMillis();
+    logger.info("AllTime={}", (endTime - startTime) / round);
+  }
+
+  @Test
+  @Ignore
+  public void testPerformanceOnComplexTree() throws IllegalPathException {
+    int deep = 5;
+    int width = 5;
+    SchemaNode root = generateComplexSchemaTree(deep, width);
+    PartialPath path = new PartialPath("root.**.d0.s");
+    long startTime = System.currentTimeMillis();
+    long calTime = 0;
+    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);
+
+        long calStartTime = System.currentTimeMillis();
+        List<MeasurementPath> res = visitor.getAllResult();
+        Assert.assertEquals((int) (1 - Math.pow(width, deep)) / (1 - width) - 1, res.size());
+        calTime += (System.currentTimeMillis() - calStartTime);
+      }
+    }
+    long endTime = System.currentTimeMillis();
+    logger.info("CalculateTime={}", calTime / round);
+    logger.info("InitialTime={}", (endTime - startTime - calTime) / round);
+    logger.info("AllTime={}", (endTime - startTime) / round);
+  }
 
   @Test
   public void testSchemaTreeVisitor() throws Exception {
@@ -343,6 +388,47 @@ public class ClusterSchemaTreeTest {
     return root;
   }
 
+  /**
+   * Generate complex schema tree with specific deep and width. For example, if deep=2 and width=3,
+   * the schema tree contains timeseries: root.d0.s, root.d1.s, root.s2.s, root.d0.d0.s,
+   * root.d0.d1.s, root.d0.d2.s, root.d1.d0.s, root.d1.d1.s, root.d1.d2.s, root.d2.d0.s,
+   * root.d2.d1.s, root.d2.d2.s
+   *
+   * @param deep deep
+   * @param width width
+   * @return root node
+   */
+  private SchemaNode generateComplexSchemaTree(int deep, int width) {
+    SchemaNode root = new SchemaInternalNode("root");
+
+    List<SchemaNode> nodes = new ArrayList<>();
+    MeasurementSchema schema = new MeasurementSchema("s", TSDataType.INT32);
+    for (int i = 0; i < width; i++) {
+      SchemaEntityNode entityNode = new SchemaEntityNode("d" + i);
+      nodes.add(entityNode);
+      root.addChild("d" + i, entityNode);
+    }
+    for (int i = 0; i < deep - 1; i++) {
+      List<SchemaNode> nextLevelNode = new ArrayList<>();
+      for (SchemaNode parent : nodes) {
+        SchemaMeasurementNode measurementNode = new SchemaMeasurementNode("s", schema);
+        parent.addChild("s", measurementNode);
+        for (int j = 0; j < width; j++) {
+          SchemaEntityNode entityNode = new SchemaEntityNode("d" + j);
+          parent.addChild("d" + j, entityNode);
+          nextLevelNode.add(entityNode);
+        }
+      }
+      nodes = nextLevelNode;
+    }
+    for (SchemaNode parent : nodes) {
+      SchemaMeasurementNode measurementNode = new SchemaMeasurementNode("s", schema);
+      parent.addChild("s", measurementNode);
+    }
+
+    return root;
+  }
+
   private void checkVisitorResult(
       SchemaTreeMeasurementVisitor visitor,
       int expectedNum,
diff --git a/server/src/test/resources/logback-test.xml b/server/src/test/resources/logback-test.xml
index 01b731ec55..d70151b7dd 100644
--- a/server/src/test/resources/logback-test.xml
+++ b/server/src/test/resources/logback-test.xml
@@ -48,6 +48,7 @@
     <logger name="org.apache.iotdb.commons.service.metric.MetricService" level="INFO"/>
     <logger name="org.apache.iotdb.db.engine.flush.FlushManager" level="INFO"/>
     <logger name="org.apache.iotdb.db.integration.IoTDBCompactionIT" level="INFO"/>
+    <logger name="org.apache.iotdb.db.mpp.common.schematree" level="INFO"/>
     <logger name="org.apache.iotdb.commons.service.RegisterManager" level="INFO"/>
     <logger name="org.apache.iotdb.db.service.DataNode" level="WARN"/>
     <logger name="org.apache.iotdb.db.service.RPCService" level="INFO"/>