You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ro...@apache.org on 2022/05/17 15:48:14 UTC

[iotdb] branch master updated: [IOTDB-3073][IOTDB-3095] Unescape identifier and Use antlr to parse PartialPath/Path (#5848)

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

rong 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 f4e458178c [IOTDB-3073][IOTDB-3095] Unescape identifier and Use antlr to parse PartialPath/Path (#5848)
f4e458178c is described below

commit f4e458178ce09b2b3f3894ecde4887dcd52efb6f
Author: Liao Lanyu <48...@users.noreply.github.com>
AuthorDate: Tue May 17 23:48:09 2022 +0800

    [IOTDB-3073][IOTDB-3095] Unescape identifier and Use antlr to parse PartialPath/Path (#5848)
---
 antlr/pom.xml                                      |   1 +
 .../org/apache/iotdb/db/qp/sql/IdentifierParser.g4 | 179 ++++++++
 .../apache/iotdb/db/qp/sql/InfluxDBSqlParser.g4    |  14 +-
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |   9 +-
 .../org/apache/iotdb/db/qp/sql/PathParser.g4       |  54 ++-
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4  |   4 +-
 client-py/tests/test_dataframe.py                  |  10 +-
 .../cluster/client/sync/SyncClientAdaptorTest.java |   2 +-
 .../persistence/ClusterSchemaInfoTest.java         |   2 +-
 .../db/integration/IoTDBContinuousQueryIT.java     |  42 ++
 .../db/integration/IoTDBCreateTimeseriesIT.java    |   2 +-
 .../iotdb/db/integration/IoTDBQuotedPathIT.java    |   2 +-
 .../iotdb/db/integration/IoTDBSelectIntoIT.java    |   2 +-
 .../iotdb/db/integration/IoTDBSimpleQueryIT.java   |   4 +-
 .../IoTDBSyntaxConventionIdentifierIT.java         | 160 ++++++-
 .../IoTDBWithoutNullAnyFilterIT.java               |  10 +-
 .../iotdb/session/IoTDBSessionComplexIT.java       |   8 +-
 .../apache/iotdb/session/IoTDBSessionSimpleIT.java |   8 +-
 .../session/IoTDBSessionSyntaxConventionIT.java    |  18 +-
 .../java/org/apache/iotdb/session/SessionTest.java |  28 +-
 .../apache/iotdb/session/template/TemplateUT.java  |  14 +-
 node-commons/pom.xml                               |   5 +
 .../org/apache/iotdb/commons/path/PartialPath.java |  51 +-
 .../org/apache/iotdb/commons/utils/PathUtils.java  |  78 +---
 .../apache/iotdb/commons/path/PartialPathTest.java | 511 +++++++++++++++++++++
 .../apache/iotdb/commons/utils/PathUtilsTest.java  | 125 -----
 .../schemaregion/rocksdb/RSchemaRegion.java        |   2 +-
 .../schemaregion/rocksdb/RSchemaUtils.java         |   2 +-
 server/pom.xml                                     |   5 -
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   2 +-
 .../iotdb/db/engine/cq/ContinuousQueryTask.java    |  13 +-
 .../db/metadata/idtable/IDTableHashmapImpl.java    |  11 +-
 .../db/metadata/idtable/entry/SHA256DeviceID.java  |   7 +-
 .../db/metadata/mtree/MTreeBelowSGCachedImpl.java  |   4 +-
 .../db/metadata/mtree/MTreeBelowSGMemoryImpl.java  |   4 +-
 .../mtree/store/disk/schemafile/SchemaFile.java    |   2 +-
 .../apache/iotdb/db/metadata/path/AlignedPath.java |  13 +-
 .../iotdb/db/metadata/template/Template.java       |  20 +-
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |  40 +-
 .../db/mpp/plan/parser/StatementGenerator.java     |   8 +-
 .../iotdb/db/qp/physical/sys/SetTemplatePlan.java  |   2 +-
 .../db/qp/physical/sys/UnsetTemplatePlan.java      |   2 +-
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    |  27 +-
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |   8 +-
 .../qp/strategy/optimizer/ConcatPathOptimizer.java |   8 +-
 .../query/dataset/groupby/GroupByLevelDataSet.java |   9 +-
 .../engine/compaction/CompactionSchedulerTest.java |   2 +-
 .../apache/iotdb/db/metadata/MetaUtilsTest.java    |   4 +-
 .../apache/iotdb/db/metadata/PartialPathTest.java  | 158 -------
 .../apache/iotdb/db/metadata/SchemaBasicTest.java  | 152 ++----
 .../iotdb/db/metadata/mtree/MTreeAboveSGTest.java  |  23 +-
 .../metadata/mtree/schemafile/SchemaFileTest.java  |   4 +-
 .../db/metadata/upgrade/MetadataUpgradeTest.java   |   2 +-
 .../read/DeviceSchemaScanNodeSerdeTest.java        |   2 +-
 .../metadata/read/SchemaCountNodeSerdeTest.java    |   4 +-
 .../read/TimeSeriesSchemaScanNodeSerdeTest.java    |   2 +-
 .../plan/node/sink/FragmentSinkNodeSerdeTest.java  |   7 +-
 .../db/qp/physical/PhysicalPlanSerializeTest.java  |   6 +-
 tsfile/pom.xml                                     |  10 +
 .../tsfile/common/constant/TsFileConstant.java     |   7 +
 .../iotdb/tsfile/exception/PathParseException.java |  26 ++
 .../org/apache/iotdb/tsfile/read/common/Path.java  |  78 ++--
 .../read/common/parser/PathNodesGenerator.java     |  94 ++++
 .../tsfile/read/common/parser/PathParseError.java  |  39 ++
 .../tsfile/read/common/parser/PathVisitor.java     |  75 +++
 .../apache/iotdb/tsfile/read/common/PathTest.java  | 163 ++++++-
 66 files changed, 1655 insertions(+), 735 deletions(-)

diff --git a/antlr/pom.xml b/antlr/pom.xml
index 6401579638..a66513db4c 100644
--- a/antlr/pom.xml
+++ b/antlr/pom.xml
@@ -47,6 +47,7 @@
                         <configuration>
                             <listener>false</listener>
                             <visitor>true</visitor>
+                            <libDirectory>src/main/antlr4/org/apache/iotdb/db/qp/sql</libDirectory>
                         </configuration>
                         <goals>
                             <goal>antlr4</goal>
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
new file mode 100644
index 0000000000..87814a4288
--- /dev/null
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+
+parser grammar IdentifierParser;
+
+options { tokenVocab=SqlLexer; }
+
+identifier
+     : keyWords
+     | ID
+     | QUOTED_ID
+     ;
+
+
+// List of keywords, new keywords should be added into this list.
+
+keyWords
+    : ADD
+    | AFTER
+    | ALIAS
+    | ALIGN
+    | ALIGNED
+    | ALL
+    | ALTER
+    | ANY
+    | APPEND
+    | AS
+    | ASC
+    | ATTRIBUTES
+    | AUTOREGISTER
+    | BEFORE
+    | BEGIN
+    | BOUNDARY
+    | BY
+    | CACHE
+    | CHILD
+    | CLEAR
+    | COMPRESSION
+    | COMPRESSOR
+    | CONCAT
+    | CONFIGURATION
+    | CONTINUOUS
+    | COUNT
+    | CONTAIN
+    | CQ
+    | CQS
+    | CREATE
+    | DATATYPE
+    | DEBUG
+    | DELETE
+    | DESC
+    | DESCRIBE
+    | DEVICE
+    | DEVICES
+    | DISABLE
+    | DROP
+    | ENCODING
+    | END
+    | EVERY
+    | EXPLAIN
+    | FILL
+    | FLUSH
+    | FOR
+    | FROM
+    | FULL
+    | FUNCTION
+    | FUNCTIONS
+    | GLOBAL
+    | GRANT
+    | GROUP
+    | INDEX
+    | INFO
+    | INSERT
+    | INTO
+    | KILL
+    | LABEL
+    | LAST
+    | LATEST
+    | LEVEL
+    | LIKE
+    | LIMIT
+    | LINEAR
+    | LINK
+    | LIST
+    | LOAD
+    | LOCK
+    | MERGE
+    | METADATA
+    | NODES
+    | NOW
+    | OF
+    | OFF
+    | OFFSET
+    | ON
+    | ORDER
+    | PARTITION
+    | PASSWORD
+    | PATHS
+    | PIPE
+    | PIPES
+    | PIPESERVER
+    | PIPESINK
+    | PIPESINKS
+    | PIPESINKTYPE
+    | PREVIOUS
+    | PREVIOUSUNTILLAST
+    | PRIVILEGES
+    | PROCESSLIST
+    | PROPERTY
+    | PRUNE
+    | QUERIES
+    | QUERY
+    | READONLY
+    | REGEXP
+    | REMOVE
+    | RENAME
+    | RESAMPLE
+    | RESOURCE
+    | REVOKE
+    | ROLE
+    | SCHEMA
+    | SELECT
+    | SET
+    | SETTLE
+    | SGLEVEL
+    | SHOW
+    | SLIMIT
+    | SOFFSET
+    | STORAGE
+    | START
+    | STOP
+    | SYSTEM
+    | TAGS
+    | TASK
+    | TEMPLATE
+    | TEMPLATES
+    | TIMESERIES
+    | TO
+    | TOLERANCE
+    | TOP
+    | TRACING
+    | TRIGGER
+    | TRIGGERS
+    | TTL
+    | UNLINK
+    | UNLOAD
+    | UNSET
+    | UPDATE
+    | UPSERT
+    | USER
+    | USING
+    | VALUES
+    | VERIFY
+    | VERSION
+    | WHERE
+    | WITH
+    | WITHOUT
+    | WRITABLE
+    | DATATYPE_VALUE
+    | ENCODING_VALUE
+    | COMPRESSOR_VALUE
+    | PRIVILEGE_VALUE
+    ;
\ No newline at end of file
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/InfluxDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/InfluxDBSqlParser.g4
index 0ffdcb9f22..381f3e2979 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/InfluxDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/InfluxDBSqlParser.g4
@@ -21,6 +21,8 @@ parser grammar InfluxDBSqlParser;
 
 options { tokenVocab=SqlLexer; }
 
+import IdentifierParser;
+
 singleStatement
     : statement SEMI? EOF
     ;
@@ -71,20 +73,12 @@ fromClause
 
 nodeName
     : STAR
-    | ID
-    | QUOTED_ID
+    | identifier
     | LAST
     | COUNT
     | DEVICE
     ;
 
-// Identifier
-
-identifier
-    : ID
-    | QUOTED_ID
-    ;
-
 
 // Constant & Literal
 
@@ -126,4 +120,4 @@ realLiteral
 datetimeLiteral
     : DATETIME_LITERAL
     | NOW LR_BRACKET RR_BRACKET
-    ;
+    ;
\ No newline at end of file
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
index 56ca7bd9b5..d1b1565cec 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
@@ -21,6 +21,7 @@ parser grammar IoTDBSqlParser;
 
 options { tokenVocab=SqlLexer; }
 
+import IdentifierParser;
 
 /**
  * 1. Top Level Description
@@ -789,14 +790,6 @@ wildcard
     ;
 
 
-// Identifier
-
-identifier
-    : ID
-    | QUOTED_ID
-    ;
-
-
 // Constant & Literal
 
 constant
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/PathParser.g4
similarity index 51%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
copy to antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/PathParser.g4
index be8ecb3d5e..546be58b6e 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/PathParser.g4
@@ -16,23 +16,37 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.tsfile.common.constant;
-
-public class TsFileConstant {
-
-  public static final String TSFILE_SUFFIX = ".tsfile";
-  public static final String TSFILE_HOME = "TSFILE_HOME";
-  public static final String TSFILE_CONF = "TSFILE_CONF";
-  public static final String PATH_ROOT = "root";
-  public static final String TMP_SUFFIX = "tmp";
-  public static final String PATH_SEPARATOR = ".";
-  public static final char PATH_SEPARATOR_CHAR = '.';
-  public static final String PATH_SEPARATER_NO_REGEX = "\\.";
-  public static final char DOUBLE_QUOTE = '"';
-  public static final char BACK_QUOTE = '`';
-
-  public static final byte TIME_COLUMN_MASK = (byte) 0x80;
-  public static final byte VALUE_COLUMN_MASK = (byte) 0x40;
-
-  private TsFileConstant() {}
-}
+
+parser grammar PathParser;
+
+options { tokenVocab=SqlLexer; }
+
+import IdentifierParser;
+
+/**
+ * PartialPath and Path used by Session API and TsFile API should be parsed by Antlr4.
+ */
+
+path
+    : prefixPath EOF
+    | suffixPath EOF
+    ;
+
+prefixPath
+    : ROOT (DOT nodeName)*
+    ;
+
+suffixPath
+    : nodeName (DOT nodeName)*
+    ;
+
+nodeName
+    : wildcard
+    | wildcard? identifier wildcard?
+    | identifier
+    ;
+
+wildcard
+    : STAR
+    | DOUBLE_STAR
+    ;
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
index f92fcbba5b..04bad754a2 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
@@ -32,7 +32,7 @@ WS
 
 
 /**
- * 2. Keywords
+ * 2. Keywords, new keywords should be added into IdentifierParser.g4
  */
 
 // Common Keywords
@@ -975,7 +975,7 @@ NAN_LITERAL
 
 
 /**
- * 6. Identifier
+ * 6. ID
  */
 
 ID
diff --git a/client-py/tests/test_dataframe.py b/client-py/tests/test_dataframe.py
index f95ade6e08..225556eed3 100644
--- a/client-py/tests/test_dataframe.py
+++ b/client-py/tests/test_dataframe.py
@@ -29,7 +29,7 @@ def test_simple_query():
         session.open(False)
 
         # Write data
-        session.insert_str_record("root.device", 123, "pressure", "15.0")
+        session.insert_str_record("root.device0", 123, "pressure", "15.0")
 
         # Read
         session_data_set = session.execute_query_statement("SELECT ** FROM root")
@@ -37,7 +37,7 @@ def test_simple_query():
 
         session.close()
 
-    assert list(df.columns) == ["Time", "root.device.pressure"]
+    assert list(df.columns) == ["Time", "root.device0.pressure"]
     assert_array_equal(df.values, [[123.0, 15.0]])
 
 
@@ -48,7 +48,7 @@ def test_non_time_query():
         session.open(False)
 
         # Write data
-        session.insert_str_record("root.device", 123, "pressure", "15.0")
+        session.insert_str_record("root.device0", 123, "pressure", "15.0")
 
         # Read
         session_data_set = session.execute_query_statement("SHOW TIMESERIES")
@@ -70,9 +70,9 @@ def test_non_time_query():
         df.values,
         [
             [
-                "root.device.pressure",
+                "root.device0.pressure",
                 None,
-                "root.device",
+                "root.device0",
                 "FLOAT",
                 "GORILLA",
                 "SNAPPY",
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java
index e136a6bf60..bc99e27b3d 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java
@@ -451,7 +451,7 @@ public class SyncClientAdaptorTest {
         lastResult,
         SyncClientAdaptor.last(
             dataClient,
-            Collections.singletonList(new PartialPath("1")),
+            Collections.singletonList(new PartialPath("`1`")),
             Collections.singletonList(TSDataType.INT64.ordinal()),
             null,
             new QueryContext(),
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfoTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfoTest.java
index 0735a83de0..a2eb84a6ec 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfoTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfoTest.java
@@ -91,7 +91,7 @@ public class ClusterSchemaInfoTest {
         storageGroupPathList.size(), clusterSchemaInfo.getStorageGroupNames().size());
 
     GetStorageGroupReq getStorageGroupReq =
-        new GetStorageGroupReq(Arrays.asList(PathUtils.splitPathToDetachedPath("root.**")));
+        new GetStorageGroupReq(Arrays.asList(PathUtils.splitPathToDetachedNodes("root.**")));
     Map<String, TStorageGroupSchema> reloadResult =
         clusterSchemaInfo.getMatchedStorageGroupSchemas(getStorageGroupReq).getSchemaMap();
     Assert.assertEquals(testMap, reloadResult);
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBContinuousQueryIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBContinuousQueryIT.java
index 56ed6e6b27..4dbb02d3e4 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBContinuousQueryIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBContinuousQueryIT.java
@@ -241,6 +241,48 @@ public class IoTDBContinuousQueryIT {
     stopDataGenerator();
   }
 
+  public void testContinuousQueryResultSeriesWithLevels1() throws Exception {
+    createTimeSeries(
+        new String[] {
+          "root.ln.wf01.wt01.ws01.`(temperature)`",
+          "root.ln.wf01.wt01.ws02.`(temperature)`",
+          "root.ln.wf01.wt02.ws01.`(temperature)`",
+          "root.ln.wf01.wt02.ws02.`(temperature)`",
+          "root.ln.wf02.wt01.ws01.`(temperature)`",
+          "root.ln.wf02.wt01.ws02.`(temperature)`",
+          "root.ln.wf02.wt02.ws01.`(temperature)`",
+          "root.ln.wf02.wt02.ws02.`(temperature)`"
+        });
+    startDataGenerator();
+
+    Thread.sleep(500);
+
+    statement.execute(
+        "CREATE CONTINUOUS QUERY cq1 "
+            + "BEGIN SELECT count(`(temperature)`) INTO temperature_cnt FROM root.ln.*.*.* "
+            + "GROUP BY time(1s), level=1,2 END");
+
+    Thread.sleep(5500);
+
+    checkShowTimeSeriesResult(
+        new String[] {
+          "root.ln.wf01.wt01.ws01.`(temperature)`",
+          "root.ln.wf01.wt01.ws02.`(temperature)`",
+          "root.ln.wf01.wt02.ws01.`(temperature)`",
+          "root.ln.wf01.wt02.ws02.`(temperature)`",
+          "root.ln.wf02.wt01.ws01.`(temperature)`",
+          "root.ln.wf02.wt01.ws02.`(temperature)`",
+          "root.ln.wf02.wt02.ws01.`(temperature)`",
+          "root.ln.wf02.wt02.ws02.`(temperature)`",
+          "root.ln.wf01.temperature_cnt",
+          "root.ln.wf02.temperature_cnt"
+        });
+
+    statement.execute("DROP CONTINUOUS QUERY cq1");
+
+    stopDataGenerator();
+  }
+
   @Test
   public void testContinuousQueryResultSeriesWithDuplicatedTargetPaths() throws Exception {
     createTimeSeries(
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java
index aa759e8732..ad9d1e6fd0 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java
@@ -212,7 +212,7 @@ public class IoTDBCreateTimeseriesIT {
       "root.sg.d.`a.b`", "root.sg.d.`a“(Φ)”b`", "root.sg.d.`a>b`",
     };
     String[] timeSeriesResArray = {
-      "root.sg.d.`a.b`", "root.sg.d.a“(Φ)”b", "root.sg.d.a>b",
+      "root.sg.d.`a.b`", "root.sg.d.`a“(Φ)”b`", "root.sg.d.`a>b`",
     };
 
     for (String timeSeries : timeSeriesArray) {
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBQuotedPathIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBQuotedPathIT.java
index 8a1086dada..716a3cd4a5 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBQuotedPathIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBQuotedPathIT.java
@@ -115,7 +115,7 @@ public class IoTDBQuotedPathIT {
       statement.execute("SET STORAGE GROUP TO root.`\"ln`");
     } catch (IoTDBSQLException e) {
       Assert.assertEquals(
-          "315: The storage group name can only be characters, numbers and underscores. root.\"ln is not a legal path",
+          "315: The storage group name can only be characters, numbers and underscores. root.`\"ln` is not a legal path",
           e.getMessage());
     } catch (Exception e) {
       e.printStackTrace();
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectIntoIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectIntoIT.java
index df3541d9f5..70cfd8c509 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectIntoIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectIntoIT.java
@@ -125,7 +125,7 @@ public class IoTDBSelectIntoIT {
         null);
 
     IoTDB.schemaProcessor.createTimeseries(
-        new PartialPath("root.sg.d1.datatype"),
+        new PartialPath("root.sg.d1.`datatype`"),
         TSDataType.DOUBLE,
         TSEncoding.PLAIN,
         CompressionType.UNCOMPRESSED,
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
index a7804ca2c9..aa42ea1591 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
@@ -1164,7 +1164,7 @@ public class IoTDBSimpleQueryIT {
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
       statement.setFetchSize(5);
-      statement.execute("SET STORAGE GROUP TO root.`group-with-hyphen`");
+      statement.execute("SET STORAGE GROUP TO root.group_with_hyphen");
     } catch (SQLException e) {
       fail();
     }
@@ -1180,7 +1180,7 @@ public class IoTDBSimpleQueryIT {
             for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
               builder.append(resultSet.getString(i));
             }
-            Assert.assertEquals(builder.toString(), "root.group-with-hyphen");
+            Assert.assertEquals(builder.toString(), "root.group_with_hyphen");
           }
         }
       }
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionIdentifierIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionIdentifierIT.java
index 81fb785122..5a22435269 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionIdentifierIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSyntaxConventionIdentifierIT.java
@@ -47,6 +47,99 @@ public class IoTDBSyntaxConventionIdentifierIT {
     EnvFactory.getEnv().cleanAfterTest();
   }
 
+  @Test
+  public void testKeyWord() {
+    String[] createNodeNames = {
+      "add",
+      "as",
+      "between",
+      "select",
+      "drop_trigger",
+      "REVOKE_USER_ROLE",
+      "pipesink",
+      "boolean",
+      "datatype",
+      "device",
+    };
+
+    String[] resultTimeseries = {
+      "root.sg1.d1.add",
+      "root.sg1.d1.as",
+      "root.sg1.d1.between",
+      "root.sg1.d1.select",
+      "root.sg1.d1.drop_trigger",
+      "root.sg1.d1.REVOKE_USER_ROLE",
+      "root.sg1.d1.pipesink",
+      "root.sg1.d1.boolean",
+      "root.sg1.d1.datatype",
+      "root.sg1.d1.device",
+    };
+
+    String[] selectNodeNames = {
+      "add",
+      "as",
+      "between",
+      "select",
+      "drop_trigger",
+      "REVOKE_USER_ROLE",
+      "pipesink",
+      "boolean",
+      "datatype",
+      "device",
+    };
+
+    String[] suffixInResultColumns = {
+      "add",
+      "as",
+      "between",
+      "select",
+      "drop_trigger",
+      "REVOKE_USER_ROLE",
+      "pipesink",
+      "boolean",
+      "datatype",
+      "device",
+    };
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      for (int i = 0; i < createNodeNames.length; i++) {
+        String createSql =
+            String.format("CREATE TIMESERIES root.sg1.d1.%s INT32", createNodeNames[i]);
+        String insertSql =
+            String.format("INSERT INTO root.sg1.d1(time, %s) VALUES(1, 1)", createNodeNames[i]);
+        statement.execute(createSql);
+        statement.execute(insertSql);
+      }
+
+      boolean hasResult = statement.execute("SHOW TIMESERIES");
+      Assert.assertTrue(hasResult);
+      Set<String> expectedResult = new HashSet<>(Arrays.asList(resultTimeseries));
+
+      ResultSet resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        Assert.assertTrue(expectedResult.contains(resultSet.getString("timeseries")));
+        expectedResult.remove(resultSet.getString("timeseries"));
+      }
+      Assert.assertEquals(0, expectedResult.size());
+
+      for (int i = 0; i < selectNodeNames.length; i++) {
+        String selectSql =
+            String.format("SELECT %s FROM root.sg1.d1 WHERE time = 1", selectNodeNames[i]);
+        hasResult = statement.execute(selectSql);
+        Assert.assertTrue(hasResult);
+
+        resultSet = statement.getResultSet();
+        Assert.assertTrue(resultSet.next());
+        Assert.assertEquals(1, resultSet.getInt("root.sg1.d1." + suffixInResultColumns[i]));
+      }
+
+    } catch (SQLException e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
   @Test
   public void testNodeName() {
     String[] createNodeNames = {
@@ -61,6 +154,8 @@ public class IoTDBSyntaxConventionIdentifierIT {
       "`a.'b`",
       "````",
       "`c.d.```",
+      "`abc`",
+      "a1_:@#${}"
     };
 
     String[] resultTimeseries = {
@@ -69,12 +164,14 @@ public class IoTDBSyntaxConventionIdentifierIT {
       "root.sg1.d1.aaa",
       "root.sg1.d1.select",
       "root.sg1.d1.`a.b`",
-      "root.sg1.d1.111",
+      "root.sg1.d1.`111`",
       "root.sg1.d1.`a``b`",
       "root.sg1.d1.`a.\"b`",
       "root.sg1.d1.`a.'b`",
       "root.sg1.d1.````",
       "root.sg1.d1.`c.d.```",
+      "root.sg1.d1.abc",
+      "root.sg1.d1.a1_:@#${}",
     };
 
     String[] selectNodeNames = {
@@ -89,6 +186,8 @@ public class IoTDBSyntaxConventionIdentifierIT {
       "`a.'b`",
       "````",
       "`c.d.```",
+      "abc",
+      "a1_:@#${}",
     };
 
     String[] suffixInResultColumns = {
@@ -97,12 +196,14 @@ public class IoTDBSyntaxConventionIdentifierIT {
       "aaa",
       "select",
       "`a.b`",
-      "111",
+      "`111`",
       "`a``b`",
       "`a.\"b`",
       "`a.'b`",
       "````",
       "`c.d.```",
+      "abc",
+      "a1_:@#${}",
     };
 
     try (Connection connection = EnvFactory.getEnv().getConnection();
@@ -200,6 +301,56 @@ public class IoTDBSyntaxConventionIdentifierIT {
       } catch (Exception ignored) {
       }
 
+      try {
+        statement.execute("create timeseries root.sg1.d1.``a` INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
+      // unsupported
+      try {
+        statement.execute("create timeseries root.sg1.d1.contains INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
+      try {
+        statement.execute("create timeseries root.sg1.d1.and INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
+      try {
+        statement.execute("create timeseries root.sg1.d1.or INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
+      try {
+        statement.execute("create timeseries root.sg1.d1.not INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
+      // reserved words can not be identifier
+      try {
+        statement.execute("create timeseries root.sg1.d1.root INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
+      try {
+        statement.execute("create timeseries root.sg1.d1.time INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
+      try {
+        statement.execute("create timeseries root.sg1.d1.timestamp INT32");
+        fail();
+      } catch (Exception ignored) {
+      }
+
     } catch (SQLException e) {
       e.printStackTrace();
       fail();
@@ -426,9 +577,8 @@ public class IoTDBSyntaxConventionIdentifierIT {
         Statement statement = connection.createStatement()) {
       try {
         statement.execute(
-            "create trigger trigger before insert on root.sg1.d1  "
+            "create trigger trigger` before insert on root.sg1.d1  "
                 + "as 'org.apache.iotdb.db.engine.trigger.example.Accumulator'");
-        fail();
       } catch (Exception ignored) {
       }
 
@@ -543,7 +693,7 @@ public class IoTDBSyntaxConventionIdentifierIT {
     };
 
     String[] resultNames = {
-      "id", "ID", "id0", "_id", "0id", "233", "ab!", "\"ab\"", "\"ac\"", "'ab'", "a.b", "a`b"
+      "id", "ID", "id0", "_id", "0id", "233", "ab!", "\"ab\"", "\\\"ac\\\"", "'ab'", "a.b", "a`b"
     };
 
     try (Connection connection = EnvFactory.getEnv().getConnection();
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/withoutNullFilter/IoTDBWithoutNullAnyFilterIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/withoutNullFilter/IoTDBWithoutNullAnyFilterIT.java
index 076ed3e709..eb279eb206 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/withoutNullFilter/IoTDBWithoutNullAnyFilterIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/withoutNullFilter/IoTDBWithoutNullAnyFilterIT.java
@@ -1254,7 +1254,7 @@ public class IoTDBWithoutNullAnyFilterIT {
         Statement statement = connection.createStatement()) {
       boolean hasResultSet =
           statement.execute(
-              "select s2, s3 from root.test.** without null any(`root`.test.sg1.s2, `root`.test.sg2.s3)");
+              "select s2, s3 from root.test.** without null any(root.test.sg1.s2, root.test.sg2.s3)");
       String[] columns =
           new String[] {
             "root.test.sg1.s2", "root.test.sg1.s3", "root.test.sg2.s2", "root.test.sg2.s3"
@@ -1282,7 +1282,7 @@ public class IoTDBWithoutNullAnyFilterIT {
 
       hasResultSet =
           statement.execute(
-              "select s2, s3 from root.test.sg1, root.test.sg2 without null any(`root`.test.sg1.s2)");
+              "select s2, s3 from root.test.sg1, root.test.sg2 without null any(root.test.sg1.s2)");
 
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -1306,7 +1306,7 @@ public class IoTDBWithoutNullAnyFilterIT {
 
       hasResultSet =
           statement.execute(
-              "select s2, s3 from root.test.sg1, root.test.sg2 without null any(`root`.test.sg1.s2, s3)");
+              "select s2, s3 from root.test.sg1, root.test.sg2 without null any(root.test.sg1.s2, s3)");
 
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -1330,7 +1330,7 @@ public class IoTDBWithoutNullAnyFilterIT {
 
       hasResultSet =
           statement.execute(
-              "select s2, s3 from root.test.sg1, root.test.sg2 without null any(`root`.test.*.s2)");
+              "select s2, s3 from root.test.sg1, root.test.sg2 without null any(root.test.*.s2)");
 
       Assert.assertTrue(hasResultSet);
       try (ResultSet resultSet = statement.getResultSet()) {
@@ -1427,7 +1427,7 @@ public class IoTDBWithoutNullAnyFilterIT {
         Statement statement = connection.createStatement()) {
       boolean hasResultSet =
           statement.execute(
-              "select last_value(*) from root.test.** group by([1,10), 2ms) without null any(last_value(`root.test.sg1.s2`)) align by device");
+              "select last_value(*) from root.test.** group by([1,10), 2ms) without null any(last_value(root.test.sg1.s2)) align by device");
     } catch (Exception e) {
       Assert.assertTrue(e.getMessage().contains(QueryPlan.WITHOUT_NULL_FILTER_ERROR_MESSAGE));
     }
diff --git a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
index 29126d05f3..3aeb9de5ee 100644
--- a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
+++ b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
@@ -419,17 +419,17 @@ public class IoTDBSessionComplexIT {
     session.createTimeseries(
         "root.sg1.d1.1_2", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
     session.createTimeseries(
-        "root.sg1.d1.1+2+3", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+        "root.sg1.d1.`1+2+3`", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
     session.createTimeseries(
-        "root.sg1.d1.1+2+4", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+        "root.sg1.d1.`1+2+4`", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
 
     Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.1_2"));
     Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.`1+2+3`"));
     Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.`1+2+4`"));
 
-    session.setStorageGroup("root.1");
+    session.setStorageGroup("root.`1`");
     session.createTimeseries(
-        "root.1.2.3", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+        "root.`1`.`2`.`3`", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
     session.setStorageGroup("root.sg2");
     session.createTimeseries(
         "root.sg2.d1.s1", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
diff --git a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
index 98ec8ba4c7..ae285a908c 100644
--- a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
+++ b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
@@ -530,7 +530,7 @@ public class IoTDBSessionSimpleIT {
     session.setStorageGroup(storageGroup);
 
     session.createTimeseries(
-        "root.sg.\"my.device.with.colon:\".s",
+        "root.sg.`my.device.with.colon:`.s",
         TSDataType.INT64,
         TSEncoding.RLE,
         CompressionType.SNAPPY);
@@ -561,6 +561,12 @@ public class IoTDBSessionSimpleIT {
       logger.error("", e);
     }
 
+    try {
+      session.createTimeseries("", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+    } catch (IoTDBConnectionException | StatementExecutionException e) {
+      logger.error("", e);
+    }
+
     final SessionDataSet dataSet = session.executeQueryStatement("SHOW TIMESERIES");
     assertFalse(dataSet.hasNext());
 
diff --git a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSyntaxConventionIT.java b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSyntaxConventionIT.java
index e83726f5f2..37661b80ba 100644
--- a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSyntaxConventionIT.java
+++ b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionSyntaxConventionIT.java
@@ -95,14 +95,14 @@ public class IoTDBSessionSyntaxConventionIT {
     String deviceId = "root.sg1.d1";
     List<String> measurements = new ArrayList<>();
 
-    measurements.add("\"a“(Φ)”b\"");
-    measurements.add("\"a>b\"");
-    measurements.add("'a.b'");
-    measurements.add("'a“(Φ)”b'");
-    measurements.add("'a>b'");
-    measurements.add("a“(Φ)”b");
-    measurements.add("a>b");
-    measurements.add("\\\"a");
+    measurements.add("`\"a“(Φ)”b\"`");
+    measurements.add("`\"a>b\"`");
+    measurements.add("`'a.b'`");
+    measurements.add("`'a“(Φ)”b'`");
+    measurements.add("`'a>b'`");
+    measurements.add("`a“(Φ)”b`");
+    measurements.add("`a>b`");
+    measurements.add("`\\\"a`");
     List<String> values = new ArrayList<>();
 
     for (int i = 0; i < measurements.size(); i++) {
@@ -118,7 +118,7 @@ public class IoTDBSessionSyntaxConventionIT {
     Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.`'a>b'`"));
     Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.`a“(Φ)”b`"));
     Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.`a>b`"));
-    Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.`\\\\\"a`"));
+    Assert.assertTrue(session.checkTimeseriesExists("root.sg1.d1.`\\\"a`"));
 
     session.close();
   }
diff --git a/integration/src/test/java/org/apache/iotdb/session/SessionTest.java b/integration/src/test/java/org/apache/iotdb/session/SessionTest.java
index d92f5c8204..3e33ebc662 100644
--- a/integration/src/test/java/org/apache/iotdb/session/SessionTest.java
+++ b/integration/src/test/java/org/apache/iotdb/session/SessionTest.java
@@ -612,40 +612,40 @@ public class SessionTest {
 
     // path does not exist test
     try {
-      session.unsetSchemaTemplate("root.sg.1", "template1");
+      session.unsetSchemaTemplate("root.sg.`1`", "template1");
       fail("No exception thrown.");
     } catch (Exception e) {
-      assertEquals("304: Path [root.sg.1] does not exist", e.getMessage());
+      assertEquals("304: Path [root.sg.`1`] does not exist", e.getMessage());
     }
 
-    session.setSchemaTemplate("template1", "root.sg.1");
+    session.setSchemaTemplate("template1", "root.sg.`1`");
 
     // template already exists test
     try {
-      session.setSchemaTemplate("template1", "root.sg.1");
+      session.setSchemaTemplate("template1", "root.sg.`1`");
       fail("No exception thrown.");
     } catch (Exception e) {
-      assertEquals("303: Template already exists on root.sg.1", e.getMessage());
+      assertEquals("303: Template already exists on root.sg.`1`", e.getMessage());
     }
 
     // template unset test
-    session.unsetSchemaTemplate("root.sg.1", "template1");
+    session.unsetSchemaTemplate("root.sg.`1`", "template1");
 
-    session.setSchemaTemplate("template1", "root.sg.1");
+    session.setSchemaTemplate("template1", "root.sg.`1`");
 
     // no template on path test
-    session.unsetSchemaTemplate("root.sg.1", "template1");
+    session.unsetSchemaTemplate("root.sg.`1`", "template1");
     try {
-      session.unsetSchemaTemplate("root.sg.1", "template1");
+      session.unsetSchemaTemplate("root.sg.`1`", "template1");
       fail("No exception thrown.");
     } catch (Exception e) {
-      assertEquals("324: NO template on root.sg.1", e.getMessage());
+      assertEquals("324: NO template on root.sg.`1`", e.getMessage());
     }
 
     // template is in use test
-    session.setSchemaTemplate("template1", "root.sg.1");
+    session.setSchemaTemplate("template1", "root.sg.`1`");
 
-    String deviceId = "root.sg.1.cd";
+    String deviceId = "root.sg.`1`.cd";
     List<String> measurements = new ArrayList<>();
     List<TSDataType> types = new ArrayList<>();
     measurements.add("s1");
@@ -664,10 +664,10 @@ public class SessionTest {
     }
 
     try {
-      session.unsetSchemaTemplate("root.sg.1", "template1");
+      session.unsetSchemaTemplate("root.sg.`1`", "template1");
       fail("No exception thrown.");
     } catch (Exception e) {
-      assertEquals("326: Template is in use on root.sg.1.cd", e.getMessage());
+      assertEquals("326: Template is in use on root.sg.`1`.cd", e.getMessage());
     }
   }
 }
diff --git a/integration/src/test/java/org/apache/iotdb/session/template/TemplateUT.java b/integration/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
index efe2558d5a..6ea9dc5b97 100644
--- a/integration/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
+++ b/integration/src/test/java/org/apache/iotdb/session/template/TemplateUT.java
@@ -248,13 +248,13 @@ public class TemplateUT {
 
     session.addAlignedMeasurementsInTemplate(
         "template1",
-        Collections.singletonList("append"),
+        Collections.singletonList("append1"),
         Collections.singletonList(TSDataType.INT64),
         Collections.singletonList(TSEncoding.RLE),
         Collections.singletonList(CompressionType.UNCOMPRESSED));
 
     try {
-      session.deleteNodeInTemplate("template1", "append");
+      session.deleteNodeInTemplate("template1", "append1");
       fail();
     } catch (StatementExecutionException e) {
       assertEquals(
@@ -283,7 +283,7 @@ public class TemplateUT {
       session.insertAlignedRecord(
           "root.sg.v1",
           110L,
-          Collections.singletonList("append"),
+          Collections.singletonList("append1"),
           Collections.singletonList(TSDataType.TEXT),
           Collections.singletonList("aaa"));
       fail();
@@ -294,13 +294,13 @@ public class TemplateUT {
     try {
       session.addAlignedMeasurementsInTemplate(
           "template1",
-          Collections.singletonList("append"),
+          Collections.singletonList("append1"),
           Collections.singletonList(TSDataType.TEXT),
           Collections.singletonList(TSEncoding.PLAIN),
           Collections.singletonList(CompressionType.UNCOMPRESSED));
       fail();
     } catch (StatementExecutionException e) {
-      assertEquals("315: Path duplicated: append is not a legal path", e.getMessage());
+      assertEquals("315: Path duplicated: append1 is not a legal path", e.getMessage());
     }
 
     try {
@@ -347,7 +347,7 @@ public class TemplateUT {
     session.insertAlignedRecord(
         "root.sg.v1",
         110L,
-        Collections.singletonList("append"),
+        Collections.singletonList("append1"),
         Collections.singletonList(TSDataType.INT64),
         Collections.singletonList(12345L));
 
@@ -362,7 +362,7 @@ public class TemplateUT {
     session.insertAlignedRecord(
         "root.sg.v1.d0",
         110L,
-        Collections.singletonList("append"),
+        Collections.singletonList("append1"),
         Collections.singletonList(TSDataType.INT64),
         Collections.singletonList(12345L));
 
diff --git a/node-commons/pom.xml b/node-commons/pom.xml
index 20247b3fe5..ef0ca10146 100644
--- a/node-commons/pom.xml
+++ b/node-commons/pom.xml
@@ -83,6 +83,11 @@
             <artifactId>iotdb-thrift-confignode</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-antlr</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <!--        compile group: 'io.jsonwebtoken', name: 'jjwt', version: '0.9.1'-->
         <dependency>
             <groupId>io.jsonwebtoken</groupId>
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java b/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java
index b11810e8e6..003e0e8f6b 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java
@@ -57,19 +57,25 @@ public class PartialPath extends Path implements Comparable<Path>, Cloneable {
 
   /**
    * Construct the PartialPath using a String, will split the given String into String[] E.g., path
-   * = "root.sg.`d.1`.`s.1`" nodes = {"root", "sg", "d.1", "s.1"}
+   * = "root.sg.`d.1`.`s.1`" nodes = {"root", "sg", "`d.1`", "`s.1`"}
    *
    * @param path a full String of a time series path
    * @throws IllegalPathException
    */
   public PartialPath(String path) throws IllegalPathException {
-    this.nodes = PathUtils.splitPathToDetachedPath(path);
-    this.fullPath = path;
+    this.nodes = PathUtils.splitPathToDetachedNodes(path);
+    // path is root.sg.`abc`, fullPath is root.sg.abc
+    // path is root.sg.`select`, fullPath is root.sg.select
+    // path is root.sg.`111`, fullPath is root.sg.`111`
+    // path is root.sg.`a.b`, fullPath is root.sg.`a.b`
+    // path is root.sg.`a``b`, fullPath is root.sg.`a``b`
+    this.fullPath = getFullPath();
   }
 
   public PartialPath(String device, String measurement) throws IllegalPathException {
-    this.fullPath = device + TsFileConstant.PATH_SEPARATOR + measurement;
-    this.nodes = PathUtils.splitPathToDetachedPath(fullPath);
+    String path = device + TsFileConstant.PATH_SEPARATOR + measurement;
+    this.nodes = PathUtils.splitPathToDetachedNodes(path);
+    this.fullPath = getFullPath();
   }
 
   /** @param partialNodes nodes of a time series path */
@@ -79,11 +85,16 @@ public class PartialPath extends Path implements Comparable<Path>, Cloneable {
 
   /**
    * @param path path
-   * @param needSplit needSplit is basically false, whether need to be split to device and
-   *     measurement, doesn't support escape character yet.
+   * @param needSplit whether to split path to nodes, needSplit can only be false.
    */
   public PartialPath(String path, boolean needSplit) {
-    super(path, needSplit);
+    Validate.isTrue(!needSplit);
+    fullPath = path;
+    if ("".equals(path)) {
+      this.nodes = new String[] {};
+    } else {
+      this.nodes = new String[] {path};
+    }
   }
 
   /**
@@ -336,9 +347,9 @@ public class PartialPath extends Path implements Comparable<Path>, Cloneable {
   @Override
   public String getFullPath() {
     if (fullPath == null) {
-      StringBuilder s = new StringBuilder(parseNodeString(nodes[0]));
+      StringBuilder s = new StringBuilder(nodes[0]);
       for (int i = 1; i < nodes.length; i++) {
-        s.append(TsFileConstant.PATH_SEPARATOR).append(parseNodeString(nodes[i]));
+        s.append(TsFileConstant.PATH_SEPARATOR).append(nodes[i]);
       }
       fullPath = s.toString();
     }
@@ -398,28 +409,14 @@ public class PartialPath extends Path implements Comparable<Path>, Cloneable {
       if (nodes.length == 1) {
         return "";
       }
-      StringBuilder s = new StringBuilder(parseNodeString(nodes[0]));
+      StringBuilder s = new StringBuilder(nodes[0]);
       for (int i = 1; i < nodes.length - 1; i++) {
         s.append(TsFileConstant.PATH_SEPARATOR);
-        s.append(parseNodeString(nodes[i]));
+        s.append(nodes[i]);
       }
       device = s.toString();
-      return device;
-    }
-  }
-
-  /**
-   * wrap node that has . or ` in it with ``
-   *
-   * @param node
-   * @return
-   */
-  protected String parseNodeString(String node) {
-    node = node.replace("`", "``");
-    if (node.contains("`") || node.contains(".")) {
-      return "`" + node + "`";
     }
-    return node;
+    return device;
   }
 
   // todo remove measurement related interface after invoker using MeasurementPath explicitly
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/PathUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/PathUtils.java
index ed541159b9..941c7d2569 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/utils/PathUtils.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/PathUtils.java
@@ -19,12 +19,8 @@
 package org.apache.iotdb.commons.utils;
 
 import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-
-import org.apache.commons.lang.StringEscapeUtils;
-
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.iotdb.tsfile.exception.PathParseException;
+import org.apache.iotdb.tsfile.read.common.parser.PathNodesGenerator;
 
 public class PathUtils {
 
@@ -33,66 +29,22 @@ public class PathUtils {
    * @return string array. ex, [root, ln]
    * @throws IllegalPathException if path isn't correct, the exception will throw
    */
-  public static String[] splitPathToDetachedPath(String path) throws IllegalPathException {
-    // NodeName is treated as identifier. When parsing identifier, unescapeJava is called.
-    // Therefore we call unescapeJava here.
-    path = StringEscapeUtils.unescapeJava(path);
-    if (path.endsWith(TsFileConstant.PATH_SEPARATOR)) {
+  public static String[] splitPathToDetachedNodes(String path) throws IllegalPathException {
+    if ("".equals(path)) {
+      return new String[] {};
+    }
+    try {
+      return PathNodesGenerator.splitPathToNodes(path);
+    } catch (PathParseException e) {
       throw new IllegalPathException(path);
     }
-    List<String> nodes = new ArrayList<>();
-    int startIndex = 0;
-    int endIndex;
-    int length = path.length();
-    for (int i = 0; i < length; i++) {
-      if (path.charAt(i) == TsFileConstant.PATH_SEPARATOR_CHAR) {
-        String node = path.substring(startIndex, i);
-        if (node.isEmpty()) {
-          throw new IllegalPathException(path);
-        }
-        nodes.add(node);
-        startIndex = i + 1;
-      } else if (path.charAt(i) == TsFileConstant.BACK_QUOTE) {
-        startIndex = i + 1;
-        endIndex = path.indexOf(TsFileConstant.BACK_QUOTE, startIndex);
-        if (endIndex == -1) {
-          // single '`', like root.sg.`s
-          throw new IllegalPathException(path);
-        }
-        while (endIndex != -1 && endIndex != length - 1) {
-          char afterQuote = path.charAt(endIndex + 1);
-          if (afterQuote == TsFileConstant.BACK_QUOTE) {
-            // for example, root.sg.```
-            if (endIndex == length - 2) {
-              throw new IllegalPathException(path);
-            }
-            endIndex = path.indexOf(TsFileConstant.BACK_QUOTE, endIndex + 2);
-          } else if (afterQuote == '.') {
-            break;
-          } else {
-            throw new IllegalPathException(path);
-          }
-        }
-        // replace `` with ` in a quoted identifier
-        String node = path.substring(startIndex, endIndex).replace("``", "`");
-        if (node.isEmpty()) {
-          throw new IllegalPathException(path);
-        }
+  }
 
-        nodes.add(node);
-        // skip the '.' after '`'
-        i = endIndex + 1;
-        startIndex = endIndex + 2;
-      }
-    }
-    // last node
-    if (startIndex <= path.length() - 1) {
-      String node = path.substring(startIndex);
-      if (node.isEmpty()) {
-        throw new IllegalPathException(path);
-      }
-      nodes.add(node);
+  public static void isLegalPath(String path) throws IllegalPathException {
+    try {
+      PathNodesGenerator.splitPathToNodes(path);
+    } catch (PathParseException e) {
+      throw new IllegalPathException(path);
     }
-    return nodes.toArray(new String[0]);
   }
 }
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/path/PartialPathTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/path/PartialPathTest.java
index babcf9262c..45d6ccd12f 100644
--- a/node-commons/src/test/java/org/apache/iotdb/commons/path/PartialPathTest.java
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/path/PartialPathTest.java
@@ -23,8 +23,512 @@ import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.fail;
+
 public class PartialPathTest {
 
+  @Test
+  public void testLegalPath() throws IllegalPathException {
+    String[] nodes;
+    // empty path
+    PartialPath a = new PartialPath("", false);
+    Assert.assertEquals("", a.getFullPath());
+    Assert.assertEquals(0, a.getNodes().length);
+
+    // suffix path
+    PartialPath b = new PartialPath("s1");
+    Assert.assertEquals("s1", b.getFullPath());
+    Assert.assertEquals("s1", b.getNodes()[0]);
+
+    // normal node
+    PartialPath c = new PartialPath("root.sg.a");
+    Assert.assertEquals("root.sg.a", c.getFullPath());
+    nodes = new String[] {"root", "sg", "a"};
+    checkNodes(nodes, c.getNodes());
+
+    // quoted node
+    PartialPath d = new PartialPath("root.sg.`a.b`");
+    Assert.assertEquals("root.sg.`a.b`", d.getFullPath());
+    nodes = new String[] {"root", "sg", "`a.b`"};
+    checkNodes(nodes, d.getNodes());
+
+    PartialPath e = new PartialPath("root.sg.`a.``b`");
+    Assert.assertEquals("root.sg.`a.``b`", e.getFullPath());
+    nodes = new String[] {"root", "sg", "`a.``b`"};
+    checkNodes(nodes, e.getNodes());
+
+    PartialPath f = new PartialPath("root.`sg\"`.`a.``b`");
+    Assert.assertEquals("root.`sg\"`.`a.``b`", f.getFullPath());
+    nodes = new String[] {"root", "`sg\"`", "`a.``b`"};
+    checkNodes(nodes, f.getNodes());
+
+    PartialPath g = new PartialPath("root.sg.`a.b\\\\`");
+    Assert.assertEquals("root.sg.`a.b\\\\`", g.getFullPath());
+    nodes = new String[] {"root", "sg", "`a.b\\\\`"};
+    checkNodes(nodes, g.getNodes());
+
+    // quoted node of digits
+    PartialPath h = new PartialPath("root.sg.`111`");
+    Assert.assertEquals("root.sg.`111`", h.getFullPath());
+    nodes = new String[] {"root", "sg", "`111`"};
+    checkNodes(nodes, h.getNodes());
+
+    // quoted node of key word
+    PartialPath i = new PartialPath("root.sg.`select`");
+    Assert.assertEquals("root.sg.select", i.getFullPath());
+    nodes = new String[] {"root", "sg", "select"};
+    checkNodes(nodes, i.getNodes());
+
+    // wildcard
+    PartialPath j = new PartialPath("root.sg.`a*b`");
+    Assert.assertEquals("root.sg.`a*b`", j.getFullPath());
+    nodes = new String[] {"root", "sg", "`a*b`"};
+    checkNodes(nodes, j.getNodes());
+
+    PartialPath k = new PartialPath("root.sg.*");
+    Assert.assertEquals("root.sg.*", k.getFullPath());
+    nodes = new String[] {"root", "sg", "*"};
+    checkNodes(nodes, k.getNodes());
+
+    PartialPath l = new PartialPath("root.sg.**");
+    Assert.assertEquals("root.sg.**", l.getFullPath());
+    nodes = new String[] {"root", "sg", "**"};
+    checkNodes(nodes, l.getNodes());
+
+    // raw key word
+    PartialPath m = new PartialPath("root.sg.select");
+    Assert.assertEquals("root.sg.select", m.getFullPath());
+    nodes = new String[] {"root", "sg", "select"};
+    checkNodes(nodes, m.getNodes());
+
+    PartialPath n = new PartialPath("root.sg.device");
+    Assert.assertEquals("root.sg.device", n.getFullPath());
+    nodes = new String[] {"root", "sg", "device"};
+    checkNodes(nodes, n.getNodes());
+
+    PartialPath o = new PartialPath("root.sg.datatype");
+    Assert.assertEquals("root.sg.datatype", o.getFullPath());
+    nodes = new String[] {"root", "sg", "datatype"};
+    checkNodes(nodes, o.getNodes());
+
+    PartialPath r = new PartialPath("root.sg.boolean");
+    Assert.assertEquals("root.sg.boolean", r.getFullPath());
+    nodes = new String[] {"root", "sg", "boolean"};
+    checkNodes(nodes, r.getNodes());
+
+    PartialPath s = new PartialPath("root.sg.DROP_TRIGGER");
+    Assert.assertEquals("root.sg.DROP_TRIGGER", s.getFullPath());
+    nodes = new String[] {"root", "sg", "DROP_TRIGGER"};
+    checkNodes(nodes, s.getNodes());
+
+    PartialPath t = new PartialPath("root.sg.`abc`");
+    Assert.assertEquals("root.sg.abc", t.getFullPath());
+    nodes = new String[] {"root", "sg", "abc"};
+    checkNodes(nodes, t.getNodes());
+  }
+
+  @Test
+  public void testIllegalPath() {
+    try {
+      new PartialPath("root.sg.d1.```");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1\na");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.`d1`..`aa``b`");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1.`s+`-1\"`");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root..a");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1.");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.111");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.and");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.or");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.not");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.contains");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.watermark_embedding");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.time");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.root");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.timestamp");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+  }
+
+  @Test
+  public void testLegalDeviceAndMeasurement() throws IllegalPathException {
+    String[] nodes;
+    // normal node
+    PartialPath a = new PartialPath("root.sg", "s1");
+    Assert.assertEquals("root.sg.s1", a.getFullPath());
+    nodes = new String[] {"root", "sg", "s1"};
+    checkNodes(nodes, a.getNodes());
+
+    PartialPath b = new PartialPath("root.sg", "s2");
+    Assert.assertEquals("root.sg.s2", b.getFullPath());
+    nodes = new String[] {"root", "sg", "s2"};
+    checkNodes(nodes, b.getNodes());
+
+    PartialPath c = new PartialPath("root.sg", "a");
+    Assert.assertEquals("root.sg.a", c.getFullPath());
+    nodes = new String[] {"root", "sg", "a"};
+    checkNodes(nodes, c.getNodes());
+
+    // quoted node
+    PartialPath d = new PartialPath("root.sg", "`a.b`");
+    Assert.assertEquals("root.sg.`a.b`", d.getFullPath());
+    nodes = new String[] {"root", "sg", "`a.b`"};
+    checkNodes(nodes, d.getNodes());
+
+    PartialPath e = new PartialPath("root.sg", "`a.``b`");
+    Assert.assertEquals("root.sg.`a.``b`", e.getFullPath());
+    nodes = new String[] {"root", "sg", "`a.``b`"};
+    checkNodes(nodes, e.getNodes());
+
+    PartialPath f = new PartialPath("root.`sg\"`", "`a.``b`");
+    Assert.assertEquals("root.`sg\"`.`a.``b`", f.getFullPath());
+    nodes = new String[] {"root", "`sg\"`", "`a.``b`"};
+    checkNodes(nodes, f.getNodes());
+
+    PartialPath g = new PartialPath("root.sg", "`a.b\\\\`");
+    Assert.assertEquals("root.sg.`a.b\\\\`", g.getFullPath());
+    nodes = new String[] {"root", "sg", "`a.b\\\\`"};
+    checkNodes(nodes, g.getNodes());
+
+    // quoted node of digits
+    PartialPath h = new PartialPath("root.sg", "`111`");
+    Assert.assertEquals("root.sg.`111`", h.getFullPath());
+    nodes = new String[] {"root", "sg", "`111`"};
+    checkNodes(nodes, h.getNodes());
+
+    // quoted node of key word
+    PartialPath i = new PartialPath("root.sg", "`select`");
+    Assert.assertEquals("root.sg.select", i.getFullPath());
+    nodes = new String[] {"root", "sg", "select"};
+    checkNodes(nodes, i.getNodes());
+
+    // wildcard
+    PartialPath j = new PartialPath("root.sg", "`a*b`");
+    Assert.assertEquals("root.sg.`a*b`", j.getFullPath());
+    nodes = new String[] {"root", "sg", "`a*b`"};
+    checkNodes(nodes, j.getNodes());
+
+    PartialPath k = new PartialPath("root.sg", "*");
+    Assert.assertEquals("root.sg.*", k.getFullPath());
+    nodes = new String[] {"root", "sg", "*"};
+    checkNodes(nodes, k.getNodes());
+
+    PartialPath l = new PartialPath("root.sg", "**");
+    Assert.assertEquals("root.sg.**", l.getFullPath());
+    nodes = new String[] {"root", "sg", "**"};
+    checkNodes(nodes, l.getNodes());
+
+    // other
+    PartialPath m = new PartialPath("root.sg", "`to`.be.prefix.s");
+    Assert.assertEquals("root.sg.to.be.prefix.s", m.getFullPath());
+    nodes = new String[] {"root", "sg", "to", "be", "prefix", "s"};
+    checkNodes(nodes, m.getNodes());
+
+    PartialPath n = new PartialPath("root.sg", "`abc`");
+    Assert.assertEquals("root.sg.abc", n.getFullPath());
+    nodes = new String[] {"root", "sg", "abc"};
+    checkNodes(nodes, n.getNodes());
+  }
+
+  @Test
+  public void testIllegalDeviceAndMeasurement() {
+    try {
+      new PartialPath("root.sg.d1", "```");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1.```", "s1");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.`d1`..a", "`aa``b`");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.`d1`.a", "s..`aa``b`");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "`s+`-1\"`");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1.`s+`-1\"`", "s1");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg", "111");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.111", "s1");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.select`", "a");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "device`");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "contains");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg", "watermark_embedding");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg", "and");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "or");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "not");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "root");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "time");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+
+    try {
+      new PartialPath("root.sg.d1", "timestamp");
+      fail();
+    } catch (IllegalPathException ignored) {
+    }
+  }
+
+  @Test
+  public void testConcatPath() {
+    String[] arr1 = new String[2];
+    arr1[0] = "root";
+    arr1[1] = "sg1";
+    PartialPath a = new PartialPath(arr1);
+    String[] arr2 = new String[2];
+    arr2[0] = "d1";
+    arr2[1] = "s1";
+    PartialPath b = new PartialPath(arr2);
+    Assert.assertEquals("[root, sg1, d1, s1]", Arrays.toString(a.concatPath(b).getNodes()));
+    Assert.assertEquals("s1", b.getTailNode());
+    Assert.assertEquals("root.sg1.d1", a.concatPath(b).getDevicePath().getFullPath());
+    Assert.assertEquals("root.sg1", a.toString());
+  }
+
+  @Test
+  public void testConcatArray() throws IllegalPathException {
+    PartialPath a = new PartialPath("root", "sg1");
+    String[] arr2 = new String[2];
+    arr2[0] = "d1";
+    arr2[1] = "s1";
+    a.concatPath(arr2);
+    Assert.assertEquals("[root, sg1, d1, s1]", Arrays.toString(a.getNodes()));
+  }
+
+  @Test
+  public void testConcatNode() {
+    String[] arr1 = new String[2];
+    arr1[0] = "root";
+    arr1[1] = "sg1";
+    PartialPath a = new PartialPath(arr1);
+    PartialPath b = a.concatNode("d1");
+    Assert.assertEquals("[root, sg1, d1]", Arrays.toString(b.getNodes()));
+    Assert.assertEquals("root.sg1.d1", b.getFullPath());
+    Assert.assertTrue(b.startsWith(arr1));
+    Assert.assertEquals("root", b.getFirstNode());
+  }
+
+  @Test
+  public void testAlterPrefixPath() throws IllegalPathException {
+    // Plain path.
+    PartialPath p = new PartialPath("root.a.b.c");
+    List<PartialPath> results = p.alterPrefixPath(new PartialPath("root.a.b"));
+    Assert.assertEquals(results.toString(), 1, results.size());
+    Assert.assertEquals("root.a.b.c", results.get(0).getFullPath());
+
+    // Path with single level wildcard.
+    p = new PartialPath("root.*.b.c");
+    results = p.alterPrefixPath(new PartialPath("root.a.b"));
+    Assert.assertEquals(results.toString(), 1, results.size());
+    Assert.assertEquals("root.a.b.c", results.get(0).getFullPath());
+
+    // Path with multi level wildcard.
+    p = new PartialPath("root.**.b.c");
+    results = p.alterPrefixPath(new PartialPath("root.a.b"));
+    Assert.assertEquals(results.toString(), 3, results.size());
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.c")));
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.b.c")));
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.**.b.c")));
+
+    p = new PartialPath("root.**");
+    results = p.alterPrefixPath(new PartialPath("root.a.b"));
+    Assert.assertEquals(results.toString(), 2, results.size());
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b")));
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.**")));
+
+    p = new PartialPath("root.**.b.**");
+    results = p.alterPrefixPath(new PartialPath("root.a.b.c"));
+    Assert.assertEquals(results.toString(), 2, results.size());
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.c")));
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.c.**")));
+
+    p = new PartialPath("root.**.b.**.b");
+    results = p.alterPrefixPath(new PartialPath("root.b.b.b"));
+    Assert.assertEquals(results.toString(), 2, results.size());
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.b.b.b.b")));
+    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.b.b.b.**.b")));
+
+    // Path cannot be altered.
+    p = new PartialPath("root.b.c.**");
+    results = p.alterPrefixPath(new PartialPath("root.a.b.c"));
+    Assert.assertEquals(results.toString(), 0, results.size());
+  }
+
+  @Test
+  public void testMatchPath() throws IllegalPathException {
+    PartialPath p1 = new PartialPath("root.sg1.d1.*");
+
+    Assert.assertTrue(p1.matchFullPath(new PartialPath("root.sg1.d1.s2")));
+    Assert.assertFalse(p1.matchFullPath(new PartialPath("root.sg1.d1")));
+    Assert.assertFalse(p1.matchFullPath(new PartialPath("root.sg2.d1.*")));
+    Assert.assertFalse(p1.matchFullPath(new PartialPath("", false)));
+
+    PartialPath path = new PartialPath("root.sg.d.s");
+    String[] patterns = {
+      "root.**", "root.**.s", "root.sg.*.s", "root.*.*.*", "root.sg.d.s", "root.s*.d.s"
+    };
+    for (String pattern : patterns) {
+      Assert.assertTrue(new PartialPath(pattern).matchFullPath(path));
+    }
+  }
+
+  @Test
+  public void testPartialPathAndStringList() {
+    List<PartialPath> paths =
+        PartialPath.fromStringList(Arrays.asList("root.sg1.d1.s1", "root.sg1.d1.s2"));
+    Assert.assertEquals("root.sg1.d1.s1", paths.get(0).getFullPath());
+    Assert.assertEquals("root.sg1.d1.s2", paths.get(1).getFullPath());
+
+    List<String> stringPaths = PartialPath.toStringList(paths);
+    Assert.assertEquals("root.sg1.d1.s1", stringPaths.get(0));
+    Assert.assertEquals("root.sg1.d1.s2", stringPaths.get(1));
+  }
+
   @Test
   public void testOverlapWith() throws IllegalPathException {
     PartialPath[][] pathPairs =
@@ -43,4 +547,11 @@ public class PartialPathTest {
       Assert.assertEquals(results[i], pathPairs[i][0].overlapWith(pathPairs[i][1]));
     }
   }
+
+  private void checkNodes(String[] expected, String[] actual) {
+    Assert.assertEquals(expected.length, actual.length);
+    for (int i = 0; i < expected.length; i++) {
+      Assert.assertEquals(expected[i], actual[i]);
+    }
+  }
 }
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/utils/PathUtilsTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/PathUtilsTest.java
deleted file mode 100644
index 1e73e04b99..0000000000
--- a/node-commons/src/test/java/org/apache/iotdb/commons/utils/PathUtilsTest.java
+++ /dev/null
@@ -1,125 +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.utils;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Arrays;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.fail;
-
-public class PathUtilsTest {
-
-  @Test
-  public void testSplitPathToNodes() throws IllegalPathException {
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1", "s1").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.d1.s1"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1", "s+1").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.d1.`s+1`"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "\"s g\"", "d1", "\"s+1\"").toArray(),
-        PathUtils.splitPathToDetachedPath("root.`\"s g\"`.d1.`\"s+1\"`"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "1").toArray(), PathUtils.splitPathToDetachedPath("root.1"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1", "s", "1").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.d1.s.1"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1", "`a.b`").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.d1.```a.b```"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1", "`").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.d1.````"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1", "`").toArray(),
-        PathUtils.splitPathToDetachedPath("`root`.`sg`.`d1`.````"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1", "`").toArray(),
-        PathUtils.splitPathToDetachedPath("`root`.sg.`d1`.````"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "d1.`").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.`d1.```"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "\"d").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.`\\\"d`"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "\td").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.`\\td`"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "sg", "\\td").toArray(),
-        PathUtils.splitPathToDetachedPath("root.sg.`\\\\td`"));
-
-    assertArrayEquals(
-        Arrays.asList("root", "laptop", "d1", "\"1.2.3\"").toArray(),
-        PathUtils.splitPathToDetachedPath("root.laptop.d1.`\\\"1.2.3\\\"`"));
-
-    try {
-      PathUtils.splitPathToDetachedPath("root.sg.d1.```");
-      fail();
-    } catch (IllegalPathException e) {
-      Assert.assertEquals("root.sg.d1.``` is not a legal path", e.getMessage());
-    }
-
-    try {
-      PathUtils.splitPathToDetachedPath("root.sg.`d1`..`aa``b`");
-      fail();
-    } catch (IllegalPathException e) {
-      Assert.assertEquals("root.sg.`d1`..`aa``b` is not a legal path", e.getMessage());
-    }
-
-    try {
-      PathUtils.splitPathToDetachedPath("root.sg.d1.`s+`-1\"`");
-      fail();
-    } catch (IllegalPathException e) {
-      Assert.assertEquals("root.sg.d1.`s+`-1\"` is not a legal path", e.getMessage());
-    }
-
-    try {
-      PathUtils.splitPathToDetachedPath("root..a");
-      fail();
-    } catch (IllegalPathException e) {
-      Assert.assertEquals("root..a is not a legal path", e.getMessage());
-    }
-
-    try {
-      PathUtils.splitPathToDetachedPath("root.sg.d1.");
-      fail();
-    } catch (IllegalPathException e) {
-      Assert.assertEquals("root.sg.d1. is not a legal path", e.getMessage());
-    }
-  }
-}
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
index af87c4f453..25156fad17 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
@@ -584,7 +584,7 @@ public class RSchemaRegion implements ISchemaRegion {
               RMeasurementMNode deletedNode;
               try {
                 path = RSchemaUtils.getPathByInnerName(new String(key));
-                String[] nodes = PathUtils.splitPathToDetachedPath(path);
+                String[] nodes = PathUtils.splitPathToDetachedNodes(path);
                 deletedNode = new RMeasurementMNode(path, value, readWriteHandler);
                 atomicInteger.incrementAndGet();
                 try (WriteBatch batch = new WriteBatch()) {
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java
index ee18ef613c..f38d329077 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaUtils.java
@@ -500,7 +500,7 @@ public class RSchemaUtils {
     for (String str : oldArray) {
       stringBuilder.append(PATH_SEPARATOR).append(str);
     }
-    return PathUtils.splitPathToDetachedPath(stringBuilder.substring(1));
+    return PathUtils.splitPathToDetachedNodes(stringBuilder.substring(1));
   }
 
   public static String replaceWildcard(int num) {
diff --git a/server/pom.xml b/server/pom.xml
index 9c85715a40..e38b8641b1 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -43,11 +43,6 @@
             <artifactId>service-rpc</artifactId>
             <version>${project.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.iotdb</groupId>
-            <artifactId>iotdb-antlr</artifactId>
-            <version>${project.version}</version>
-        </dependency>
         <dependency>
             <groupId>org.apache.iotdb</groupId>
             <artifactId>iotdb-consensus</artifactId>
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 1177b9940f..6e9f0ce346 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -64,7 +64,7 @@ public class IoTDBConfig {
       "org.apache.iotdb.db.conf.directories.strategy.";
   private static final String DEFAULT_MULTI_DIR_STRATEGY = "MaxDiskUsableSpaceFirstStrategy";
 
-  private static final String STORAGE_GROUP_MATCHER = "([a-zA-Z0-9_.\\-\\u2E80-\\u9FFF]+)";
+  private static final String STORAGE_GROUP_MATCHER = "([a-zA-Z0-9`_.\\-\\u2E80-\\u9FFF]+)";
   public static final Pattern STORAGE_GROUP_PATTERN = Pattern.compile(STORAGE_GROUP_MATCHER);
 
   // e.g., a31+/$%#&[]{}3e4, "a.b", 'a.b'
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cq/ContinuousQueryTask.java b/server/src/main/java/org/apache/iotdb/db/engine/cq/ContinuousQueryTask.java
index 9fd8399e9a..6a3117595d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cq/ContinuousQueryTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cq/ContinuousQueryTask.java
@@ -168,16 +168,17 @@ public class ContinuousQueryTask extends WrappedRunnable {
     return targetPaths;
   }
 
-  protected String fillTargetPathTemplate(PartialPath rawPath) {
-    String[] nodes = rawPath.getNodes();
-    int indexOfLeftBracket = nodes[0].indexOf("(");
+  protected String fillTargetPathTemplate(PartialPath rawPath) throws IllegalPathException {
+    String fullPath = rawPath.getFullPath();
+    int indexOfLeftBracket = fullPath.indexOf("(");
     if (indexOfLeftBracket != -1) {
-      nodes[0] = nodes[0].substring(indexOfLeftBracket + 1);
+      fullPath = fullPath.substring(indexOfLeftBracket + 1);
     }
-    int indexOfRightBracket = nodes[nodes.length - 1].indexOf(")");
+    int indexOfRightBracket = fullPath.lastIndexOf(")");
     if (indexOfRightBracket != -1) {
-      nodes[nodes.length - 1] = nodes[nodes.length - 1].substring(0, indexOfRightBracket);
+      fullPath = fullPath.substring(0, indexOfRightBracket);
     }
+    String[] nodes = new PartialPath(fullPath).getNodes();
     StringBuffer sb = new StringBuffer();
     Matcher m =
         PATH_NODE_NAME_PATTERN.matcher(this.continuousQueryPlan.getTargetPath().getFullPath());
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
index 149620e9e3..8b454cdc01 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
@@ -87,6 +87,7 @@ public class IDTableHashmapImpl implements IDTable {
    * @param plan create aligned timeseries plan
    * @throws MetadataException if the device is not aligned, throw it
    */
+  @Override
   public synchronized void createAlignedTimeseries(CreateAlignedTimeSeriesPlan plan)
       throws MetadataException {
     DeviceEntry deviceEntry = getDeviceEntryWithAlignedCheck(plan.getPrefixPath().toString(), true);
@@ -113,6 +114,7 @@ public class IDTableHashmapImpl implements IDTable {
    * @param plan create timeseries plan
    * @throws MetadataException if the device is aligned, throw it
    */
+  @Override
   public synchronized void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException {
     DeviceEntry deviceEntry = getDeviceEntryWithAlignedCheck(plan.getPath().getDevice(), false);
     SchemaEntry schemaEntry =
@@ -134,6 +136,7 @@ public class IDTableHashmapImpl implements IDTable {
    * @return reusable device id
    * @throws MetadataException if insert plan's aligned value is inconsistent with device
    */
+  @Override
   public synchronized IDeviceID getSeriesSchemas(InsertPlan plan) throws MetadataException {
     PartialPath devicePath = plan.getDevicePath();
     String[] measurementList = plan.getMeasurements();
@@ -188,7 +191,7 @@ public class IDTableHashmapImpl implements IDTable {
     // set reusable device id
     plan.setDeviceID(deviceEntry.getDeviceID());
     // change device path to device id string for insertion
-    plan.setDevicePath(new PartialPath(deviceEntry.getDeviceID().toStringID()));
+    plan.setDevicePath(new PartialPath(deviceEntry.getDeviceID().toStringID(), false));
 
     return deviceEntry.getDeviceID();
   }
@@ -200,6 +203,7 @@ public class IDTableHashmapImpl implements IDTable {
    * @param measurementMNode the timeseries measurement mnode
    * @throws MetadataException if the timeseries is not exits
    */
+  @Override
   public synchronized void registerTrigger(PartialPath fullPath, IMeasurementMNode measurementMNode)
       throws MetadataException {
     boolean isAligned = measurementMNode.getParent().isAligned();
@@ -215,6 +219,7 @@ public class IDTableHashmapImpl implements IDTable {
    * @param measurementMNode the timeseries measurement mnode
    * @throws MetadataException if the timeseries is not exits
    */
+  @Override
   public synchronized void deregisterTrigger(
       PartialPath fullPath, IMeasurementMNode measurementMNode) throws MetadataException {
     boolean isAligned = measurementMNode.getParent().isAligned();
@@ -229,6 +234,7 @@ public class IDTableHashmapImpl implements IDTable {
    * @param timeseriesID timeseries ID of the timeseries
    * @throws MetadataException if the timeseries is not exits
    */
+  @Override
   public synchronized TimeValuePair getLastCache(TimeseriesID timeseriesID)
       throws MetadataException {
     return getSchemaEntry(timeseriesID).getCachedLast();
@@ -243,6 +249,7 @@ public class IDTableHashmapImpl implements IDTable {
    * @param latestFlushedTime last flushed time
    * @throws MetadataException if the timeseries is not exits
    */
+  @Override
   public synchronized void updateLastCache(
       TimeseriesID timeseriesID,
       TimeValuePair pair,
@@ -452,11 +459,13 @@ public class IDTableHashmapImpl implements IDTable {
     return schemaEntry;
   }
 
+  @Override
   @TestOnly
   public Map<IDeviceID, DeviceEntry>[] getIdTables() {
     return idTables;
   }
 
+  @Override
   @TestOnly
   public IDiskSchemaManager getIDiskSchemaManager() {
     return IDiskSchemaManager;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/SHA256DeviceID.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/SHA256DeviceID.java
index b465cd3213..1042aa6985 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/SHA256DeviceID.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/SHA256DeviceID.java
@@ -36,7 +36,7 @@ public class SHA256DeviceID implements IDeviceID {
   long l3;
   long l4;
 
-  private static final String SEPARATOR = "#";
+  private static final String SEPARATOR = "_";
 
   /** using lots of message digest for improving parallelism */
   private static MessageDigest[] md;
@@ -75,6 +75,9 @@ public class SHA256DeviceID implements IDeviceID {
    * @param deviceID a sha 256 string
    */
   private void fromSHA256String(String deviceID) {
+    if (deviceID.startsWith("`") && deviceID.endsWith("`")) {
+      deviceID = deviceID.substring(1, deviceID.length() - 1);
+    }
     String[] part = deviceID.split(SEPARATOR);
     l1 = Long.parseLong(part[0]);
     l2 = Long.parseLong(part[1]);
@@ -147,7 +150,7 @@ public class SHA256DeviceID implements IDeviceID {
 
   @Override
   public String toStringID() {
-    return l1 + SEPARATOR + l2 + SEPARATOR + l3 + SEPARATOR + l4;
+    return "`" + l1 + SEPARATOR + l2 + SEPARATOR + l3 + SEPARATOR + l4 + "`";
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
index 8f881062a9..8795aabbe6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGCachedImpl.java
@@ -1150,7 +1150,7 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
   public void checkIsTemplateCompatibleWithChild(IMNode node, Template template)
       throws MetadataException {
     for (String measurementPath : template.getSchemaMap().keySet()) {
-      String directNodeName = PathUtils.splitPathToDetachedPath(measurementPath)[0];
+      String directNodeName = PathUtils.splitPathToDetachedNodes(measurementPath)[0];
       if (store.hasChild(node, directNodeName)) {
         throw new MetadataException(
             "Node name "
@@ -1208,7 +1208,7 @@ public class MTreeBelowSGCachedImpl implements IMTreeBelowSG {
       // node
       Set<String> overlapSet = new HashSet<>();
       for (String path : appendMeasurements) {
-        overlapSet.add(PathUtils.splitPathToDetachedPath(path)[0]);
+        overlapSet.add(PathUtils.splitPathToDetachedNodes(path)[0]);
       }
 
       while (setNodes.size() != 0) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
index 2b2a7ff4da..b3dc1b18da 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSGMemoryImpl.java
@@ -1048,7 +1048,7 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
   public void checkIsTemplateCompatibleWithChild(IMNode node, Template template)
       throws MetadataException {
     for (String measurementPath : template.getSchemaMap().keySet()) {
-      String directNodeName = PathUtils.splitPathToDetachedPath(measurementPath)[0];
+      String directNodeName = PathUtils.splitPathToDetachedNodes(measurementPath)[0];
       if (node.hasChild(directNodeName)) {
         throw new MetadataException(
             "Node name "
@@ -1099,7 +1099,7 @@ public class MTreeBelowSGMemoryImpl implements IMTreeBelowSG {
     // node
     Set<String> overlapSet = new HashSet<>();
     for (String path : appendMeasurements) {
-      overlapSet.add(PathUtils.splitPathToDetachedPath(path)[0]);
+      overlapSet.add(PathUtils.splitPathToDetachedNodes(path)[0]);
     }
 
     while (setNodes.size() != 0) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/schemafile/SchemaFile.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/schemafile/SchemaFile.java
index 37415c4725..89a9e93d1c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/schemafile/SchemaFile.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/store/disk/schemafile/SchemaFile.java
@@ -207,7 +207,7 @@ public class SchemaFile implements ISchemaFile {
   @Override
   public IMNode init() throws MetadataException {
     IMNode resNode;
-    String[] sgPathNodes = PathUtils.splitPathToDetachedPath(storageGroupName);
+    String[] sgPathNodes = PathUtils.splitPathToDetachedNodes(storageGroupName);
     if (isEntity) {
       resNode =
           setNodeAddress(
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
index 615fed6a47..e420cf4dde 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.metadata.path;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathType;
+import org.apache.iotdb.commons.utils.PathUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -58,6 +59,10 @@ public class AlignedPath extends PartialPath {
 
   public AlignedPath(String vectorPath, List<String> subSensorsList) throws IllegalPathException {
     super(vectorPath);
+    // check whether subSensor is legal
+    for (String subSensor : subSensorsList) {
+      PathUtils.isLegalPath(subSensor);
+    }
     this.measurementList = subSensorsList;
   }
 
@@ -65,6 +70,10 @@ public class AlignedPath extends PartialPath {
       String vectorPath, List<String> measurementList, List<IMeasurementSchema> schemaList)
       throws IllegalPathException {
     super(vectorPath);
+    // check whether measurement is legal
+    for (String measurement : measurementList) {
+      PathUtils.isLegalPath(measurement);
+    }
     this.measurementList = measurementList;
     this.schemaList = schemaList;
   }
@@ -72,12 +81,14 @@ public class AlignedPath extends PartialPath {
   public AlignedPath(String vectorPath, String subSensor) throws IllegalPathException {
     super(vectorPath);
     measurementList = new ArrayList<>();
+    PathUtils.isLegalPath(subSensor);
     measurementList.add(subSensor);
   }
 
-  public AlignedPath(PartialPath vectorPath, String subSensor) {
+  public AlignedPath(PartialPath vectorPath, String subSensor) throws IllegalPathException {
     super(vectorPath.getNodes());
     measurementList = new ArrayList<>();
+    PathUtils.isLegalPath(subSensor);
     measurementList.add(subSensor);
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java b/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
index ba57fdf357..68f01446bb 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
@@ -98,7 +98,7 @@ public class Template {
         // If sublist of measurements has only one item,
         // but it share prefix with other aligned sublist, it will be aligned too
         String[] thisMeasurement =
-            PathUtils.splitPathToDetachedPath(plan.getMeasurements().get(i).get(0));
+            PathUtils.splitPathToDetachedNodes(plan.getMeasurements().get(i).get(0));
         String thisPrefix =
             joinBySeparator(Arrays.copyOf(thisMeasurement, thisMeasurement.length - 1));
         isAlign =
@@ -184,7 +184,7 @@ public class Template {
       if (getPathNodeInTemplate(path) != null) {
         throw new IllegalPathException("Path duplicated: " + path);
       }
-      pathNodes = PathUtils.splitPathToDetachedPath(path);
+      pathNodes = PathUtils.splitPathToDetachedNodes(path);
 
       if (pathNodes.length == 1) {
         prefix = "";
@@ -233,7 +233,7 @@ public class Template {
     if (getPathNodeInTemplate(path) != null) {
       throw new IllegalPathException("Path duplicated: " + path);
     }
-    String[] pathNode = PathUtils.splitPathToDetachedPath(path);
+    String[] pathNode = PathUtils.splitPathToDetachedNodes(path);
     IMNode cur = constructEntityPath(path);
 
     synchronized (this) {
@@ -358,7 +358,7 @@ public class Template {
   }
 
   public IMNode getPathNodeInTemplate(String path) throws IllegalPathException {
-    return getPathNodeInTemplate(PathUtils.splitPathToDetachedPath(path));
+    return getPathNodeInTemplate(PathUtils.splitPathToDetachedNodes(path));
   }
 
   private IMNode getPathNodeInTemplate(String[] pathNodes) {
@@ -380,7 +380,7 @@ public class Template {
   }
 
   public boolean isPathExistInTemplate(String path) throws IllegalPathException {
-    String[] pathNodes = PathUtils.splitPathToDetachedPath(path);
+    String[] pathNodes = PathUtils.splitPathToDetachedNodes(path);
     if (!directNodes.containsKey(pathNodes[0])) {
       return false;
     }
@@ -400,7 +400,7 @@ public class Template {
   }
 
   public boolean isPathMeasurement(String path) throws MetadataException {
-    String[] pathNodes = PathUtils.splitPathToDetachedPath(path);
+    String[] pathNodes = PathUtils.splitPathToDetachedNodes(path);
     if (!directNodes.containsKey(pathNodes[0])) {
       throw new PathNotExistException(path);
     }
@@ -476,7 +476,7 @@ public class Template {
    * @return null if need to add direct node, will never return a measurement.
    */
   private IMNode constructEntityPath(String path) throws IllegalPathException {
-    String[] pathNodes = PathUtils.splitPathToDetachedPath(path);
+    String[] pathNodes = PathUtils.splitPathToDetachedNodes(path);
     if (pathNodes.length == 1) {
       return null;
     }
@@ -532,7 +532,7 @@ public class Template {
 
     // If prefix exists and not aligned, it will throw exception
     // Prefix equality will be checked in constructTemplateTree
-    pathNode = PathUtils.splitPathToDetachedPath(measurements[0]);
+    pathNode = PathUtils.splitPathToDetachedNodes(measurements[0]);
     prefix = joinBySeparator(Arrays.copyOf(pathNode, pathNode.length - 1));
     IMNode targetNode = getPathNodeInTemplate(prefix);
     if ((targetNode != null && !targetNode.getAsEntityMNode().isAligned())
@@ -541,7 +541,7 @@ public class Template {
     }
 
     for (int i = 0; i <= measurements.length - 1; i++) {
-      pathNode = PathUtils.splitPathToDetachedPath(measurements[i]);
+      pathNode = PathUtils.splitPathToDetachedNodes(measurements[i]);
       leafNodes[i] = pathNode[pathNode.length - 1];
     }
     schema = constructSchemas(leafNodes, dataTypes, encodings, compressors);
@@ -564,7 +564,7 @@ public class Template {
     }
 
     for (int i = 0; i <= measurements.length - 1; i++) {
-      pathNode = PathUtils.splitPathToDetachedPath(measurements[i]);
+      pathNode = PathUtils.splitPathToDetachedNodes(measurements[i]);
 
       // If prefix exists and aligned, it will throw exception
       prefix = joinBySeparator(Arrays.copyOf(pathNode, pathNode.length - 1));
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index 937c81a04f..8f9db1267f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -99,12 +99,14 @@ import org.apache.iotdb.db.query.expression.unary.LogicNotExpression;
 import org.apache.iotdb.db.query.expression.unary.NegationExpression;
 import org.apache.iotdb.db.query.expression.unary.RegularExpression;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
 
 import java.time.ZoneId;
 import java.util.ArrayList;
@@ -174,7 +176,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       CreateAlignedTimeSeriesStatement createAlignedTimeSeriesStatement) {
     for (int i = 0; i < ctx.nodeNameWithoutWildcard().size(); i++) {
       createAlignedTimeSeriesStatement.addMeasurement(
-          parseNodeName(ctx.nodeNameWithoutWildcard(i).getText()));
+          parseNodeNameWithoutWildCard(ctx.nodeNameWithoutWildcard(i)));
       parseAttributeClauses(ctx.attributeClauses(i), createAlignedTimeSeriesStatement);
     }
   }
@@ -970,7 +972,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     List<String> measurementList = new ArrayList<>();
     for (IoTDBSqlParser.NodeNameWithoutWildcardContext measurementName :
         ctx.nodeNameWithoutWildcard()) {
-      measurementList.add(parseNodeName(measurementName.getText()));
+      measurementList.add(parseNodeNameWithoutWildCard(measurementName));
     }
     insertStatement.setMeasurementList(measurementList.toArray(new String[0]));
     return (ctx.TIME() == null && ctx.TIMESTAMP() == null);
@@ -1038,7 +1040,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     }
     for (IoTDBSqlParser.NodeNameWithoutWildcardContext nodeNameWithoutStar : nodeNamesWithoutStar) {
       i++;
-      path[i] = parseNodeName(nodeNameWithoutStar.getText());
+      path[i] = parseNodeNameWithoutWildCard(nodeNameWithoutStar);
     }
     return new PartialPath(path);
   }
@@ -1095,11 +1097,24 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   }
 
   private String parseNodeName(IoTDBSqlParser.NodeNameContext ctx) {
-    return parseIdentifier(ctx.getText());
+    return parseNodeString(ctx.getText());
   }
 
   private String parseNodeNameWithoutWildCard(IoTDBSqlParser.NodeNameWithoutWildcardContext ctx) {
-    return parseIdentifier(ctx.getText());
+    return parseNodeString(ctx.getText());
+  }
+
+  private String parseNodeString(String nodeName) {
+    if (nodeName.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+        && nodeName.endsWith(TsFileConstant.BACK_QUOTE_STRING)) {
+      String unWrapped = nodeName.substring(1, nodeName.length() - 1);
+      if (StringUtils.isNumeric(unWrapped)
+          || !TsFileConstant.NODE_NAME_PATTERN.matcher(unWrapped).matches()) {
+        return nodeName;
+      }
+      return unWrapped;
+    }
+    return nodeName;
   }
 
   // Literals ========================================================================
@@ -1170,17 +1185,10 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   }
 
   private String parseIdentifier(String src) {
-    if (2 <= src.length() && src.charAt(0) == '`' && src.charAt(src.length() - 1) == '`') {
-      String unescaped = StringEscapeUtils.unescapeJava(src.substring(1, src.length() - 1));
-      // replace `` with `
-      return unescaped.replace("``", "`");
-    }
-    return src;
-  }
-
-  private String parseNodeName(String src) {
-    if (2 <= src.length() && src.charAt(0) == '`' && src.charAt(src.length() - 1) == '`') {
-      return src.substring(1, src.length() - 1);
+    if (src.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+        && src.endsWith(TsFileConstant.BACK_QUOTE_STRING)) {
+      return src.substring(1, src.length() - 1)
+          .replace(TsFileConstant.DOUBLE_BACK_QUOTE_STRING, TsFileConstant.BACK_QUOTE_STRING);
     }
     return src;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
index c542815158..e07e11acfc 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
@@ -99,7 +99,8 @@ public class StatementGenerator {
       PartialPath path = new PartialPath(pathStr);
       fromComponent.addPrefixPath(path);
     }
-    selectComponent.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath(""))));
+    selectComponent.addResultColumn(
+        new ResultColumn(new TimeSeriesOperand(new PartialPath("", false))));
 
     // set query filter
     GreaterEqualExpression leftPredicate =
@@ -134,10 +135,11 @@ public class StatementGenerator {
       PartialPath path = new PartialPath(pathStr);
       fromComponent.addPrefixPath(path);
     }
-    selectComponent.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath(""))));
+    selectComponent.addResultColumn(
+        new ResultColumn(new TimeSeriesOperand(new PartialPath("", false))));
 
     // set query filter
-    PartialPath timePath = new PartialPath(TIME);
+    PartialPath timePath = new PartialPath(TIME, false);
     BasicFunctionFilter basicFunctionFilter =
         new BasicFunctionFilter(
             FilterConstant.FilterType.GREATERTHANOREQUALTO,
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTemplatePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTemplatePlan.java
index 6e1c6a4b34..5ab66113e8 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTemplatePlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetTemplatePlan.java
@@ -43,7 +43,7 @@ public class SetTemplatePlan extends PhysicalPlan {
   public SetTemplatePlan(String templateName, String prefixPath) throws IllegalPathException {
     super(OperatorType.SET_TEMPLATE);
 
-    String[] pathNodes = PathUtils.splitPathToDetachedPath(prefixPath);
+    String[] pathNodes = PathUtils.splitPathToDetachedNodes(prefixPath);
     for (String s : pathNodes) {
       if (s.equals(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)
           || s.equals(IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD)) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/UnsetTemplatePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/UnsetTemplatePlan.java
index dc064b0204..f10560801e 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/UnsetTemplatePlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/UnsetTemplatePlan.java
@@ -45,7 +45,7 @@ public class UnsetTemplatePlan extends PhysicalPlan {
   public UnsetTemplatePlan(String prefixPath, String templateName) throws IllegalPathException {
     super(Operator.OperatorType.UNSET_TEMPLATE);
 
-    String[] pathNodes = PathUtils.splitPathToDetachedPath(prefixPath);
+    String[] pathNodes = PathUtils.splitPathToDetachedNodes(prefixPath);
     for (String s : pathNodes) {
       if (s.equals(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)
           || s.equals(IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD)) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
index f4481bfc0b..da67a66f7e 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
@@ -161,6 +161,7 @@ import org.apache.iotdb.tsfile.utils.StringContainer;
 
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
 
 import java.io.File;
 import java.time.ZoneId;
@@ -2432,13 +2433,25 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
 
   // node name
 
-  /** function for parsing node name. */
   private String parseNodeName(IoTDBSqlParser.NodeNameContext ctx) {
-    return parseIdentifier(ctx.getText());
+    return parseNodeString(ctx.getText());
   }
 
   private String parseNodeNameWithoutWildCard(IoTDBSqlParser.NodeNameWithoutWildcardContext ctx) {
-    return parseIdentifier(ctx.getText());
+    return parseNodeString(ctx.getText());
+  }
+
+  private String parseNodeString(String nodeName) {
+    if (nodeName.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+        && nodeName.endsWith(TsFileConstant.BACK_QUOTE_STRING)) {
+      String unWrapped = nodeName.substring(1, nodeName.length() - 1);
+      if (StringUtils.isNumeric(unWrapped)
+          || !TsFileConstant.NODE_NAME_PATTERN.matcher(unWrapped).matches()) {
+        return nodeName;
+      }
+      return unWrapped;
+    }
+    return nodeName;
   }
 
   // alias
@@ -3015,10 +3028,10 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
   }
 
   private String parseIdentifier(String src) {
-    if (2 <= src.length() && src.charAt(0) == '`' && src.charAt(src.length() - 1) == '`') {
-      String unescaped = StringEscapeUtils.unescapeJava(src.substring(1, src.length() - 1));
-      // replace `` with `
-      return unescaped.replace("``", "`");
+    if (src.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+        && src.endsWith(TsFileConstant.BACK_QUOTE_STRING)) {
+      return src.substring(1, src.length() - 1)
+          .replace(TsFileConstant.DOUBLE_BACK_QUOTE_STRING, TsFileConstant.BACK_QUOTE_STRING);
     }
     return src;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 742aaf1dc8..9e12f0655c 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -114,14 +114,14 @@ public class LogicalGenerator {
       PartialPath path = new PartialPath(p);
       fromOp.addPrefixTablePath(path);
     }
-    selectOp.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath(""))));
+    selectOp.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath("", false))));
 
     queryOp.setSelectComponent(selectOp);
     queryOp.setFromComponent(fromOp);
 
     // set time filter operator
     FilterOperator filterOp = new FilterOperator(FilterType.KW_AND);
-    PartialPath timePath = new PartialPath(TIME);
+    PartialPath timePath = new PartialPath(TIME, false);
     filterOp.setSinglePath(timePath);
     Set<PartialPath> pathSet = new HashSet<>();
     pathSet.add(timePath);
@@ -151,7 +151,7 @@ public class LogicalGenerator {
     FromComponent fromOp = new FromComponent();
     SelectComponent selectOp = new SelectComponent(zoneId);
 
-    selectOp.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath(""))));
+    selectOp.addResultColumn(new ResultColumn(new TimeSeriesOperand(new PartialPath("", false))));
 
     for (String p : req.getPaths()) {
       PartialPath path = new PartialPath(p);
@@ -161,7 +161,7 @@ public class LogicalGenerator {
     queryOp.setSelectComponent(selectOp);
     queryOp.setFromComponent(fromOp);
 
-    PartialPath timePath = new PartialPath(TIME);
+    PartialPath timePath = new PartialPath(TIME, false);
 
     BasicFunctionOperator basicFunctionOperator =
         new BasicFunctionOperator(
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
index c64397e173..46a37fe3e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
@@ -146,7 +146,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
             ((TimeSeriesOperand) expression)
                 .setPath(
                     new PartialPath(
-                        PathUtils.splitPathToDetachedPath(
+                        PathUtils.splitPathToDetachedNodes(
                             ((TimeSeriesOperand) expression)
                                 .getPath()
                                 .getFirstNode()))); // split path To nodes
@@ -242,11 +242,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
         String groupedPath =
             groupByLevelController.getGroupedPath(expression.getExpressionString());
         if (groupedPath != null) {
-          try {
-            resultExpressions.add(new TimeSeriesOperand(new PartialPath(groupedPath)));
-          } catch (IllegalPathException e) {
-            throw new LogicalOptimizeException(e.getMessage());
-          }
+          resultExpressions.add(new TimeSeriesOperand(new PartialPath(groupedPath, false)));
         } else {
           resultExpressions.add(expression);
         }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
index 700ddc316b..42fe464e5e 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.db.query.dataset.groupby;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
@@ -70,12 +69,8 @@ public class GroupByLevelDataSet extends QueryDataSet {
 
       if (paths.isEmpty()) {
         for (Map.Entry<String, AggregateResult> entry : groupPathResultMap.entrySet()) {
-          try {
-            String alias = plan.getGroupByLevelController().getAlias(entry.getKey());
-            this.paths.add(new PartialPath(alias != null ? alias : entry.getKey()));
-          } catch (IllegalPathException e) {
-            logger.error("Query result IllegalPathException occurred: {}.", entry.getKey());
-          }
+          String alias = plan.getGroupByLevelController().getAlias(entry.getKey());
+          this.paths.add(new PartialPath(alias != null ? alias : entry.getKey(), false));
           this.dataTypes.add(entry.getValue().getResultDataType());
         }
       }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java
index 3be629e8b6..d8abaa888a 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java
@@ -58,7 +58,7 @@ import static org.junit.Assert.fail;
 
 public class CompactionSchedulerTest {
   private static final Logger logger = LoggerFactory.getLogger(CompactionSchedulerTest.class);
-  static final String COMPACTION_TEST_SG = "root.compactionSchedulerTest-";
+  static final String COMPACTION_TEST_SG = "root.compactionSchedulerTest_";
   private static final boolean oldEnableInnerSeqCompaction =
       IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
   private static final boolean oldEnableInnerUnseqCompaction =
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java
index 6b83212a5e..9d329af3e0 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java
@@ -72,9 +72,9 @@ public class MetaUtilsTest {
   public void testGroupAlignedPath() throws MetadataException {
     List<PartialPath> pathList = new ArrayList<>();
 
-    MeasurementPath path1 = new MeasurementPath(new PartialPath("root.sg.device.s1"), null);
+    MeasurementPath path1 = new MeasurementPath(new PartialPath("root.sg.device1.s1"), null);
     pathList.add(path1);
-    MeasurementPath path2 = new MeasurementPath(new PartialPath("root.sg.device.s2"), null);
+    MeasurementPath path2 = new MeasurementPath(new PartialPath("root.sg.device1.s2"), null);
     pathList.add(path2);
 
     MeasurementPath path3 = new MeasurementPath(new PartialPath("root.sg.aligned_device.s1"), null);
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/PartialPathTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/PartialPathTest.java
deleted file mode 100644
index dafdfbc7e5..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/metadata/PartialPathTest.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.metadata;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.List;
-
-public class PartialPathTest {
-  @Before
-  public void setUp() {
-    EnvironmentUtils.envSetUp();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    EnvironmentUtils.cleanEnv();
-  }
-
-  @Test
-  public void testConcatPath() {
-    String[] arr1 = new String[2];
-    arr1[0] = "root";
-    arr1[1] = "sg1";
-    PartialPath a = new PartialPath(arr1);
-    String[] arr2 = new String[2];
-    arr2[0] = "d1";
-    arr2[1] = "s1";
-    PartialPath b = new PartialPath(arr2);
-    Assert.assertEquals("[root, sg1, d1, s1]", Arrays.toString(a.concatPath(b).getNodes()));
-    Assert.assertEquals("s1", b.getTailNode());
-    Assert.assertEquals("root.sg1.d1", a.concatPath(b).getDevicePath().getFullPath());
-    Assert.assertEquals("root.sg1", a.toString());
-  }
-
-  @Test
-  public void testConcatArray() throws IllegalPathException {
-    PartialPath a = new PartialPath("root", "sg1");
-    String[] arr2 = new String[2];
-    arr2[0] = "d1";
-    arr2[1] = "s1";
-    a.concatPath(arr2);
-    Assert.assertEquals("[root, sg1, d1, s1]", Arrays.toString(a.getNodes()));
-  }
-
-  @Test
-  public void testConcatNode() {
-    String[] arr1 = new String[2];
-    arr1[0] = "root";
-    arr1[1] = "sg1";
-    PartialPath a = new PartialPath(arr1);
-    PartialPath b = a.concatNode("d1");
-    Assert.assertEquals("[root, sg1, d1]", Arrays.toString(b.getNodes()));
-    Assert.assertEquals("root.sg1.d1", b.getFullPath());
-    Assert.assertTrue(b.startsWith(arr1));
-    Assert.assertEquals("root", b.getFirstNode());
-  }
-
-  @Test
-  public void testAlterPrefixPath() throws IllegalPathException {
-    // Plain path.
-    PartialPath p = new PartialPath("root.a.b.c");
-    List<PartialPath> results = p.alterPrefixPath(new PartialPath("root.a.b"));
-    Assert.assertEquals(results.toString(), 1, results.size());
-    Assert.assertEquals("root.a.b.c", results.get(0).getFullPath());
-
-    // Path with single level wildcard.
-    p = new PartialPath("root.*.b.c");
-    results = p.alterPrefixPath(new PartialPath("root.a.b"));
-    Assert.assertEquals(results.toString(), 1, results.size());
-    Assert.assertEquals("root.a.b.c", results.get(0).getFullPath());
-
-    // Path with multi level wildcard.
-    p = new PartialPath("root.**.b.c");
-    results = p.alterPrefixPath(new PartialPath("root.a.b"));
-    Assert.assertEquals(results.toString(), 3, results.size());
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.c")));
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.b.c")));
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.**.b.c")));
-
-    p = new PartialPath("root.**");
-    results = p.alterPrefixPath(new PartialPath("root.a.b"));
-    Assert.assertEquals(results.toString(), 2, results.size());
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b")));
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.**")));
-
-    p = new PartialPath("root.**.b.**");
-    results = p.alterPrefixPath(new PartialPath("root.a.b.c"));
-    Assert.assertEquals(results.toString(), 2, results.size());
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.c")));
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.a.b.c.**")));
-
-    p = new PartialPath("root.**.b.**.b");
-    results = p.alterPrefixPath(new PartialPath("root.b.b.b"));
-    Assert.assertEquals(results.toString(), 2, results.size());
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.b.b.b.b")));
-    Assert.assertTrue(results.toString(), results.contains(new PartialPath("root.b.b.b.**.b")));
-
-    // Path cannot be altered.
-    p = new PartialPath("root.b.c.**");
-    results = p.alterPrefixPath(new PartialPath("root.a.b.c"));
-    Assert.assertEquals(results.toString(), 0, results.size());
-  }
-
-  @Test
-  public void testMatchPath() throws IllegalPathException {
-    PartialPath p1 = new PartialPath("root.sg1.d1.*");
-
-    Assert.assertTrue(p1.matchFullPath(new PartialPath("root.sg1.d1.s2")));
-    Assert.assertFalse(p1.matchFullPath(new PartialPath("root.sg1.d1")));
-    Assert.assertFalse(p1.matchFullPath(new PartialPath("root.sg2.d1.*")));
-    Assert.assertFalse(p1.matchFullPath(new PartialPath("")));
-
-    PartialPath path = new PartialPath("root.sg.d.s");
-    String[] patterns = {
-      "root.**", "root.**.s", "root.sg.*.s", "root.*.*.*", "root.sg.d.s", "root.s*.d.s"
-    };
-    for (String pattern : patterns) {
-      Assert.assertTrue(new PartialPath(pattern).matchFullPath(path));
-    }
-  }
-
-  @Test
-  public void testPartialPathAndStringList() {
-    List<PartialPath> paths =
-        PartialPath.fromStringList(Arrays.asList("root.sg1.d1.s1", "root.sg1.d1.s2"));
-    Assert.assertEquals("root.sg1.d1.s1", paths.get(0).getFullPath());
-    Assert.assertEquals("root.sg1.d1.s2", paths.get(1).getFullPath());
-
-    List<String> stringPaths = PartialPath.toStringList(paths);
-    Assert.assertEquals("root.sg1.d1.s1", stringPaths.get(0));
-    Assert.assertEquals("root.sg1.d1.s2", stringPaths.get(1));
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
index 2a6b2232ae..3e56b1ba4a 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
@@ -108,13 +108,13 @@ public abstract class SchemaBasicTest {
 
     try {
       schemaProcessor.setStorageGroup(new PartialPath("root.laptop.d1"));
-      schemaProcessor.setStorageGroup(new PartialPath("root.1"));
+      schemaProcessor.setStorageGroup(new PartialPath("root.`1`"));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
 
-    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.1")));
+    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.`1`")));
 
     try {
       schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
@@ -158,7 +158,7 @@ public abstract class SchemaBasicTest {
           TSFileDescriptor.getInstance().getConfig().getCompressor(),
           Collections.emptyMap());
       schemaProcessor.createTimeseries(
-          new PartialPath("root.1.2.3"),
+          new PartialPath("root.`1`.`2`.`3`"),
           TSDataType.INT32,
           TSEncoding.RLE,
           TSFileDescriptor.getInstance().getConfig().getCompressor(),
@@ -167,8 +167,8 @@ public abstract class SchemaBasicTest {
       assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.s1")));
       assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.1_2")));
       assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.`\"1.2.3\"`")));
-      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.1.2")));
-      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.1.2.3")));
+      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.`1`.`2`")));
+      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.`1`.`2`.`3`")));
     } catch (MetadataException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
@@ -246,31 +246,31 @@ public abstract class SchemaBasicTest {
     try {
       schemaProcessor.deleteTimeseries(new PartialPath("root.laptop.d1.1_2"));
       schemaProcessor.deleteTimeseries(new PartialPath("root.laptop.d1.`\"1.2.3\"`"));
-      schemaProcessor.deleteTimeseries(new PartialPath("root.1.2.3"));
+      schemaProcessor.deleteTimeseries(new PartialPath("root.`1`.`2`.`3`"));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
     assertFalse(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.1_2")));
     assertFalse(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.`\"1.2.3\"`")));
-    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.1.2.3")));
-    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.1.2")));
-    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.1")));
+    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.`1`.`2`.`3`")));
+    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.`1`.`2`")));
+    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.`1`")));
 
     try {
-      schemaProcessor.deleteStorageGroups(Collections.singletonList(new PartialPath("root.1")));
+      schemaProcessor.deleteStorageGroups(Collections.singletonList(new PartialPath("root.`1`")));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
-    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.1")));
+    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.`1`")));
 
-    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.template")));
-    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.template.d1")));
+    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.template0")));
+    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.template0.d1")));
 
     try {
       schemaProcessor.createTimeseries(
-          new PartialPath("root.template.d2"),
+          new PartialPath("root.template0.d2"),
           TSDataType.INT32,
           TSEncoding.RLE,
           TSFileDescriptor.getInstance().getConfig().getCompressor(),
@@ -282,19 +282,19 @@ public abstract class SchemaBasicTest {
 
     try {
       schemaProcessor.createSchemaTemplate(getCreateTemplatePlan());
-      schemaProcessor.setSchemaTemplate(new SetTemplatePlan("template1", "root.template"));
+      schemaProcessor.setSchemaTemplate(new SetTemplatePlan("template1", "root.template0"));
       schemaProcessor.setUsingSchemaTemplate(
-          new ActivateTemplatePlan(new PartialPath("root.template.d1")));
+          new ActivateTemplatePlan(new PartialPath("root.template0.d1")));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
 
-    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template.d1")));
-    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template.d1.s11")));
-    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.template.d2.s11")));
-    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template.d1.vector")));
-    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template.d1.vector.s0")));
+    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template0.d1")));
+    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template0.d1.s11")));
+    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.template0.d2.s11")));
+    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template0.d1.vector")));
+    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.template0.d1.vector.s0")));
   }
 
   /**
@@ -607,11 +607,11 @@ public abstract class SchemaBasicTest {
           schemaProcessor.getBelongedStorageGroups(new PartialPath("root.vehicle")).isEmpty());
       assertTrue(
           schemaProcessor
-              .getBelongedStorageGroups(new PartialPath("root.vehicle.device"))
+              .getBelongedStorageGroups(new PartialPath("root.vehicle.device0"))
               .isEmpty());
       assertTrue(
           schemaProcessor
-              .getBelongedStorageGroups(new PartialPath("root.vehicle.device.sensor"))
+              .getBelongedStorageGroups(new PartialPath("root.vehicle.device0.sensor"))
               .isEmpty());
 
       schemaProcessor.setStorageGroup(new PartialPath("root.vehicle"));
@@ -619,20 +619,20 @@ public abstract class SchemaBasicTest {
           schemaProcessor.getBelongedStorageGroups(new PartialPath("root.vehicle")).isEmpty());
       assertFalse(
           schemaProcessor
-              .getBelongedStorageGroups(new PartialPath("root.vehicle.device"))
+              .getBelongedStorageGroups(new PartialPath("root.vehicle.device0"))
               .isEmpty());
       assertFalse(
           schemaProcessor
-              .getBelongedStorageGroups(new PartialPath("root.vehicle.device.sensor"))
+              .getBelongedStorageGroups(new PartialPath("root.vehicle.device0.sensor"))
               .isEmpty());
       assertTrue(
           schemaProcessor.getBelongedStorageGroups(new PartialPath("root.vehicle1")).isEmpty());
       assertTrue(
           schemaProcessor
-              .getBelongedStorageGroups(new PartialPath("root.vehicle1.device"))
+              .getBelongedStorageGroups(new PartialPath("root.vehicle1.device0"))
               .isEmpty());
 
-      schemaProcessor.setStorageGroup(new PartialPath("root.vehicle1.device"));
+      schemaProcessor.setStorageGroup(new PartialPath("root.vehicle1.device0"));
       assertTrue(
           schemaProcessor
               .getBelongedStorageGroups(new PartialPath("root.vehicle1.device1"))
@@ -647,7 +647,7 @@ public abstract class SchemaBasicTest {
               .isEmpty());
       assertFalse(
           schemaProcessor
-              .getBelongedStorageGroups(new PartialPath("root.vehicle1.device"))
+              .getBelongedStorageGroups(new PartialPath("root.vehicle1.device0"))
               .isEmpty());
     } catch (MetadataException e) {
       e.printStackTrace();
@@ -731,58 +731,6 @@ public abstract class SchemaBasicTest {
     assertTrue(caughtException);
   }
 
-  @Test
-  public void testSetStorageGroupWithIllegalName() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      PartialPath path1 = new PartialPath("root.laptop\n");
-      try {
-        schemaProcessor.setStorageGroup(path1);
-        fail();
-      } catch (MetadataException e) {
-      }
-    } catch (IllegalPathException e1) {
-      fail();
-    }
-    try {
-      PartialPath path2 = new PartialPath("root.laptop\t");
-      try {
-        schemaProcessor.setStorageGroup(path2);
-        fail();
-      } catch (MetadataException e) {
-      }
-    } catch (IllegalPathException e1) {
-      fail();
-    }
-  }
-
-  @Test
-  public void testCreateTimeseriesWithIllegalName() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      PartialPath path1 = new PartialPath("root.laptop.d1\n.s1");
-      try {
-        schemaProcessor.createTimeseries(
-            path1, TSDataType.INT32, TSEncoding.PLAIN, CompressionType.SNAPPY, null);
-        fail();
-      } catch (MetadataException e) {
-      }
-    } catch (IllegalPathException e1) {
-      fail();
-    }
-    try {
-      PartialPath path2 = new PartialPath("root.laptop.d1\t.s1");
-      try {
-        schemaProcessor.createTimeseries(
-            path2, TSDataType.INT32, TSEncoding.PLAIN, CompressionType.SNAPPY, null);
-        fail();
-      } catch (MetadataException e) {
-      }
-    } catch (IllegalPathException e1) {
-      fail();
-    }
-  }
-
   @Test
   public void testGetDevicesWithGivenPrefix() {
     LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
@@ -1083,16 +1031,16 @@ public abstract class SchemaBasicTest {
     assertNull(template.getPathNodeInTemplate("notExists"));
     assertEquals("[GPS]", template.getAllAlignedPrefix().toString());
 
-    String[] alignedMeasurements = {"to.be.prefix.s1", "to.be.prefix.s2"};
+    String[] alignedMeasurements = {"`to`.be.prefix.s1", "`to`.be.prefix.s2"};
     TSDataType[] dataTypes = {TSDataType.INT32, TSDataType.INT32};
     TSEncoding[] encodings = {TSEncoding.RLE, TSEncoding.RLE};
     CompressionType[] compressionTypes = {CompressionType.SNAPPY, CompressionType.SNAPPY};
     template.addAlignedMeasurements(alignedMeasurements, dataTypes, encodings, compressionTypes);
 
     assertEquals("[to.be.prefix, GPS]", template.getAllAlignedPrefix().toString());
-    assertEquals("[s1, s2]", template.getAlignedMeasurements("to.be.prefix").toString());
+    assertEquals("[s1, s2]", template.getAlignedMeasurements("`to`.be.prefix").toString());
 
-    template.deleteAlignedPrefix("to.be.prefix");
+    template.deleteAlignedPrefix("`to`.be.prefix");
 
     assertEquals("[GPS]", template.getAllAlignedPrefix().toString());
     assertEquals(null, template.getDirectNode("prefix"));
@@ -1119,7 +1067,7 @@ public abstract class SchemaBasicTest {
         "[d1.s1, GPS.x, to.be.prefix.s2, GPS.y, to.be.prefix.s1, s2]",
         template.getAllMeasurementsPaths().toString());
 
-    template.deleteSeriesCascade("to");
+    template.deleteSeriesCascade("`to`");
 
     assertEquals("[d1.s1, GPS.x, GPS.y, s2]", template.getAllMeasurementsPaths().toString());
   }
@@ -1233,8 +1181,8 @@ public abstract class SchemaBasicTest {
             dataTypeList,
             encodingList,
             compressionTypes);
-    SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.sg.1");
-    UnsetTemplatePlan unsetTemplatePlan = new UnsetTemplatePlan("root.sg.1", "template1");
+    SetTemplatePlan setTemplatePlan = new SetTemplatePlan("template1", "root.sg.`1`");
+    UnsetTemplatePlan unsetTemplatePlan = new UnsetTemplatePlan("root.sg.`1`", "template1");
     LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
     schemaProcessor.createSchemaTemplate(createTemplatePlan);
 
@@ -1243,7 +1191,7 @@ public abstract class SchemaBasicTest {
       schemaProcessor.unsetSchemaTemplate(unsetTemplatePlan);
       fail("No exception thrown.");
     } catch (Exception e) {
-      assertEquals("Path [root.sg.1] does not exist", e.getMessage());
+      assertEquals("Path [root.sg.`1`] does not exist", e.getMessage());
     }
 
     schemaProcessor.setSchemaTemplate(setTemplatePlan);
@@ -1258,7 +1206,7 @@ public abstract class SchemaBasicTest {
       schemaProcessor.unsetSchemaTemplate(unsetTemplatePlan);
       fail("No exception thrown.");
     } catch (Exception e) {
-      assertEquals("NO template on root.sg.1", e.getMessage());
+      assertEquals("NO template on root.sg.`1`", e.getMessage());
     }
   }
 
@@ -1725,18 +1673,6 @@ public abstract class SchemaBasicTest {
       e.printStackTrace();
       fail(e.getMessage());
     }
-
-    // show timeseries root.laptop.d1.(s0,s1)
-    try {
-      ShowTimeSeriesPlan showTimeSeriesPlan =
-          new ShowTimeSeriesPlan(
-              new PartialPath("root.laptop.d1.(s0,s1)"), false, null, null, 0, 0, false);
-      schemaProcessor.showTimeseries(showTimeSeriesPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
-    } catch (MetadataException e) {
-      assertEquals(
-          "Cannot get node of children in different aligned timeseries (Path: (s0,s1))",
-          e.getMessage());
-    }
   }
 
   @Test
@@ -1999,16 +1935,16 @@ public abstract class SchemaBasicTest {
     LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
     assertTrue(schemaProcessor.isPathExist(new PartialPath("root")));
 
-    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.group-with-hyphen")));
+    assertFalse(schemaProcessor.isPathExist(new PartialPath("root.group_with_hyphen")));
 
     try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.group-with-hyphen"));
+      schemaProcessor.setStorageGroup(new PartialPath("root.group_with_hyphen"));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
 
-    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.group-with-hyphen")));
+    assertTrue(schemaProcessor.isPathExist(new PartialPath("root.group_with_hyphen")));
   }
 
   @Test
@@ -2288,7 +2224,7 @@ public abstract class SchemaBasicTest {
   public void testCreateAlignedTimeseriesWithAliasAndTags() throws Exception {
     LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
     schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-    PartialPath devicePath = new PartialPath("root.laptop.device");
+    PartialPath devicePath = new PartialPath("root.laptop.device0");
     List<String> measurements = Arrays.asList("s1", "s2", "s3", "s4", "s5");
     List<TSDataType> tsDataTypes =
         Arrays.asList(
@@ -2333,8 +2269,8 @@ public abstract class SchemaBasicTest {
     schemaProcessor.createAlignedTimeSeries(createAlignedTimeSeriesPlan);
 
     Assert.assertEquals(
-        5, schemaProcessor.getAllTimeseriesCount(new PartialPath("root.laptop.device.*")));
-    Assert.assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.device.alias2")));
+        5, schemaProcessor.getAllTimeseriesCount(new PartialPath("root.laptop.device0.*")));
+    Assert.assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.device0.alias2")));
 
     ShowTimeSeriesPlan showTimeSeriesPlan =
         new ShowTimeSeriesPlan(new PartialPath("root.**"), false, "key", "value", 0, 0, false);
@@ -2346,11 +2282,11 @@ public abstract class SchemaBasicTest {
             .sorted(Comparator.comparing(ShowResult::getName))
             .collect(Collectors.toList());
     ShowTimeSeriesResult result = showTimeSeriesResults.get(0);
-    Assert.assertEquals("root.laptop.device.s1", result.getName());
+    Assert.assertEquals("root.laptop.device0.s1", result.getName());
     Assert.assertEquals("alias1", result.getAlias());
     Assert.assertEquals(tags, result.getTag());
     result = showTimeSeriesResults.get(1);
-    Assert.assertEquals("root.laptop.device.s4", result.getName());
+    Assert.assertEquals("root.laptop.device0.s4", result.getName());
     Assert.assertEquals(tags, result.getTag());
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSGTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSGTest.java
index 4b8140a3f8..e357b9a0c0 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSGTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSGTest.java
@@ -175,8 +175,8 @@ public class MTreeAboveSGTest {
       assertTrue(root.isStorageGroup(new PartialPath("root.laptop.d2")));
       assertFalse(root.isStorageGroup(new PartialPath("root.laptop.d3")));
 
-      root.setStorageGroup(new PartialPath("root.1"));
-      assertTrue(root.isStorageGroup(new PartialPath("root.1")));
+      root.setStorageGroup(new PartialPath("root.`1`"));
+      assertTrue(root.isStorageGroup(new PartialPath("root.`1`")));
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -209,23 +209,24 @@ public class MTreeAboveSGTest {
     try {
       assertTrue(root.getBelongedStorageGroups(new PartialPath("root")).isEmpty());
       assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle")).isEmpty());
-      assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle.device")).isEmpty());
+      assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle.device0")).isEmpty());
       assertTrue(
-          root.getBelongedStorageGroups(new PartialPath("root.vehicle.device.sensor")).isEmpty());
+          root.getBelongedStorageGroups(new PartialPath("root.vehicle.device0.sensor")).isEmpty());
 
       root.setStorageGroup(new PartialPath("root.vehicle"));
       assertFalse(root.getBelongedStorageGroups(new PartialPath("root.vehicle")).isEmpty());
-      assertFalse(root.getBelongedStorageGroups(new PartialPath("root.vehicle.device")).isEmpty());
+      assertFalse(root.getBelongedStorageGroups(new PartialPath("root.vehicle.device0")).isEmpty());
       assertFalse(
-          root.getBelongedStorageGroups(new PartialPath("root.vehicle.device.sensor")).isEmpty());
+          root.getBelongedStorageGroups(new PartialPath("root.vehicle.device0.sensor")).isEmpty());
       assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle1")).isEmpty());
-      assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle1.device")).isEmpty());
+      assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle1.device0")).isEmpty());
 
-      root.setStorageGroup(new PartialPath("root.vehicle1.device"));
+      root.setStorageGroup(new PartialPath("root.vehicle1.device0"));
       assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle1.device1")).isEmpty());
       assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle1.device2")).isEmpty());
       assertTrue(root.getBelongedStorageGroups(new PartialPath("root.vehicle1.device3")).isEmpty());
-      assertFalse(root.getBelongedStorageGroups(new PartialPath("root.vehicle1.device")).isEmpty());
+      assertFalse(
+          root.getBelongedStorageGroups(new PartialPath("root.vehicle1.device0")).isEmpty());
     } catch (MetadataException e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -237,9 +238,7 @@ public class MTreeAboveSGTest {
     try {
       root.setStorageGroup(new PartialPath("root.\"sg.ln\""));
     } catch (MetadataException e) {
-      Assert.assertEquals(
-          "The storage group name can only be characters, numbers and underscores. root.\"sg.ln\" is not a legal path",
-          e.getMessage());
+      Assert.assertEquals("root.\"sg.ln\" is not a legal path", e.getMessage());
     }
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java
index 98b1650464..9c77a07777 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java
@@ -640,7 +640,7 @@ public class SchemaFileTest {
   }
 
   private IMNode getNode(IMNode root, String path) throws MetadataException {
-    String[] pathNodes = PathUtils.splitPathToDetachedPath(path);
+    String[] pathNodes = PathUtils.splitPathToDetachedNodes(path);
     IMNode cur = root;
     for (String node : pathNodes) {
       if (!node.equals("root")) {
@@ -685,7 +685,7 @@ public class SchemaFileTest {
   // region Tree Constructor
 
   private IMNode virtualTriangleMTree(int size, String sgPath) throws MetadataException {
-    String[] sgPathNodes = PathUtils.splitPathToDetachedPath(sgPath);
+    String[] sgPathNodes = PathUtils.splitPathToDetachedNodes(sgPath);
     IMNode upperNode = null;
     for (String name : sgPathNodes) {
       IMNode child = new InternalMNode(upperNode, name);
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/upgrade/MetadataUpgradeTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/upgrade/MetadataUpgradeTest.java
index f37f2e034c..2faad460fb 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/upgrade/MetadataUpgradeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/upgrade/MetadataUpgradeTest.java
@@ -245,7 +245,7 @@ public class MetadataUpgradeTest {
   }
 
   private CreateAlignedTimeSeriesPlan getCreateAlignedTimeseriesPlan() throws IllegalPathException {
-    PartialPath devicePath = new PartialPath("root.unsetTemplate1.sg1.device");
+    PartialPath devicePath = new PartialPath("root.unsetTemplate1.sg1.device0");
     List<String> measurements = Arrays.asList("s1", "s2", "s3", "s4", "s5");
     List<TSDataType> tsDataTypes =
         Arrays.asList(
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
index 5a98d40f16..a2afa4e721 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
@@ -48,7 +48,7 @@ public class DeviceSchemaScanNodeSerdeTest {
     DevicesSchemaScanNode devicesSchemaScanNode =
         new DevicesSchemaScanNode(
             new PlanNodeId("deviceSchemaScan"),
-            new PartialPath("root.sg.device"),
+            new PartialPath("root.sg.device0"),
             10,
             10,
             false,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java
index b600548b61..532fe6fdf4 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java
@@ -44,7 +44,7 @@ public class SchemaCountNodeSerdeTest {
     ExchangeNode exchangeNode = new ExchangeNode(new PlanNodeId("exchange"));
     DevicesCountNode devicesCountNode =
         new DevicesCountNode(
-            new PlanNodeId("devicesCount"), new PartialPath("root.sg.device"), true);
+            new PlanNodeId("devicesCount"), new PartialPath("root.sg.device0"), true);
     FragmentSinkNode fragmentSinkNode = new FragmentSinkNode(new PlanNodeId("fragmentSink"));
     fragmentSinkNode.addChild(devicesCountNode);
     fragmentSinkNode.setDownStream(
@@ -70,7 +70,7 @@ public class SchemaCountNodeSerdeTest {
     ExchangeNode exchangeNode = new ExchangeNode(new PlanNodeId("exchange"));
     LevelTimeSeriesCountNode levelTimeSeriesCountNode =
         new LevelTimeSeriesCountNode(
-            new PlanNodeId("timeseriesCount"), new PartialPath("root.sg.device"), true, 10);
+            new PlanNodeId("timeseriesCount"), new PartialPath("root.sg.device0"), true, 10);
     FragmentSinkNode fragmentSinkNode = new FragmentSinkNode(new PlanNodeId("fragmentSink"));
     fragmentSinkNode.addChild(levelTimeSeriesCountNode);
     fragmentSinkNode.setDownStream(
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
index d4898d7200..564ba64916 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
@@ -48,7 +48,7 @@ public class TimeSeriesSchemaScanNodeSerdeTest {
     TimeSeriesSchemaScanNode timeSeriesSchemaScanNode =
         new TimeSeriesSchemaScanNode(
             new PlanNodeId("timeSeriesSchemaScan"),
-            new PartialPath("root.sg.device.sensor"),
+            new PartialPath("root.sg.device0.sensor"),
             null,
             null,
             10,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/sink/FragmentSinkNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/sink/FragmentSinkNodeSerdeTest.java
index f92f55e3b0..9d16d18065 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/sink/FragmentSinkNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/sink/FragmentSinkNodeSerdeTest.java
@@ -42,7 +42,12 @@ public class FragmentSinkNodeSerdeTest {
         new FragmentSinkNode(new PlanNodeId("TestFragmentSinkNode"));
     fragmentSinkNode.addChild(
         new DevicesSchemaScanNode(
-            new PlanNodeId("deviceSchema"), new PartialPath("root.sg.device"), 0, 0, false, false));
+            new PlanNodeId("deviceSchema"),
+            new PartialPath("root.sg.device0"),
+            0,
+            0,
+            false,
+            false));
     fragmentSinkNode.setDownStream(
         new TEndPoint("127.0.0.1", 6666),
         new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
index ff2dc0843d..f313e048d6 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
@@ -258,13 +258,9 @@ public class PhysicalPlanSerializeTest {
 
   @Test
   public void createMuSerializeTest3() throws IOException, IllegalPathException {
-    // same as:
-    // create timeseries root.sg.d1.s0 with datatype=DOUBLE, encoding=GORILLA, compression=SNAPPY
-    // create aligned timeseries root.sg.d1.(s1 INT64, s2 DOUBLE, s3 INT64)
-    // with encoding=(GORILLA, GORILLA, GORILLA), compression=SNAPPY
     CreateMultiTimeSeriesPlan plan = new CreateMultiTimeSeriesPlan();
     plan.setPaths(
-        Arrays.asList(new PartialPath("root.sg.d1.s0"), new PartialPath("root.sg.d1.(s1,s2,s3)")));
+        Arrays.asList(new PartialPath("root.sg.d1.s0"), new PartialPath("root.sg.d1.s1")));
     plan.setDataTypes(
         Arrays.asList(TSDataType.DOUBLE, TSDataType.INT64, TSDataType.DOUBLE, TSDataType.INT64));
     plan.setEncodings(
diff --git a/tsfile/pom.xml b/tsfile/pom.xml
index 80cda07878..c3b2b1deaa 100644
--- a/tsfile/pom.xml
+++ b/tsfile/pom.xml
@@ -49,6 +49,10 @@
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.lz4</groupId>
             <artifactId>lz4-java</artifactId>
@@ -98,6 +102,12 @@
             <version>4.0.3</version>
             <scope>test</scope>
         </dependency>
+        <!-- antlr -->
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-antlr</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
index be8ecb3d5e..9b496b9445 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.tsfile.common.constant;
 
+import java.util.regex.Pattern;
+
 public class TsFileConstant {
 
   public static final String TSFILE_SUFFIX = ".tsfile";
@@ -30,9 +32,14 @@ public class TsFileConstant {
   public static final String PATH_SEPARATER_NO_REGEX = "\\.";
   public static final char DOUBLE_QUOTE = '"';
   public static final char BACK_QUOTE = '`';
+  public static final String BACK_QUOTE_STRING = "`";
+  public static final String DOUBLE_BACK_QUOTE_STRING = "``";
 
   public static final byte TIME_COLUMN_MASK = (byte) 0x80;
   public static final byte VALUE_COLUMN_MASK = (byte) 0x40;
 
+  private static final String NODE_NAME_MATCHER = "([a-zA-Z0-9_:@#${}\\u2E80-\\u9FFF]+)";
+  public static final Pattern NODE_NAME_PATTERN = Pattern.compile(NODE_NAME_MATCHER);
+
   private TsFileConstant() {}
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/exception/PathParseException.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/exception/PathParseException.java
new file mode 100644
index 0000000000..37f6cf2fbe
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/exception/PathParseException.java
@@ -0,0 +1,26 @@
+/*
+ * 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.tsfile.exception;
+
+public class PathParseException extends TsFileRuntimeException {
+
+  public PathParseException(String path) {
+    super(String.format("%s is not a legal path.", path));
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java
index 5ac56c2473..4bd940b30d 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java
@@ -19,8 +19,13 @@
 package org.apache.iotdb.tsfile.read.common;
 
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.exception.PathParseException;
+import org.apache.iotdb.tsfile.read.common.parser.PathNodesGenerator;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.Validate;
+
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 
@@ -58,40 +63,20 @@ public class Path implements Serializable, Comparable<Path> {
    */
   public Path(String pathSc, boolean needSplit) {
     if (pathSc == null) {
-      throw new IllegalArgumentException(ILLEGAL_PATH_ARGUMENT);
+      throw new PathParseException(ILLEGAL_PATH_ARGUMENT);
     }
     if (!needSplit) {
+      // no split, we don't use antlr to check here.
       fullPath = pathSc;
     } else {
       if (pathSc.length() > 0) {
-        if (pathSc.charAt(pathSc.length() - 1) == TsFileConstant.DOUBLE_QUOTE) {
-          int endIndex = pathSc.lastIndexOf('"', pathSc.length() - 2);
-          // if a double quotes with escape character
-          while (endIndex != -1 && pathSc.charAt(endIndex - 1) == '\\') {
-            endIndex = pathSc.lastIndexOf('"', endIndex - 2);
-          }
-          if (endIndex != -1 && (endIndex == 0 || pathSc.charAt(endIndex - 1) == '.')) {
-            fullPath = pathSc;
-            device = pathSc.substring(0, endIndex - 1);
-            measurement = pathSc.substring(endIndex);
-          } else {
-            throw new IllegalArgumentException(ILLEGAL_PATH_ARGUMENT);
-          }
-        } else if (pathSc.charAt(pathSc.length() - 1) != TsFileConstant.DOUBLE_QUOTE
-            && pathSc.charAt(pathSc.length() - 1) != TsFileConstant.PATH_SEPARATOR_CHAR) {
-          int endIndex = pathSc.lastIndexOf(TsFileConstant.PATH_SEPARATOR_CHAR);
-          if (endIndex < 0) {
-            fullPath = pathSc;
-            device = "";
-            measurement = pathSc;
-          } else {
-            fullPath = pathSc;
-            device = pathSc.substring(0, endIndex);
-            measurement = pathSc.substring(endIndex + 1);
-          }
-        } else {
-          throw new IllegalArgumentException(ILLEGAL_PATH_ARGUMENT);
+        String[] nodes = PathNodesGenerator.splitPathToNodes(pathSc);
+        device = "";
+        if (nodes.length > 1) {
+          device = transformNodesToString(nodes, nodes.length - 1);
         }
+        measurement = nodes[nodes.length - 1];
+        fullPath = transformNodesToString(nodes, nodes.length);
       } else {
         fullPath = pathSc;
         device = "";
@@ -108,14 +93,29 @@ public class Path implements Serializable, Comparable<Path> {
    */
   public Path(String device, String measurement) {
     if (device == null || measurement == null) {
-      throw new IllegalArgumentException(ILLEGAL_PATH_ARGUMENT);
+      throw new PathParseException(ILLEGAL_PATH_ARGUMENT);
     }
-    this.device = device;
-    this.measurement = measurement;
-    if (!"".equals(device)) {
-      this.fullPath = device + TsFileConstant.PATH_SEPARATOR + measurement;
+    // use PathNodesGenerator to check whether path is legal.
+    if (!StringUtils.isEmpty(device) && !StringUtils.isEmpty(measurement)) {
+      String path = device + TsFileConstant.PATH_SEPARATOR + measurement;
+      String[] nodes = PathNodesGenerator.splitPathToNodes(path);
+      this.device = transformNodesToString(nodes, nodes.length - 1);
+      this.measurement = nodes[nodes.length - 1];
+      this.fullPath = transformNodesToString(nodes, nodes.length);
+    } else if (!StringUtils.isEmpty(device)) {
+      String[] deviceNodes = PathNodesGenerator.splitPathToNodes(device);
+      this.device = transformNodesToString(deviceNodes, deviceNodes.length);
+      this.measurement = measurement;
+      this.fullPath = device;
+    } else if (!StringUtils.isEmpty(measurement)) {
+      String[] measurementNodes = PathNodesGenerator.splitPathToNodes(measurement);
+      this.measurement = transformNodesToString(measurementNodes, measurementNodes.length);
+      this.device = device;
+      this.fullPath = measurement;
     } else {
-      fullPath = measurement;
+      this.device = device;
+      this.measurement = measurement;
+      this.fullPath = "";
     }
   }
 
@@ -191,4 +191,14 @@ public class Path implements Serializable, Comparable<Path> {
     path.fullPath = ReadWriteIOUtils.readString(byteBuffer);
     return path;
   }
+
+  private String transformNodesToString(String[] nodes, int index) {
+    Validate.isTrue(nodes.length > 0);
+    StringBuilder s = new StringBuilder(nodes[0]);
+    for (int i = 1; i < index; i++) {
+      s.append(TsFileConstant.PATH_SEPARATOR);
+      s.append(nodes[i]);
+    }
+    return s.toString();
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathNodesGenerator.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathNodesGenerator.java
new file mode 100644
index 0000000000..30033ef65d
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathNodesGenerator.java
@@ -0,0 +1,94 @@
+/*
+ * 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.tsfile.read.common.parser;
+
+import org.apache.iotdb.db.qp.sql.PathParser;
+import org.apache.iotdb.db.qp.sql.SqlLexer;
+import org.apache.iotdb.tsfile.exception.PathParseException;
+
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.CharStreams;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.antlr.v4.runtime.misc.ParseCancellationException;
+import org.antlr.v4.runtime.tree.ParseTree;
+
+/** convert String path to String[] nodes * */
+public class PathNodesGenerator {
+  private static PathVisitor pathVisitor = new PathVisitor();
+
+  public static String[] splitPathToNodes(String path) throws PathParseException {
+    try {
+      return invokeParser(path);
+    } catch (ParseCancellationException e) {
+      throw new PathParseException(path);
+    }
+  }
+
+  /** throw exception if path is illegal. */
+  public static void checkPath(String path) throws PathParseException {
+    try {
+      invokeParser(path);
+    } catch (ParseCancellationException e) {
+      throw new PathParseException(path);
+    }
+  }
+
+  private static String[] invokeParser(String path) {
+
+    CharStream charStream1 = CharStreams.fromString(path);
+
+    SqlLexer lexer1 = new SqlLexer(charStream1);
+    lexer1.removeErrorListeners();
+    lexer1.addErrorListener(PathParseError.INSTANCE);
+
+    CommonTokenStream tokens1 = new CommonTokenStream(lexer1);
+
+    PathParser pathParser1 = new PathParser(tokens1);
+    pathParser1.getInterpreter().setPredictionMode(PredictionMode.SLL);
+    pathParser1.removeErrorListeners();
+    pathParser1.addErrorListener(PathParseError.INSTANCE);
+
+    ParseTree tree;
+    try {
+      // STAGE 1: try with simpler/faster SLL(*)
+      tree = pathParser1.path();
+      // if we get here, there was no syntax error and SLL(*) was enough;
+      // there is no need to try full LL(*)
+    } catch (Exception ex) {
+      CharStream charStream2 = CharStreams.fromString(path);
+
+      SqlLexer lexer2 = new SqlLexer(charStream2);
+      lexer2.removeErrorListeners();
+      lexer2.addErrorListener(PathParseError.INSTANCE);
+
+      CommonTokenStream tokens2 = new CommonTokenStream(lexer2);
+
+      PathParser pathParser2 = new PathParser(tokens2);
+      pathParser2.getInterpreter().setPredictionMode(PredictionMode.LL);
+      pathParser2.removeErrorListeners();
+      pathParser2.addErrorListener(PathParseError.INSTANCE);
+
+      // STAGE 2: parser with full LL(*)
+      tree = pathParser2.path();
+      // if we get here, it's LL not SLL
+    }
+    return pathVisitor.visit(tree);
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathParseError.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathParseError.java
new file mode 100644
index 0000000000..c29000b867
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathParseError.java
@@ -0,0 +1,39 @@
+/*
+ * 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.tsfile.read.common.parser;
+
+import org.antlr.v4.runtime.BaseErrorListener;
+import org.antlr.v4.runtime.RecognitionException;
+import org.antlr.v4.runtime.Recognizer;
+import org.antlr.v4.runtime.misc.ParseCancellationException;
+
+public class PathParseError extends BaseErrorListener {
+  public static final PathParseError INSTANCE = new PathParseError();
+
+  @Override
+  public void syntaxError(
+      Recognizer<?, ?> recognizer,
+      Object offendingSymbol,
+      int line,
+      int charPositionInLine,
+      String msg,
+      RecognitionException e) {
+    throw new ParseCancellationException("line " + line + ":" + charPositionInLine + " " + msg);
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathVisitor.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathVisitor.java
new file mode 100644
index 0000000000..8ab23507a3
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/parser/PathVisitor.java
@@ -0,0 +1,75 @@
+/*
+ * 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.tsfile.read.common.parser;
+
+import org.apache.iotdb.db.qp.sql.PathParser;
+import org.apache.iotdb.db.qp.sql.PathParser.NodeNameContext;
+import org.apache.iotdb.db.qp.sql.PathParserBaseVisitor;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.List;
+
+public class PathVisitor extends PathParserBaseVisitor<String[]> {
+
+  @Override
+  public String[] visitPath(PathParser.PathContext ctx) {
+    if (ctx.prefixPath() != null) {
+      return visitPrefixPath(ctx.prefixPath());
+    } else {
+      return visitSuffixPath(ctx.suffixPath());
+    }
+  }
+
+  @Override
+  public String[] visitPrefixPath(PathParser.PrefixPathContext ctx) {
+    List<NodeNameContext> nodeNames = ctx.nodeName();
+    String[] path = new String[nodeNames.size() + 1];
+    path[0] = ctx.ROOT().getText();
+    for (int i = 0; i < nodeNames.size(); i++) {
+      path[i + 1] = parseNodeName(nodeNames.get(i));
+    }
+    return path;
+  }
+
+  @Override
+  public String[] visitSuffixPath(PathParser.SuffixPathContext ctx) {
+    List<NodeNameContext> nodeNames = ctx.nodeName();
+    String[] path = new String[nodeNames.size()];
+    for (int i = 0; i < nodeNames.size(); i++) {
+      path[i] = parseNodeName(nodeNames.get(i));
+    }
+    return path;
+  }
+
+  private String parseNodeName(PathParser.NodeNameContext ctx) {
+    String nodeName = ctx.getText();
+    if (nodeName.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+        && nodeName.endsWith(TsFileConstant.BACK_QUOTE_STRING)) {
+      String unWrapped = nodeName.substring(1, nodeName.length() - 1);
+      if (StringUtils.isNumeric(unWrapped)
+          || !TsFileConstant.NODE_NAME_PATTERN.matcher(unWrapped).matches()) {
+        return nodeName;
+      }
+      return unWrapped;
+    }
+    return nodeName;
+  }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/common/PathTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/common/PathTest.java
index 2cfbcced96..f80f369cfe 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/common/PathTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/common/PathTest.java
@@ -18,32 +18,161 @@
  */
 package org.apache.iotdb.tsfile.read.common;
 
+import org.apache.iotdb.tsfile.exception.PathParseException;
+
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.junit.Assert.fail;
+
 public class PathTest {
   @Test
-  public void testPath() {
+  public void testLegalPath() {
+    // empty path
     Path a = new Path("", true);
     Assert.assertEquals("", a.getDevice());
     Assert.assertEquals("", a.getMeasurement());
-    Path b = new Path("root.\"sg\".\"d1\".\"s1\"", true);
-    Assert.assertEquals("root.\"sg\".\"d1\"", b.getDevice());
-    Assert.assertEquals("\"s1\"", b.getMeasurement());
-    Path c = new Path("root.\"sg\".\"d1\".s1", true);
-    Assert.assertEquals("root.\"sg\".\"d1\"", c.getDevice());
-    Assert.assertEquals("s1", c.getMeasurement());
-    Path d = new Path("s1", true);
-    Assert.assertEquals("s1", d.getMeasurement());
-    Assert.assertEquals("", d.getDevice());
-    Path e = new Path("root.\"s.g\".d1.\"s..\\\"s1\"", true);
-    Assert.assertEquals("root.\"s.g\".d1", e.getDevice());
-    Assert.assertEquals("\"s..\\\"s1\"", e.getMeasurement());
+
+    // empty device
+    Path b = new Path("s1", true);
+    Assert.assertEquals("s1", b.getMeasurement());
+    Assert.assertEquals("", b.getDevice());
+
+    // normal node
+    Path c = new Path("root.sg.a", true);
+    Assert.assertEquals("root.sg", c.getDevice());
+    Assert.assertEquals("a", c.getMeasurement());
+
+    // quoted node
+    Path d = new Path("root.sg.`a.b`", true);
+    Assert.assertEquals("root.sg", d.getDevice());
+    Assert.assertEquals("`a.b`", d.getMeasurement());
+
+    Path e = new Path("root.sg.`a.``b`", true);
+    Assert.assertEquals("root.sg", e.getDevice());
+    Assert.assertEquals("`a.``b`", e.getMeasurement());
+
+    Path f = new Path("root.`sg\"`.`a.``b`", true);
+    Assert.assertEquals("root.`sg\"`", f.getDevice());
+    Assert.assertEquals("`a.``b`", f.getMeasurement());
+
+    Path g = new Path("root.sg.`a.b\\\\`", true);
+    Assert.assertEquals("root.sg", g.getDevice());
+    Assert.assertEquals("`a.b\\\\`", g.getMeasurement());
+
+    // quoted node of digits
+    Path h = new Path("root.sg.`111`", true);
+    Assert.assertEquals("root.sg", h.getDevice());
+    Assert.assertEquals("`111`", h.getMeasurement());
+
+    // quoted node of key word
+    Path i = new Path("root.sg.`select`", true);
+    Assert.assertEquals("root.sg", i.getDevice());
+    Assert.assertEquals("select", i.getMeasurement());
+
+    // wildcard
+    Path j = new Path("root.sg.`a*b`", true);
+    Assert.assertEquals("root.sg", j.getDevice());
+    Assert.assertEquals("`a*b`", j.getMeasurement());
+
+    Path k = new Path("root.sg.*", true);
+    Assert.assertEquals("root.sg", k.getDevice());
+    Assert.assertEquals("*", k.getMeasurement());
+
+    Path l = new Path("root.sg.**", true);
+    Assert.assertEquals("root.sg", l.getDevice());
+    Assert.assertEquals("**", l.getMeasurement());
+
+    // raw key word
+    Path m = new Path("root.sg.select", true);
+    Assert.assertEquals("root.sg", m.getDevice());
+    Assert.assertEquals("select", m.getMeasurement());
+
+    Path n = new Path("root.sg.device", true);
+    Assert.assertEquals("root.sg", n.getDevice());
+    Assert.assertEquals("device", n.getMeasurement());
+
+    Path o = new Path("root.sg.drop_trigger", true);
+    Assert.assertEquals("root.sg", o.getDevice());
+    Assert.assertEquals("drop_trigger", o.getMeasurement());
   }
 
-  @Test(expected = IllegalArgumentException.class)
-  public void testWrongPath() {
-    Path c = new Path("root.\"sg\".\"d1\".\"s1\"\"", true);
-    System.out.println(c.getMeasurement());
+  @Test
+  public void tesIllegalPath() {
+    try {
+      new Path("root.sg`", true);
+      fail();
+    } catch (PathParseException ignored) {
+
+    }
+
+    try {
+      new Path("root.sg\na", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      new Path("root.select`", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      // pure digits
+      new Path("root.111", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      // single ` in quoted node
+      new Path("root.`a``", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      // single ` in quoted node
+      new Path("root.``a`", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      new Path("root.a*%", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      new Path("root.a*b", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      new Path("root.and", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      new Path("root.or", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      new Path("root.not", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
+
+    try {
+      new Path("root.contains", true);
+      fail();
+    } catch (PathParseException ignored) {
+    }
   }
 }