You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by sp...@apache.org on 2023/04/25 12:42:54 UTC

[iotdb] branch master updated: [IOTDB-5812] Reduce useless create of PartialPath in auth module (#9691)

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

spricoder 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 7d4fdc781f [IOTDB-5812] Reduce useless create of PartialPath in auth module (#9691)
7d4fdc781f is described below

commit 7d4fdc781f772d20404291292cb935f370eabf92
Author: ZhangHongYin <46...@users.noreply.github.com>
AuthorDate: Tue Apr 25 20:42:49 2023 +0800

    [IOTDB-5812] Reduce useless create of PartialPath in auth module (#9691)
    
    * modify path to PartialPath
    
    * Optimize the serialize and deserialize of PathPrivilege
    
    * Optimize auth string list to binary in thrift
    
    * Modify test
    
    * Remove Illegal Path
---
 .../consensus/request/auth/AuthorPlan.java         | 29 +++++--
 .../iotdb/confignode/manager/ConfigManager.java    |  2 +-
 .../apache/iotdb/confignode/manager/IManager.java  |  2 +-
 .../confignode/manager/PermissionManager.java      |  3 +-
 .../iotdb/confignode/persistence/AuthorInfo.java   | 34 ++++----
 .../thrift/ConfigNodeRPCServiceProcessor.java      | 10 ++-
 .../request/ConfigPhysicalPlanSerDeTest.java       |  8 +-
 .../confignode/persistence/AuthorInfoTest.java     | 30 +++----
 .../confignode/it/IoTDBClusterAuthorityIT.java     | 88 ++++++++++++--------
 .../commons/auth/authorizer/BasicAuthorizer.java   | 35 ++++----
 .../iotdb/commons/auth/authorizer/IAuthorizer.java | 33 ++++----
 .../commons/auth/authorizer/OpenIdAuthorizer.java  |  3 +-
 .../iotdb/commons/auth/entity/PathPrivilege.java   | 29 ++++---
 .../org/apache/iotdb/commons/auth/entity/Role.java | 13 +--
 .../org/apache/iotdb/commons/auth/entity/User.java | 13 +--
 .../iotdb/commons/auth/role/BasicRoleManager.java  |  5 +-
 .../iotdb/commons/auth/role/IRoleManager.java      |  6 +-
 .../iotdb/commons/auth/user/BasicUserManager.java  |  5 +-
 .../iotdb/commons/auth/user/IUserManager.java      |  6 +-
 .../org/apache/iotdb/commons/utils/AuthUtils.java  | 96 ++++++++++++++--------
 .../org/apache/iotdb/commons/utils/IOUtils.java    |  8 +-
 .../org/apache/iotdb/db/auth/AuthorityChecker.java | 10 +--
 .../apache/iotdb/db/auth/AuthorizerManager.java    | 15 ++--
 .../iotdb/db/auth/ClusterAuthorityFetcher.java     | 38 +++++----
 .../apache/iotdb/db/auth/IAuthorityFetcher.java    |  3 +-
 .../iotdb/db/auth/AuthorizerManagerTest.java       | 26 ++++--
 .../auth/authorizer/LocalFileAuthorizerTest.java   |  9 +-
 .../iotdb/db/auth/entity/PathPrivilegeTest.java    |  8 +-
 .../org/apache/iotdb/db/auth/entity/RoleTest.java  |  8 +-
 .../org/apache/iotdb/db/auth/entity/UserTest.java  |  8 +-
 .../db/auth/role/LocalFileRoleAccessorTest.java    |  6 +-
 .../db/auth/role/LocalFileRoleManagerTest.java     |  8 +-
 .../db/auth/user/LocalFileUserAccessorTest.java    |  6 +-
 .../db/auth/user/LocalFileUserManagerTest.java     |  8 +-
 .../security/encrypt/MessageDigestEncryptTest.java |  6 +-
 .../src/main/thrift/confignode.thrift              |  4 +-
 36 files changed, 379 insertions(+), 242 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java
index 3fa08c63cb..5255d47c46 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java
@@ -19,27 +19,33 @@
 package org.apache.iotdb.confignode.consensus.request.auth;
 
 import org.apache.iotdb.commons.auth.AuthException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
 import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
 import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
+import org.slf4j.Logger;
+
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
 
 public class AuthorPlan extends ConfigPhysicalPlan {
+  private static final Logger logger = org.slf4j.LoggerFactory.getLogger(AuthorPlan.class);
 
   private ConfigPhysicalPlanType authorType;
   private String roleName;
   private String password;
   private String newPassword;
   private Set<Integer> permissions;
-  private List<String> nodeNameList;
+  private List<PartialPath> nodeNameList;
   private String userName;
 
   public AuthorPlan(ConfigPhysicalPlanType type) {
@@ -66,7 +72,7 @@ public class AuthorPlan extends ConfigPhysicalPlan {
       String password,
       String newPassword,
       Set<Integer> permissions,
-      List<String> nodeNameList)
+      List<PartialPath> nodeNameList)
       throws AuthException {
     this(authorType);
     this.authorType = authorType;
@@ -118,11 +124,11 @@ public class AuthorPlan extends ConfigPhysicalPlan {
     this.permissions = permissions;
   }
 
-  public List<String> getNodeNameList() {
+  public List<PartialPath> getNodeNameList() {
     return nodeNameList;
   }
 
-  public void setNodeNameList(List<String> nodeNameList) {
+  public void setNodeNameList(List<PartialPath> nodeNameList) {
     this.nodeNameList = nodeNameList;
   }
 
@@ -150,7 +156,10 @@ public class AuthorPlan extends ConfigPhysicalPlan {
         stream.writeInt(permission);
       }
     }
-    BasicStructureSerDeUtil.write(nodeNameList, stream);
+    BasicStructureSerDeUtil.write(nodeNameList.size(), stream);
+    for (PartialPath partialPath : nodeNameList) {
+      BasicStructureSerDeUtil.write(partialPath.getFullPath(), stream);
+    }
   }
 
   @Override
@@ -169,7 +178,15 @@ public class AuthorPlan extends ConfigPhysicalPlan {
         permissions.add(buffer.getInt());
       }
     }
-    nodeNameList = BasicStructureSerDeUtil.readStringList(buffer);
+    int nodeNameListSize = BasicStructureSerDeUtil.readInt(buffer);
+    nodeNameList = new ArrayList<>(nodeNameListSize);
+    try {
+      for (int i = 0; i < nodeNameListSize; i++) {
+        nodeNameList.add(new PartialPath(BasicStructureSerDeUtil.readString(buffer)));
+      }
+    } catch (MetadataException e) {
+      logger.error("Invalid path when deserialize authPlan: {}", nodeNameList, e);
+    }
   }
 
   private short getPlanType(ConfigPhysicalPlanType configPhysicalPlanType) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 96666e1eda..d3cbaa2518 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -966,7 +966,7 @@ public class ConfigManager implements IManager {
 
   @Override
   public TPermissionInfoResp checkUserPrivileges(
-      String username, List<String> paths, int permission) {
+      String username, List<PartialPath> paths, int permission) {
     TSStatus status = confirmLeader();
     if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
       return permissionManager.checkUserPrivileges(username, paths, permission);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index 0cc60c8f9f..329372437b 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -375,7 +375,7 @@ public interface IManager {
   TPermissionInfoResp login(String username, String password);
 
   /** Check User Privileges */
-  TPermissionInfoResp checkUserPrivileges(String username, List<String> paths, int permission);
+  TPermissionInfoResp checkUserPrivileges(String username, List<PartialPath> paths, int permission);
 
   /**
    * Register ConfigNode when it is first startup
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
index 098a67a88d..a238d00b99 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.confignode.manager;
 
 import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.confignode.client.DataNodeRequestType;
 import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
 import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
@@ -86,7 +87,7 @@ public class PermissionManager {
   }
 
   public TPermissionInfoResp checkUserPrivileges(
-      String username, List<String> paths, int permission) {
+      String username, List<PartialPath> paths, int permission) {
     return authorInfo.checkUserPrivileges(username, paths, permission);
   }
 
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java
index 0d72f5a59a..fc498381b3 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.commons.auth.entity.User;
 import org.apache.iotdb.commons.conf.CommonConfig;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.commons.utils.FileUtils;
@@ -107,11 +108,11 @@ public class AuthorInfo implements SnapshotProcessor {
   }
 
   public TPermissionInfoResp checkUserPrivileges(
-      String username, List<String> paths, int permission) {
+      String username, List<PartialPath> paths, int permission) {
     boolean status = true;
     TPermissionInfoResp result = new TPermissionInfoResp();
     try {
-      for (String path : paths) {
+      for (PartialPath path : paths) {
         if (!checkOnePath(username, path, permission)) {
           status = false;
           break;
@@ -135,7 +136,8 @@ public class AuthorInfo implements SnapshotProcessor {
     return result;
   }
 
-  private boolean checkOnePath(String username, String path, int permission) throws AuthException {
+  private boolean checkOnePath(String username, PartialPath path, int permission)
+      throws AuthException {
     try {
       if (authorizer.checkUserPrivileges(username, path, permission)) {
         return true;
@@ -154,7 +156,7 @@ public class AuthorInfo implements SnapshotProcessor {
     String password = authorPlan.getPassword();
     String newPassword = authorPlan.getNewPassword();
     Set<Integer> permissions = authorPlan.getPermissions();
-    List<String> nodeNameList = authorPlan.getNodeNameList();
+    List<PartialPath> nodeNameList = authorPlan.getNodeNameList();
     try {
       switch (authorType) {
         case UpdateUser:
@@ -174,14 +176,14 @@ public class AuthorInfo implements SnapshotProcessor {
           break;
         case GrantRole:
           for (int i : permissions) {
-            for (String path : nodeNameList) {
+            for (PartialPath path : nodeNameList) {
               authorizer.grantPrivilegeToRole(roleName, path, i);
             }
           }
           break;
         case GrantUser:
           for (int i : permissions) {
-            for (String path : nodeNameList) {
+            for (PartialPath path : nodeNameList) {
               authorizer.grantPrivilegeToUser(userName, path, i);
             }
           }
@@ -191,14 +193,14 @@ public class AuthorInfo implements SnapshotProcessor {
           break;
         case RevokeUser:
           for (int i : permissions) {
-            for (String path : nodeNameList) {
+            for (PartialPath path : nodeNameList) {
               authorizer.revokePrivilegeFromUser(userName, path, i);
             }
           }
           break;
         case RevokeRole:
           for (int i : permissions) {
-            for (String path : nodeNameList) {
+            for (PartialPath path : nodeNameList) {
               authorizer.revokePrivilegeFromRole(roleName, path, i);
             }
           }
@@ -285,8 +287,8 @@ public class AuthorInfo implements SnapshotProcessor {
         rolePrivilegesSet.add(pathPrivilege.toString());
         continue;
       }
-      for (String path : plan.getNodeNameList()) {
-        if (AuthUtils.pathBelongsTo(pathPrivilege.getPath(), path)) {
+      for (PartialPath path : plan.getNodeNameList()) {
+        if (path.matchFullPath(pathPrivilege.getPath())) {
           rolePrivilegesSet.add(pathPrivilege.toString());
         }
       }
@@ -324,8 +326,8 @@ public class AuthorInfo implements SnapshotProcessor {
           userPrivilegeSet.add(pathPrivilege.toString());
           continue;
         }
-        for (String path : plan.getNodeNameList()) {
-          if (AuthUtils.pathBelongsTo(pathPrivilege.getPath(), path)
+        for (PartialPath path : plan.getNodeNameList()) {
+          if (path.matchFullPath(pathPrivilege.getPath())
               && !userPrivilegeSet.contains(pathPrivilege.toString())) {
             rolePrivileges.add("");
             userPrivilegeSet.add(pathPrivilege.toString());
@@ -346,8 +348,8 @@ public class AuthorInfo implements SnapshotProcessor {
             rolePrivilegeSet.add(pathPrivilege.toString());
             continue;
           }
-          for (String path : plan.getNodeNameList()) {
-            if (AuthUtils.pathBelongsTo(pathPrivilege.getPath(), path)
+          for (PartialPath path : plan.getNodeNameList()) {
+            if (path.matchFullPath(pathPrivilege.getPath())
                 && !rolePrivilegeSet.contains(pathPrivilege.toString())) {
               rolePrivileges.add(roleN);
               rolePrivilegeSet.add(pathPrivilege.toString());
@@ -402,7 +404,7 @@ public class AuthorInfo implements SnapshotProcessor {
     User user = authorizer.getUser(username);
     if (user.getPrivilegeList() != null) {
       for (PathPrivilege pathPrivilege : user.getPrivilegeList()) {
-        userPrivilegeList.add(pathPrivilege.getPath());
+        userPrivilegeList.add(pathPrivilege.getPath().getFullPath());
         String privilegeIdList = pathPrivilege.getPrivileges().toString();
         userPrivilegeList.add(privilegeIdList.substring(1, privilegeIdList.length() - 1));
       }
@@ -418,7 +420,7 @@ public class AuthorInfo implements SnapshotProcessor {
         Role role = authorizer.getRole(roleName);
         List<String> rolePrivilegeList = new ArrayList<>();
         for (PathPrivilege pathPrivilege : role.getPrivilegeList()) {
-          rolePrivilegeList.add(pathPrivilege.getPath());
+          rolePrivilegeList.add(pathPrivilege.getPath().getFullPath());
           String privilegeIdList = pathPrivilege.getPrivileges().toString();
           rolePrivilegeList.add(privilegeIdList.substring(1, privilegeIdList.length() - 1));
         }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 88f4dd07d8..22367d7c16 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -32,6 +32,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.commons.utils.StatusUtils;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.confignode.conf.ConfigNodeConfig;
@@ -504,7 +505,7 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
               req.getPassword(),
               req.getNewPassword(),
               req.getPermissions(),
-              req.getNodeNameList());
+              AuthUtils.deserializePartialPathList(ByteBuffer.wrap(req.getNodeNameList())));
     } catch (AuthException e) {
       LOGGER.error(e.getMessage());
     }
@@ -527,7 +528,7 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
               req.getPassword(),
               req.getNewPassword(),
               req.getPermissions(),
-              req.getNodeNameList());
+              AuthUtils.deserializePartialPathList(ByteBuffer.wrap(req.getNodeNameList())));
     } catch (AuthException e) {
       LOGGER.error(e.getMessage());
     }
@@ -544,8 +545,9 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
 
   @Override
   public TPermissionInfoResp checkUserPrivileges(TCheckUserPrivilegesReq req) {
-    return configManager.checkUserPrivileges(
-        req.getUsername(), req.getPaths(), req.getPermission());
+    List<PartialPath> partialPaths =
+        AuthUtils.deserializePartialPathList(ByteBuffer.wrap(req.getPaths()));
+    return configManager.checkUserPrivileges(req.getUsername(), partialPaths, req.getPermission());
   }
 
   @Override
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
index f3a179f420..7295abfa33 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
@@ -524,7 +524,7 @@ public class ConfigPhysicalPlanSerDeTest {
   }
 
   @Test
-  public void AuthorPlanTest() throws IOException, AuthException {
+  public void AuthorPlanTest() throws IOException, AuthException, IllegalPathException {
 
     AuthorPlan req0;
     AuthorPlan req1;
@@ -572,9 +572,9 @@ public class ConfigPhysicalPlanSerDeTest {
     Assert.assertEquals(req0, req1);
 
     // grant user
-    List<String> nodeNameList = new ArrayList<>();
-    nodeNameList.add("root.ln.**");
-    nodeNameList.add("root.abc.**");
+    List<PartialPath> nodeNameList = new ArrayList<>();
+    nodeNameList.add(new PartialPath("root.ln.**"));
+    nodeNameList.add(new PartialPath("root.abc.**"));
     req0 =
         new AuthorPlan(
             ConfigPhysicalPlanType.GrantUser, "tempuser", "", "", "", permissions, nodeNameList);
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/persistence/AuthorInfoTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/persistence/AuthorInfoTest.java
index cc0e2888ca..2d9802444f 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/persistence/AuthorInfoTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/persistence/AuthorInfoTest.java
@@ -23,6 +23,8 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.PrivilegeType;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
 import org.apache.iotdb.confignode.consensus.request.auth.AuthorPlan;
 import org.apache.iotdb.confignode.consensus.response.auth.PermissionInfoResp;
@@ -70,7 +72,7 @@ public class AuthorInfoTest {
   }
 
   @Test
-  public void permissionTest() throws TException, AuthException {
+  public void permissionTest() throws TException, AuthException, IllegalPathException {
 
     TSStatus status;
 
@@ -98,8 +100,8 @@ public class AuthorInfoTest {
     privilege.add("root.** : CREATE_USER");
     privilege.add("root.** : CREATE_USER");
 
-    List<String> paths = new ArrayList<>();
-    paths.add("root.ln");
+    List<PartialPath> paths = new ArrayList<>();
+    paths.add(new PartialPath("root.ln"));
 
     cleanUserAndRole();
 
@@ -205,8 +207,8 @@ public class AuthorInfoTest {
     Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
     // grant user
-    List<String> nodeNameList = new ArrayList<>();
-    nodeNameList.add("root.ln.**");
+    List<PartialPath> nodeNameList = new ArrayList<>();
+    nodeNameList.add(new PartialPath("root.ln.**"));
     authorPlan =
         new AuthorPlan(
             ConfigPhysicalPlanType.GrantUser, "user0", "", "", "", privilegeList, nodeNameList);
@@ -279,7 +281,7 @@ public class AuthorInfoTest {
             "",
             "",
             new HashSet<>(),
-            Collections.singletonList("root.**"));
+            Collections.singletonList(new PartialPath("root.**")));
     permissionInfoResp = authorInfo.executeListUserPrivileges(authorPlan);
     status = permissionInfoResp.getStatus();
     Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -328,7 +330,7 @@ public class AuthorInfoTest {
             "",
             "",
             new HashSet<>(),
-            Collections.singletonList("root.**"));
+            Collections.singletonList(new PartialPath("root.**")));
     permissionInfoResp = authorInfo.executeListRolePrivileges(authorPlan);
     status = permissionInfoResp.getStatus();
     Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -505,7 +507,7 @@ public class AuthorInfoTest {
   }
 
   @Test
-  public void testMultPathsPermission() throws TException, AuthException {
+  public void testMultPathsPermission() throws TException, AuthException, IllegalPathException {
     TSStatus status;
 
     AuthorPlan authorPlan;
@@ -530,13 +532,13 @@ public class AuthorInfoTest {
     allPrivilege.addAll(rolePrivilege);
     Collections.sort(allPrivilege);
 
-    List<String> userPaths = new ArrayList<>();
-    userPaths.add("root.ln.**");
-    userPaths.add("root.sg.**");
+    List<PartialPath> userPaths = new ArrayList<>();
+    userPaths.add(new PartialPath("root.ln.**"));
+    userPaths.add(new PartialPath("root.sg.**"));
 
-    List<String> rolePaths = new ArrayList<>();
-    rolePaths.add("root.role_1.**");
-    rolePaths.add("root.abc.**");
+    List<PartialPath> rolePaths = new ArrayList<>();
+    rolePaths.add(new PartialPath("root.role_1.**"));
+    rolePaths.add(new PartialPath("root.abc.**"));
 
     cleanUserAndRole();
 
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBClusterAuthorityIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBClusterAuthorityIT.java
index 4842a108d0..41119bd73e 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBClusterAuthorityIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBClusterAuthorityIT.java
@@ -23,6 +23,9 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.auth.entity.PrivilegeType;
 import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.confignode.rpc.thrift.IConfigNodeRPCService;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerResp;
@@ -65,7 +68,8 @@ public class IoTDBClusterAuthorityIT {
     EnvFactory.getEnv().cleanClusterEnvironment();
   }
 
-  private void cleanUserAndRole(IConfigNodeRPCService.Iface client) throws TException {
+  private void cleanUserAndRole(IConfigNodeRPCService.Iface client)
+      throws TException, IllegalPathException {
     TSStatus status;
 
     // clean user
@@ -77,7 +81,7 @@ public class IoTDBClusterAuthorityIT {
             "",
             "",
             new HashSet<>(),
-            Collections.singletonList(""));
+            AuthUtils.serializePartialPathList(new ArrayList<>()));
     TAuthorizerResp authorizerResp = client.queryPermission(authorizerReq);
     status = authorizerResp.getStatus();
     assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -93,7 +97,7 @@ public class IoTDBClusterAuthorityIT {
                 "",
                 "",
                 new HashSet<>(),
-                Collections.singletonList(""));
+                AuthUtils.serializePartialPathList(new ArrayList<>()));
         status = client.operatePermission(authorizerReq);
         assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
       }
@@ -101,7 +105,7 @@ public class IoTDBClusterAuthorityIT {
   }
 
   @Test
-  public void permissionTest() {
+  public void permissionTest() throws IllegalPathException {
     TSStatus status;
     List<String> userList = new ArrayList<>();
     userList.add("root");
@@ -127,8 +131,8 @@ public class IoTDBClusterAuthorityIT {
     privilege.add("root.** : CREATE_USER");
     privilege.add("root.** : CREATE_USER");
 
-    List<String> paths = new ArrayList<>();
-    paths.add("root.ln.**");
+    List<PartialPath> paths = new ArrayList<>();
+    paths.add(new PartialPath("root.ln.**"));
 
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
@@ -143,7 +147,7 @@ public class IoTDBClusterAuthorityIT {
               "passwd",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
       authorizerReq.setUserName("tempuser1");
@@ -152,7 +156,10 @@ public class IoTDBClusterAuthorityIT {
 
       // check user privileges
       checkUserPrivilegesReq =
-          new TCheckUserPrivilegesReq("tempuser0", paths, PrivilegeType.DELETE_USER.ordinal());
+          new TCheckUserPrivilegesReq(
+              "tempuser0",
+              AuthUtils.serializePartialPathList(paths),
+              PrivilegeType.DELETE_USER.ordinal());
       status = client.checkUserPrivileges(checkUserPrivilegesReq).getStatus();
       assertEquals(TSStatusCode.NO_PERMISSION.getStatusCode(), status.getCode());
 
@@ -165,14 +172,20 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
       // list user
       authorizerReq =
           new TAuthorizerReq(
-              AuthorType.LIST_USER.ordinal(), "", "", "", "", new HashSet<>(), new ArrayList<>());
+              AuthorType.LIST_USER.ordinal(),
+              "",
+              "",
+              "",
+              "",
+              new HashSet<>(),
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -188,7 +201,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
       authorizerReq.setRoleName("temprole1");
@@ -204,14 +217,20 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
       // list role
       authorizerReq =
           new TAuthorizerReq(
-              AuthorType.LIST_ROLE.ordinal(), "", "", "", "", new HashSet<>(), new ArrayList<>());
+              AuthorType.LIST_ROLE.ordinal(),
+              "",
+              "",
+              "",
+              "",
+              new HashSet<>(),
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -227,13 +246,13 @@ public class IoTDBClusterAuthorityIT {
               "",
               "newpwd",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
       // grant user
-      List<String> nodeNameList = new ArrayList<>();
-      nodeNameList.add("root.ln.**");
+      List<PartialPath> nodeNameList = new ArrayList<>();
+      nodeNameList.add(new PartialPath("root.ln.**"));
       authorizerReq =
           new TAuthorizerReq(
               AuthorType.GRANT_USER.ordinal(),
@@ -242,13 +261,16 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               privilegeList,
-              nodeNameList);
+              AuthUtils.serializePartialPathList(nodeNameList));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
       // check user privileges
       checkUserPrivilegesReq =
-          new TCheckUserPrivilegesReq("tempuser0", paths, PrivilegeType.DELETE_USER.ordinal());
+          new TCheckUserPrivilegesReq(
+              "tempuser0",
+              AuthUtils.serializePartialPathList(paths),
+              PrivilegeType.DELETE_USER.ordinal());
       status = client.checkUserPrivileges(checkUserPrivilegesReq).getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
@@ -261,7 +283,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               privilegeList,
-              nodeNameList);
+              AuthUtils.serializePartialPathList(nodeNameList));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
@@ -274,7 +296,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              nodeNameList);
+              AuthUtils.serializePartialPathList(nodeNameList));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
@@ -287,7 +309,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               revokePrivilege,
-              nodeNameList);
+              AuthUtils.serializePartialPathList(nodeNameList));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
@@ -300,7 +322,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               revokePrivilege,
-              nodeNameList);
+              AuthUtils.serializePartialPathList(nodeNameList));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
@@ -313,7 +335,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              nodeNameList);
+              AuthUtils.serializePartialPathList(nodeNameList));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -329,7 +351,8 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              Collections.singletonList("root.**"));
+              AuthUtils.serializePartialPathList(
+                  Collections.singletonList(new PartialPath("root.**"))));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -345,7 +368,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -361,7 +384,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              nodeNameList);
+              AuthUtils.serializePartialPathList(nodeNameList));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -378,7 +401,8 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              Collections.singletonList("root.**"));
+              AuthUtils.serializePartialPathList(
+                  Collections.singletonList(new PartialPath("root.**"))));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -394,7 +418,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -410,7 +434,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -426,7 +450,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
@@ -443,7 +467,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       status = client.operatePermission(authorizerReq);
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
 
@@ -456,7 +480,7 @@ public class IoTDBClusterAuthorityIT {
               "",
               "",
               new HashSet<>(),
-              new ArrayList<>());
+              AuthUtils.serializePartialPathList(new ArrayList<>()));
       authorizerResp = client.queryPermission(authorizerReq);
       status = authorizerResp.getStatus();
       assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/BasicAuthorizer.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/BasicAuthorizer.java
index 5e682da97f..7c1c891bb9 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/BasicAuthorizer.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/BasicAuthorizer.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.commons.auth.role.IRoleManager;
 import org.apache.iotdb.commons.auth.user.IUserManager;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
 import org.apache.iotdb.commons.utils.AuthUtils;
@@ -137,16 +138,16 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public void grantPrivilegeToUser(String username, String path, int privilegeId)
+  public void grantPrivilegeToUser(String username, PartialPath path, int privilegeId)
       throws AuthException {
-    String newPath = path;
+    PartialPath newPath = path;
     if (isAdmin(username)) {
       throw new AuthException(
           TSStatusCode.NO_PERMISSION,
           "Invalid operation, administrator already has all privileges");
     }
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
-      newPath = AuthUtils.ROOT_PATH_PRIVILEGE;
+      newPath = AuthUtils.ROOT_PATH_PRIVILEGE_PATH;
     }
     if (!userManager.grantPrivilegeToUser(username, newPath, privilegeId)) {
       throw new AuthException(
@@ -157,17 +158,17 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public void revokePrivilegeFromUser(String username, String path, int privilegeId)
+  public void revokePrivilegeFromUser(String username, PartialPath path, int privilegeId)
       throws AuthException {
     if (isAdmin(username)) {
       throw new AuthException(
           TSStatusCode.NO_PERMISSION, "Invalid operation, administrator must have all privileges");
     }
-    String p = path;
+    PartialPath newPath = path;
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
-      p = AuthUtils.ROOT_PATH_PRIVILEGE;
+      newPath = AuthUtils.ROOT_PATH_PRIVILEGE_PATH;
     }
-    if (!userManager.revokePrivilegeFromUser(username, p, privilegeId)) {
+    if (!userManager.revokePrivilegeFromUser(username, newPath, privilegeId)) {
       throw new AuthException(
           TSStatusCode.NOT_HAS_PRIVILEGE,
           String.format(
@@ -209,13 +210,13 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public void grantPrivilegeToRole(String roleName, String path, int privilegeId)
+  public void grantPrivilegeToRole(String roleName, PartialPath path, int privilegeId)
       throws AuthException {
-    String p = path;
+    PartialPath newPath = path;
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
-      p = AuthUtils.ROOT_PATH_PRIVILEGE;
+      newPath = AuthUtils.ROOT_PATH_PRIVILEGE_PATH;
     }
-    if (!roleManager.grantPrivilegeToRole(roleName, p, privilegeId)) {
+    if (!roleManager.grantPrivilegeToRole(roleName, newPath, privilegeId)) {
       throw new AuthException(
           TSStatusCode.ALREADY_HAS_PRIVILEGE,
           String.format(
@@ -224,13 +225,13 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public void revokePrivilegeFromRole(String roleName, String path, int privilegeId)
+  public void revokePrivilegeFromRole(String roleName, PartialPath path, int privilegeId)
       throws AuthException {
-    String p = path;
+    PartialPath newPath = path;
     if (!PrivilegeType.isPathRelevant(privilegeId)) {
-      p = AuthUtils.ROOT_PATH_PRIVILEGE;
+      newPath = AuthUtils.ROOT_PATH_PRIVILEGE_PATH;
     }
-    if (!roleManager.revokePrivilegeFromRole(roleName, p, privilegeId)) {
+    if (!roleManager.revokePrivilegeFromRole(roleName, newPath, privilegeId)) {
       throw new AuthException(
           TSStatusCode.NOT_HAS_PRIVILEGE,
           String.format(
@@ -276,7 +277,7 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public Set<Integer> getPrivileges(String username, String path) throws AuthException {
+  public Set<Integer> getPrivileges(String username, PartialPath path) throws AuthException {
     if (isAdmin(username)) {
       return ADMIN_PRIVILEGES;
     }
@@ -306,7 +307,7 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   }
 
   @Override
-  public boolean checkUserPrivileges(String username, String path, int privilegeId)
+  public boolean checkUserPrivileges(String username, PartialPath path, int privilegeId)
       throws AuthException {
     if (isAdmin(username)) {
       return true;
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/IAuthorizer.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/IAuthorizer.java
index d06df9b2cf..4138d99e49 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/IAuthorizer.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/IAuthorizer.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.commons.auth.authorizer;
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.entity.User;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
 
 import java.util.List;
@@ -69,7 +70,8 @@ public interface IAuthorizer extends SnapshotProcessor {
    * @throws AuthException If the user does not exist or the privilege or the seriesPath is illegal
    *     or the permission already exists.
    */
-  void grantPrivilegeToUser(String username, String path, int privilegeId) throws AuthException;
+  void grantPrivilegeToUser(String username, PartialPath path, int privilegeId)
+      throws AuthException;
 
   /**
    * Revoke a privilege on seriesPath from a user.
@@ -81,7 +83,8 @@ public interface IAuthorizer extends SnapshotProcessor {
    * @throws AuthException If the user does not exist or the privilege or the seriesPath is illegal
    *     or if the permission does not exist.
    */
-  void revokePrivilegeFromUser(String username, String path, int privilegeId) throws AuthException;
+  void revokePrivilegeFromUser(String username, PartialPath path, int privilegeId)
+      throws AuthException;
 
   /**
    * Add a role.
@@ -109,7 +112,8 @@ public interface IAuthorizer extends SnapshotProcessor {
    * @throws AuthException If the role does not exist or the privilege or the seriesPath is illegal
    *     or the privilege already exists.
    */
-  void grantPrivilegeToRole(String roleName, String path, int privilegeId) throws AuthException;
+  void grantPrivilegeToRole(String roleName, PartialPath path, int privilegeId)
+      throws AuthException;
 
   /**
    * Remove a privilege on a seriesPath from a role.
@@ -121,7 +125,8 @@ public interface IAuthorizer extends SnapshotProcessor {
    * @throws AuthException If the role does not exist or the privilege or the seriesPath is illegal
    *     or the privilege does not exists.
    */
-  void revokePrivilegeFromRole(String roleName, String path, int privilegeId) throws AuthException;
+  void revokePrivilegeFromRole(String roleName, PartialPath path, int privilegeId)
+      throws AuthException;
 
   /**
    * Add a role to a user.
@@ -150,7 +155,7 @@ public interface IAuthorizer extends SnapshotProcessor {
    * @return A set of integers each present a privilege.
    * @throws AuthException if exception raised when finding the privileges.
    */
-  Set<Integer> getPrivileges(String username, String path) throws AuthException;
+  Set<Integer> getPrivileges(String username, PartialPath path) throws AuthException;
 
   /**
    * Modify the password of a user.
@@ -171,7 +176,8 @@ public interface IAuthorizer extends SnapshotProcessor {
    * @return True if the user has such privilege, false if the user does not have such privilege.
    * @throws AuthException If the seriesPath or the privilege is illegal.
    */
-  boolean checkUserPrivileges(String username, String path, int privilegeId) throws AuthException;
+  boolean checkUserPrivileges(String username, PartialPath path, int privilegeId)
+      throws AuthException;
 
   /** Reset the Authorizer to initiative status. */
   void reset() throws AuthException;
@@ -209,8 +215,7 @@ public interface IAuthorizer extends SnapshotProcessor {
   /**
    * Whether data water-mark is enabled for user 'userName'.
    *
-   * @param userName
-   * @return
+   * @param userName the name of user
    * @throws AuthException if the user does not exist
    */
   boolean isUserUseWaterMark(String userName) throws AuthException;
@@ -218,8 +223,8 @@ public interface IAuthorizer extends SnapshotProcessor {
   /**
    * Enable or disable data water-mark for user 'userName'.
    *
-   * @param userName
-   * @param useWaterMark
+   * @param userName the name of user
+   * @param useWaterMark whether to use water-mark or not
    * @throws AuthException if the user does not exist.
    */
   void setUserUseWaterMark(String userName, boolean useWaterMark) throws AuthException;
@@ -246,18 +251,18 @@ public interface IAuthorizer extends SnapshotProcessor {
   Map<String, Role> getAllRoles();
 
   /**
-   * clear all old users info, replace the old users with the new one
+   * clear all old user info, replace the old users with the new one
    *
    * @param users new users info
-   * @throws AuthException
+   * @throws AuthException IOException
    */
   void replaceAllUsers(Map<String, User> users) throws AuthException;
 
   /**
-   * clear all old roles info, replace the old roles with the new one
+   * clear all old role info, replace the old roles with the new one
    *
    * @param roles new roles info
-   * @throws AuthException
+   * @throws AuthException IOException
    */
   void replaceAllRoles(Map<String, Role> roles) throws AuthException;
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/OpenIdAuthorizer.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/OpenIdAuthorizer.java
index b1fd48e9b6..c43a34acf6 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/OpenIdAuthorizer.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/OpenIdAuthorizer.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.commons.auth.role.LocalFileRoleManager;
 import org.apache.iotdb.commons.auth.user.LocalFileUserManager;
 import org.apache.iotdb.commons.conf.CommonConfig;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 import com.nimbusds.jose.JOSEException;
@@ -255,7 +256,7 @@ public class OpenIdAuthorizer extends BasicAuthorizer {
   }
 
   @Override
-  public boolean checkUserPrivileges(String username, String path, int privilegeId)
+  public boolean checkUserPrivileges(String username, PartialPath path, int privilegeId)
       throws AuthException {
     return isAdmin(username);
   }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java
index 70d138005a..b1fda65ddd 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java
@@ -19,10 +19,16 @@
 
 package org.apache.iotdb.commons.auth.entity;
 
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
 import org.apache.iotdb.commons.utils.SerializeUtils;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -32,12 +38,12 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * This class represents a privilege on a specific seriesPath. If the privilege is seriesPath-free,
- * the seriesPath will be null.
+ * the seriesPath will be null.I
  */
 public class PathPrivilege {
-
+  private static final Logger logger = LoggerFactory.getLogger(PathPrivilege.class);
   private Set<Integer> privileges;
-  private String path;
+  private PartialPath path;
 
   /**
    * This field records how many times this privilege is referenced during a life cycle (from being
@@ -56,7 +62,7 @@ public class PathPrivilege {
     // Empty constructor
   }
 
-  public PathPrivilege(String path) {
+  public PathPrivilege(PartialPath path) {
     this.path = path;
     this.privileges = new HashSet<>();
   }
@@ -69,11 +75,11 @@ public class PathPrivilege {
     this.privileges = privileges;
   }
 
-  public String getPath() {
+  public PartialPath getPath() {
     return path;
   }
 
-  public void setPath(String path) {
+  public void setPath(PartialPath path) {
     this.path = path;
   }
 
@@ -104,7 +110,7 @@ public class PathPrivilege {
 
   @Override
   public String toString() {
-    StringBuilder builder = new StringBuilder(path);
+    StringBuilder builder = new StringBuilder(path.getFullPath());
     builder.append(" :");
     for (Integer privilegeId : privileges) {
       builder.append(" ").append(PrivilegeType.values()[privilegeId]);
@@ -117,14 +123,17 @@ public class PathPrivilege {
     DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
 
     SerializeUtils.serializeIntSet(privileges, dataOutputStream);
-    SerializeUtils.serialize(path, dataOutputStream);
-
+    try {
+      path.serialize(dataOutputStream);
+    } catch (IOException exception) {
+      logger.error("Unexpected exception when serialize path", exception);
+    }
     return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
   }
 
   public void deserialize(ByteBuffer buffer) {
     privileges = new HashSet<>();
     SerializeUtils.deserializeIntSet(privileges, buffer);
-    path = SerializeUtils.deserializeString(buffer);
+    path = (PartialPath) PathDeserializeUtil.deserialize(buffer);
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java
index eb3001d2d3..d942089697 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java
@@ -19,6 +19,7 @@
 package org.apache.iotdb.commons.auth.entity;
 
 import org.apache.iotdb.commons.auth.AuthException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.commons.utils.SerializeUtils;
 
@@ -62,20 +63,20 @@ public class Role {
     this.privilegeList = privilegeList;
   }
 
-  public boolean hasPrivilege(String path, int privilegeId) {
+  public boolean hasPrivilege(PartialPath path, int privilegeId) {
     return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
   }
 
-  public void addPrivilege(String path, int privilegeId) {
+  public void addPrivilege(PartialPath path, int privilegeId) {
     AuthUtils.addPrivilege(path, privilegeId, privilegeList);
   }
 
-  public void removePrivilege(String path, int privilegeId) {
+  public void removePrivilege(PartialPath path, int privilegeId) {
     AuthUtils.removePrivilege(path, privilegeId, privilegeList);
   }
 
   /** set privileges of path. */
-  public void setPrivileges(String path, Set<Integer> privileges) {
+  public void setPrivileges(PartialPath path, Set<Integer> privileges) {
     for (PathPrivilege pathPrivilege : privilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
         pathPrivilege.setPrivileges(privileges);
@@ -83,11 +84,11 @@ public class Role {
     }
   }
 
-  public Set<Integer> getPrivileges(String path) throws AuthException {
+  public Set<Integer> getPrivileges(PartialPath path) throws AuthException {
     return AuthUtils.getPrivileges(path, privilegeList);
   }
 
-  public boolean checkPrivilege(String path, int privilegeId) throws AuthException {
+  public boolean checkPrivilege(PartialPath path, int privilegeId) throws AuthException {
     return AuthUtils.checkPrivilege(path, privilegeId, privilegeList);
   }
 
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/User.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/User.java
index 67c27dd5b4..4edc513c8c 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/User.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/User.java
@@ -19,6 +19,7 @@
 package org.apache.iotdb.commons.auth.entity;
 
 import org.apache.iotdb.commons.auth.AuthException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.commons.utils.SerializeUtils;
 
@@ -105,15 +106,15 @@ public class User {
     this.lastActiveTime = lastActiveTime;
   }
 
-  public boolean hasPrivilege(String path, int privilegeId) {
+  public boolean hasPrivilege(PartialPath path, int privilegeId) {
     return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
   }
 
-  public void addPrivilege(String path, int privilegeId) {
+  public void addPrivilege(PartialPath path, int privilegeId) {
     AuthUtils.addPrivilege(path, privilegeId, privilegeList);
   }
 
-  public void removePrivilege(String path, int privilegeId) {
+  public void removePrivilege(PartialPath path, int privilegeId) {
     AuthUtils.removePrivilege(path, privilegeId, privilegeList);
   }
 
@@ -123,7 +124,7 @@ public class User {
    * @param path -path
    * @param privileges -set of integer to determine privilege
    */
-  public void setPrivileges(String path, Set<Integer> privileges) {
+  public void setPrivileges(PartialPath path, Set<Integer> privileges) {
     for (PathPrivilege pathPrivilege : privilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
         pathPrivilege.setPrivileges(privileges);
@@ -135,11 +136,11 @@ public class User {
     return roleList.contains(roleName);
   }
 
-  public Set<Integer> getPrivileges(String path) throws AuthException {
+  public Set<Integer> getPrivileges(PartialPath path) throws AuthException {
     return AuthUtils.getPrivileges(path, privilegeList);
   }
 
-  public boolean checkPrivilege(String path, int privilegeId) throws AuthException {
+  public boolean checkPrivilege(PartialPath path, int privilegeId) throws AuthException {
     return AuthUtils.checkPrivilege(path, privilegeId, privilegeList);
   }
 
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/BasicRoleManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/BasicRoleManager.java
index 7b0c6f9832..142532434b 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/BasicRoleManager.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/BasicRoleManager.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.commons.auth.role;
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.concurrent.HashLock;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 
@@ -106,7 +107,7 @@ public abstract class BasicRoleManager implements IRoleManager {
   }
 
   @Override
-  public boolean grantPrivilegeToRole(String rolename, String path, int privilegeId)
+  public boolean grantPrivilegeToRole(String rolename, PartialPath path, int privilegeId)
       throws AuthException {
     AuthUtils.validatePrivilegeOnPath(path, privilegeId);
     lock.writeLock(rolename);
@@ -134,7 +135,7 @@ public abstract class BasicRoleManager implements IRoleManager {
   }
 
   @Override
-  public boolean revokePrivilegeFromRole(String rolename, String path, int privilegeId)
+  public boolean revokePrivilegeFromRole(String rolename, PartialPath path, int privilegeId)
       throws AuthException {
     AuthUtils.validatePrivilegeOnPath(path, privilegeId);
     lock.writeLock(rolename);
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/IRoleManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/IRoleManager.java
index ccadbd5917..496c031b17 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/IRoleManager.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/role/IRoleManager.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.commons.auth.role;
 
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.Role;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
 
 import java.util.List;
@@ -65,7 +66,8 @@ public interface IRoleManager extends SnapshotProcessor {
    * @return True if the permission is successfully added, false if the permission already exists.
    * @throws AuthException If the role does not exist or the privilege or the seriesPath is illegal.
    */
-  boolean grantPrivilegeToRole(String rolename, String path, int privilegeId) throws AuthException;
+  boolean grantPrivilegeToRole(String rolename, PartialPath path, int privilegeId)
+      throws AuthException;
 
   /**
    * Revoke a privilege on seriesPath from a role.
@@ -78,7 +80,7 @@ public interface IRoleManager extends SnapshotProcessor {
    *     exists.
    * @throws AuthException If the role does not exist or the privilege or the seriesPath is illegal.
    */
-  boolean revokePrivilegeFromRole(String rolename, String path, int privilegeId)
+  boolean revokePrivilegeFromRole(String rolename, PartialPath path, int privilegeId)
       throws AuthException;
 
   /** Re-initialize this object. */
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/BasicUserManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/BasicUserManager.java
index 787eb560b2..4b872db74e 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/BasicUserManager.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/BasicUserManager.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.User;
 import org.apache.iotdb.commons.concurrent.HashLock;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 
@@ -153,7 +154,7 @@ public abstract class BasicUserManager implements IUserManager {
   }
 
   @Override
-  public boolean grantPrivilegeToUser(String username, String path, int privilegeId)
+  public boolean grantPrivilegeToUser(String username, PartialPath path, int privilegeId)
       throws AuthException {
     AuthUtils.validatePrivilegeOnPath(path, privilegeId);
     lock.writeLock(username);
@@ -181,7 +182,7 @@ public abstract class BasicUserManager implements IUserManager {
   }
 
   @Override
-  public boolean revokePrivilegeFromUser(String username, String path, int privilegeId)
+  public boolean revokePrivilegeFromUser(String username, PartialPath path, int privilegeId)
       throws AuthException {
     AuthUtils.validatePrivilegeOnPath(path, privilegeId);
     lock.writeLock(username);
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/IUserManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/IUserManager.java
index 6abc22e9d8..f403db6195 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/IUserManager.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/IUserManager.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.commons.auth.user;
 
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.User;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
 
 import java.util.List;
@@ -66,7 +67,8 @@ public interface IUserManager extends SnapshotProcessor {
    * @return True if the permission is successfully added, false if the permission already exists.
    * @throws AuthException If the user does not exist or the privilege or the seriesPath is illegal.
    */
-  boolean grantPrivilegeToUser(String username, String path, int privilegeId) throws AuthException;
+  boolean grantPrivilegeToUser(String username, PartialPath path, int privilegeId)
+      throws AuthException;
 
   /**
    * Revoke a privilege on seriesPath from a user.
@@ -79,7 +81,7 @@ public interface IUserManager extends SnapshotProcessor {
    *     exists.
    * @throws AuthException If the user does not exist or the privilege or the seriesPath is illegal.
    */
-  boolean revokePrivilegeFromUser(String username, String path, int privilegeId)
+  boolean revokePrivilegeFromUser(String username, PartialPath path, int privilegeId)
       throws AuthException;
 
   /**
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/AuthUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/AuthUtils.java
index b13be3217d..dd26ff8f4e 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/utils/AuthUtils.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/AuthUtils.java
@@ -23,14 +23,22 @@ import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.PrivilegeType;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathDeserializeUtil;
 import org.apache.iotdb.commons.security.encrypt.AsymmetricEncryptFactory;
 import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp;
 import org.apache.iotdb.confignode.rpc.thrift.TRoleResp;
 import org.apache.iotdb.confignode.rpc.thrift.TUserResp;
 import org.apache.iotdb.rpc.TSStatusCode;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -39,15 +47,25 @@ import java.util.Map;
 import java.util.Set;
 
 public class AuthUtils {
+  private static final Logger logger = LoggerFactory.getLogger(AuthUtils.class);
   private static final String ROOT_PREFIX = IoTDBConstant.PATH_ROOT;
-  public static final String ROOT_PATH_PRIVILEGE =
-      IoTDBConstant.PATH_ROOT
-          + IoTDBConstant.PATH_SEPARATOR
-          + IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+  public static PartialPath ROOT_PATH_PRIVILEGE_PATH;
   private static final int MIN_PASSWORD_LENGTH = 4;
   private static final int MIN_USERNAME_LENGTH = 4;
   private static final int MIN_ROLENAME_LENGTH = 4;
 
+  static {
+    try {
+      ROOT_PATH_PRIVILEGE_PATH =
+          new PartialPath(
+              IoTDBConstant.PATH_ROOT
+                  + IoTDBConstant.PATH_SEPARATOR
+                  + IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD);
+    } catch (MetadataException e) {
+      // do nothing
+    }
+  }
+
   private AuthUtils() {
     // Empty constructor
   }
@@ -135,8 +153,8 @@ public class AuthUtils {
    * @param path series path
    * @throws AuthException contains message why path is invalid
    */
-  public static void validatePath(String path) throws AuthException {
-    if (!path.startsWith(ROOT_PREFIX)) {
+  public static void validatePath(PartialPath path) throws AuthException {
+    if (!path.getFirstNode().equals(ROOT_PREFIX)) {
       throw new AuthException(
           TSStatusCode.ILLEGAL_PARAMETER,
           String.format(
@@ -151,10 +169,11 @@ public class AuthUtils {
    * @param privilegeId privilege Id
    * @throws AuthException contains message why path is invalid
    */
-  public static void validatePrivilegeOnPath(String path, int privilegeId) throws AuthException {
+  public static void validatePrivilegeOnPath(PartialPath path, int privilegeId)
+      throws AuthException {
     validatePrivilege(privilegeId);
     PrivilegeType type = PrivilegeType.values()[privilegeId];
-    if (!path.equals(ROOT_PATH_PRIVILEGE)) {
+    if (!path.equals(ROOT_PATH_PRIVILEGE_PATH)) {
       validatePath(path);
       switch (type) {
         case READ_TIMESERIES:
@@ -205,25 +224,6 @@ public class AuthUtils {
         .encrypt(password);
   }
 
-  /**
-   * Check if pathA belongs to pathB according to path pattern.
-   *
-   * @param pathA sub-path
-   * @param pathB path
-   * @exception AuthException throw if pathA or pathB is invalid
-   * @return True if pathA is a sub pattern of pathB, e.g. pathA = "root.a.b.c" and pathB =
-   *     "root.a.b.*", "root.a.**", "root.a.*.c", "root.**.c" or "root.*.b.**"
-   */
-  public static boolean pathBelongsTo(String pathA, String pathB) throws AuthException {
-    try {
-      PartialPath partialPathA = new PartialPath(pathA);
-      PartialPath partialPathB = new PartialPath(pathB);
-      return partialPathB.matchFullPath(partialPathA);
-    } catch (IllegalPathException e) {
-      throw new AuthException(TSStatusCode.ILLEGAL_PARAMETER, e);
-    }
-  }
-
   /**
    * Check privilege
    *
@@ -234,14 +234,14 @@ public class AuthUtils {
    * @return True if privilege-check passed
    */
   public static boolean checkPrivilege(
-      String path, int privilegeId, List<PathPrivilege> privilegeList) throws AuthException {
+      PartialPath path, int privilegeId, List<PathPrivilege> privilegeList) throws AuthException {
     if (privilegeList == null) {
       return false;
     }
     for (PathPrivilege pathPrivilege : privilegeList) {
       if (path != null) {
         if (pathPrivilege.getPath() != null
-            && AuthUtils.pathBelongsTo(path, pathPrivilege.getPath())
+            && pathPrivilege.getPath().matchFullPath(path)
             && pathPrivilege.getPrivileges().contains(privilegeId)) {
           return true;
         }
@@ -263,7 +263,7 @@ public class AuthUtils {
    * @exception AuthException throw if path is invalid or path in privilege is invalid
    * @return The privileges granted to the role
    */
-  public static Set<Integer> getPrivileges(String path, List<PathPrivilege> privilegeList)
+  public static Set<Integer> getPrivileges(PartialPath path, List<PathPrivilege> privilegeList)
       throws AuthException {
     if (privilegeList == null) {
       return new HashSet<>();
@@ -271,8 +271,7 @@ public class AuthUtils {
     Set<Integer> privileges = new HashSet<>();
     for (PathPrivilege pathPrivilege : privilegeList) {
       if (path != null) {
-        if (pathPrivilege.getPath() != null
-            && AuthUtils.pathBelongsTo(path, pathPrivilege.getPath())) {
+        if (pathPrivilege.getPath() != null && pathPrivilege.getPath().matchFullPath(path)) {
           privileges.addAll(pathPrivilege.getPrivileges());
         }
       } else {
@@ -293,7 +292,7 @@ public class AuthUtils {
    * @return True if series path has this privilege
    */
   public static boolean hasPrivilege(
-      String path, int privilegeId, List<PathPrivilege> privilegeList) {
+      PartialPath path, int privilegeId, List<PathPrivilege> privilegeList) {
     for (PathPrivilege pathPrivilege : privilegeList) {
       if (pathPrivilege.getPath().equals(path)
           && pathPrivilege.getPrivileges().contains(privilegeId)) {
@@ -311,7 +310,8 @@ public class AuthUtils {
    * @param privilegeId privilege Id
    * @param privilegeList privileges in List structure of user or role
    */
-  public static void addPrivilege(String path, int privilegeId, List<PathPrivilege> privilegeList) {
+  public static void addPrivilege(
+      PartialPath path, int privilegeId, List<PathPrivilege> privilegeList) {
     PathPrivilege targetPathPrivilege = null;
     // check PathPrivilege of target path is already existed
     for (PathPrivilege pathPrivilege : privilegeList) {
@@ -343,7 +343,7 @@ public class AuthUtils {
    * @param privilegeList privileges in List structure of user or role
    */
   public static void removePrivilege(
-      String path, int privilegeId, List<PathPrivilege> privilegeList) {
+      PartialPath path, int privilegeId, List<PathPrivilege> privilegeList) {
     PathPrivilege targetPathPrivilege = null;
     for (PathPrivilege pathPrivilege : privilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
@@ -411,4 +411,28 @@ public class AuthUtils {
     }
     return result;
   }
+
+  public static ByteBuffer serializePartialPathList(List<PartialPath> paths) {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+
+    try {
+      dataOutputStream.writeInt(paths.size());
+      for (PartialPath path : paths) {
+        path.serialize(dataOutputStream);
+      }
+    } catch (IOException e) {
+      logger.error("Failed to serialize PartialPath list", e);
+    }
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  public static List<PartialPath> deserializePartialPathList(ByteBuffer buffer) {
+    int size = buffer.getInt();
+    List<PartialPath> paths = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      paths.add((PartialPath) PathDeserializeUtil.deserialize(buffer));
+    }
+    return paths;
+  }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/IOUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/IOUtils.java
index 6af7e42737..7349941293 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/utils/IOUtils.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/IOUtils.java
@@ -19,6 +19,8 @@
 package org.apache.iotdb.commons.utils;
 
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 
 import java.io.DataInputStream;
 import java.io.File;
@@ -127,10 +129,10 @@ public class IOUtils {
    */
   public static PathPrivilege readPathPrivilege(
       DataInputStream inputStream, String encoding, ThreadLocal<byte[]> strBufferLocal)
-      throws IOException {
+      throws IOException, IllegalPathException {
     String path = IOUtils.readString(inputStream, encoding, strBufferLocal);
     int privilegeNum = inputStream.readInt();
-    PathPrivilege pathPrivilege = new PathPrivilege(path);
+    PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath(path));
     for (int i = 0; i < privilegeNum; i++) {
       pathPrivilege.getPrivileges().add(inputStream.readInt());
     }
@@ -153,7 +155,7 @@ public class IOUtils {
       String encoding,
       ThreadLocal<ByteBuffer> encodingBufferLocal)
       throws IOException {
-    writeString(outputStream, pathPrivilege.getPath(), encoding, encodingBufferLocal);
+    writeString(outputStream, pathPrivilege.getPath().getFullPath(), encoding, encodingBufferLocal);
     writeInt(outputStream, pathPrivilege.getPrivileges().size(), encodingBufferLocal);
     for (Integer i : pathPrivilege.getPrivileges()) {
       writeInt(outputStream, i, encodingBufferLocal);
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java b/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
index ad88a301df..18877189ae 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
@@ -80,13 +80,13 @@ public class AuthorityChecker {
       return true;
     }
 
-    List<String> allPath = new ArrayList<>();
+    List<PartialPath> allPath = new ArrayList<>();
     if (paths != null && !paths.isEmpty()) {
       for (PartialPath path : paths) {
-        allPath.add(path == null ? AuthUtils.ROOT_PATH_PRIVILEGE : path.getFullPath());
+        allPath.add(path == null ? AuthUtils.ROOT_PATH_PRIVILEGE_PATH : path);
       }
     } else {
-      allPath.add(AuthUtils.ROOT_PATH_PRIVILEGE);
+      allPath.add(AuthUtils.ROOT_PATH_PRIVILEGE_PATH);
     }
 
     TSStatus status = authorizerManager.checkPath(username, allPath, permission);
@@ -96,8 +96,8 @@ public class AuthorityChecker {
   private static boolean checkOnePath(String username, PartialPath path, int permission)
       throws AuthException {
     try {
-      String fullPath = path == null ? AuthUtils.ROOT_PATH_PRIVILEGE : path.getFullPath();
-      if (authorizerManager.checkUserPrivileges(username, fullPath, permission)) {
+      PartialPath newPath = path == null ? AuthUtils.ROOT_PATH_PRIVILEGE_PATH : path;
+      if (authorizerManager.checkUserPrivileges(username, newPath, permission)) {
         return true;
       }
     } catch (AuthException e) {
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java b/server/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java
index 56d83fe72c..2237d13188 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.commons.auth.authorizer.BasicAuthorizer;
 import org.apache.iotdb.commons.auth.authorizer.IAuthorizer;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.entity.User;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
 import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
 import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
@@ -108,7 +109,7 @@ public class AuthorizerManager implements IAuthorizer {
   }
 
   @Override
-  public void grantPrivilegeToUser(String username, String path, int privilegeId)
+  public void grantPrivilegeToUser(String username, PartialPath path, int privilegeId)
       throws AuthException {
     authReadWriteLock.writeLock().lock();
     try {
@@ -119,7 +120,7 @@ public class AuthorizerManager implements IAuthorizer {
   }
 
   @Override
-  public void revokePrivilegeFromUser(String username, String path, int privilegeId)
+  public void revokePrivilegeFromUser(String username, PartialPath path, int privilegeId)
       throws AuthException {
     authReadWriteLock.writeLock().lock();
     try {
@@ -150,7 +151,7 @@ public class AuthorizerManager implements IAuthorizer {
   }
 
   @Override
-  public void grantPrivilegeToRole(String roleName, String path, int privilegeId)
+  public void grantPrivilegeToRole(String roleName, PartialPath path, int privilegeId)
       throws AuthException {
     authReadWriteLock.writeLock().lock();
     try {
@@ -161,7 +162,7 @@ public class AuthorizerManager implements IAuthorizer {
   }
 
   @Override
-  public void revokePrivilegeFromRole(String roleName, String path, int privilegeId)
+  public void revokePrivilegeFromRole(String roleName, PartialPath path, int privilegeId)
       throws AuthException {
     authReadWriteLock.writeLock().lock();
     try {
@@ -192,7 +193,7 @@ public class AuthorizerManager implements IAuthorizer {
   }
 
   @Override
-  public Set<Integer> getPrivileges(String username, String path) throws AuthException {
+  public Set<Integer> getPrivileges(String username, PartialPath path) throws AuthException {
     authReadWriteLock.readLock().lock();
     try {
       return authorizer.getPrivileges(username, path);
@@ -212,7 +213,7 @@ public class AuthorizerManager implements IAuthorizer {
   }
 
   @Override
-  public boolean checkUserPrivileges(String username, String path, int privilegeId)
+  public boolean checkUserPrivileges(String username, PartialPath path, int privilegeId)
       throws AuthException {
     authReadWriteLock.readLock().lock();
     try {
@@ -358,7 +359,7 @@ public class AuthorizerManager implements IAuthorizer {
   }
 
   /** Check the path */
-  public TSStatus checkPath(String username, List<String> allPath, int permission) {
+  public TSStatus checkPath(String username, List<PartialPath> allPath, int permission) {
     authReadWriteLock.readLock().lock();
     try {
       return authorityFetcher.checkUserPrivileges(username, allPath, permission);
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java b/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java
index 6c48a86e96..99343df434 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
 import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
@@ -52,17 +53,15 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 public class ClusterAuthorityFetcher implements IAuthorityFetcher {
   private static final Logger logger = LoggerFactory.getLogger(ClusterAuthorityFetcher.class);
 
-  private IAuthorCache iAuthorCache;
+  private final IAuthorCache iAuthorCache;
   private IAuthorizer authorizer;
 
   private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER =
@@ -78,10 +77,10 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
   }
 
   @Override
-  public TSStatus checkUserPrivileges(String username, List<String> allPath, int permission) {
+  public TSStatus checkUserPrivileges(String username, List<PartialPath> allPath, int permission) {
     User user = iAuthorCache.getUserCache(username);
     if (user != null) {
-      for (String path : allPath) {
+      for (PartialPath path : allPath) {
         try {
           if (!user.isOpenIdUser() || !authorizer.checkUserPrivileges(username, path, permission)) {
             if (!user.checkPrivilege(path, permission)) {
@@ -227,8 +226,10 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
     }
   }
 
-  public TSStatus checkPath(String username, List<String> allPath, int permission) {
-    TCheckUserPrivilegesReq req = new TCheckUserPrivilegesReq(username, allPath, permission);
+  public TSStatus checkPath(String username, List<PartialPath> allPath, int permission) {
+    TCheckUserPrivilegesReq req =
+        new TCheckUserPrivilegesReq(
+            username, AuthUtils.serializePartialPathList(allPath), permission);
     TPermissionInfoResp permissionInfoResp;
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
@@ -259,7 +260,11 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
     for (int i = 0; i < privilegeList.size(); i += 2) {
       String path = privilegeList.get(i);
       String privilege = privilegeList.get(i + 1);
-      pathPrivilegeList.add(toPathPrivilege(path, privilege));
+      try {
+        pathPrivilegeList.add(toPathPrivilege(new PartialPath(path), privilege));
+      } catch (MetadataException e) {
+        logger.error("Failed to parse path {}.", path, e);
+      }
     }
     user.setOpenIdUser(tPermissionInfoResp.getUserInfo().isIsOpenIdUser());
     user.setPrivilegeList(pathPrivilegeList);
@@ -279,7 +284,11 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
     for (int i = 0; i < privilegeList.size(); i += 2) {
       String path = privilegeList.get(i);
       String privilege = privilegeList.get(i + 1);
-      pathPrivilegeList.add(toPathPrivilege(path, privilege));
+      try {
+        pathPrivilegeList.add(toPathPrivilege(new PartialPath(path), privilege));
+      } catch (MetadataException e) {
+        logger.error("Failed to parse path {}.", path, e);
+      }
     }
     role.setPrivilegeList(pathPrivilegeList);
     return role;
@@ -292,7 +301,7 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
    * @param privilege privilegeIds
    * @return
    */
-  private PathPrivilege toPathPrivilege(String path, String privilege) {
+  private PathPrivilege toPathPrivilege(PartialPath path, String privilege) {
     PathPrivilege pathPrivilege = new PathPrivilege();
     String[] privileges = privilege.replace(" ", "").split(",");
     Set<Integer> privilegeIds = new HashSet<>();
@@ -306,6 +315,9 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
 
   private TAuthorizerReq statementToAuthorizerReq(AuthorStatement authorStatement)
       throws AuthException {
+    if (authorStatement.getAuthorType() == null) {
+      authorStatement.setNodeNameList(new ArrayList<>());
+    }
     return new TAuthorizerReq(
         authorStatement.getAuthorType().ordinal(),
         authorStatement.getUserName() == null ? "" : authorStatement.getUserName(),
@@ -313,10 +325,6 @@ public class ClusterAuthorityFetcher implements IAuthorityFetcher {
         authorStatement.getPassWord() == null ? "" : authorStatement.getPassWord(),
         authorStatement.getNewPassword() == null ? "" : authorStatement.getNewPassword(),
         AuthUtils.strToPermissions(authorStatement.getPrivilegeList()),
-        authorStatement.getNodeNameList() == null
-            ? Collections.emptyList()
-            : authorStatement.getNodeNameList().stream()
-                .map(PartialPath::getFullPath)
-                .collect(Collectors.toList()));
+        AuthUtils.serializePartialPathList(authorStatement.getNodeNameList()));
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/IAuthorityFetcher.java b/server/src/main/java/org/apache/iotdb/db/auth/IAuthorityFetcher.java
index a8d58c8877..f31419463d 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/IAuthorityFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/IAuthorityFetcher.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.db.auth;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
 
@@ -31,7 +32,7 @@ public interface IAuthorityFetcher {
 
   TSStatus checkUser(String username, String password);
 
-  TSStatus checkUserPrivileges(String username, List<String> allPath, int permission);
+  TSStatus checkUserPrivileges(String username, List<PartialPath> allPath, int permission);
 
   SettableFuture<ConfigTaskResult> operatePermission(AuthorStatement authorStatement);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/AuthorizerManagerTest.java b/server/src/test/java/org/apache/iotdb/db/auth/AuthorizerManagerTest.java
index 6d22218d66..2fac0adae4 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/AuthorizerManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/AuthorizerManagerTest.java
@@ -23,6 +23,8 @@ import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.PrivilegeType;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.entity.User;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp;
 import org.apache.iotdb.confignode.rpc.thrift.TRoleResp;
 import org.apache.iotdb.confignode.rpc.thrift.TUserResp;
@@ -44,7 +46,7 @@ public class AuthorizerManagerTest {
   ClusterAuthorityFetcher authorityFetcher = new ClusterAuthorityFetcher(new BasicAuthorityCache());
 
   @Test
-  public void permissionCacheTest() {
+  public void permissionCacheTest() throws IllegalPathException {
     User user = new User();
     Role role1 = new Role();
     Role role2 = new Role();
@@ -54,7 +56,7 @@ public class AuthorizerManagerTest {
     List<PathPrivilege> privilegeList = new ArrayList<>();
     privilegesIds.add(PrivilegeType.CREATE_ROLE.ordinal());
     privilegesIds.add(PrivilegeType.REVOKE_USER_ROLE.ordinal());
-    privilege.setPath("root.ln");
+    privilege.setPath(new PartialPath("root.ln"));
     privilege.setPrivileges(privilegesIds);
     privilegeList.add(privilege);
     role1.setName("role1");
@@ -78,7 +80,7 @@ public class AuthorizerManagerTest {
 
     // User permission information
     for (PathPrivilege pathPrivilege : user.getPrivilegeList()) {
-      userPrivilegeList.add(pathPrivilege.getPath());
+      userPrivilegeList.add(pathPrivilege.getPath().getFullPath());
       String privilegeIdList = pathPrivilege.getPrivileges().toString();
       userPrivilegeList.add(privilegeIdList.substring(1, privilegeIdList.length() - 1));
     }
@@ -104,14 +106,18 @@ public class AuthorizerManagerTest {
         TSStatusCode.SUCCESS_STATUS.getStatusCode(),
         authorityFetcher
             .checkUserPrivileges(
-                "user", Collections.singletonList("root.ln"), PrivilegeType.CREATE_ROLE.ordinal())
+                "user",
+                Collections.singletonList(new PartialPath("root.ln")),
+                PrivilegeType.CREATE_ROLE.ordinal())
             .getCode());
     // User does not have permission
     Assert.assertEquals(
         TSStatusCode.NO_PERMISSION.getStatusCode(),
         authorityFetcher
             .checkUserPrivileges(
-                "user", Collections.singletonList("root.ln"), PrivilegeType.CREATE_USER.ordinal())
+                "user",
+                Collections.singletonList(new PartialPath("root.ln")),
+                PrivilegeType.CREATE_USER.ordinal())
             .getCode());
 
     // Authenticate users with roles
@@ -125,7 +131,7 @@ public class AuthorizerManagerTest {
       rolePrivilegeList = new ArrayList<>();
       tRoleResp.setRoleName(role.getName());
       for (PathPrivilege pathPrivilege : role.getPrivilegeList()) {
-        rolePrivilegeList.add(pathPrivilege.getPath());
+        rolePrivilegeList.add(pathPrivilege.getPath().getFullPath());
         String privilegeIdList = pathPrivilege.getPrivileges().toString();
         rolePrivilegeList.add(privilegeIdList.substring(1, privilegeIdList.length() - 1));
       }
@@ -145,14 +151,18 @@ public class AuthorizerManagerTest {
         TSStatusCode.SUCCESS_STATUS.getStatusCode(),
         authorityFetcher
             .checkUserPrivileges(
-                "user", Collections.singletonList("root.ln"), PrivilegeType.CREATE_ROLE.ordinal())
+                "user",
+                Collections.singletonList(new PartialPath("root.ln")),
+                PrivilegeType.CREATE_ROLE.ordinal())
             .getCode());
     // role does not have permission
     Assert.assertEquals(
         TSStatusCode.NO_PERMISSION.getStatusCode(),
         authorityFetcher
             .checkUserPrivileges(
-                "user", Collections.singletonList("root.ln"), PrivilegeType.CREATE_USER.ordinal())
+                "user",
+                Collections.singletonList(new PartialPath("root.ln")),
+                PrivilegeType.CREATE_USER.ordinal())
             .getCode());
 
     authorityFetcher.getAuthorCache().invalidateCache(user.getName(), "");
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizerTest.java b/server/src/test/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizerTest.java
index cd951ccdc7..fc659a02db 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/authorizer/LocalFileAuthorizerTest.java
@@ -23,6 +23,8 @@ import org.apache.iotdb.commons.auth.authorizer.IAuthorizer;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.entity.User;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.auth.AuthorizerManager;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 
@@ -43,7 +45,7 @@ public class LocalFileAuthorizerTest {
 
   IAuthorizer authorizer;
   User user;
-  String nodeName = "root.laptop.d1";
+  PartialPath nodeName;
   String roleName = "role";
 
   @Before
@@ -51,6 +53,7 @@ public class LocalFileAuthorizerTest {
     EnvironmentUtils.envSetUp();
     authorizer = AuthorizerManager.getInstance();
     user = new User("user", "password");
+    nodeName = new PartialPath("root.laptop.d1");
   }
 
   @After
@@ -136,8 +139,8 @@ public class LocalFileAuthorizerTest {
     }
 
     try {
-      authorizer.revokePrivilegeFromUser("root", "root", 1);
-    } catch (AuthException e) {
+      authorizer.revokePrivilegeFromUser("root", new PartialPath("root"), 1);
+    } catch (AuthException | MetadataException e) {
       Assert.assertEquals(
           "Invalid operation, administrator must have all privileges", e.getMessage());
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/entity/PathPrivilegeTest.java b/server/src/test/java/org/apache/iotdb/db/auth/entity/PathPrivilegeTest.java
index 0a88856ab1..757f88da18 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/entity/PathPrivilegeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/entity/PathPrivilegeTest.java
@@ -19,6 +19,8 @@
 package org.apache.iotdb.db.auth.entity;
 
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -28,13 +30,13 @@ import java.util.Collections;
 public class PathPrivilegeTest {
 
   @Test
-  public void testPathPrivilege() {
+  public void testPathPrivilege() throws IllegalPathException {
     PathPrivilege pathPrivilege = new PathPrivilege();
-    pathPrivilege.setPath("root.ln");
+    pathPrivilege.setPath(new PartialPath("root.ln"));
     pathPrivilege.setPrivileges(Collections.singleton(1));
     Assert.assertEquals("root.ln : INSERT_TIMESERIES", pathPrivilege.toString());
     PathPrivilege pathPrivilege1 = new PathPrivilege();
-    pathPrivilege1.setPath("root.sg");
+    pathPrivilege1.setPath(new PartialPath("root.sg"));
     pathPrivilege1.setPrivileges(Collections.singleton(1));
     Assert.assertNotEquals(pathPrivilege, pathPrivilege1);
     pathPrivilege.deserialize(pathPrivilege1.serialize());
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/entity/RoleTest.java b/server/src/test/java/org/apache/iotdb/db/auth/entity/RoleTest.java
index a529e646a9..724b6097de 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/entity/RoleTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/entity/RoleTest.java
@@ -20,6 +20,8 @@ package org.apache.iotdb.db.auth.entity;
 
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.Role;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -29,11 +31,11 @@ import java.util.Collections;
 public class RoleTest {
 
   @Test
-  public void testRole() {
+  public void testRole() throws IllegalPathException {
     Role role = new Role("role");
-    PathPrivilege pathPrivilege = new PathPrivilege("root.ln");
+    PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath("root.ln"));
     role.setPrivilegeList(Collections.singletonList(pathPrivilege));
-    role.setPrivileges("root.ln", Collections.singleton(1));
+    role.setPrivileges(new PartialPath("root.ln"), Collections.singleton(1));
     Assert.assertEquals(
         "Role{name='role', privilegeList=[root.ln : INSERT_TIMESERIES]}", role.toString());
     Role role1 = new Role("role1");
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/entity/UserTest.java b/server/src/test/java/org/apache/iotdb/db/auth/entity/UserTest.java
index 23455d39af..467e1777e1 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/entity/UserTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/entity/UserTest.java
@@ -20,6 +20,8 @@ package org.apache.iotdb.db.auth.entity;
 
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.User;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -29,11 +31,11 @@ import java.util.Collections;
 public class UserTest {
 
   @Test
-  public void testUser() {
+  public void testUser() throws IllegalPathException {
     User user = new User("user", "password");
-    PathPrivilege pathPrivilege = new PathPrivilege("root.ln");
+    PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath("root.ln"));
     user.setPrivilegeList(Collections.singletonList(pathPrivilege));
-    user.setPrivileges("root.ln", Collections.singleton(1));
+    user.setPrivileges(new PartialPath("root.ln"), Collections.singleton(1));
     Assert.assertEquals(
         "User{name='user', password='password', privilegeList=[root.ln : INSERT_TIMESERIES], roleList=[], isOpenIdUser=false, useWaterMark=false, lastActiveTime=0}",
         user.toString());
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessorTest.java b/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessorTest.java
index 142eeff92e..a03f1af54e 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleAccessorTest.java
@@ -21,6 +21,8 @@ package org.apache.iotdb.db.auth.role;
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.role.LocalFileRoleAccessor;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 
@@ -58,12 +60,12 @@ public class LocalFileRoleAccessorTest {
   }
 
   @Test
-  public void test() throws IOException {
+  public void test() throws IOException, IllegalPathException {
     Role[] roles = new Role[5];
     for (int i = 0; i < roles.length; i++) {
       roles[i] = new Role("role" + i);
       for (int j = 0; j <= i; j++) {
-        PathPrivilege pathPrivilege = new PathPrivilege("root.a.b.c" + j);
+        PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath("root.a.b.c" + j));
         pathPrivilege.getPrivileges().add(j);
         roles[i].getPrivilegeList().add(pathPrivilege);
       }
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleManagerTest.java b/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleManagerTest.java
index fb0c29d435..6660fb2713 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/role/LocalFileRoleManagerTest.java
@@ -22,6 +22,8 @@ import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.role.LocalFileRoleManager;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 
@@ -58,12 +60,12 @@ public class LocalFileRoleManagerTest {
   }
 
   @Test
-  public void test() throws AuthException {
+  public void test() throws AuthException, IllegalPathException {
     Role[] roles = new Role[5];
     for (int i = 0; i < roles.length; i++) {
       roles[i] = new Role("role" + i);
       for (int j = 0; j <= i; j++) {
-        PathPrivilege pathPrivilege = new PathPrivilege("root.a.b.c" + j);
+        PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath("root.a.b.c" + j));
         pathPrivilege.getPrivileges().add(j);
         roles[i].getPrivilegeList().add(pathPrivilege);
       }
@@ -103,7 +105,7 @@ public class LocalFileRoleManagerTest {
 
     // grant privilege
     role = manager.getRole(roles[0].getName());
-    String path = "root.a.b.c";
+    PartialPath path = new PartialPath("root.a.b.c");
     int privilegeId = 0;
     assertFalse(role.hasPrivilege(path, privilegeId));
     assertTrue(manager.grantPrivilegeToRole(role.getName(), path, privilegeId));
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessorTest.java b/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessorTest.java
index b45d56618d..a84244474d 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserAccessorTest.java
@@ -21,6 +21,8 @@ package org.apache.iotdb.db.auth.user;
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.User;
 import org.apache.iotdb.commons.auth.user.LocalFileUserAccessor;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 
@@ -59,12 +61,12 @@ public class LocalFileUserAccessorTest {
   }
 
   @Test
-  public void test() throws IOException {
+  public void test() throws IOException, IllegalPathException {
     User[] users = new User[5];
     for (int i = 0; i < users.length; i++) {
       users[i] = new User("user" + i, "password" + i);
       for (int j = 0; j <= i; j++) {
-        PathPrivilege pathPrivilege = new PathPrivilege("root.a.b.c" + j);
+        PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath("root.a.b.c" + j));
         pathPrivilege.getPrivileges().add(j);
         users[i].getPrivilegeList().add(pathPrivilege);
         users[i].getRoleList().add("role" + j);
diff --git a/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserManagerTest.java b/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserManagerTest.java
index 87cf266a4b..6b6a3b8487 100644
--- a/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/auth/user/LocalFileUserManagerTest.java
@@ -23,6 +23,8 @@ import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.User;
 import org.apache.iotdb.commons.auth.user.LocalFileUserManager;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -76,12 +78,12 @@ public class LocalFileUserManagerTest {
   }
 
   @Test
-  public void test() throws AuthException {
+  public void test() throws AuthException, IllegalPathException {
     User[] users = new User[5];
     for (int i = 0; i < users.length; i++) {
       users[i] = new User("user" + i, "password" + i);
       for (int j = 0; j <= i; j++) {
-        PathPrivilege pathPrivilege = new PathPrivilege("root.a.b.c" + j);
+        PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath("root.a.b.c" + j));
         pathPrivilege.getPrivileges().add(j);
         users[i].getPrivilegeList().add(pathPrivilege);
         users[i].getRoleList().add("role" + j);
@@ -124,7 +126,7 @@ public class LocalFileUserManagerTest {
 
     // grant privilege
     user = manager.getUser(users[0].getName());
-    String path = "root.a.b.c";
+    PartialPath path = new PartialPath("root.a.b.c");
     int privilegeId = 0;
     assertFalse(user.hasPrivilege(path, privilegeId));
     assertTrue(manager.grantPrivilegeToUser(user.getName(), path, privilegeId));
diff --git a/server/src/test/java/org/apache/iotdb/db/security/encrypt/MessageDigestEncryptTest.java b/server/src/test/java/org/apache/iotdb/db/security/encrypt/MessageDigestEncryptTest.java
index eb3ee49d91..8e140145d6 100644
--- a/server/src/test/java/org/apache/iotdb/db/security/encrypt/MessageDigestEncryptTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/security/encrypt/MessageDigestEncryptTest.java
@@ -23,6 +23,8 @@ import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.User;
 import org.apache.iotdb.commons.auth.user.LocalFileUserManager;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.security.encrypt.MessageDigestEncrypt;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -62,12 +64,12 @@ public class MessageDigestEncryptTest {
   }
 
   @Test
-  public void testMessageDigestEncrypt() throws AuthException {
+  public void testMessageDigestEncrypt() throws AuthException, IllegalPathException {
     User[] users = new User[5];
     for (int i = 0; i < users.length; i++) {
       users[i] = new User("user" + i, "password" + i);
       for (int j = 0; j <= i; j++) {
-        PathPrivilege pathPrivilege = new PathPrivilege("root.a.b.c" + j);
+        PathPrivilege pathPrivilege = new PathPrivilege(new PartialPath("root.a.b.c" + j));
         pathPrivilege.getPrivileges().add(j);
         users[i].getPrivilegeList().add(pathPrivilege);
         users[i].getRoleList().add("role" + j);
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index 38afe6ef41..8ae61f5353 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -293,7 +293,7 @@ struct TAuthorizerReq {
   4: required string password
   5: required string newPassword
   6: required set<i32> permissions
-  7: required list<string> nodeNameList
+  7: required binary nodeNameList
 }
 
 struct TAuthorizerResp {
@@ -327,7 +327,7 @@ struct TLoginReq {
 
 struct TCheckUserPrivilegesReq {
   1: required string username
-  2: required list<string> paths
+  2: required binary paths
   3: required i32 permission
 }