You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by "JackieTien97 (via GitHub)" <gi...@apache.org> on 2023/08/28 13:33:36 UTC

[GitHub] [iotdb] JackieTien97 commented on a diff in pull request #10939: IOTDB. auth refine.

JackieTien97 commented on code in PR #10939:
URL: https://github.com/apache/iotdb/pull/10939#discussion_r1307087534


##########
mvn:
##########


Review Comment:
   delete this empty file



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java:
##########
@@ -48,6 +48,7 @@ public class AuthorPlan extends ConfigPhysicalPlan {
   private Set<Integer> permissions;
   private List<PartialPath> nodeNameList;
   private String userName;
+  private String currentUser;

Review Comment:
   It seems that we don't need this field, we already do the authority check before, and only when the authority check passed, we will seed AuthorReq to CN.



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java:
##########
@@ -987,6 +989,25 @@ public TPermissionInfoResp checkUserPrivileges(
     }
   }
 
+  public TAuthizedPatternTreeResp fetchAuthizedPatternTree(String username, int permission) {
+    TSStatus status = confirmLeader();
+    if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      try {
+        return permissionManager.fetchAuthizedPTree(username, permission);
+      } catch (AuthException e) {
+        TAuthizedPatternTreeResp resp = AuthUtils.generateEmptyAuthizedPTree(username, permission);
+        status
+            .setCode(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode())
+            .setMessage(e.getMessage());
+        return resp;
+      }
+    } else {
+      TAuthizedPatternTreeResp resp = AuthUtils.generateEmptyAuthizedPTree(username, permission);
+      resp.setStatus(status);
+      return resp;

Review Comment:
   if it's not leader, we just need to directly return the status(it's a redirection  status_code).



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -178,29 +188,45 @@ public TSStatus authorNonQuery(AuthorPlan authorPlan) {
         case GrantRole:
           for (int i : permissions) {
             for (PartialPath path : nodeNameList) {
-              authorizer.grantPrivilegeToRole(roleName, path, i);
+              // LSL  need to check if its grant opt.
+              authorizer.grantPrivilegeToRole(
+                  authorPlan.getCurrentUser(), roleName, path, i, false);
             }
           }
           break;
         case GrantUser:
           for (int i : permissions) {
+            if (nodeNameList == null) {
+              authorizer.grantPrivilegeToUser(null, userName, null, i, false);
+              continue;
+            }
             for (PartialPath path : nodeNameList) {
-              authorizer.grantPrivilegeToUser(userName, path, i);
+              // need to check if its grant opt.
+              authorizer.grantPrivilegeToUser(null, userName, path, i, false);
             }
           }
           break;
         case GrantRoleToUser:
+          // LSL need to check if current user has this role;

Review Comment:
   Do we have like `GRANT ROLE <ROLENAME> TO <USERNAME> WITH GRANT OPTION`?



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -178,29 +188,45 @@ public TSStatus authorNonQuery(AuthorPlan authorPlan) {
         case GrantRole:
           for (int i : permissions) {
             for (PartialPath path : nodeNameList) {
-              authorizer.grantPrivilegeToRole(roleName, path, i);
+              // LSL  need to check if its grant opt.
+              authorizer.grantPrivilegeToRole(
+                  authorPlan.getCurrentUser(), roleName, path, i, false);
             }
           }
           break;
         case GrantUser:
           for (int i : permissions) {
+            if (nodeNameList == null) {
+              authorizer.grantPrivilegeToUser(null, userName, null, i, false);

Review Comment:
   grant opt should be passed from `AuthorPlan`



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -113,6 +119,10 @@ public TPermissionInfoResp checkUserPrivileges(
     boolean status = true;
     TPermissionInfoResp result = new TPermissionInfoResp();
     try {
+      if (paths.isEmpty()) {
+        if (authorizer.checkUserPrivileges(username, null, permission)) ;

Review Comment:
   ```suggestion
           authorizer.checkUserPrivileges(username, null, permission);
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -178,29 +188,45 @@ public TSStatus authorNonQuery(AuthorPlan authorPlan) {
         case GrantRole:
           for (int i : permissions) {
             for (PartialPath path : nodeNameList) {
-              authorizer.grantPrivilegeToRole(roleName, path, i);
+              // LSL  need to check if its grant opt.
+              authorizer.grantPrivilegeToRole(
+                  authorPlan.getCurrentUser(), roleName, path, i, false);
             }
           }
           break;
         case GrantUser:
           for (int i : permissions) {
+            if (nodeNameList == null) {
+              authorizer.grantPrivilegeToUser(null, userName, null, i, false);
+              continue;
+            }
             for (PartialPath path : nodeNameList) {
-              authorizer.grantPrivilegeToUser(userName, path, i);
+              // need to check if its grant opt.
+              authorizer.grantPrivilegeToUser(null, userName, path, i, false);

Review Comment:
   grant opt should be passed from `AuthorPlan`



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/InvalidAuthCacheProcedure.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.confignode.procedure.impl.sync;
+
+import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.confignode.client.DataNodeRequestType;
+import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
+import org.apache.iotdb.confignode.manager.PermissionManager;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.InvalidAuthCacheState;
+import org.apache.iotdb.confignode.procedure.store.ProcedureType;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.Set;
+import java.util.Iterator;
+import java.util.Objects;
+
+public class InvalidAuthCacheProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, InvalidAuthCacheState> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(InvalidAuthCacheProcedure.class);
+
+  private PermissionManager permissionManager;
+
+  private String user;
+  private String role;
+
+  private int timeoutMS;
+
+  private static final int RETRY_THRESHOLD = 1;
+  private static final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig();
+
+  private List<Pair<TDataNodeConfiguration, Long>> dataNodesToInvalid;
+
+  private Set<TDataNodeConfiguration> invalidedDNs;
+
+  public InvalidAuthCacheProcedure() {
+    super();
+  }
+
+  public InvalidAuthCacheProcedure(String user, String role, List<TDataNodeConfiguration> alldns) {
+    super();
+    this.user = user;
+    this.role = role;
+    this.dataNodesToInvalid = new ArrayList<>();
+    for (TDataNodeConfiguration item : alldns) {
+      this.dataNodesToInvalid.add(
+          new Pair<TDataNodeConfiguration, Long>(item, System.currentTimeMillis()));
+    }
+    invalidedDNs = new HashSet<>();
+    this.timeoutMS = commonConfig.getDatanodeTokenTimeoutMS();
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, InvalidAuthCacheState state) {
+    if (dataNodesToInvalid.isEmpty()) {
+      return Flow.NO_MORE_STATE;
+    }
+    try {
+      switch (state) {
+        case INIT:
+          LOGGER.info("Start to invalid auth cache for " + "user: %s, role %s", user, role);

Review Comment:
   ```suggestion
             LOGGER.info("Start to invalid auth cache for user: {}, role: {}", user, role);
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/InvalidAuthCacheProcedure.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.confignode.procedure.impl.sync;
+
+import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.confignode.client.DataNodeRequestType;
+import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
+import org.apache.iotdb.confignode.manager.PermissionManager;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.InvalidAuthCacheState;
+import org.apache.iotdb.confignode.procedure.store.ProcedureType;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.Set;
+import java.util.Iterator;
+import java.util.Objects;
+
+public class InvalidAuthCacheProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, InvalidAuthCacheState> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(InvalidAuthCacheProcedure.class);
+
+  private PermissionManager permissionManager;
+
+  private String user;
+  private String role;
+
+  private int timeoutMS;
+
+  private static final int RETRY_THRESHOLD = 1;
+  private static final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig();
+
+  private List<Pair<TDataNodeConfiguration, Long>> dataNodesToInvalid;
+
+  private Set<TDataNodeConfiguration> invalidedDNs;
+
+  public InvalidAuthCacheProcedure() {
+    super();
+  }
+
+  public InvalidAuthCacheProcedure(String user, String role, List<TDataNodeConfiguration> alldns) {
+    super();
+    this.user = user;
+    this.role = role;
+    this.dataNodesToInvalid = new ArrayList<>();
+    for (TDataNodeConfiguration item : alldns) {
+      this.dataNodesToInvalid.add(
+          new Pair<TDataNodeConfiguration, Long>(item, System.currentTimeMillis()));
+    }
+    invalidedDNs = new HashSet<>();
+    this.timeoutMS = commonConfig.getDatanodeTokenTimeoutMS();
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, InvalidAuthCacheState state) {
+    if (dataNodesToInvalid.isEmpty()) {
+      return Flow.NO_MORE_STATE;
+    }
+    try {
+      switch (state) {
+        case INIT:
+          LOGGER.info("Start to invalid auth cache for " + "user: %s, role %s", user, role);
+          // shall we need to check if the user/role has been deleted?
+          if (dataNodesToInvalid.isEmpty()) {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALID_DONE);
+          }
+          setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALIDING);
+          break;
+        case DATANODE_AUTHCACHE_INVALIDING:
+          if (dataNodesToInvalid.isEmpty()) {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALID_DONE);
+          }
+          TInvalidatePermissionCacheReq req = new TInvalidatePermissionCacheReq();
+          TSStatus status;
+          req.setUsername(user);
+          req.setRoleName(role);
+          Iterator<Pair<TDataNodeConfiguration, Long>> it = dataNodesToInvalid.iterator();
+          while (it.hasNext()) {
+            if (it.next().getRight() + this.timeoutMS < System.currentTimeMillis()) {
+              invalidedDNs.add(it.next().getLeft());
+              it.remove();
+              continue;
+            }
+            status =
+                SyncDataNodeClientPool.getInstance()
+                    .sendSyncRequestToDataNodeWithRetry(
+                        it.next().getLeft().getLocation().getInternalEndPoint(),
+                        req,
+                        DataNodeRequestType.INVALIDATE_PERMISSION_CACHE);
+            if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+              invalidedDNs.add(it.next().getLeft());
+              it.remove();
+            }
+          }
+          if (dataNodesToInvalid.isEmpty()) {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALID_DONE);
+          } else {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALIDING);
+          }
+          break;
+        case DATANODE_AUTHCACHE_INVALID_DONE:
+          LOGGER.info("finish invalid auth cache for user:%s, role %s", user, role);
+          break;
+      }
+    } catch (Exception e) {
+      if (isRollbackSupported(state)) {
+        LOGGER.error("Fail in invalid auth cache", e);
+        setFailure(new ProcedureException(e.getMessage()));
+      } else {
+        LOGGER.error(
+            "Retrievable error trying to invalid auth cache :[user : %s, role : %s] in datanode",

Review Comment:
   ```suggestion
           LOGGER.error(
               "Retrievable error trying to invalid auth cache :[user : {} role : {}] in datanode",
   ```



##########
iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/AuthorInfoTest.java:
##########
@@ -89,10 +89,13 @@ public void permissionTest() throws TException, AuthException, IllegalPathExcept
     TCheckUserPrivilegesReq checkUserPrivilegesReq;
 
     Set<Integer> privilegeList = new HashSet<>();
-    privilegeList.add(PrivilegeType.MANAGE_USER.ordinal());
+    privilegeList.add(PrivilegeType.READ_DATA.ordinal());
+
+    Set<Integer> sysPriList = new HashSet<>();
+    sysPriList.add(PrivilegeType.MANAGE_ROLE.ordinal());
 
     Set<Integer> revokePrivilege = new HashSet<>();
-    revokePrivilege.add(PrivilegeType.MANAGE_USER.ordinal());
+    revokePrivilege.add(PrivilegeType.READ_DATA.ordinal());
 
     List<String> privilege = new ArrayList<>();
     privilege.add("root.** : MANAGE_USER");

Review Comment:
   `MANAGE_USER` is a sys privilege, why it has `root.**`?



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java:
##########
@@ -62,52 +62,65 @@ private AuthorityChecker() {
    *
    * @param username username
    * @param paths paths in List structure
-   * @param type Statement Type
-   * @param targetUser target user
    * @return if permission-check is passed
    */
-  public static boolean checkPermission(
-      String username, List<? extends PartialPath> paths, StatementType type, String targetUser) {
+  public static boolean checkFullPathListPermission(
+      String username, List<PartialPath> paths, int permission) throws AuthException {
+    // this function will throw auth_exception for illegal permission;
+    AuthUtils.validatePrivilege(paths.get(0), permission);
+
     if (SUPER_USER.equals(username)) {
       return true;
     }
 
-    int[] permissions = translateToPermissionId(type);
-    for (int permission : permissions) {
-      if (permission == -1) {
-        continue;
-      } else if (permission == PrivilegeType.ALTER_PASSWORD.ordinal()
-          && username.equals(targetUser)) {
-        // A user can modify his own password
-        return true;
-      }
+    TSStatus status = authorizerManager.checkFullPathPrivilegeList(username, paths, permission);
+    if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      return true;
+    }
 
-      List<PartialPath> allPath = new ArrayList<>();
-      if (paths != null && !paths.isEmpty()) {
-        for (PartialPath path : paths) {
-          allPath.add(path == null ? AuthUtils.ROOT_PATH_PRIVILEGE_PATH : path);
-        }
-      } else {
-        allPath.add(AuthUtils.ROOT_PATH_PRIVILEGE_PATH);
-      }
+    return false;
+  }
 
-      TSStatus status = authorizerManager.checkPath(username, allPath, permission);
+  private static boolean checkFullPathPermission(
+      String username, PartialPath fullPath, int permission) throws AuthException {
+    try {
+      AuthUtils.validatePrivilege(fullPath, permission);

Review Comment:
   No need to do this check.



##########
iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4:
##########
@@ -911,11 +911,12 @@ PRIVILEGE_VALUE
     | WRITE_SCHEMA
     | MANAGE_USER
     | MANAGE_ROLE
-    | GRANT_PRIVILEGE
-    | ALTER_PASSWORD
     | USE_TRIGGER
+    | USE_UDF
     | USE_CQ
     | USE_PIPE
+    | EXTEND_TEMPLATE
+    | AUDIT

Review Comment:
   add all of these new keywords into `IdentifierParser.g4`



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java:
##########
@@ -112,27 +112,39 @@ public TPermissionInfoResp checkUserPrivileges(
     return authorInfo.checkUserPrivileges(username, paths, permission);
   }
 
+  public TAuthizedPatternTreeResp fetchAuthizedPTree(String username, int permission)
+      throws AuthException {
+    return authorInfo.generateAuthizedPTree(username, permission);
+  }
+
   /**
    * When the permission information of a user or role is changed will clear all datanode
-   * permissions related to the user or role.
+   * permissions related to the user or role. We use procedure to invalid cache: procedure can make
+   * sure all datanode can be invalided.
    */
   public TSStatus invalidateCache(String username, String roleName) {
     List<TDataNodeConfiguration> allDataNodes =
         configManager.getNodeManager().getRegisteredDataNodes();
     TInvalidatePermissionCacheReq req = new TInvalidatePermissionCacheReq();
-    TSStatus status;
-    req.setUsername(username);
-    req.setRoleName(roleName);
-    for (TDataNodeConfiguration dataNodeInfo : allDataNodes) {
-      status =
-          SyncDataNodeClientPool.getInstance()
-              .sendSyncRequestToDataNodeWithRetry(
-                  dataNodeInfo.getLocation().getInternalEndPoint(),
-                  req,
-                  DataNodeRequestType.INVALIDATE_PERMISSION_CACHE);
-      if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-        return status;
-      }
+    //    TSStatus status;
+    //    req.setUsername(username);
+    //    req.setRoleName(roleName);
+    //    List<TDataNodeConfiguration> dnsToInvalid = new ArrayList<>();
+    //    for (TDataNodeConfiguration dataNodeInfo : allDataNodes) {
+    //      status =
+    //          SyncDataNodeClientPool.getInstance()
+    //              .sendSyncRequestToDataNodeWithRetry(
+    //                  dataNodeInfo.getLocation().getInternalEndPoint(),
+    //                  req,
+    //                  DataNodeRequestType.INVALIDATE_PERMISSION_CACHE);
+    //      if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+    //        dnsToInvalid.add(dataNodeInfo);
+    //      }
+    //    }

Review Comment:
   ```suggestion
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -178,29 +188,45 @@ public TSStatus authorNonQuery(AuthorPlan authorPlan) {
         case GrantRole:
           for (int i : permissions) {
             for (PartialPath path : nodeNameList) {
-              authorizer.grantPrivilegeToRole(roleName, path, i);
+              // LSL  need to check if its grant opt.
+              authorizer.grantPrivilegeToRole(
+                  authorPlan.getCurrentUser(), roleName, path, i, false);

Review Comment:
   grant opt should be passed from `AuthorPlan`



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -367,6 +400,47 @@ public PermissionInfoResp executeListUserPrivileges(AuthorPlan plan) throws Auth
     return result;
   }
 
+  public TAuthizedPatternTreeResp generateAuthizedPTree(String username, int permission)
+      throws AuthException {
+    TAuthizedPatternTreeResp resp = new TAuthizedPatternTreeResp();
+    User user = authorizer.getUser(username);
+    PathPatternTree pPtree = new PathPatternTree();
+    if (user == null) {
+      resp.setStatus(RpcUtils.getStatus(TSStatusCode.USER_NOT_EXIST, "No such user : " + username));
+      resp.setUsername(username);
+      resp.setPrivilegeId(permission);
+      return resp;
+    }
+    for (PathPrivilege path : user.getPathPrivilegeList()) {
+      if (path.getPrivileges().contains(permission)) {
+        pPtree.appendPathPattern(path.getPath());
+      }
+    }
+    for (String rolename : user.getRoleList()) {
+      Role role = authorizer.getRole(rolename);
+      if (role != null) {
+        for (PathPrivilege path : role.getPathPrivilegeList()) {
+          if (path.getPrivileges().contains(permission)) {
+            pPtree.appendPathPattern(path.getPath());
+          }
+        }
+      }
+    }
+    pPtree.constructTree();
+    resp.setUsername(username);
+    resp.setPrivilegeId(permission);
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+    try {
+      pPtree.serialize(dataOutputStream);
+    } catch (Exception ignore) {
+
+    }

Review Comment:
   should never ignore it, we need to use AuthException to wrap it.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java:
##########
@@ -378,6 +387,15 @@ public TSStatus checkUser(String username, String password) {
     }
   }
 
+  public PathPatternTree getAuthizedPattern(String username, int permission) {
+    authReadWriteLock.readLock().lock();
+    try {
+      return authorityFetcher.getAuthizedPatternTree(username, permission);
+    } finally {
+      authReadWriteLock.readLock().unlock();
+    }
+  }
+

Review Comment:
   these methods should not be in `AuthorizerManager`



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -30,33 +30,23 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Objects;
 import java.util.Set;
-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.I
- */
+/** This class represents a privilege on a specific seriesPath. */
 public class PathPrivilege {
   private static final Logger logger = LoggerFactory.getLogger(PathPrivilege.class);
-  private Set<Integer> privileges;
-  private PartialPath path;
 
-  /**
-   * This field records how many times this privilege is referenced during a life cycle (from being
-   * loaded to being discarded). When serialized to a file, this determines the order of
-   * serialization. The higher this values is, the sooner this privilege will be serialized. As a
-   * result, the hot privileges will be in the first place so that the hit time will decrease when
-   * being queried.
-   */
-  private AtomicInteger referenceCnt = new AtomicInteger(0);
+  // privilege capacity: read_data, write_data, read_schema, write_schema;
+  private static final Integer priCap = 4;

Review Comment:
   ```suggestion
     private static final int priCap = 4;
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -178,29 +188,45 @@ public TSStatus authorNonQuery(AuthorPlan authorPlan) {
         case GrantRole:
           for (int i : permissions) {
             for (PartialPath path : nodeNameList) {

Review Comment:
   what if nodeNameList == null?



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java:
##########
@@ -987,6 +989,25 @@ public TPermissionInfoResp checkUserPrivileges(
     }
   }
 
+  public TAuthizedPatternTreeResp fetchAuthizedPatternTree(String username, int permission) {
+    TSStatus status = confirmLeader();
+    if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      try {
+        return permissionManager.fetchAuthizedPTree(username, permission);
+      } catch (AuthException e) {
+        TAuthizedPatternTreeResp resp = AuthUtils.generateEmptyAuthizedPTree(username, permission);
+        status
+            .setCode(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode())
+            .setMessage(e.getMessage());
+        return resp;

Review Comment:
   you should directly use the `TSStatusCode` in `AuthException`, and it seems that you forgot to set the status into `TAuthizedPatternTreeResp`.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -77,43 +82,131 @@ public ClusterAuthorityFetcher(IAuthorCache iAuthorCache) {
   }
 
   @Override
-  public TSStatus checkUserPrivileges(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkUserPathPrivileges(
+      String username, List<PartialPath> allPath, int permission) {
+    checkCacheAvailable();
     User user = iAuthorCache.getUserCache(username);
     if (user != null) {
-      for (PartialPath path : allPath) {
-        try {
-          if (!user.isOpenIdUser() || !authorizer.checkUserPrivileges(username, path, permission)) {
-            if (!user.checkPrivilege(path, permission)) {
-              if (user.getRoleList().isEmpty()) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
-              }
-              boolean status = false;
-              for (String roleName : user.getRoleList()) {
-                Role role = iAuthorCache.getRoleCache(roleName);
-                // It is detected that the role of the user does not exist in the cache, indicating
-                // that the permission information of the role has changed.
-                // The user cache needs to be initialized
-                if (role == null) {
-                  iAuthorCache.invalidateCache(username, "");
-                  return checkPath(username, allPath, permission);
-                }
-                status = role.checkPrivilege(path, permission);
-                if (status) {
-                  break;
-                }
+      if (!user.isOpenIdUser()) {
+        for (PartialPath path : allPath) {
+          // check user first
+          if (!user.checkPathPrivilege(path, permission)) {
+            if (user.getRoleList().isEmpty()) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+            boolean status = false;
+            for (String rolename : user.getRoleList()) {
+              Role cacheRole = iAuthorCache.getRoleCache(rolename);
+              if (cacheRole == null) {
+                return checkPathFromConfigNode(username, allPath, permission);
               }
-              if (!status) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+              if (cacheRole.checkPathPrivilege(path, permission)) {
+                status = true;
+                break;
               }
             }
+            if (!status) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+          }
+        }
+      }
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    } else {
+      return checkPathFromConfigNode(username, allPath, permission);
+    }
+  }
+
+  @Override
+  public PathPatternTree getAuthizedPatternTree(String username, int permission) {
+    boolean originFetch = false;
+    User user = iAuthorCache.getUserCache(username);
+    if (user != null) {
+      if (!user.getRoleList().isEmpty()) {
+        for (String role : user.getRoleList()) {
+          if (iAuthorCache.getRoleCache(role) == null) {
+            originFetch = true;
+          }
+        }
+      }
+    } else {
+      originFetch = true;
+    }
+    if (!originFetch) {
+      PathPatternTree patternTree = new PathPatternTree();
+      for (PathPrivilege path : user.getPathPrivilegeList()) {
+        if (path.getPrivileges().contains(permission)) {
+          patternTree.appendPathPattern(path.getPath());
+        }
+      }
+      for (String role : user.getRoleList()) {
+        Role cachedRole = iAuthorCache.getRoleCache(role);
+        for (PathPrivilege path : cachedRole.getPathPrivilegeList()) {
+          if (path.getPrivileges().contains(permission)) {
+            patternTree.appendPathPattern(path.getPath());
+          }
+        }
+      }
+      patternTree.constructTree();
+      return patternTree;
+    } else {
+      return fetchAuthizedPatternTree(username, permission);
+    }
+  }
+
+  public PathPatternTree fetchAuthizedPatternTree(String username, int permission) {

Review Comment:
   ```suggestion
     private PathPatternTree fetchAuthizedPatternTree(String username, int permission) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PrivilegeType.java:
##########
@@ -34,50 +27,56 @@ public enum PrivilegeType {
   WRITE_SCHEMA(true),
   MANAGE_USER,
   MANAGE_ROLE,
-  GRANT_PRIVILEGE,
-  ALTER_PASSWORD,
-  USE_TRIGGER(true),
+  USE_TRIGGER,
+
+  USE_UDF,
+
   USE_CQ,
   USE_PIPE,
-  MANAGE_DATABASE(true),
+  EXTEND_TEMPLATE,
+  MANAGE_DATABASE,
   MAINTAIN,
-  READ(true, false, READ_DATA, READ_SCHEMA),
-  WRITE(true, false, WRITE_DATA, WRITE_SCHEMA),
-  ALL(
-      true,
-      false,
-      READ,
-      WRITE,
-      MANAGE_USER,
-      MANAGE_ROLE,
-      GRANT_PRIVILEGE,
-      ALTER_PASSWORD,
-      USE_TRIGGER,
-      USE_CQ,
-      USE_PIPE,
-      MANAGE_DATABASE,
-      MAINTAIN);
+  AUDIT;
+
+  private enum scope {
+    NULL,
+    SYSTEMPRIVILEGE,
+    PATHPRIVILEGE,
+    ROLEPRIVILEGE;

Review Comment:
   ```suggestion
   ```
   is not used.



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -314,13 +342,18 @@ public PermissionInfoResp executeListUserPrivileges(AuthorPlan plan) throws Auth
     List<String> userPrivilegesList = new ArrayList<>();
 
     if (IoTDBConstant.PATH_ROOT.equals(plan.getUserName())) {
-      for (PrivilegeType privilegeType : PrivilegeType.ALL.getStorablePrivilege()) {
+      for (PrivilegeType privilegeType : PrivilegeType.values()) {
         userPrivilegesList.add(privilegeType.toString());
       }
+      userPrivilegesList.add("FLUSH");
+      userPrivilegesList.add("MERGE");
+      userPrivilegesList.add("CLEAR CACHE");
+      userPrivilegesList.add("ALTER SYSTEM");
+      userPrivilegesList.add("SCHEMA SNAPSHOT");

Review Comment:
   These are not complete.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PrivilegeType.java:
##########
@@ -34,50 +27,56 @@ public enum PrivilegeType {
   WRITE_SCHEMA(true),
   MANAGE_USER,
   MANAGE_ROLE,
-  GRANT_PRIVILEGE,
-  ALTER_PASSWORD,
-  USE_TRIGGER(true),
+  USE_TRIGGER,
+
+  USE_UDF,
+
   USE_CQ,
   USE_PIPE,
-  MANAGE_DATABASE(true),
+  EXTEND_TEMPLATE,
+  MANAGE_DATABASE,
   MAINTAIN,
-  READ(true, false, READ_DATA, READ_SCHEMA),
-  WRITE(true, false, WRITE_DATA, WRITE_SCHEMA),
-  ALL(
-      true,
-      false,
-      READ,
-      WRITE,
-      MANAGE_USER,
-      MANAGE_ROLE,
-      GRANT_PRIVILEGE,
-      ALTER_PASSWORD,
-      USE_TRIGGER,
-      USE_CQ,
-      USE_PIPE,
-      MANAGE_DATABASE,
-      MAINTAIN);
+  AUDIT;
+
+  private enum scope {
+    NULL,
+    SYSTEMPRIVILEGE,
+    PATHPRIVILEGE,
+    ROLEPRIVILEGE;
+  }
 
   private static final int PRIVILEGE_COUNT = values().length;
 
   private final boolean isPathRelevant;
-  private final boolean isStorable;
-  private final List<PrivilegeType> subPrivileges = new ArrayList<>();
 
   PrivilegeType() {
     this.isPathRelevant = false;
-    this.isStorable = true;
   }
 
   PrivilegeType(boolean isPathRelevant) {
     this.isPathRelevant = isPathRelevant;
-    this.isStorable = true;
   }
 
-  PrivilegeType(boolean isPathRelevant, boolean isStorable, PrivilegeType... privilegeTypes) {
-    this.isPathRelevant = isPathRelevant;
-    this.isStorable = isStorable;
-    this.subPrivileges.addAll(Arrays.asList(privilegeTypes));
+  scope getAuthScope() {

Review Comment:
   this method is not used.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PrivilegeType.java:
##########
@@ -34,50 +27,56 @@ public enum PrivilegeType {
   WRITE_SCHEMA(true),
   MANAGE_USER,
   MANAGE_ROLE,
-  GRANT_PRIVILEGE,
-  ALTER_PASSWORD,
-  USE_TRIGGER(true),
+  USE_TRIGGER,
+
+  USE_UDF,
+
   USE_CQ,
   USE_PIPE,
-  MANAGE_DATABASE(true),
+  EXTEND_TEMPLATE,
+  MANAGE_DATABASE,
   MAINTAIN,
-  READ(true, false, READ_DATA, READ_SCHEMA),
-  WRITE(true, false, WRITE_DATA, WRITE_SCHEMA),
-  ALL(
-      true,
-      false,
-      READ,
-      WRITE,
-      MANAGE_USER,
-      MANAGE_ROLE,
-      GRANT_PRIVILEGE,
-      ALTER_PASSWORD,
-      USE_TRIGGER,
-      USE_CQ,
-      USE_PIPE,
-      MANAGE_DATABASE,
-      MAINTAIN);
+  AUDIT;
+
+  private enum scope {

Review Comment:
   ```suggestion
     private enum Scope {
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -301,6 +328,7 @@ public PermissionInfoResp executeListRolePrivileges(AuthorPlan plan) throws Auth
     return result;
   }
 
+  // LSL

Review Comment:
   ```suggestion
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -272,6 +298,7 @@ public PermissionInfoResp executeListRoles(AuthorPlan plan) throws AuthException
     return result;
   }
 
+  // LSL

Review Comment:
   ```suggestion
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -77,43 +82,131 @@ public ClusterAuthorityFetcher(IAuthorCache iAuthorCache) {
   }
 
   @Override
-  public TSStatus checkUserPrivileges(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkUserPathPrivileges(
+      String username, List<PartialPath> allPath, int permission) {
+    checkCacheAvailable();
     User user = iAuthorCache.getUserCache(username);
     if (user != null) {
-      for (PartialPath path : allPath) {
-        try {
-          if (!user.isOpenIdUser() || !authorizer.checkUserPrivileges(username, path, permission)) {
-            if (!user.checkPrivilege(path, permission)) {
-              if (user.getRoleList().isEmpty()) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
-              }
-              boolean status = false;
-              for (String roleName : user.getRoleList()) {
-                Role role = iAuthorCache.getRoleCache(roleName);
-                // It is detected that the role of the user does not exist in the cache, indicating
-                // that the permission information of the role has changed.
-                // The user cache needs to be initialized
-                if (role == null) {
-                  iAuthorCache.invalidateCache(username, "");
-                  return checkPath(username, allPath, permission);
-                }
-                status = role.checkPrivilege(path, permission);
-                if (status) {
-                  break;
-                }
+      if (!user.isOpenIdUser()) {
+        for (PartialPath path : allPath) {
+          // check user first
+          if (!user.checkPathPrivilege(path, permission)) {
+            if (user.getRoleList().isEmpty()) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+            boolean status = false;
+            for (String rolename : user.getRoleList()) {
+              Role cacheRole = iAuthorCache.getRoleCache(rolename);
+              if (cacheRole == null) {
+                return checkPathFromConfigNode(username, allPath, permission);
               }
-              if (!status) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+              if (cacheRole.checkPathPrivilege(path, permission)) {
+                status = true;
+                break;
               }
             }
+            if (!status) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+          }
+        }
+      }
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    } else {
+      return checkPathFromConfigNode(username, allPath, permission);
+    }
+  }
+
+  @Override
+  public PathPatternTree getAuthizedPatternTree(String username, int permission) {
+    boolean originFetch = false;
+    User user = iAuthorCache.getUserCache(username);
+    if (user != null) {
+      if (!user.getRoleList().isEmpty()) {
+        for (String role : user.getRoleList()) {
+          if (iAuthorCache.getRoleCache(role) == null) {
+            originFetch = true;
+          }
+        }
+      }
+    } else {
+      originFetch = true;
+    }

Review Comment:
   We can combine these two steps into one, because cache miss happens infrequently, we don't need to call `getUserCache` and `getRoleCache` each time.



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java:
##########
@@ -403,8 +477,9 @@ public TPermissionInfoResp getUserPermissionInfo(String username) throws AuthExc
 
     // User permission information
     User user = authorizer.getUser(username);
-    if (user.getPrivilegeList() != null) {
-      for (PathPrivilege pathPrivilege : user.getPrivilegeList()) {
+    // LSL

Review Comment:
   ```suggestion
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -226,7 +332,12 @@ public TSStatus checkUser(String username, String password) {
     }
   }
 
-  public TSStatus checkPath(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkSysPriFromConfigNode(String username, int permission) {
+    return checkPathFromConfigNode(username, new ArrayList<>(), permission);

Review Comment:
   ```suggestion
       return checkPathFromConfigNode(username, Collections.emptyList(), permission);
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java:
##########
@@ -112,27 +112,39 @@ public TPermissionInfoResp checkUserPrivileges(
     return authorInfo.checkUserPrivileges(username, paths, permission);
   }
 
+  public TAuthizedPatternTreeResp fetchAuthizedPTree(String username, int permission)
+      throws AuthException {
+    return authorInfo.generateAuthizedPTree(username, permission);
+  }
+
   /**
    * When the permission information of a user or role is changed will clear all datanode
-   * permissions related to the user or role.
+   * permissions related to the user or role. We use procedure to invalid cache: procedure can make
+   * sure all datanode can be invalided.
    */
   public TSStatus invalidateCache(String username, String roleName) {

Review Comment:
   this should be called after consensus write operation, and these two should be wrapped in one procedure.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java:
##########
@@ -62,52 +62,65 @@ private AuthorityChecker() {
    *
    * @param username username
    * @param paths paths in List structure
-   * @param type Statement Type
-   * @param targetUser target user
    * @return if permission-check is passed
    */
-  public static boolean checkPermission(
-      String username, List<? extends PartialPath> paths, StatementType type, String targetUser) {
+  public static boolean checkFullPathListPermission(
+      String username, List<PartialPath> paths, int permission) throws AuthException {
+    // this function will throw auth_exception for illegal permission;
+    AuthUtils.validatePrivilege(paths.get(0), permission);

Review Comment:
   No need to do this check.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -77,43 +82,131 @@ public ClusterAuthorityFetcher(IAuthorCache iAuthorCache) {
   }
 
   @Override
-  public TSStatus checkUserPrivileges(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkUserPathPrivileges(
+      String username, List<PartialPath> allPath, int permission) {
+    checkCacheAvailable();
     User user = iAuthorCache.getUserCache(username);
     if (user != null) {
-      for (PartialPath path : allPath) {
-        try {
-          if (!user.isOpenIdUser() || !authorizer.checkUserPrivileges(username, path, permission)) {
-            if (!user.checkPrivilege(path, permission)) {
-              if (user.getRoleList().isEmpty()) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
-              }
-              boolean status = false;
-              for (String roleName : user.getRoleList()) {
-                Role role = iAuthorCache.getRoleCache(roleName);
-                // It is detected that the role of the user does not exist in the cache, indicating
-                // that the permission information of the role has changed.
-                // The user cache needs to be initialized
-                if (role == null) {
-                  iAuthorCache.invalidateCache(username, "");
-                  return checkPath(username, allPath, permission);
-                }
-                status = role.checkPrivilege(path, permission);
-                if (status) {
-                  break;
-                }
+      if (!user.isOpenIdUser()) {
+        for (PartialPath path : allPath) {
+          // check user first
+          if (!user.checkPathPrivilege(path, permission)) {
+            if (user.getRoleList().isEmpty()) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+            boolean status = false;
+            for (String rolename : user.getRoleList()) {
+              Role cacheRole = iAuthorCache.getRoleCache(rolename);
+              if (cacheRole == null) {
+                return checkPathFromConfigNode(username, allPath, permission);
               }
-              if (!status) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+              if (cacheRole.checkPathPrivilege(path, permission)) {
+                status = true;
+                break;
               }
             }
+            if (!status) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+          }
+        }
+      }
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    } else {
+      return checkPathFromConfigNode(username, allPath, permission);
+    }
+  }
+
+  @Override
+  public PathPatternTree getAuthizedPatternTree(String username, int permission) {
+    boolean originFetch = false;

Review Comment:
   ```suggestion
       boolean cacheMiss = false;
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/InvalidAuthCacheProcedure.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.confignode.procedure.impl.sync;
+
+import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.confignode.client.DataNodeRequestType;
+import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
+import org.apache.iotdb.confignode.manager.PermissionManager;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.InvalidAuthCacheState;
+import org.apache.iotdb.confignode.procedure.store.ProcedureType;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.Set;
+import java.util.Iterator;
+import java.util.Objects;
+
+public class InvalidAuthCacheProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, InvalidAuthCacheState> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(InvalidAuthCacheProcedure.class);
+
+  private PermissionManager permissionManager;
+
+  private String user;
+  private String role;
+
+  private int timeoutMS;
+
+  private static final int RETRY_THRESHOLD = 1;
+  private static final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig();
+
+  private List<Pair<TDataNodeConfiguration, Long>> dataNodesToInvalid;
+
+  private Set<TDataNodeConfiguration> invalidedDNs;
+
+  public InvalidAuthCacheProcedure() {
+    super();
+  }
+
+  public InvalidAuthCacheProcedure(String user, String role, List<TDataNodeConfiguration> alldns) {
+    super();
+    this.user = user;
+    this.role = role;
+    this.dataNodesToInvalid = new ArrayList<>();
+    for (TDataNodeConfiguration item : alldns) {
+      this.dataNodesToInvalid.add(
+          new Pair<TDataNodeConfiguration, Long>(item, System.currentTimeMillis()));

Review Comment:
   ```suggestion
             new Pair<>(item, System.currentTimeMillis()));
   ```



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/InvalidAuthCacheState.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.confignode.procedure.state;
+
+public enum InvalidAuthCacheState {
+  INIT,
+  DATANODE_AUTHCACHE_INVALIDING,
+  DATANODE_AUTHCACHE_INVALID_DONE

Review Comment:
   `INIT` and `DATANODE_AUTHCACHE_INVALID_DONE` is unnecessary.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java:
##########
@@ -62,52 +62,65 @@ private AuthorityChecker() {
    *
    * @param username username
    * @param paths paths in List structure
-   * @param type Statement Type
-   * @param targetUser target user
    * @return if permission-check is passed
    */
-  public static boolean checkPermission(
-      String username, List<? extends PartialPath> paths, StatementType type, String targetUser) {
+  public static boolean checkFullPathListPermission(
+      String username, List<PartialPath> paths, int permission) throws AuthException {
+    // this function will throw auth_exception for illegal permission;
+    AuthUtils.validatePrivilege(paths.get(0), permission);
+
     if (SUPER_USER.equals(username)) {
       return true;
     }
 
-    int[] permissions = translateToPermissionId(type);
-    for (int permission : permissions) {
-      if (permission == -1) {
-        continue;
-      } else if (permission == PrivilegeType.ALTER_PASSWORD.ordinal()
-          && username.equals(targetUser)) {
-        // A user can modify his own password
-        return true;
-      }
+    TSStatus status = authorizerManager.checkFullPathPrivilegeList(username, paths, permission);
+    if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      return true;
+    }
 
-      List<PartialPath> allPath = new ArrayList<>();
-      if (paths != null && !paths.isEmpty()) {
-        for (PartialPath path : paths) {
-          allPath.add(path == null ? AuthUtils.ROOT_PATH_PRIVILEGE_PATH : path);
-        }
-      } else {
-        allPath.add(AuthUtils.ROOT_PATH_PRIVILEGE_PATH);
-      }
+    return false;
+  }
 
-      TSStatus status = authorizerManager.checkPath(username, allPath, permission);
+  private static boolean checkFullPathPermission(
+      String username, PartialPath fullPath, int permission) throws AuthException {
+    try {
+      AuthUtils.validatePrivilege(fullPath, permission);
+      if (SUPER_USER.equals(username)) {
+        return true;
+      }
+      TSStatus status = authorizerManager.checkFullPathPrivilege(username, fullPath, permission);
       if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         return true;
       }
+    } catch (AuthException e) {
+      logger.error(
+          "Error occurs when checking the seriesPath {} for user {}", fullPath, username, e);
+      throw new AuthException(TSStatusCode.ILLEGAL_PARAMETER, e);
     }
     return false;
   }
 
-  private static boolean checkOnePath(String username, PartialPath path, int permission)
+  public static PathPatternTree getAuthorizedPathTree(String username, int permission)
+      throws AuthException {
+    AuthUtils.validatePrivilege(permission);

Review Comment:
   No need to do this check.



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/InvalidAuthCacheProcedure.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.confignode.procedure.impl.sync;
+
+import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.confignode.client.DataNodeRequestType;
+import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
+import org.apache.iotdb.confignode.manager.PermissionManager;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.InvalidAuthCacheState;
+import org.apache.iotdb.confignode.procedure.store.ProcedureType;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.Set;
+import java.util.Iterator;
+import java.util.Objects;
+
+public class InvalidAuthCacheProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, InvalidAuthCacheState> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(InvalidAuthCacheProcedure.class);
+
+  private PermissionManager permissionManager;
+
+  private String user;
+  private String role;
+
+  private int timeoutMS;
+
+  private static final int RETRY_THRESHOLD = 1;
+  private static final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig();
+
+  private List<Pair<TDataNodeConfiguration, Long>> dataNodesToInvalid;
+
+  private Set<TDataNodeConfiguration> invalidedDNs;

Review Comment:
   ```suggestion
   ```
   It seems that this field is never used.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java:
##########
@@ -151,11 +159,12 @@ public void deleteRole(String roleName) throws AuthException {
   }
 
   @Override
-  public void grantPrivilegeToRole(String roleName, PartialPath path, int privilegeId)
+  public void grantPrivilegeToRole(
+      String currentUser, String roleName, PartialPath path, int privilegeId, boolean grantOpt)
       throws AuthException {
     authReadWriteLock.writeLock().lock();
     try {
-      authorizer.grantPrivilegeToRole(roleName, path, privilegeId);
+      authorizer.grantPrivilegeToRole(currentUser, roleName, path, privilegeId, grantOpt);

Review Comment:
   ```suggestion
         authorizer.grantPrivilegeToRole(roleName, path, privilegeId, grantOpt);
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java:
##########
@@ -62,52 +62,65 @@ private AuthorityChecker() {
    *
    * @param username username
    * @param paths paths in List structure
-   * @param type Statement Type
-   * @param targetUser target user
    * @return if permission-check is passed
    */
-  public static boolean checkPermission(
-      String username, List<? extends PartialPath> paths, StatementType type, String targetUser) {
+  public static boolean checkFullPathListPermission(
+      String username, List<PartialPath> paths, int permission) throws AuthException {
+    // this function will throw auth_exception for illegal permission;
+    AuthUtils.validatePrivilege(paths.get(0), permission);
+
     if (SUPER_USER.equals(username)) {
       return true;
     }
 
-    int[] permissions = translateToPermissionId(type);
-    for (int permission : permissions) {
-      if (permission == -1) {
-        continue;
-      } else if (permission == PrivilegeType.ALTER_PASSWORD.ordinal()
-          && username.equals(targetUser)) {
-        // A user can modify his own password
-        return true;
-      }
+    TSStatus status = authorizerManager.checkFullPathPrivilegeList(username, paths, permission);
+    if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      return true;
+    }
 
-      List<PartialPath> allPath = new ArrayList<>();
-      if (paths != null && !paths.isEmpty()) {
-        for (PartialPath path : paths) {
-          allPath.add(path == null ? AuthUtils.ROOT_PATH_PRIVILEGE_PATH : path);
-        }
-      } else {
-        allPath.add(AuthUtils.ROOT_PATH_PRIVILEGE_PATH);
-      }
+    return false;
+  }
 
-      TSStatus status = authorizerManager.checkPath(username, allPath, permission);
+  private static boolean checkFullPathPermission(
+      String username, PartialPath fullPath, int permission) throws AuthException {
+    try {
+      AuthUtils.validatePrivilege(fullPath, permission);
+      if (SUPER_USER.equals(username)) {
+        return true;
+      }
+      TSStatus status = authorizerManager.checkFullPathPrivilege(username, fullPath, permission);
       if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         return true;
       }
+    } catch (AuthException e) {
+      logger.error(
+          "Error occurs when checking the seriesPath {} for user {}", fullPath, username, e);
+      throw new AuthException(TSStatusCode.ILLEGAL_PARAMETER, e);
     }
     return false;
   }
 
-  private static boolean checkOnePath(String username, PartialPath path, int permission)
+  public static PathPatternTree getAuthorizedPathTree(String username, int permission)
+      throws AuthException {
+    AuthUtils.validatePrivilege(permission);
+    PathPatternTree pathTree = authorizerManager.getAuthizedPattern(username, permission);
+    return pathTree;
+  }
+
+  public static boolean checkSystemPermission(String username, int permission)
       throws AuthException {
     try {
-      PartialPath newPath = path == null ? AuthUtils.ROOT_PATH_PRIVILEGE_PATH : path;
-      if (authorizerManager.checkUserPrivileges(username, newPath, permission)) {
+      AuthUtils.validatePrivilege(null, permission);

Review Comment:
   No need to do this check.



##########
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/InvalidAuthCacheProcedure.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.confignode.procedure.impl.sync;
+
+import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.confignode.client.DataNodeRequestType;
+import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
+import org.apache.iotdb.confignode.manager.PermissionManager;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
+import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
+import org.apache.iotdb.confignode.procedure.state.InvalidAuthCacheState;
+import org.apache.iotdb.confignode.procedure.store.ProcedureType;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.Set;
+import java.util.Iterator;
+import java.util.Objects;
+
+public class InvalidAuthCacheProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, InvalidAuthCacheState> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(InvalidAuthCacheProcedure.class);
+
+  private PermissionManager permissionManager;
+
+  private String user;
+  private String role;
+
+  private int timeoutMS;
+
+  private static final int RETRY_THRESHOLD = 1;
+  private static final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig();
+
+  private List<Pair<TDataNodeConfiguration, Long>> dataNodesToInvalid;
+
+  private Set<TDataNodeConfiguration> invalidedDNs;
+
+  public InvalidAuthCacheProcedure() {
+    super();
+  }
+
+  public InvalidAuthCacheProcedure(String user, String role, List<TDataNodeConfiguration> alldns) {
+    super();
+    this.user = user;
+    this.role = role;
+    this.dataNodesToInvalid = new ArrayList<>();
+    for (TDataNodeConfiguration item : alldns) {
+      this.dataNodesToInvalid.add(
+          new Pair<TDataNodeConfiguration, Long>(item, System.currentTimeMillis()));
+    }
+    invalidedDNs = new HashSet<>();
+    this.timeoutMS = commonConfig.getDatanodeTokenTimeoutMS();
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, InvalidAuthCacheState state) {
+    if (dataNodesToInvalid.isEmpty()) {
+      return Flow.NO_MORE_STATE;
+    }
+    try {
+      switch (state) {
+        case INIT:
+          LOGGER.info("Start to invalid auth cache for " + "user: %s, role %s", user, role);
+          // shall we need to check if the user/role has been deleted?
+          if (dataNodesToInvalid.isEmpty()) {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALID_DONE);
+          }
+          setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALIDING);
+          break;
+        case DATANODE_AUTHCACHE_INVALIDING:
+          if (dataNodesToInvalid.isEmpty()) {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALID_DONE);
+          }
+          TInvalidatePermissionCacheReq req = new TInvalidatePermissionCacheReq();
+          TSStatus status;
+          req.setUsername(user);
+          req.setRoleName(role);
+          Iterator<Pair<TDataNodeConfiguration, Long>> it = dataNodesToInvalid.iterator();
+          while (it.hasNext()) {
+            if (it.next().getRight() + this.timeoutMS < System.currentTimeMillis()) {
+              invalidedDNs.add(it.next().getLeft());
+              it.remove();
+              continue;
+            }
+            status =
+                SyncDataNodeClientPool.getInstance()
+                    .sendSyncRequestToDataNodeWithRetry(
+                        it.next().getLeft().getLocation().getInternalEndPoint(),
+                        req,
+                        DataNodeRequestType.INVALIDATE_PERMISSION_CACHE);
+            if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+              invalidedDNs.add(it.next().getLeft());
+              it.remove();
+            }
+          }
+          if (dataNodesToInvalid.isEmpty()) {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALID_DONE);
+          } else {
+            setNextState(InvalidAuthCacheState.DATANODE_AUTHCACHE_INVALIDING);
+          }
+          break;
+        case DATANODE_AUTHCACHE_INVALID_DONE:
+          LOGGER.info("finish invalid auth cache for user:%s, role %s", user, role);
+          break;
+      }
+    } catch (Exception e) {
+      if (isRollbackSupported(state)) {
+        LOGGER.error("Fail in invalid auth cache", e);
+        setFailure(new ProcedureException(e.getMessage()));
+      } else {
+        LOGGER.error(
+            "Retrievable error trying to invalid auth cache :[user : %s, role : %s] in datanode",
+            user, role, e);
+        if (getCycles() > RETRY_THRESHOLD) {
+          setFailure(
+              new ProcedureException(
+                  String.format(
+                      "Fail to invalid auth cahce, user: %s, role: %s, datanode:%s",

Review Comment:
   ```suggestion
                         "Fail to invalid auth cache, user: %s, role: %s, datanode:%s",
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java:
##########
@@ -400,6 +418,47 @@ public SettableFuture<ConfigTaskResult> operatePermission(AuthorStatement author
     }
   }
 
+  public TSStatus checkFullPathPrivilege(String username, PartialPath fullPath, int permission) {
+    authReadWriteLock.writeLock().lock();
+    try {
+      List<PartialPath> paths = new ArrayList<>();
+      paths.add(fullPath);
+      return authorityFetcher.checkUserPathPrivileges(username, paths, permission);
+    } finally {
+      authReadWriteLock.writeLock().unlock();
+    }
+  }
+
+  public TSStatus checkFullPathPrivilegeList(
+      String username, List<PartialPath> paths, int permission) {
+    authReadWriteLock.readLock().lock();
+    try {
+      return authorityFetcher.checkUserPathPrivileges(username, paths, permission);
+    } finally {
+      authReadWriteLock.writeLock().unlock();
+    }
+  }
+
+  public TSStatus checkUserSysPrivilege(String username, int permission) {

Review Comment:
   same as above.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java:
##########
@@ -109,11 +116,12 @@ public void deleteUser(String username) throws AuthException {
   }
 
   @Override
-  public void grantPrivilegeToUser(String username, PartialPath path, int privilegeId)
+  public void grantPrivilegeToUser(
+      String currentUser, String username, PartialPath path, int privilegeId, boolean grantOpt)
       throws AuthException {
     authReadWriteLock.writeLock().lock();
     try {
-      authorizer.grantPrivilegeToUser(username, path, privilegeId);
+      authorizer.grantPrivilegeToUser(currentUser, username, path, privilegeId, grantOpt);

Review Comment:
   ```suggestion
         authorizer.grantPrivilegeToUser(username, path, privilegeId, grantOpt);
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java:
##########
@@ -400,6 +418,47 @@ public SettableFuture<ConfigTaskResult> operatePermission(AuthorStatement author
     }
   }
 
+  public TSStatus checkFullPathPrivilege(String username, PartialPath fullPath, int permission) {
+    authReadWriteLock.writeLock().lock();
+    try {
+      List<PartialPath> paths = new ArrayList<>();
+      paths.add(fullPath);
+      return authorityFetcher.checkUserPathPrivileges(username, paths, permission);
+    } finally {
+      authReadWriteLock.writeLock().unlock();
+    }
+  }
+
+  public TSStatus checkFullPathPrivilegeList(
+      String username, List<PartialPath> paths, int permission) {
+    authReadWriteLock.readLock().lock();
+    try {
+      return authorityFetcher.checkUserPathPrivileges(username, paths, permission);
+    } finally {
+      authReadWriteLock.writeLock().unlock();
+    }
+  }
+
+  public TSStatus checkUserSysPrivilege(String username, int permission) {
+    authReadWriteLock.readLock().lock();
+    try {
+      return authorityFetcher.checkUserSysPrivileges(username, permission);
+    } finally {
+      authReadWriteLock.readLock().unlock();
+    }
+  }
+
+  public void refreshToken() {

Review Comment:
   put it directly into `ClusterAuthorityFetcher`



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java:
##########
@@ -400,6 +418,47 @@ public SettableFuture<ConfigTaskResult> operatePermission(AuthorStatement author
     }
   }
 
+  public TSStatus checkFullPathPrivilege(String username, PartialPath fullPath, int permission) {

Review Comment:
   same as above.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -226,7 +332,12 @@ public TSStatus checkUser(String username, String password) {
     }
   }
 
-  public TSStatus checkPath(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkSysPriFromConfigNode(String username, int permission) {

Review Comment:
   ```suggestion
     private TSStatus checkSysPriFromConfigNode(String username, int permission) {
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorizerManager.java:
##########
@@ -400,6 +418,47 @@ public SettableFuture<ConfigTaskResult> operatePermission(AuthorStatement author
     }
   }
 
+  public TSStatus checkFullPathPrivilege(String username, PartialPath fullPath, int permission) {
+    authReadWriteLock.writeLock().lock();
+    try {
+      List<PartialPath> paths = new ArrayList<>();
+      paths.add(fullPath);
+      return authorityFetcher.checkUserPathPrivileges(username, paths, permission);
+    } finally {
+      authReadWriteLock.writeLock().unlock();
+    }
+  }
+
+  public TSStatus checkFullPathPrivilegeList(

Review Comment:
   same as above.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -226,7 +332,12 @@ public TSStatus checkUser(String username, String password) {
     }
   }
 
-  public TSStatus checkPath(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkSysPriFromConfigNode(String username, int permission) {
+    return checkPathFromConfigNode(username, new ArrayList<>(), permission);
+  }
+
+  public TSStatus checkPathFromConfigNode(

Review Comment:
   ```suggestion
     private TSStatus checkPathFromConfigNode(
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java:
##########
@@ -62,52 +62,65 @@ private AuthorityChecker() {
    *
    * @param username username
    * @param paths paths in List structure
-   * @param type Statement Type
-   * @param targetUser target user
    * @return if permission-check is passed
    */
-  public static boolean checkPermission(
-      String username, List<? extends PartialPath> paths, StatementType type, String targetUser) {
+  public static boolean checkFullPathListPermission(
+      String username, List<PartialPath> paths, int permission) throws AuthException {
+    // this function will throw auth_exception for illegal permission;
+    AuthUtils.validatePrivilege(paths.get(0), permission);
+
     if (SUPER_USER.equals(username)) {
       return true;
     }
 
-    int[] permissions = translateToPermissionId(type);
-    for (int permission : permissions) {
-      if (permission == -1) {
-        continue;
-      } else if (permission == PrivilegeType.ALTER_PASSWORD.ordinal()
-          && username.equals(targetUser)) {
-        // A user can modify his own password
-        return true;
-      }
+    TSStatus status = authorizerManager.checkFullPathPrivilegeList(username, paths, permission);

Review Comment:
   use `IAuthorityFetcher` instead of `authorizerManager`.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -77,43 +82,131 @@ public ClusterAuthorityFetcher(IAuthorCache iAuthorCache) {
   }
 
   @Override
-  public TSStatus checkUserPrivileges(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkUserPathPrivileges(
+      String username, List<PartialPath> allPath, int permission) {
+    checkCacheAvailable();
     User user = iAuthorCache.getUserCache(username);
     if (user != null) {
-      for (PartialPath path : allPath) {
-        try {
-          if (!user.isOpenIdUser() || !authorizer.checkUserPrivileges(username, path, permission)) {
-            if (!user.checkPrivilege(path, permission)) {
-              if (user.getRoleList().isEmpty()) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
-              }
-              boolean status = false;
-              for (String roleName : user.getRoleList()) {
-                Role role = iAuthorCache.getRoleCache(roleName);
-                // It is detected that the role of the user does not exist in the cache, indicating
-                // that the permission information of the role has changed.
-                // The user cache needs to be initialized
-                if (role == null) {
-                  iAuthorCache.invalidateCache(username, "");
-                  return checkPath(username, allPath, permission);
-                }
-                status = role.checkPrivilege(path, permission);
-                if (status) {
-                  break;
-                }
+      if (!user.isOpenIdUser()) {
+        for (PartialPath path : allPath) {
+          // check user first
+          if (!user.checkPathPrivilege(path, permission)) {
+            if (user.getRoleList().isEmpty()) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+            boolean status = false;
+            for (String rolename : user.getRoleList()) {
+              Role cacheRole = iAuthorCache.getRoleCache(rolename);
+              if (cacheRole == null) {
+                return checkPathFromConfigNode(username, allPath, permission);
               }
-              if (!status) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+              if (cacheRole.checkPathPrivilege(path, permission)) {
+                status = true;
+                break;
               }
             }
+            if (!status) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+          }
+        }
+      }
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    } else {
+      return checkPathFromConfigNode(username, allPath, permission);
+    }
+  }
+
+  @Override
+  public PathPatternTree getAuthizedPatternTree(String username, int permission) {
+    boolean originFetch = false;
+    User user = iAuthorCache.getUserCache(username);
+    if (user != null) {
+      if (!user.getRoleList().isEmpty()) {
+        for (String role : user.getRoleList()) {
+          if (iAuthorCache.getRoleCache(role) == null) {
+            originFetch = true;
+          }
+        }
+      }
+    } else {
+      originFetch = true;
+    }
+    if (!originFetch) {
+      PathPatternTree patternTree = new PathPatternTree();
+      for (PathPrivilege path : user.getPathPrivilegeList()) {
+        if (path.getPrivileges().contains(permission)) {
+          patternTree.appendPathPattern(path.getPath());
+        }
+      }
+      for (String role : user.getRoleList()) {
+        Role cachedRole = iAuthorCache.getRoleCache(role);
+        for (PathPrivilege path : cachedRole.getPathPrivilegeList()) {
+          if (path.getPrivileges().contains(permission)) {
+            patternTree.appendPathPattern(path.getPath());
+          }
+        }
+      }
+      patternTree.constructTree();
+      return patternTree;
+    } else {
+      return fetchAuthizedPatternTree(username, permission);
+    }
+  }
+
+  public PathPatternTree fetchAuthizedPatternTree(String username, int permission) {
+    TCheckUserPrivilegesReq req = new TCheckUserPrivilegesReq(username, null, permission);
+    TAuthizedPatternTreeResp authizedPatternTree = new TAuthizedPatternTreeResp();
+    try (ConfigNodeClient configNodeClient =
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
+      authizedPatternTree = configNodeClient.fetchAuthizedPatternTree(req);
+    } catch (ClientManagerException | TException e) {
+      logger.error("Failed to connect to config node.");
+      authizedPatternTree = new TAuthizedPatternTreeResp();
+      authizedPatternTree.setStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.EXECUTE_STATEMENT_ERROR, "Failed to connect to config node."));
+    }
+    if (authizedPatternTree.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      iAuthorCache.putUserCache(username, cacheUser(authizedPatternTree.getPermissionInfo()));
+      PathPatternTree patternTree =
+          PathPatternTree.deserialize(ByteBuffer.wrap(authizedPatternTree.getPathPatternTree()));
+      patternTree.constructTree();

Review Comment:
   ```suggestion
   ```
   no need to call this method after `deserialize`



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());

Review Comment:
   ```suggestion
             PrivilegeType.values()[privilege], path);
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }
+    return true;
   }
 
-  public AtomicInteger getReferenceCnt() {
-    return referenceCnt;
+  public boolean revokeGrantOpt(Integer privilege) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "path %s dont have privilege %s",
+          path.toString(), PrivilegeType.values()[privilege].toString());
+      return false;
+    }
+    grantOpts.remove(privilege);
+    return true;
   }
 
-  public void setReferenceCnt(AtomicInteger referenceCnt) {
-    this.referenceCnt = referenceCnt;
+  private Integer posToPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.READ_DATA.ordinal();
+      case 1:
+        return PrivilegeType.WRITE_DATA.ordinal();
+      case 2:
+        return PrivilegeType.READ_SCHEMA.ordinal();
+      case 3:
+        return PrivilegeType.WRITE_SCHEMA.ordinal();
+      default:
+        return -1; // this should raise an error ?
+    }
+  }
+
+  private Integer priToPos(PrivilegeType pri) {
+    switch (pri) {
+      case READ_DATA:
+        return 0;
+      case WRITE_DATA:
+        return 1;
+      case READ_SCHEMA:
+        return 2;
+      case WRITE_SCHEMA:
+        return 3;
+      default:
+        return -1;
+    }
+  }
+
+  private boolean basicCheck(Integer privilege) {
+    if (!(privilege == PrivilegeType.READ_DATA.ordinal()
+        || privilege == PrivilegeType.WRITE_DATA.ordinal()
+        || privilege == PrivilegeType.READ_SCHEMA.ordinal()
+        || privilege == PrivilegeType.WRITE_SCHEMA.ordinal())) {
+      logger.warn(
+          String.format(
+              "Get an illegal privilege %s for path %s",
+              PrivilegeType.values()[privilege].toString(), path.toString()));
+      return false;
+    }
+    return true;
+  }
+
+  public void setAllPrivileges(int privs) {
+    for (int i = 0; i < priCap; i++) {
+      if (((0x1 << i) & privs) != 0) {
+        privileges.add(posToPri(i));
+      }
+      if ((((0x1 << (i + 4) & privs) != 0))) {
+        grantOpts.add(posToPri(i));
+      }
+    }
+  }
+
+  public int getAllPrivileges() {
+    int privilege = 0;
+    for (Integer pri : privileges) {

Review Comment:
   ```suggestion
       for (int pri : privileges) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/user/LocalFileUserManager.java:
##########
@@ -33,6 +35,9 @@ public LocalFileUserManager(String userDirPath) throws AuthException {
 
   @Override
   public boolean processTakeSnapshot(File snapshotDir) throws TException, IOException {
+    for (Map.Entry<String, User> entry : userMap.entrySet()) {
+      accessor.saveUser(entry.getValue());
+    }

Review Comment:
   you can put this call into snapshot take.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java:
##########
@@ -2178,6 +2178,7 @@ public Statement visitCreateRole(IoTDBSqlParser.CreateRoleContext ctx) {
 
   // Alter Password
 
+  // 这里需要修改,对于更改密码的操作,用户可以修改自己的密码,非本用户需要获取 manage 权限

Review Comment:
   Use English and does this TODO already done?



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }
+    return true;
   }
 
-  public AtomicInteger getReferenceCnt() {
-    return referenceCnt;
+  public boolean revokeGrantOpt(Integer privilege) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "path %s dont have privilege %s",
+          path.toString(), PrivilegeType.values()[privilege].toString());
+      return false;
+    }
+    grantOpts.remove(privilege);
+    return true;
   }
 
-  public void setReferenceCnt(AtomicInteger referenceCnt) {
-    this.referenceCnt = referenceCnt;
+  private Integer posToPri(int pos) {

Review Comment:
   ```suggestion
     private int posToPri(int pos) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {

Review Comment:
   ```suggestion
     public boolean revokePrivilege(int privilege) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -30,33 +30,23 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Objects;
 import java.util.Set;
-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.I
- */
+/** This class represents a privilege on a specific seriesPath. */
 public class PathPrivilege {
   private static final Logger logger = LoggerFactory.getLogger(PathPrivilege.class);
-  private Set<Integer> privileges;
-  private PartialPath path;
 
-  /**
-   * This field records how many times this privilege is referenced during a life cycle (from being
-   * loaded to being discarded). When serialized to a file, this determines the order of
-   * serialization. The higher this values is, the sooner this privilege will be serialized. As a
-   * result, the hot privileges will be in the first place so that the hit time will decrease when
-   * being queried.
-   */
-  private AtomicInteger referenceCnt = new AtomicInteger(0);
+  // privilege capacity: read_data, write_data, read_schema, write_schema;
+  private static final Integer priCap = 4;

Review Comment:
   Can we directly use Enum to init this field? Because in future, other  people add a new auth, he may forget to change here



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }
+    return true;
   }
 
-  public AtomicInteger getReferenceCnt() {
-    return referenceCnt;
+  public boolean revokeGrantOpt(Integer privilege) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "path %s dont have privilege %s",
+          path.toString(), PrivilegeType.values()[privilege].toString());

Review Comment:
   ```suggestion
             "path {} doesn't have privilege {}",
             path, PrivilegeType.values()[privilege]);
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;
 
   public Role() {
     // empty constructor
   }
 
   public Role(String name) {
     this.name = name;
-    this.privilegeList = new ArrayList<>();
+    this.pathPrivilegeList = new ArrayList<>();
+    this.sysPrivilegeSet = new HashSet<>();
+    this.sysPriGrantOpt = new HashSet<>();
   }
 
+  /** ------------- get func -----------------* */
   public String getName() {
     return name;
   }
 
-  public void setName(String name) {
-    this.name = name;
+  public List<PathPrivilege> getPathPrivilegeList() {
+    return pathPrivilegeList;
   }
 
-  public List<PathPrivilege> getPrivilegeList() {
-    return privilegeList;
+  public Set<Integer> getSysPrivilege() {
+    return sysPrivilegeSet;
   }
 
-  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
-    this.privilegeList = privilegeList;
+  public Set<Integer> getPathPrivileges(PartialPath path) throws AuthException {
+    return AuthUtils.getPrivileges(path, pathPrivilegeList);
   }
 
-  public boolean hasPrivilege(PartialPath path, int privilegeId) {
-    return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
+  public Set<Integer> getSysPriGrantOpt() {
+    return sysPriGrantOpt;
   }
 
-  public void addPrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.addPrivilege(path, privilegeId, privilegeList);
+  public int getAllSysPrivileges() {
+    int privs = 0;
+    for (Integer sysPri : sysPrivilegeSet) {
+      privs |= (0b1 << sysPriTopos(sysPri));
+    }
+    for (Integer sysPriGrantOpt : sysPriGrantOpt) {
+      privs |= 0b1 << (sysPriTopos(sysPriGrantOpt) + 16);
+    }
+    return privs;
   }
 
-  public void removePrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.removePrivilege(path, privilegeId, privilegeList);
+  /** -------------- set func ----------------* */
+  public void setName(String name) {
+    this.name = name;
   }
 
-  /** set privileges of path. */
-  public void setPrivileges(PartialPath path, Set<Integer> privileges) {
-    for (PathPrivilege pathPrivilege : privilegeList) {
+  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
+    this.pathPrivilegeList = privilegeList;
+  }
+
+  public void setPathPrivileges(PartialPath path, Set<Integer> privileges) {
+    for (PathPrivilege pathPrivilege : pathPrivilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
         pathPrivilege.setPrivileges(privileges);
       }
     }
   }
 
-  public Set<Integer> getPrivileges(PartialPath path) throws AuthException {
-    return AuthUtils.getPrivileges(path, privilegeList);
+  public void addPathPrivilege(PartialPath path, int privilegeId, boolean grantOpt) {
+    AuthUtils.addPrivilege(path, privilegeId, pathPrivilegeList, grantOpt);
+  }
+
+  public void removePathPrivilege(PartialPath path, int privilegeId) {
+    AuthUtils.removePrivilege(path, privilegeId, pathPrivilegeList);
+  }
+
+  public void setSysPrivilegeSet(Set<Integer> privilegeSet) {
+    this.sysPrivilegeSet = privilegeSet;
+  }
+
+  public void setSysPriGrantOpt(Set<Integer> grantOpt) {
+    this.sysPriGrantOpt = grantOpt;
+  }
+
+  private int posToSysPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.MANAGE_DATABASE.ordinal();
+      case 1:
+        return PrivilegeType.MANAGE_USER.ordinal();
+      case 2:
+        return PrivilegeType.MANAGE_ROLE.ordinal();
+      case 3:
+        return PrivilegeType.USE_TRIGGER.ordinal();
+      case 4:
+        return PrivilegeType.USE_UDF.ordinal();
+      case 5:
+        return PrivilegeType.USE_CQ.ordinal();
+      case 6:
+        return PrivilegeType.USE_PIPE.ordinal();
+      case 7:
+        return PrivilegeType.EXTEND_TEMPLATE.ordinal();
+      case 8:
+        return PrivilegeType.AUDIT.ordinal();
+      case 9:
+        return PrivilegeType.MAINTAIN.ordinal();
+      default:
+        return -1;
+    }
+  }
+
+  private int sysPriTopos(int privilegeId) {
+    PrivilegeType type = PrivilegeType.values()[privilegeId];
+    switch (type) {
+      case MANAGE_DATABASE:
+        return 0;
+      case MANAGE_USER:
+        return 1;
+      case MANAGE_ROLE:
+        return 2;
+      case USE_TRIGGER:
+        return 3;
+      case USE_UDF:
+        return 4;
+      case USE_CQ:
+        return 5;
+      case USE_PIPE:
+        return 6;
+      case EXTEND_TEMPLATE:
+        return 7;
+      case AUDIT:
+        return 8;
+      case MAINTAIN:
+        return 9;
+      default:
+        return -1;
+    }
+  }
+
+  public void setSysPrivilegeSet(int privilegeMask) {
+    if (sysPrivilegeSet == null) {
+      sysPrivilegeSet = new HashSet<>();
+    }
+    if (sysPriGrantOpt == null) {
+      sysPriGrantOpt = new HashSet<>();
+    }
+    for (int i = 0; i < sysPriSize; i++) {
+      if ((privilegeMask & (0b1 << i)) != 0) {
+        sysPrivilegeSet.add(posToSysPri(i));
+      }
+      if ((privilegeMask & (0b1 << i + 16)) != 0) {

Review Comment:
   ```suggestion
         if ((privilegeMask & (1 << (i + 16))) != 0) {
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java:
##########
@@ -1185,7 +1185,7 @@ public THeartbeatResp getDataNodeHeartBeat(THeartbeatReq req) throws TException
 
       resp.setLoadSample(loadSample);
     }
-
+    AuthorizerManager.getInstance().refreshToken();

Review Comment:
   call ClusterAuthorityFetcher.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/BasicAuthorizer.java:
##########
@@ -138,22 +138,24 @@ public void deleteUser(String username) throws AuthException {
   }
 
   @Override
-  public void grantPrivilegeToUser(String username, PartialPath path, int privilegeId)
+  public void grantPrivilegeToUser(
+      String currentName, String username, PartialPath path, int privilegeId, boolean grantOpt)
       throws AuthException {
-    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_PATH;
-    }
-    if (!userManager.grantPrivilegeToUser(username, newPath, privilegeId)) {
-      throw new AuthException(
-          TSStatusCode.ALREADY_HAS_PRIVILEGE,
-          String.format(
-              "User %s already has %s on %s", username, PrivilegeType.values()[privilegeId], path));
+    // currentName equals null mean followers apply author plan.
+    if (currentName == null

Review Comment:
   no need to use currentName



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }

Review Comment:
   ```suggestion
       privileges.add(privilege);
       
       if (grantOpt) {
         grantOpts.add(privilege);
       }
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/BasicAuthorizer.java:
##########
@@ -209,29 +207,30 @@ public void deleteRole(String roleName) throws AuthException {
     }
   }
 
+  // When role/user A grant privilege to B,
+  // we have to make sure that A have the privilege and grant option.
   @Override
-  public void grantPrivilegeToRole(String roleName, PartialPath path, int privilegeId)
+  public void grantPrivilegeToRole(
+      String currentName, String roleName, PartialPath path, int privilegeId, boolean grantOpt)
       throws AuthException {
-    PartialPath newPath = path;
-    if (!PrivilegeType.isPathRelevant(privilegeId)) {
-      newPath = AuthUtils.ROOT_PATH_PRIVILEGE_PATH;
-    }
-    if (!roleManager.grantPrivilegeToRole(roleName, newPath, privilegeId)) {
-      throw new AuthException(
-          TSStatusCode.ALREADY_HAS_PRIVILEGE,
-          String.format(
-              "Role %s already has %s on %s", roleName, PrivilegeType.values()[privilegeId], path));
+
+    if (currentName == null
+        || isAdmin(currentName)

Review Comment:
   no need to check.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;

Review Comment:
   ```suggestion
     public static final int SYS_PRI_SIZE = 10;
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }
+    return true;
   }
 
-  public AtomicInteger getReferenceCnt() {
-    return referenceCnt;
+  public boolean revokeGrantOpt(Integer privilege) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "path %s dont have privilege %s",
+          path.toString(), PrivilegeType.values()[privilege].toString());
+      return false;
+    }
+    grantOpts.remove(privilege);
+    return true;
   }
 
-  public void setReferenceCnt(AtomicInteger referenceCnt) {
-    this.referenceCnt = referenceCnt;
+  private Integer posToPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.READ_DATA.ordinal();
+      case 1:
+        return PrivilegeType.WRITE_DATA.ordinal();
+      case 2:
+        return PrivilegeType.READ_SCHEMA.ordinal();
+      case 3:
+        return PrivilegeType.WRITE_SCHEMA.ordinal();
+      default:
+        return -1; // this should raise an error ?
+    }
+  }
+
+  private Integer priToPos(PrivilegeType pri) {

Review Comment:
   ```suggestion
     private int priToPos(PrivilegeType pri) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {

Review Comment:
   ```suggestion
     public boolean grantPrivilege(int privilege, boolean grantOpt) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;
 
   public Role() {
     // empty constructor
   }
 
   public Role(String name) {
     this.name = name;
-    this.privilegeList = new ArrayList<>();
+    this.pathPrivilegeList = new ArrayList<>();
+    this.sysPrivilegeSet = new HashSet<>();
+    this.sysPriGrantOpt = new HashSet<>();
   }
 
+  /** ------------- get func -----------------* */
   public String getName() {
     return name;
   }
 
-  public void setName(String name) {
-    this.name = name;
+  public List<PathPrivilege> getPathPrivilegeList() {
+    return pathPrivilegeList;
   }
 
-  public List<PathPrivilege> getPrivilegeList() {
-    return privilegeList;
+  public Set<Integer> getSysPrivilege() {
+    return sysPrivilegeSet;
   }
 
-  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
-    this.privilegeList = privilegeList;
+  public Set<Integer> getPathPrivileges(PartialPath path) throws AuthException {
+    return AuthUtils.getPrivileges(path, pathPrivilegeList);
   }
 
-  public boolean hasPrivilege(PartialPath path, int privilegeId) {
-    return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
+  public Set<Integer> getSysPriGrantOpt() {
+    return sysPriGrantOpt;
   }
 
-  public void addPrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.addPrivilege(path, privilegeId, privilegeList);
+  public int getAllSysPrivileges() {
+    int privs = 0;
+    for (Integer sysPri : sysPrivilegeSet) {
+      privs |= (0b1 << sysPriTopos(sysPri));
+    }
+    for (Integer sysPriGrantOpt : sysPriGrantOpt) {
+      privs |= 0b1 << (sysPriTopos(sysPriGrantOpt) + 16);
+    }
+    return privs;
   }
 
-  public void removePrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.removePrivilege(path, privilegeId, privilegeList);
+  /** -------------- set func ----------------* */
+  public void setName(String name) {
+    this.name = name;
   }
 
-  /** set privileges of path. */
-  public void setPrivileges(PartialPath path, Set<Integer> privileges) {
-    for (PathPrivilege pathPrivilege : privilegeList) {
+  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
+    this.pathPrivilegeList = privilegeList;
+  }
+
+  public void setPathPrivileges(PartialPath path, Set<Integer> privileges) {
+    for (PathPrivilege pathPrivilege : pathPrivilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
         pathPrivilege.setPrivileges(privileges);
       }
     }
   }
 
-  public Set<Integer> getPrivileges(PartialPath path) throws AuthException {
-    return AuthUtils.getPrivileges(path, privilegeList);
+  public void addPathPrivilege(PartialPath path, int privilegeId, boolean grantOpt) {
+    AuthUtils.addPrivilege(path, privilegeId, pathPrivilegeList, grantOpt);
+  }
+
+  public void removePathPrivilege(PartialPath path, int privilegeId) {
+    AuthUtils.removePrivilege(path, privilegeId, pathPrivilegeList);
+  }
+
+  public void setSysPrivilegeSet(Set<Integer> privilegeSet) {
+    this.sysPrivilegeSet = privilegeSet;
+  }
+
+  public void setSysPriGrantOpt(Set<Integer> grantOpt) {
+    this.sysPriGrantOpt = grantOpt;
+  }
+
+  private int posToSysPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.MANAGE_DATABASE.ordinal();
+      case 1:
+        return PrivilegeType.MANAGE_USER.ordinal();
+      case 2:
+        return PrivilegeType.MANAGE_ROLE.ordinal();
+      case 3:
+        return PrivilegeType.USE_TRIGGER.ordinal();
+      case 4:
+        return PrivilegeType.USE_UDF.ordinal();
+      case 5:
+        return PrivilegeType.USE_CQ.ordinal();
+      case 6:
+        return PrivilegeType.USE_PIPE.ordinal();
+      case 7:
+        return PrivilegeType.EXTEND_TEMPLATE.ordinal();
+      case 8:
+        return PrivilegeType.AUDIT.ordinal();
+      case 9:
+        return PrivilegeType.MAINTAIN.ordinal();
+      default:
+        return -1;
+    }
+  }
+
+  private int sysPriTopos(int privilegeId) {
+    PrivilegeType type = PrivilegeType.values()[privilegeId];
+    switch (type) {
+      case MANAGE_DATABASE:
+        return 0;
+      case MANAGE_USER:
+        return 1;
+      case MANAGE_ROLE:
+        return 2;
+      case USE_TRIGGER:
+        return 3;
+      case USE_UDF:
+        return 4;
+      case USE_CQ:
+        return 5;
+      case USE_PIPE:
+        return 6;
+      case EXTEND_TEMPLATE:
+        return 7;
+      case AUDIT:
+        return 8;
+      case MAINTAIN:
+        return 9;
+      default:
+        return -1;
+    }
+  }
+
+  public void setSysPrivilegeSet(int privilegeMask) {
+    if (sysPrivilegeSet == null) {
+      sysPrivilegeSet = new HashSet<>();
+    }
+    if (sysPriGrantOpt == null) {
+      sysPriGrantOpt = new HashSet<>();
+    }
+    for (int i = 0; i < sysPriSize; i++) {
+      if ((privilegeMask & (0b1 << i)) != 0) {
+        sysPrivilegeSet.add(posToSysPri(i));
+      }
+      if ((privilegeMask & (0b1 << i + 16)) != 0) {
+        sysPriGrantOpt.add(posToSysPri(i));
+      }
+    }
+  }
+  // ??
+  public void addSysPrivilege(int privilegeId) {
+    if (!sysPrivilegeSet.contains(privilegeId)) {
+      sysPrivilegeSet.add(privilegeId);
+    }
+  }
+
+  public void removeSysPrivilege(int privilegeId) {
+    if (sysPrivilegeSet.contains(PrivilegeType.values()[privilegeId])) {
+      sysPrivilegeSet.remove(PrivilegeType.values()[privilegeId]);
+    }

Review Comment:
   ```suggestion
       sysPrivilegeSet.remove(privilegeId);
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/auth/ClusterAuthorityFetcher.java:
##########
@@ -77,43 +82,131 @@ public ClusterAuthorityFetcher(IAuthorCache iAuthorCache) {
   }
 
   @Override
-  public TSStatus checkUserPrivileges(String username, List<PartialPath> allPath, int permission) {
+  public TSStatus checkUserPathPrivileges(
+      String username, List<PartialPath> allPath, int permission) {
+    checkCacheAvailable();
     User user = iAuthorCache.getUserCache(username);
     if (user != null) {
-      for (PartialPath path : allPath) {
-        try {
-          if (!user.isOpenIdUser() || !authorizer.checkUserPrivileges(username, path, permission)) {
-            if (!user.checkPrivilege(path, permission)) {
-              if (user.getRoleList().isEmpty()) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
-              }
-              boolean status = false;
-              for (String roleName : user.getRoleList()) {
-                Role role = iAuthorCache.getRoleCache(roleName);
-                // It is detected that the role of the user does not exist in the cache, indicating
-                // that the permission information of the role has changed.
-                // The user cache needs to be initialized
-                if (role == null) {
-                  iAuthorCache.invalidateCache(username, "");
-                  return checkPath(username, allPath, permission);
-                }
-                status = role.checkPrivilege(path, permission);
-                if (status) {
-                  break;
-                }
+      if (!user.isOpenIdUser()) {
+        for (PartialPath path : allPath) {
+          // check user first
+          if (!user.checkPathPrivilege(path, permission)) {
+            if (user.getRoleList().isEmpty()) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+            boolean status = false;
+            for (String rolename : user.getRoleList()) {
+              Role cacheRole = iAuthorCache.getRoleCache(rolename);
+              if (cacheRole == null) {
+                return checkPathFromConfigNode(username, allPath, permission);
               }
-              if (!status) {
-                return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+              if (cacheRole.checkPathPrivilege(path, permission)) {
+                status = true;
+                break;
               }
             }
+            if (!status) {
+              return RpcUtils.getStatus(TSStatusCode.NO_PERMISSION);
+            }
+          }
+        }
+      }
+      return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+    } else {
+      return checkPathFromConfigNode(username, allPath, permission);
+    }
+  }
+
+  @Override
+  public PathPatternTree getAuthizedPatternTree(String username, int permission) {
+    boolean originFetch = false;
+    User user = iAuthorCache.getUserCache(username);
+    if (user != null) {
+      if (!user.getRoleList().isEmpty()) {
+        for (String role : user.getRoleList()) {
+          if (iAuthorCache.getRoleCache(role) == null) {
+            originFetch = true;
+          }
+        }
+      }
+    } else {
+      originFetch = true;
+    }
+    if (!originFetch) {
+      PathPatternTree patternTree = new PathPatternTree();
+      for (PathPrivilege path : user.getPathPrivilegeList()) {
+        if (path.getPrivileges().contains(permission)) {
+          patternTree.appendPathPattern(path.getPath());
+        }
+      }
+      for (String role : user.getRoleList()) {
+        Role cachedRole = iAuthorCache.getRoleCache(role);
+        for (PathPrivilege path : cachedRole.getPathPrivilegeList()) {
+          if (path.getPrivileges().contains(permission)) {
+            patternTree.appendPathPattern(path.getPath());
+          }
+        }
+      }
+      patternTree.constructTree();
+      return patternTree;
+    } else {
+      return fetchAuthizedPatternTree(username, permission);
+    }
+  }
+
+  public PathPatternTree fetchAuthizedPatternTree(String username, int permission) {
+    TCheckUserPrivilegesReq req = new TCheckUserPrivilegesReq(username, null, permission);
+    TAuthizedPatternTreeResp authizedPatternTree = new TAuthizedPatternTreeResp();
+    try (ConfigNodeClient configNodeClient =
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
+      authizedPatternTree = configNodeClient.fetchAuthizedPatternTree(req);
+    } catch (ClientManagerException | TException e) {
+      logger.error("Failed to connect to config node.");
+      authizedPatternTree = new TAuthizedPatternTreeResp();
+      authizedPatternTree.setStatus(
+          RpcUtils.getStatus(
+              TSStatusCode.EXECUTE_STATEMENT_ERROR, "Failed to connect to config node."));
+    }
+    if (authizedPatternTree.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      iAuthorCache.putUserCache(username, cacheUser(authizedPatternTree.getPermissionInfo()));
+      PathPatternTree patternTree =
+          PathPatternTree.deserialize(ByteBuffer.wrap(authizedPatternTree.getPathPatternTree()));
+      patternTree.constructTree();
+      return patternTree;
+    }
+    return null;

Review Comment:
   throw Execption instead of returning null, otherwise the caller may be NPE.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;
 
   public Role() {
     // empty constructor
   }
 
   public Role(String name) {
     this.name = name;
-    this.privilegeList = new ArrayList<>();
+    this.pathPrivilegeList = new ArrayList<>();
+    this.sysPrivilegeSet = new HashSet<>();
+    this.sysPriGrantOpt = new HashSet<>();
   }
 
+  /** ------------- get func -----------------* */
   public String getName() {
     return name;
   }
 
-  public void setName(String name) {
-    this.name = name;
+  public List<PathPrivilege> getPathPrivilegeList() {
+    return pathPrivilegeList;
   }
 
-  public List<PathPrivilege> getPrivilegeList() {
-    return privilegeList;
+  public Set<Integer> getSysPrivilege() {
+    return sysPrivilegeSet;
   }
 
-  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
-    this.privilegeList = privilegeList;
+  public Set<Integer> getPathPrivileges(PartialPath path) throws AuthException {
+    return AuthUtils.getPrivileges(path, pathPrivilegeList);
   }
 
-  public boolean hasPrivilege(PartialPath path, int privilegeId) {
-    return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
+  public Set<Integer> getSysPriGrantOpt() {
+    return sysPriGrantOpt;
   }
 
-  public void addPrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.addPrivilege(path, privilegeId, privilegeList);
+  public int getAllSysPrivileges() {
+    int privs = 0;
+    for (Integer sysPri : sysPrivilegeSet) {
+      privs |= (0b1 << sysPriTopos(sysPri));
+    }
+    for (Integer sysPriGrantOpt : sysPriGrantOpt) {
+      privs |= 0b1 << (sysPriTopos(sysPriGrantOpt) + 16);
+    }
+    return privs;
   }
 
-  public void removePrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.removePrivilege(path, privilegeId, privilegeList);
+  /** -------------- set func ----------------* */
+  public void setName(String name) {
+    this.name = name;
   }
 
-  /** set privileges of path. */
-  public void setPrivileges(PartialPath path, Set<Integer> privileges) {
-    for (PathPrivilege pathPrivilege : privilegeList) {
+  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
+    this.pathPrivilegeList = privilegeList;
+  }
+
+  public void setPathPrivileges(PartialPath path, Set<Integer> privileges) {
+    for (PathPrivilege pathPrivilege : pathPrivilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
         pathPrivilege.setPrivileges(privileges);
       }
     }
   }
 
-  public Set<Integer> getPrivileges(PartialPath path) throws AuthException {
-    return AuthUtils.getPrivileges(path, privilegeList);
+  public void addPathPrivilege(PartialPath path, int privilegeId, boolean grantOpt) {
+    AuthUtils.addPrivilege(path, privilegeId, pathPrivilegeList, grantOpt);
+  }
+
+  public void removePathPrivilege(PartialPath path, int privilegeId) {
+    AuthUtils.removePrivilege(path, privilegeId, pathPrivilegeList);
+  }
+
+  public void setSysPrivilegeSet(Set<Integer> privilegeSet) {
+    this.sysPrivilegeSet = privilegeSet;
+  }
+
+  public void setSysPriGrantOpt(Set<Integer> grantOpt) {
+    this.sysPriGrantOpt = grantOpt;
+  }
+
+  private int posToSysPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.MANAGE_DATABASE.ordinal();
+      case 1:
+        return PrivilegeType.MANAGE_USER.ordinal();
+      case 2:
+        return PrivilegeType.MANAGE_ROLE.ordinal();
+      case 3:
+        return PrivilegeType.USE_TRIGGER.ordinal();
+      case 4:
+        return PrivilegeType.USE_UDF.ordinal();
+      case 5:
+        return PrivilegeType.USE_CQ.ordinal();
+      case 6:
+        return PrivilegeType.USE_PIPE.ordinal();
+      case 7:
+        return PrivilegeType.EXTEND_TEMPLATE.ordinal();
+      case 8:
+        return PrivilegeType.AUDIT.ordinal();
+      case 9:
+        return PrivilegeType.MAINTAIN.ordinal();
+      default:
+        return -1;
+    }
+  }
+
+  private int sysPriTopos(int privilegeId) {
+    PrivilegeType type = PrivilegeType.values()[privilegeId];
+    switch (type) {
+      case MANAGE_DATABASE:
+        return 0;
+      case MANAGE_USER:
+        return 1;
+      case MANAGE_ROLE:
+        return 2;
+      case USE_TRIGGER:
+        return 3;
+      case USE_UDF:
+        return 4;
+      case USE_CQ:
+        return 5;
+      case USE_PIPE:
+        return 6;
+      case EXTEND_TEMPLATE:
+        return 7;
+      case AUDIT:
+        return 8;
+      case MAINTAIN:
+        return 9;
+      default:
+        return -1;
+    }
+  }
+
+  public void setSysPrivilegeSet(int privilegeMask) {
+    if (sysPrivilegeSet == null) {
+      sysPrivilegeSet = new HashSet<>();
+    }
+    if (sysPriGrantOpt == null) {
+      sysPriGrantOpt = new HashSet<>();
+    }
+    for (int i = 0; i < sysPriSize; i++) {
+      if ((privilegeMask & (0b1 << i)) != 0) {
+        sysPrivilegeSet.add(posToSysPri(i));
+      }
+      if ((privilegeMask & (0b1 << i + 16)) != 0) {
+        sysPriGrantOpt.add(posToSysPri(i));
+      }
+    }
+  }
+  // ??
+  public void addSysPrivilege(int privilegeId) {
+    if (!sysPrivilegeSet.contains(privilegeId)) {
+      sysPrivilegeSet.add(privilegeId);
+    }
+  }
+
+  public void removeSysPrivilege(int privilegeId) {
+    if (sysPrivilegeSet.contains(PrivilegeType.values()[privilegeId])) {
+      sysPrivilegeSet.remove(PrivilegeType.values()[privilegeId]);
+    }
   }
 
-  public boolean checkPrivilege(PartialPath path, int privilegeId) throws AuthException {
-    return AuthUtils.checkPrivilege(path, privilegeId, privilegeList);
+  /** ------------ check func ---------------* */
+  public boolean hasPrivilege(PartialPath path, int privilegeId) {
+    if (path == null) {
+      return sysPrivilegeSet.contains(privilegeId);
+    } else {
+      return AuthUtils.hasPrivilege(path, privilegeId, pathPrivilegeList);
+    }
+  }
+
+  public boolean checkPathPrivilege(PartialPath path, int privilegeId) {
+    return AuthUtils.checkPathPrivilege(path, privilegeId, pathPrivilegeList);
+  }
+
+  public boolean checkPathPrivilegeGrantOpt(PartialPath path, int privilegeId) {
+    return AuthUtils.checkPathPrivilegeGrantOpt(path, privilegeId, pathPrivilegeList);
   }
 
+  public boolean checkSysPrivilege(int privilegeId) {
+    return sysPrivilegeSet.contains(PrivilegeType.values()[privilegeId]);

Review Comment:
   ```suggestion
       return sysPrivilegeSet.contains(privilegeId);
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }

Review Comment:
   ```suggestion
       grantOpts.remove(privilege);
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PrivilegeType.java:
##########
@@ -34,50 +27,56 @@ public enum PrivilegeType {
   WRITE_SCHEMA(true),
   MANAGE_USER,
   MANAGE_ROLE,
-  GRANT_PRIVILEGE,
-  ALTER_PASSWORD,
-  USE_TRIGGER(true),
+  USE_TRIGGER,
+
+  USE_UDF,
+
   USE_CQ,
   USE_PIPE,
-  MANAGE_DATABASE(true),
+  EXTEND_TEMPLATE,
+  MANAGE_DATABASE,
   MAINTAIN,
-  READ(true, false, READ_DATA, READ_SCHEMA),
-  WRITE(true, false, WRITE_DATA, WRITE_SCHEMA),
-  ALL(
-      true,
-      false,
-      READ,
-      WRITE,
-      MANAGE_USER,
-      MANAGE_ROLE,
-      GRANT_PRIVILEGE,
-      ALTER_PASSWORD,
-      USE_TRIGGER,
-      USE_CQ,
-      USE_PIPE,
-      MANAGE_DATABASE,
-      MAINTAIN);
+  AUDIT;
+
+  private enum scope {
+    NULL,
+    SYSTEMPRIVILEGE,
+    PATHPRIVILEGE,
+    ROLEPRIVILEGE;
+  }
 
   private static final int PRIVILEGE_COUNT = values().length;
 
   private final boolean isPathRelevant;
-  private final boolean isStorable;
-  private final List<PrivilegeType> subPrivileges = new ArrayList<>();
 
   PrivilegeType() {
     this.isPathRelevant = false;
-    this.isStorable = true;
   }
 
   PrivilegeType(boolean isPathRelevant) {
     this.isPathRelevant = isPathRelevant;
-    this.isStorable = true;
   }
 
-  PrivilegeType(boolean isPathRelevant, boolean isStorable, PrivilegeType... privilegeTypes) {
-    this.isPathRelevant = isPathRelevant;
-    this.isStorable = isStorable;
-    this.subPrivileges.addAll(Arrays.asList(privilegeTypes));
+  scope getAuthScope() {

Review Comment:
   delete it.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/BasicAuthorizer.java:
##########
@@ -317,20 +316,109 @@ public boolean checkUserPrivileges(String username, PartialPath path, int privil
       throw new AuthException(
           TSStatusCode.USER_NOT_EXIST, String.format(NO_SUCH_USER_EXCEPTION, username));
     }
-    // get privileges of the user
-    if (user.checkPrivilege(path, privilegeId)) {
-      return true;
-    }
-    // merge the privileges of the roles of the user
-    for (String roleName : user.getRoleList()) {
-      Role role = roleManager.getRole(roleName);
-      if (role.checkPrivilege(path, privilegeId)) {
+    if (path != null) {
+      // get privileges of the user
+      if (user.checkPathPrivilege(path, privilegeId)) {
         return true;
       }
+      // merge the privileges of the roles of the user
+      for (String roleName : user.getRoleList()) {
+        Role role = roleManager.getRole(roleName);
+        if (role.checkPathPrivilege(path, privilegeId)) {
+          return true;
+        }
+      }
+    } else {
+      if (user.checkSysPrivilege(privilegeId)) {
+        return true;
+      }
+      for (String roleName : user.getRoleList()) {
+        Role role = roleManager.getRole(roleName);
+        if (role.checkSysPrivilege(privilegeId)) {
+          return true;
+        }
+      }
     }
+
     return false;
   }
 
+  public boolean checkUserPrivilegeGrantOpt(String username, PartialPath path, int privilegeId)

Review Comment:
   provide it as a rpc interface.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/authorizer/IAuthorizer.java:
##########
@@ -70,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, PartialPath path, int privilegeId)
+  void grantPrivilegeToUser(
+      String currnetUser, String username, PartialPath path, int privilegeId, boolean grantOpt)

Review Comment:
   ```suggestion
         String username, PartialPath path, int privilegeId, boolean grantOpt)
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;
 
   public Role() {
     // empty constructor
   }
 
   public Role(String name) {
     this.name = name;
-    this.privilegeList = new ArrayList<>();
+    this.pathPrivilegeList = new ArrayList<>();
+    this.sysPrivilegeSet = new HashSet<>();
+    this.sysPriGrantOpt = new HashSet<>();
   }
 
+  /** ------------- get func -----------------* */
   public String getName() {
     return name;
   }
 
-  public void setName(String name) {
-    this.name = name;
+  public List<PathPrivilege> getPathPrivilegeList() {
+    return pathPrivilegeList;
   }
 
-  public List<PathPrivilege> getPrivilegeList() {
-    return privilegeList;
+  public Set<Integer> getSysPrivilege() {
+    return sysPrivilegeSet;
   }
 
-  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
-    this.privilegeList = privilegeList;
+  public Set<Integer> getPathPrivileges(PartialPath path) throws AuthException {
+    return AuthUtils.getPrivileges(path, pathPrivilegeList);
   }
 
-  public boolean hasPrivilege(PartialPath path, int privilegeId) {
-    return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
+  public Set<Integer> getSysPriGrantOpt() {
+    return sysPriGrantOpt;
   }
 
-  public void addPrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.addPrivilege(path, privilegeId, privilegeList);
+  public int getAllSysPrivileges() {
+    int privs = 0;
+    for (Integer sysPri : sysPrivilegeSet) {

Review Comment:
   ```suggestion
       for (int sysPri : sysPrivilegeSet) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;

Review Comment:
   Can we directly use Enum to init this field? Because in future, other  people add a new auth, he may forget to change here.



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",

Review Comment:
   ```suggestion
             "not find privilege {} on path {}",
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }
+    return true;
   }
 
-  public AtomicInteger getReferenceCnt() {
-    return referenceCnt;
+  public boolean revokeGrantOpt(Integer privilege) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "path %s dont have privilege %s",
+          path.toString(), PrivilegeType.values()[privilege].toString());
+      return false;
+    }
+    grantOpts.remove(privilege);
+    return true;
   }
 
-  public void setReferenceCnt(AtomicInteger referenceCnt) {
-    this.referenceCnt = referenceCnt;
+  private Integer posToPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.READ_DATA.ordinal();
+      case 1:
+        return PrivilegeType.WRITE_DATA.ordinal();
+      case 2:
+        return PrivilegeType.READ_SCHEMA.ordinal();
+      case 3:
+        return PrivilegeType.WRITE_SCHEMA.ordinal();
+      default:
+        return -1; // this should raise an error ?
+    }
+  }
+
+  private Integer priToPos(PrivilegeType pri) {
+    switch (pri) {
+      case READ_DATA:
+        return 0;
+      case WRITE_DATA:
+        return 1;
+      case READ_SCHEMA:
+        return 2;
+      case WRITE_SCHEMA:
+        return 3;
+      default:
+        return -1;
+    }
+  }
+
+  private boolean basicCheck(Integer privilege) {
+    if (!(privilege == PrivilegeType.READ_DATA.ordinal()
+        || privilege == PrivilegeType.WRITE_DATA.ordinal()
+        || privilege == PrivilegeType.READ_SCHEMA.ordinal()
+        || privilege == PrivilegeType.WRITE_SCHEMA.ordinal())) {
+      logger.warn(
+          String.format(
+              "Get an illegal privilege %s for path %s",
+              PrivilegeType.values()[privilege].toString(), path.toString()));
+      return false;
+    }
+    return true;
+  }
+
+  public void setAllPrivileges(int privs) {
+    for (int i = 0; i < priCap; i++) {
+      if (((0x1 << i) & privs) != 0) {
+        privileges.add(posToPri(i));
+      }
+      if ((((0x1 << (i + 4) & privs) != 0))) {

Review Comment:
   ```suggestion
         if (((0x1 << (i + 16) & privs) != 0)) {
   ```
   16 or 4?



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;
 
   public Role() {
     // empty constructor
   }
 
   public Role(String name) {
     this.name = name;
-    this.privilegeList = new ArrayList<>();
+    this.pathPrivilegeList = new ArrayList<>();
+    this.sysPrivilegeSet = new HashSet<>();
+    this.sysPriGrantOpt = new HashSet<>();
   }
 
+  /** ------------- get func -----------------* */
   public String getName() {
     return name;
   }
 
-  public void setName(String name) {
-    this.name = name;
+  public List<PathPrivilege> getPathPrivilegeList() {
+    return pathPrivilegeList;
   }
 
-  public List<PathPrivilege> getPrivilegeList() {
-    return privilegeList;
+  public Set<Integer> getSysPrivilege() {
+    return sysPrivilegeSet;
   }
 
-  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
-    this.privilegeList = privilegeList;
+  public Set<Integer> getPathPrivileges(PartialPath path) throws AuthException {
+    return AuthUtils.getPrivileges(path, pathPrivilegeList);
   }
 
-  public boolean hasPrivilege(PartialPath path, int privilegeId) {
-    return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
+  public Set<Integer> getSysPriGrantOpt() {
+    return sysPriGrantOpt;
   }
 
-  public void addPrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.addPrivilege(path, privilegeId, privilegeList);
+  public int getAllSysPrivileges() {
+    int privs = 0;
+    for (Integer sysPri : sysPrivilegeSet) {
+      privs |= (0b1 << sysPriTopos(sysPri));
+    }
+    for (Integer sysPriGrantOpt : sysPriGrantOpt) {
+      privs |= 0b1 << (sysPriTopos(sysPriGrantOpt) + 16);
+    }
+    return privs;
   }
 
-  public void removePrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.removePrivilege(path, privilegeId, privilegeList);
+  /** -------------- set func ----------------* */
+  public void setName(String name) {
+    this.name = name;
   }
 
-  /** set privileges of path. */
-  public void setPrivileges(PartialPath path, Set<Integer> privileges) {
-    for (PathPrivilege pathPrivilege : privilegeList) {
+  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
+    this.pathPrivilegeList = privilegeList;
+  }
+
+  public void setPathPrivileges(PartialPath path, Set<Integer> privileges) {
+    for (PathPrivilege pathPrivilege : pathPrivilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
         pathPrivilege.setPrivileges(privileges);
       }
     }
   }
 
-  public Set<Integer> getPrivileges(PartialPath path) throws AuthException {
-    return AuthUtils.getPrivileges(path, privilegeList);
+  public void addPathPrivilege(PartialPath path, int privilegeId, boolean grantOpt) {
+    AuthUtils.addPrivilege(path, privilegeId, pathPrivilegeList, grantOpt);
+  }
+
+  public void removePathPrivilege(PartialPath path, int privilegeId) {
+    AuthUtils.removePrivilege(path, privilegeId, pathPrivilegeList);
+  }
+
+  public void setSysPrivilegeSet(Set<Integer> privilegeSet) {
+    this.sysPrivilegeSet = privilegeSet;
+  }
+
+  public void setSysPriGrantOpt(Set<Integer> grantOpt) {
+    this.sysPriGrantOpt = grantOpt;
+  }
+
+  private int posToSysPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.MANAGE_DATABASE.ordinal();
+      case 1:
+        return PrivilegeType.MANAGE_USER.ordinal();
+      case 2:
+        return PrivilegeType.MANAGE_ROLE.ordinal();
+      case 3:
+        return PrivilegeType.USE_TRIGGER.ordinal();
+      case 4:
+        return PrivilegeType.USE_UDF.ordinal();
+      case 5:
+        return PrivilegeType.USE_CQ.ordinal();
+      case 6:
+        return PrivilegeType.USE_PIPE.ordinal();
+      case 7:
+        return PrivilegeType.EXTEND_TEMPLATE.ordinal();
+      case 8:
+        return PrivilegeType.AUDIT.ordinal();
+      case 9:
+        return PrivilegeType.MAINTAIN.ordinal();
+      default:
+        return -1;
+    }
+  }
+
+  private int sysPriTopos(int privilegeId) {
+    PrivilegeType type = PrivilegeType.values()[privilegeId];
+    switch (type) {
+      case MANAGE_DATABASE:
+        return 0;
+      case MANAGE_USER:
+        return 1;
+      case MANAGE_ROLE:
+        return 2;
+      case USE_TRIGGER:
+        return 3;
+      case USE_UDF:
+        return 4;
+      case USE_CQ:
+        return 5;
+      case USE_PIPE:
+        return 6;
+      case EXTEND_TEMPLATE:
+        return 7;
+      case AUDIT:
+        return 8;
+      case MAINTAIN:
+        return 9;
+      default:
+        return -1;
+    }
+  }
+
+  public void setSysPrivilegeSet(int privilegeMask) {
+    if (sysPrivilegeSet == null) {
+      sysPrivilegeSet = new HashSet<>();
+    }
+    if (sysPriGrantOpt == null) {
+      sysPriGrantOpt = new HashSet<>();
+    }
+    for (int i = 0; i < sysPriSize; i++) {
+      if ((privilegeMask & (0b1 << i)) != 0) {
+        sysPrivilegeSet.add(posToSysPri(i));
+      }
+      if ((privilegeMask & (0b1 << i + 16)) != 0) {
+        sysPriGrantOpt.add(posToSysPri(i));
+      }
+    }
+  }
+  // ??
+  public void addSysPrivilege(int privilegeId) {
+    if (!sysPrivilegeSet.contains(privilegeId)) {
+      sysPrivilegeSet.add(privilegeId);
+    }

Review Comment:
   ```suggestion
       sysPrivilegeSet.add(privilegeId);
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;
 
   public Role() {
     // empty constructor
   }
 
   public Role(String name) {
     this.name = name;
-    this.privilegeList = new ArrayList<>();
+    this.pathPrivilegeList = new ArrayList<>();
+    this.sysPrivilegeSet = new HashSet<>();
+    this.sysPriGrantOpt = new HashSet<>();
   }
 
+  /** ------------- get func -----------------* */
   public String getName() {
     return name;
   }
 
-  public void setName(String name) {
-    this.name = name;
+  public List<PathPrivilege> getPathPrivilegeList() {
+    return pathPrivilegeList;
   }
 
-  public List<PathPrivilege> getPrivilegeList() {
-    return privilegeList;
+  public Set<Integer> getSysPrivilege() {
+    return sysPrivilegeSet;
   }
 
-  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
-    this.privilegeList = privilegeList;
+  public Set<Integer> getPathPrivileges(PartialPath path) throws AuthException {
+    return AuthUtils.getPrivileges(path, pathPrivilegeList);
   }
 
-  public boolean hasPrivilege(PartialPath path, int privilegeId) {
-    return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
+  public Set<Integer> getSysPriGrantOpt() {
+    return sysPriGrantOpt;
   }
 
-  public void addPrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.addPrivilege(path, privilegeId, privilegeList);
+  public int getAllSysPrivileges() {
+    int privs = 0;
+    for (Integer sysPri : sysPrivilegeSet) {
+      privs |= (0b1 << sysPriTopos(sysPri));
+    }
+    for (Integer sysPriGrantOpt : sysPriGrantOpt) {
+      privs |= 0b1 << (sysPriTopos(sysPriGrantOpt) + 16);
+    }
+    return privs;
   }
 
-  public void removePrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.removePrivilege(path, privilegeId, privilegeList);
+  /** -------------- set func ----------------* */
+  public void setName(String name) {
+    this.name = name;
   }
 
-  /** set privileges of path. */
-  public void setPrivileges(PartialPath path, Set<Integer> privileges) {
-    for (PathPrivilege pathPrivilege : privilegeList) {
+  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
+    this.pathPrivilegeList = privilegeList;
+  }
+
+  public void setPathPrivileges(PartialPath path, Set<Integer> privileges) {
+    for (PathPrivilege pathPrivilege : pathPrivilegeList) {
       if (pathPrivilege.getPath().equals(path)) {
         pathPrivilege.setPrivileges(privileges);
       }
     }
   }
 
-  public Set<Integer> getPrivileges(PartialPath path) throws AuthException {
-    return AuthUtils.getPrivileges(path, privilegeList);
+  public void addPathPrivilege(PartialPath path, int privilegeId, boolean grantOpt) {
+    AuthUtils.addPrivilege(path, privilegeId, pathPrivilegeList, grantOpt);
+  }
+
+  public void removePathPrivilege(PartialPath path, int privilegeId) {
+    AuthUtils.removePrivilege(path, privilegeId, pathPrivilegeList);
+  }
+
+  public void setSysPrivilegeSet(Set<Integer> privilegeSet) {
+    this.sysPrivilegeSet = privilegeSet;
+  }
+
+  public void setSysPriGrantOpt(Set<Integer> grantOpt) {
+    this.sysPriGrantOpt = grantOpt;
+  }
+
+  private int posToSysPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.MANAGE_DATABASE.ordinal();
+      case 1:
+        return PrivilegeType.MANAGE_USER.ordinal();
+      case 2:
+        return PrivilegeType.MANAGE_ROLE.ordinal();
+      case 3:
+        return PrivilegeType.USE_TRIGGER.ordinal();
+      case 4:
+        return PrivilegeType.USE_UDF.ordinal();
+      case 5:
+        return PrivilegeType.USE_CQ.ordinal();
+      case 6:
+        return PrivilegeType.USE_PIPE.ordinal();
+      case 7:
+        return PrivilegeType.EXTEND_TEMPLATE.ordinal();
+      case 8:
+        return PrivilegeType.AUDIT.ordinal();
+      case 9:
+        return PrivilegeType.MAINTAIN.ordinal();
+      default:
+        return -1;
+    }
+  }
+
+  private int sysPriTopos(int privilegeId) {
+    PrivilegeType type = PrivilegeType.values()[privilegeId];
+    switch (type) {
+      case MANAGE_DATABASE:
+        return 0;
+      case MANAGE_USER:
+        return 1;
+      case MANAGE_ROLE:
+        return 2;
+      case USE_TRIGGER:
+        return 3;
+      case USE_UDF:
+        return 4;
+      case USE_CQ:
+        return 5;
+      case USE_PIPE:
+        return 6;
+      case EXTEND_TEMPLATE:
+        return 7;
+      case AUDIT:
+        return 8;
+      case MAINTAIN:
+        return 9;
+      default:
+        return -1;
+    }
+  }
+
+  public void setSysPrivilegeSet(int privilegeMask) {
+    if (sysPrivilegeSet == null) {
+      sysPrivilegeSet = new HashSet<>();
+    }
+    if (sysPriGrantOpt == null) {
+      sysPriGrantOpt = new HashSet<>();
+    }
+    for (int i = 0; i < sysPriSize; i++) {
+      if ((privilegeMask & (0b1 << i)) != 0) {

Review Comment:
   ```suggestion
         if ((privilegeMask & (1 << i)) != 0) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }
+    return true;
   }
 
-  public AtomicInteger getReferenceCnt() {
-    return referenceCnt;
+  public boolean revokeGrantOpt(Integer privilege) {

Review Comment:
   ```suggestion
     public boolean revokeGrantOpt(int privilege) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/PathPrivilege.java:
##########
@@ -75,20 +66,128 @@ public void setPrivileges(Set<Integer> privileges) {
     this.privileges = privileges;
   }
 
-  public PartialPath getPath() {
-    return path;
+  public Set<Integer> getGrantOpt() {
+    return grantOpts;
   }
 
-  public void setPath(PartialPath path) {
-    this.path = path;
+  public void setGrantOpt(Set<Integer> grantOpts) {
+    this.grantOpts = grantOpts;
+  }
+
+  public boolean grantPrivilege(Integer privilege, boolean grantOpt) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+    }
+    if (grantOpt && !grantOpts.contains(privilege)) {
+      grantOpts.add(privilege);
+    }
+    return true;
+  }
+
+  public boolean revokePrivilege(Integer privilege) {
+    basicCheck(privilege);
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "not find privilege %s on path %s",
+          PrivilegeType.values()[privilege].toString(), path.toString());
+      return false;
+    }
+    privileges.remove(privilege);
+    if (grantOpts.contains(privilege)) {
+      grantOpts.remove(privilege);
+    }
+    return true;
   }
 
-  public AtomicInteger getReferenceCnt() {
-    return referenceCnt;
+  public boolean revokeGrantOpt(Integer privilege) {
+    if (!basicCheck(privilege)) {
+      return false;
+    }
+    if (!privileges.contains(privilege)) {
+      logger.warn(
+          "path %s dont have privilege %s",
+          path.toString(), PrivilegeType.values()[privilege].toString());
+      return false;
+    }
+    grantOpts.remove(privilege);
+    return true;
   }
 
-  public void setReferenceCnt(AtomicInteger referenceCnt) {
-    this.referenceCnt = referenceCnt;
+  private Integer posToPri(int pos) {
+    switch (pos) {
+      case 0:
+        return PrivilegeType.READ_DATA.ordinal();
+      case 1:
+        return PrivilegeType.WRITE_DATA.ordinal();
+      case 2:
+        return PrivilegeType.READ_SCHEMA.ordinal();
+      case 3:
+        return PrivilegeType.WRITE_SCHEMA.ordinal();
+      default:
+        return -1; // this should raise an error ?
+    }
+  }
+
+  private Integer priToPos(PrivilegeType pri) {
+    switch (pri) {
+      case READ_DATA:
+        return 0;
+      case WRITE_DATA:
+        return 1;
+      case READ_SCHEMA:
+        return 2;
+      case WRITE_SCHEMA:
+        return 3;
+      default:
+        return -1;
+    }
+  }
+
+  private boolean basicCheck(Integer privilege) {
+    if (!(privilege == PrivilegeType.READ_DATA.ordinal()
+        || privilege == PrivilegeType.WRITE_DATA.ordinal()
+        || privilege == PrivilegeType.READ_SCHEMA.ordinal()
+        || privilege == PrivilegeType.WRITE_SCHEMA.ordinal())) {
+      logger.warn(
+          String.format(
+              "Get an illegal privilege %s for path %s",
+              PrivilegeType.values()[privilege].toString(), path.toString()));
+      return false;
+    }
+    return true;
+  }
+
+  public void setAllPrivileges(int privs) {
+    for (int i = 0; i < priCap; i++) {
+      if (((0x1 << i) & privs) != 0) {
+        privileges.add(posToPri(i));
+      }
+      if ((((0x1 << (i + 4) & privs) != 0))) {
+        grantOpts.add(posToPri(i));
+      }
+    }
+  }
+
+  public int getAllPrivileges() {
+    int privilege = 0;
+    for (Integer pri : privileges) {
+      privilege |= 1 << priToPos(PrivilegeType.values()[pri]);
+    }
+    for (Integer grantOpt : grantOpts) {

Review Comment:
   ```suggestion
       for (int grantOpt : grantOpts) {
   ```



##########
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/auth/entity/Role.java:
##########
@@ -36,62 +37,196 @@
 public class Role {
 
   private String name;
-  private List<PathPrivilege> privilegeList;
+  private List<PathPrivilege> pathPrivilegeList;
+
+  private Set<Integer> sysPrivilegeSet;
+
+  private Set<Integer> sysPriGrantOpt;
+
+  public static final int sysPriSize = 10;
 
   public Role() {
     // empty constructor
   }
 
   public Role(String name) {
     this.name = name;
-    this.privilegeList = new ArrayList<>();
+    this.pathPrivilegeList = new ArrayList<>();
+    this.sysPrivilegeSet = new HashSet<>();
+    this.sysPriGrantOpt = new HashSet<>();
   }
 
+  /** ------------- get func -----------------* */
   public String getName() {
     return name;
   }
 
-  public void setName(String name) {
-    this.name = name;
+  public List<PathPrivilege> getPathPrivilegeList() {
+    return pathPrivilegeList;
   }
 
-  public List<PathPrivilege> getPrivilegeList() {
-    return privilegeList;
+  public Set<Integer> getSysPrivilege() {
+    return sysPrivilegeSet;
   }
 
-  public void setPrivilegeList(List<PathPrivilege> privilegeList) {
-    this.privilegeList = privilegeList;
+  public Set<Integer> getPathPrivileges(PartialPath path) throws AuthException {
+    return AuthUtils.getPrivileges(path, pathPrivilegeList);
   }
 
-  public boolean hasPrivilege(PartialPath path, int privilegeId) {
-    return AuthUtils.hasPrivilege(path, privilegeId, privilegeList);
+  public Set<Integer> getSysPriGrantOpt() {
+    return sysPriGrantOpt;
   }
 
-  public void addPrivilege(PartialPath path, int privilegeId) {
-    AuthUtils.addPrivilege(path, privilegeId, privilegeList);
+  public int getAllSysPrivileges() {
+    int privs = 0;
+    for (Integer sysPri : sysPrivilegeSet) {
+      privs |= (0b1 << sysPriTopos(sysPri));
+    }
+    for (Integer sysPriGrantOpt : sysPriGrantOpt) {

Review Comment:
   ```suggestion
       for (int sysPriGrantOpt : sysPriGrantOpt) {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org